You are viewing a plain text version of this content. The canonical link for it is here.
Posted to issues@flink.apache.org by GitBox <gi...@apache.org> on 2020/05/15 14:02:59 UTC

[GitHub] [flink] leonardBang opened a new pull request #12176: [FLINK-17029][jdbc]Introduce a new JDBC connector with new property keys

leonardBang opened a new pull request #12176:
URL: https://github.com/apache/flink/pull/12176


   
   <!--
   *Thank you very much for contributing to Apache Flink - we are happy that you want to help us improve Flink. To help the community review your contribution in the best possible way, please go through the checklist below, which will get the contribution into a shape in which it can be best reviewed.*
   
   *Please understand that we do not do this to make contributions to Flink a hassle. In order to uphold a high standard of quality for code contributions, while at the same time managing a large number of contributions, we need contributors to prepare the contributions well, and give reviewers enough contextual information for the review. Please also understand that contributions that do not follow this guide will take longer to review and thus typically be picked up with lower priority by the community.*
   
   ## Contribution Checklist
   
     - Make sure that the pull request corresponds to a [JIRA issue](https://issues.apache.org/jira/projects/FLINK/issues). Exceptions are made for typos in JavaDoc or documentation files, which need no JIRA issue.
   
     - Name the pull request in the form "[FLINK-XXXX] [component] Title of the pull request", where *FLINK-XXXX* should be replaced by the actual issue number. Skip *component* if you are unsure about which is the best component.
     Typo fixes that have no associated JIRA issue should be named following this pattern: `[hotfix] [docs] Fix typo in event time introduction` or `[hotfix] [javadocs] Expand JavaDoc for PuncuatedWatermarkGenerator`.
   
     - Fill out the template below to describe the changes contributed by the pull request. That will give reviewers the context they need to do the review.
   
     - Make sure that the change passes the automated tests, i.e., `mvn clean verify` passes. You can set up Travis CI to do that following [this guide](https://flink.apache.org/contributing/contribute-code.html#open-a-pull-request).
   
     - Each pull request should address only one issue, not mix up code from multiple issues.
   
     - Each commit in the pull request has a meaningful commit message (including the JIRA id)
   
     - Once all items of the checklist are addressed, remove the above text and this checklist, leaving only the filled out template below.
   
   
   **(The sections below can be removed for hotfixes of typos)**
   -->
   
   ## What is the purpose of the change
   
   *This pull request introduce a new JDBC connector with new property keys which follows [FLIP-95](https://cwiki.apache.org/confluence/display/FLINK/FLIP-95%3A+New+TableSource+and+TableSink+interfaces).*
   
   
   ## Brief change log
   
     - (1) introduce **JdbcDynamicTableSource**、**JdbcDynamicTableSink**、and new **Factory** to find them
     - (2) introduce **JdbcDynamicInputFormat**、**JdbcDynamicOutputFormat** that use new internal data structure **RowData**
   - (3) introduce **JdbcDynamicLookupFunction** that we used to support` temporal table join.`
   
   ## Verifying this change
   - (1) Add three ITCases **JdbcDynamicTableSoureITCase**、**JdbcDynamicTableSinkITCase**、**JdbcTemporayTableITCase**
   - (2) Add unit tests for **JdbcDynamicInputFormat** and **JdbcDynamicOutputFormat**
   
   ## Does this pull request potentially affect one of the following parts:
   
     - Dependencies (does it add or upgrade a dependency): (no)
     - The public API, i.e., is any changed class annotated with `@Public(Evolving)`: (no)
     - The serializers: (no)
     - The runtime per-record code paths (performance sensitive): ( no)
     - Anything that affects deployment or recovery: JobManager (and its components), Checkpointing, Yarn/Mesos, ZooKeeper: (no)
     - The S3 file system connector: (no)
   
   ## Documentation
   
     - Does this pull request introduce a new feature? (yes)
     - If yes, how is the feature documented? (JavaDocs)


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

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [flink] wuchong commented on a change in pull request #12176: [FLINK-17029][jdbc]Introduce a new JDBC connector with new property keys

Posted by GitBox <gi...@apache.org>.
wuchong commented on a change in pull request #12176:
URL: https://github.com/apache/flink/pull/12176#discussion_r426133850



##########
File path: flink-connectors/flink-connector-jdbc/src/main/java/org/apache/flink/connector/jdbc/table/JdbcDynamicTableSource.java
##########
@@ -0,0 +1,162 @@
+/*
+ * 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.connector.jdbc.table;
+
+import org.apache.flink.annotation.Internal;
+import org.apache.flink.api.java.typeutils.RowTypeInfo;
+import org.apache.flink.connector.jdbc.dialect.JdbcDialect;
+import org.apache.flink.connector.jdbc.internal.options.JdbcLookupOptions;
+import org.apache.flink.connector.jdbc.internal.options.JdbcOptions;
+import org.apache.flink.connector.jdbc.internal.options.JdbcReadOptions;
+import org.apache.flink.connector.jdbc.split.JdbcNumericBetweenParametersProvider;
+import org.apache.flink.table.api.TableSchema;
+import org.apache.flink.table.connector.ChangelogMode;
+import org.apache.flink.table.connector.source.DynamicTableSource;
+import org.apache.flink.table.connector.source.InputFormatProvider;
+import org.apache.flink.table.connector.source.LookupTableSource;
+import org.apache.flink.table.connector.source.ScanTableSource;
+import org.apache.flink.table.connector.source.TableFunctionProvider;
+import org.apache.flink.table.types.DataType;
+import org.apache.flink.table.types.logical.LogicalType;
+import org.apache.flink.table.types.logical.RowType;
+
+import java.util.Arrays;
+import java.util.Objects;
+
+import static org.apache.flink.table.types.utils.TypeConversions.fromDataTypeToLegacyInfo;
+
+/**
+ * A {@link DynamicTableSource} for JDBC.
+ */
+@Internal
+public class JdbcDynamicTableSource implements ScanTableSource, LookupTableSource {
+
+	private static final String name = "JdbcTableSource";
+	private final JdbcOptions options;
+	private final JdbcReadOptions readOptions;
+	private final JdbcLookupOptions lookupOptions;
+	private final TableSchema schema;
+	private final int[] selectFields;
+
+	public JdbcDynamicTableSource(
+		JdbcOptions options,
+		JdbcReadOptions readOptions,
+		JdbcLookupOptions lookupOptions,
+		TableSchema schema,
+		int[] selectFields) {
+		this.options = options;
+		this.readOptions = readOptions;
+		this.lookupOptions = lookupOptions;
+		this.schema = schema;
+		this.selectFields = selectFields;
+	}
+
+	@Override
+	public LookupRuntimeProvider getLookupRuntimeProvider(LookupTableSource.Context context) {
+		// JDBC only support non-nested look up keys
+		String[] keyNames = new String[context.getKeys().length];
+		for (int i = 0; i < keyNames.length; i++) {
+			int index = context.getKeys()[i][0];
+			keyNames[i] = schema.getFieldNames()[index];
+		}
+		return TableFunctionProvider.of(JdbcDynamicLookupFunction.builder()
+			.setFieldNames(schema.getFieldNames())
+			.setFieldTypes(schema.getFieldDataTypes())
+			.setKeyNames(keyNames)
+			.setOptions(options)
+			.setLookupOptions(lookupOptions)
+			.build());
+	}
+
+	@Override
+	public ScanRuntimeProvider getScanRuntimeProvider(ScanTableSource.Context runtimeProviderContext) {
+		final DataType rowDataType = schema.toPhysicalRowDataType();
+		final RowTypeInfo rowTypeInfo = (RowTypeInfo) fromDataTypeToLegacyInfo(rowDataType);
+		final JdbcDynamicInputFormat.Builder builder = JdbcDynamicInputFormat.builder()
+			.setDrivername(options.getDriverName())
+			.setDBUrl(options.getDbURL())
+			.setUsername(options.getUsername().orElse(null))
+			.setPassword(options.getPassword().orElse(null))
+			.setRowTypeInfo(new RowTypeInfo(rowTypeInfo.getFieldTypes(), rowTypeInfo.getFieldNames()));
+
+		if (readOptions.getFetchSize() != 0) {
+			builder.setFetchSize(readOptions.getFetchSize());
+		}
+		final JdbcDialect dialect = options.getDialect();
+		String query = dialect.getSelectFromStatement(
+			options.getTableName(), rowTypeInfo.getFieldNames(), new String[0]);
+		if (readOptions.getPartitionColumnName().isPresent()) {
+			long lowerBound = readOptions.getPartitionLowerBound().get();
+			long upperBound = readOptions.getPartitionUpperBound().get();
+			int numPartitions = readOptions.getNumPartitions().get();
+			builder.setParametersProvider(
+				new JdbcNumericBetweenParametersProvider(lowerBound, upperBound).ofBatchNum(numPartitions));
+			query += " WHERE " +
+				dialect.quoteIdentifier(readOptions.getPartitionColumnName().get()) +
+				" BETWEEN ? AND ?";
+		}
+		builder.setQuery(query);
+		final RowType rowType = RowType.of(
+			Arrays.stream(schema.getFieldDataTypes())
+				.map(DataType::getLogicalType)
+				.toArray(LogicalType[]::new),
+			schema.getFieldNames());
+		builder.setRowConverter(dialect.getInputConverter(rowType));
+
+		return InputFormatProvider.of(builder.build());
+	}
+
+	@Override
+	public ChangelogMode getChangelogMode() {
+		return ChangelogMode.insertOnly();
+	}
+
+	@Override
+	public DynamicTableSource copy() {
+		return new JdbcDynamicTableSource(options, readOptions, lookupOptions, schema, selectFields);
+	}
+
+	@Override
+	public String asSummaryString() {
+		return name;

Review comment:
       ditto.

##########
File path: flink-connectors/flink-connector-jdbc/src/main/java/org/apache/flink/connector/jdbc/table/JdbcDynamicTableSource.java
##########
@@ -0,0 +1,162 @@
+/*
+ * 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.connector.jdbc.table;
+
+import org.apache.flink.annotation.Internal;
+import org.apache.flink.api.java.typeutils.RowTypeInfo;
+import org.apache.flink.connector.jdbc.dialect.JdbcDialect;
+import org.apache.flink.connector.jdbc.internal.options.JdbcLookupOptions;
+import org.apache.flink.connector.jdbc.internal.options.JdbcOptions;
+import org.apache.flink.connector.jdbc.internal.options.JdbcReadOptions;
+import org.apache.flink.connector.jdbc.split.JdbcNumericBetweenParametersProvider;
+import org.apache.flink.table.api.TableSchema;
+import org.apache.flink.table.connector.ChangelogMode;
+import org.apache.flink.table.connector.source.DynamicTableSource;
+import org.apache.flink.table.connector.source.InputFormatProvider;
+import org.apache.flink.table.connector.source.LookupTableSource;
+import org.apache.flink.table.connector.source.ScanTableSource;
+import org.apache.flink.table.connector.source.TableFunctionProvider;
+import org.apache.flink.table.types.DataType;
+import org.apache.flink.table.types.logical.LogicalType;
+import org.apache.flink.table.types.logical.RowType;
+
+import java.util.Arrays;
+import java.util.Objects;
+
+import static org.apache.flink.table.types.utils.TypeConversions.fromDataTypeToLegacyInfo;
+
+/**
+ * A {@link DynamicTableSource} for JDBC.
+ */
+@Internal
+public class JdbcDynamicTableSource implements ScanTableSource, LookupTableSource {
+
+	private static final String name = "JdbcTableSource";
+	private final JdbcOptions options;
+	private final JdbcReadOptions readOptions;
+	private final JdbcLookupOptions lookupOptions;
+	private final TableSchema schema;
+	private final int[] selectFields;
+
+	public JdbcDynamicTableSource(
+		JdbcOptions options,
+		JdbcReadOptions readOptions,
+		JdbcLookupOptions lookupOptions,
+		TableSchema schema,
+		int[] selectFields) {
+		this.options = options;
+		this.readOptions = readOptions;
+		this.lookupOptions = lookupOptions;
+		this.schema = schema;
+		this.selectFields = selectFields;
+	}
+
+	@Override
+	public LookupRuntimeProvider getLookupRuntimeProvider(LookupTableSource.Context context) {
+		// JDBC only support non-nested look up keys

Review comment:
       Add a check that the length of inner array of `context#getKeys()` should be 1.

##########
File path: flink-connectors/flink-connector-jdbc/src/main/java/org/apache/flink/connector/jdbc/table/JdbcDynamicTableSourceSinkFactory.java
##########
@@ -0,0 +1,283 @@
+/*
+ * 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.connector.jdbc.table;
+
+import org.apache.flink.annotation.Internal;
+import org.apache.flink.configuration.ConfigOption;
+import org.apache.flink.configuration.ConfigOptions;
+import org.apache.flink.configuration.ReadableConfig;
+import org.apache.flink.connector.jdbc.JdbcExecutionOptions;
+import org.apache.flink.connector.jdbc.dialect.JdbcDialects;
+import org.apache.flink.connector.jdbc.internal.options.JdbcDmlOptions;
+import org.apache.flink.connector.jdbc.internal.options.JdbcLookupOptions;
+import org.apache.flink.connector.jdbc.internal.options.JdbcOptions;
+import org.apache.flink.connector.jdbc.internal.options.JdbcReadOptions;
+import org.apache.flink.table.api.TableSchema;
+import org.apache.flink.table.connector.sink.DynamicTableSink;
+import org.apache.flink.table.connector.source.DynamicTableSource;
+import org.apache.flink.table.factories.DynamicTableSinkFactory;
+import org.apache.flink.table.factories.DynamicTableSourceFactory;
+import org.apache.flink.table.factories.FactoryUtil;
+import org.apache.flink.table.types.DataType;
+import org.apache.flink.table.utils.TableSchemaUtils;
+
+import java.time.Duration;
+import java.util.HashSet;
+import java.util.Optional;
+import java.util.Set;
+
+/**
+ * Factory for creating configured instances of {@link JdbcDynamicTableSource}
+ * and {@link JdbcDynamicTableSink}.
+ */
+@Internal
+public class JdbcDynamicTableSourceSinkFactory implements DynamicTableSourceFactory, DynamicTableSinkFactory {
+
+	public static final ConfigOption<String> IDENTIFIER = ConfigOptions
+		.key("connector")
+		.stringType()
+		.defaultValue("jdbc")
+		.withDescription("-- required: specify this table type is jdbc.");

Review comment:
       Remove the `-- ` prefix, and upper case the first character. 

##########
File path: flink-connectors/flink-connector-jdbc/src/main/java/org/apache/flink/connector/jdbc/table/JdbcDynamicTableSink.java
##########
@@ -0,0 +1,113 @@
+/*
+ * 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.connector.jdbc.table;
+
+import org.apache.flink.annotation.Internal;
+import org.apache.flink.api.common.typeinfo.TypeInformation;
+import org.apache.flink.connector.jdbc.JdbcExecutionOptions;
+import org.apache.flink.connector.jdbc.internal.options.JdbcDmlOptions;
+import org.apache.flink.connector.jdbc.internal.options.JdbcOptions;
+import org.apache.flink.table.connector.ChangelogMode;
+import org.apache.flink.table.connector.sink.DynamicTableSink;
+import org.apache.flink.table.connector.sink.OutputFormatProvider;
+import org.apache.flink.table.data.RowData;
+import org.apache.flink.table.types.DataType;
+import org.apache.flink.types.RowKind;
+
+import java.util.Arrays;
+import java.util.Objects;
+
+import static org.apache.flink.connector.jdbc.table.JdbcDynamicOutputFormat.DynamicOutputFormatBuilder;
+
+/**
+ * A {@link DynamicTableSink} for JDBC.
+ */
+@Internal
+public class JdbcDynamicTableSink implements DynamicTableSink {
+
+	private static final String name = "JdbcTableSink";
+	private final JdbcOptions jdbcOptions;
+	private final JdbcExecutionOptions executionOptions;
+	private final JdbcDmlOptions dmlOptions;
+	private final DataType rowDataType;
+	private final DataType[] fieldTypes;

Review comment:
       You can keep the physical `TableSchema` in the sink, it's easy to get the row DataType and field DataTypes. 

##########
File path: flink-connectors/flink-connector-jdbc/src/main/java/org/apache/flink/connector/jdbc/table/JdbcDynamicTableSourceSinkFactory.java
##########
@@ -0,0 +1,283 @@
+/*
+ * 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.connector.jdbc.table;
+
+import org.apache.flink.annotation.Internal;
+import org.apache.flink.configuration.ConfigOption;
+import org.apache.flink.configuration.ConfigOptions;
+import org.apache.flink.configuration.ReadableConfig;
+import org.apache.flink.connector.jdbc.JdbcExecutionOptions;
+import org.apache.flink.connector.jdbc.dialect.JdbcDialects;
+import org.apache.flink.connector.jdbc.internal.options.JdbcDmlOptions;
+import org.apache.flink.connector.jdbc.internal.options.JdbcLookupOptions;
+import org.apache.flink.connector.jdbc.internal.options.JdbcOptions;
+import org.apache.flink.connector.jdbc.internal.options.JdbcReadOptions;
+import org.apache.flink.table.api.TableSchema;
+import org.apache.flink.table.connector.sink.DynamicTableSink;
+import org.apache.flink.table.connector.source.DynamicTableSource;
+import org.apache.flink.table.factories.DynamicTableSinkFactory;
+import org.apache.flink.table.factories.DynamicTableSourceFactory;
+import org.apache.flink.table.factories.FactoryUtil;
+import org.apache.flink.table.types.DataType;
+import org.apache.flink.table.utils.TableSchemaUtils;
+
+import java.time.Duration;
+import java.util.HashSet;
+import java.util.Optional;
+import java.util.Set;
+
+/**
+ * Factory for creating configured instances of {@link JdbcDynamicTableSource}
+ * and {@link JdbcDynamicTableSink}.
+ */
+@Internal
+public class JdbcDynamicTableSourceSinkFactory implements DynamicTableSourceFactory, DynamicTableSinkFactory {
+
+	public static final ConfigOption<String> IDENTIFIER = ConfigOptions
+		.key("connector")
+		.stringType()
+		.defaultValue("jdbc")
+		.withDescription("-- required: specify this table type is jdbc.");
+	public static final ConfigOption<String> URL = ConfigOptions
+		.key("url")
+		.stringType()
+		.noDefaultValue()
+		.withDescription("-- required: the jdbc database url.");
+	public static final ConfigOption<String> TABLE = ConfigOptions
+		.key("table")
+		.stringType()
+		.noDefaultValue()
+		.withDescription("-- required: the jdbc table name.");
+	public static final ConfigOption<String> USERNAME = ConfigOptions
+		.key("username")
+		.stringType()
+		.noDefaultValue()
+		.withDescription("-- optional: the jdbc user name.");
+	public static final ConfigOption<String> PASSWORD = ConfigOptions
+		.key("password")
+		.stringType()
+		.noDefaultValue()
+		.withDescription("-- optional: the jdbc password.");
+	private static final ConfigOption<String> DRIVER = ConfigOptions
+		.key("driver")
+		.stringType()
+		.noDefaultValue()
+		.withDescription("-- optional: the class name of the JDBC driver to use to connect to this URL. " +
+			"If not set, it will automatically be derived from the URL.");
+
+	// read config options
+	private static final ConfigOption<String> READ_PARTITION_COLUMN = ConfigOptions
+		.key("read.partition.column")
+		.stringType()
+		.noDefaultValue()
+		.withDescription("-- optional: the column name used for partitioning the input.");
+	private static final ConfigOption<Integer> READ_PARTITION_NUM = ConfigOptions
+		.key("read.partition.num")
+		.intType()
+		.noDefaultValue()
+		.withDescription("-- optional: the number of partitions.");
+	private static final ConfigOption<Long> READ_PARTITION_LOWER_BOUND = ConfigOptions
+		.key("read.partition.lower-bound")
+		.longType()
+		.noDefaultValue()
+		.withDescription("-- optional: the smallest value of the first partition.");
+	private static final ConfigOption<Long> READ_PARTITION_UPPER_BOUND = ConfigOptions
+		.key("read.partition.upper-bound")
+		.longType()
+		.noDefaultValue()
+		.withDescription("-- optional: the largest value of the last partition.");
+	private static final ConfigOption<Integer> READ_FETCH_SIZE = ConfigOptions
+		.key("read.fetch-size")
+		.intType()
+		.defaultValue(0)
+		.withDescription("-- optional, Gives the reader a hint as to the number of rows that should be fetched, from" +
+			" the database when reading per round trip. If the value specified is zero, then the hint is ignored. The" +
+			" default value is zero.");
+
+	// look up config options
+	private static final ConfigOption<Long> LOOKUP_CACHE_MAX_ROWS = ConfigOptions
+		.key("lookup.cache.max-rows")
+		.longType()
+		.defaultValue(-1L)
+		.withDescription("-- optional, max number of rows of lookup cache, over this value, the oldest rows will " +
+			"be eliminated. \"cache.max-rows\" and \"cache.ttl\" options must all be specified if any of them is " +
+			"specified. Cache is not enabled as default.");
+	private static final ConfigOption<Duration> LOOKUP_CACHE_TTL = ConfigOptions
+		.key("lookup.cache.ttl")
+		.durationType()
+		.defaultValue(Duration.ofSeconds(10))
+		.withDescription("-- optional, the cache time to live.");
+	private static final ConfigOption<Integer> LOOKUP_MAX_RETRIES = ConfigOptions
+		.key("lookup.max-retries")
+		.intType()
+		.defaultValue(3)
+		.withDescription("-- optional, max retry times if lookup database failed.");
+
+	// write config options
+	private static final ConfigOption<Integer> WRITE_FLUSH_MAX_ROWS = ConfigOptions
+		.key("write.flush.max-rows")
+		.intType()
+		.defaultValue(5000)
+		.withDescription("-- optional, flush max size (includes all append, upsert and delete records), over this number" +
+			" of records, will flush data. The default value is 5000.");
+	private static final ConfigOption<Long> WRITE_FLUSH_INTERVAL = ConfigOptions
+		.key("write.flush.interval")
+		.longType()
+		.defaultValue(0L)
+		.withDescription("-- optional, flush interval mills, over this time, asynchronous threads will flush data. The " +
+			"default value is 0, which means no asynchronous flush thread will be scheduled.");
+	private static final ConfigOption<Integer> WRITE_MAX_RETRIES = ConfigOptions
+		.key("write.max-retries")
+		.intType()
+		.defaultValue(3)
+		.withDescription("-- optional, max retry times if writing records to database failed.");
+
+	@Override
+	public DynamicTableSink createDynamicTableSink(Context context) {
+		final FactoryUtil.TableFactoryHelper helper = FactoryUtil.createTableFactoryHelper(this, context);
+		helper.validate();
+		final JdbcOptions jdbcOptions = getJdbcOptions(helper.getOptions());
+		final DataType rowDataType = context.getCatalogTable().getSchema().toPhysicalRowDataType();
+		final TableSchema formatSchema = TableSchemaUtils.getPhysicalSchema(context.getCatalogTable().getSchema());
+		final DataType[] fieldDataTypes = formatSchema.getFieldDataTypes();
+
+		return new JdbcDynamicTableSink(
+			jdbcOptions,
+			getJdbcExecutionOptions(helper.getOptions()),
+			getJdbcDmlOptions(jdbcOptions, context.getCatalogTable().getSchema()),
+			rowDataType,
+			fieldDataTypes);
+	}
+
+	@Override
+	public DynamicTableSource createDynamicTableSource(Context context) {
+		final FactoryUtil.TableFactoryHelper helper = FactoryUtil.createTableFactoryHelper(this, context);
+		helper.validate();
+		TableSchema formatSchema = TableSchemaUtils.getPhysicalSchema(context.getCatalogTable().getSchema());
+
+		int[] selectFields = new int[formatSchema.getFieldNames().length];
+		for (int i = 0; i < selectFields.length; i++) {
+			selectFields[i] = i;
+		}
+		return new JdbcDynamicTableSource(
+			getJdbcOptions(helper.getOptions()),
+			getJdbcReadOptions(helper.getOptions()),
+			getJdbcLookupOptions(helper.getOptions()),
+			formatSchema,
+			selectFields);
+	}
+
+	private JdbcOptions getJdbcOptions(ReadableConfig readableConfig) {
+		final String url = readableConfig.get(URL);
+		final JdbcOptions.Builder builder = JdbcOptions.builder()
+			.setDBUrl(url)
+			.setTableName(readableConfig.get(TABLE))
+			.setDialect(JdbcDialects.get(url).get());
+
+		readableConfig.getOptional(DRIVER).ifPresent(builder::setDriverName);
+		readableConfig.getOptional(USERNAME).ifPresent(builder::setUsername);
+		readableConfig.getOptional(PASSWORD).ifPresent(builder::setPassword);
+		return builder.build();
+	}
+
+	private JdbcReadOptions getJdbcReadOptions(ReadableConfig readableConfig) {
+		final Optional<String> partitionColumnName = readableConfig.getOptional(READ_PARTITION_COLUMN);
+		final Optional<Long> partitionLower = readableConfig.getOptional(READ_PARTITION_LOWER_BOUND);
+		final Optional<Long> partitionUpper = readableConfig.getOptional(READ_PARTITION_UPPER_BOUND);
+		final Optional<Integer> numPartitions = readableConfig.getOptional(READ_PARTITION_NUM);
+
+		final JdbcReadOptions.Builder builder = JdbcReadOptions.builder();
+		if (partitionColumnName.isPresent()) {

Review comment:
       Add a validation that other 3 options shouldn't be empty.

##########
File path: flink-connectors/flink-connector-jdbc/src/main/java/org/apache/flink/connector/jdbc/table/JdbcDynamicTableSource.java
##########
@@ -0,0 +1,162 @@
+/*
+ * 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.connector.jdbc.table;
+
+import org.apache.flink.annotation.Internal;
+import org.apache.flink.api.java.typeutils.RowTypeInfo;
+import org.apache.flink.connector.jdbc.dialect.JdbcDialect;
+import org.apache.flink.connector.jdbc.internal.options.JdbcLookupOptions;
+import org.apache.flink.connector.jdbc.internal.options.JdbcOptions;
+import org.apache.flink.connector.jdbc.internal.options.JdbcReadOptions;
+import org.apache.flink.connector.jdbc.split.JdbcNumericBetweenParametersProvider;
+import org.apache.flink.table.api.TableSchema;
+import org.apache.flink.table.connector.ChangelogMode;
+import org.apache.flink.table.connector.source.DynamicTableSource;
+import org.apache.flink.table.connector.source.InputFormatProvider;
+import org.apache.flink.table.connector.source.LookupTableSource;
+import org.apache.flink.table.connector.source.ScanTableSource;
+import org.apache.flink.table.connector.source.TableFunctionProvider;
+import org.apache.flink.table.types.DataType;
+import org.apache.flink.table.types.logical.LogicalType;
+import org.apache.flink.table.types.logical.RowType;
+
+import java.util.Arrays;
+import java.util.Objects;
+
+import static org.apache.flink.table.types.utils.TypeConversions.fromDataTypeToLegacyInfo;
+
+/**
+ * A {@link DynamicTableSource} for JDBC.
+ */
+@Internal
+public class JdbcDynamicTableSource implements ScanTableSource, LookupTableSource {
+
+	private static final String name = "JdbcTableSource";
+	private final JdbcOptions options;
+	private final JdbcReadOptions readOptions;
+	private final JdbcLookupOptions lookupOptions;
+	private final TableSchema schema;
+	private final int[] selectFields;
+
+	public JdbcDynamicTableSource(
+		JdbcOptions options,
+		JdbcReadOptions readOptions,
+		JdbcLookupOptions lookupOptions,
+		TableSchema schema,
+		int[] selectFields) {
+		this.options = options;
+		this.readOptions = readOptions;
+		this.lookupOptions = lookupOptions;
+		this.schema = schema;
+		this.selectFields = selectFields;
+	}
+
+	@Override
+	public LookupRuntimeProvider getLookupRuntimeProvider(LookupTableSource.Context context) {
+		// JDBC only support non-nested look up keys
+		String[] keyNames = new String[context.getKeys().length];
+		for (int i = 0; i < keyNames.length; i++) {
+			int index = context.getKeys()[i][0];
+			keyNames[i] = schema.getFieldNames()[index];
+		}
+		return TableFunctionProvider.of(JdbcDynamicLookupFunction.builder()
+			.setFieldNames(schema.getFieldNames())
+			.setFieldTypes(schema.getFieldDataTypes())
+			.setKeyNames(keyNames)
+			.setOptions(options)
+			.setLookupOptions(lookupOptions)
+			.build());
+	}
+
+	@Override
+	public ScanRuntimeProvider getScanRuntimeProvider(ScanTableSource.Context runtimeProviderContext) {
+		final DataType rowDataType = schema.toPhysicalRowDataType();
+		final RowTypeInfo rowTypeInfo = (RowTypeInfo) fromDataTypeToLegacyInfo(rowDataType);
+		final JdbcDynamicInputFormat.Builder builder = JdbcDynamicInputFormat.builder()
+			.setDrivername(options.getDriverName())
+			.setDBUrl(options.getDbURL())
+			.setUsername(options.getUsername().orElse(null))
+			.setPassword(options.getPassword().orElse(null))
+			.setRowTypeInfo(new RowTypeInfo(rowTypeInfo.getFieldTypes(), rowTypeInfo.getFieldNames()));
+
+		if (readOptions.getFetchSize() != 0) {
+			builder.setFetchSize(readOptions.getFetchSize());
+		}
+		final JdbcDialect dialect = options.getDialect();
+		String query = dialect.getSelectFromStatement(
+			options.getTableName(), rowTypeInfo.getFieldNames(), new String[0]);
+		if (readOptions.getPartitionColumnName().isPresent()) {
+			long lowerBound = readOptions.getPartitionLowerBound().get();
+			long upperBound = readOptions.getPartitionUpperBound().get();
+			int numPartitions = readOptions.getNumPartitions().get();
+			builder.setParametersProvider(
+				new JdbcNumericBetweenParametersProvider(lowerBound, upperBound).ofBatchNum(numPartitions));
+			query += " WHERE " +
+				dialect.quoteIdentifier(readOptions.getPartitionColumnName().get()) +
+				" BETWEEN ? AND ?";
+		}
+		builder.setQuery(query);
+		final RowType rowType = RowType.of(
+			Arrays.stream(schema.getFieldDataTypes())
+				.map(DataType::getLogicalType)
+				.toArray(LogicalType[]::new),
+			schema.getFieldNames());

Review comment:
       final RowType rowType = (RowType) schema.toRowDataType().getLogicalType();

##########
File path: flink-connectors/flink-connector-jdbc/src/main/java/org/apache/flink/connector/jdbc/table/JdbcDynamicTableSourceSinkFactory.java
##########
@@ -0,0 +1,283 @@
+/*
+ * 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.connector.jdbc.table;
+
+import org.apache.flink.annotation.Internal;
+import org.apache.flink.configuration.ConfigOption;
+import org.apache.flink.configuration.ConfigOptions;
+import org.apache.flink.configuration.ReadableConfig;
+import org.apache.flink.connector.jdbc.JdbcExecutionOptions;
+import org.apache.flink.connector.jdbc.dialect.JdbcDialects;
+import org.apache.flink.connector.jdbc.internal.options.JdbcDmlOptions;
+import org.apache.flink.connector.jdbc.internal.options.JdbcLookupOptions;
+import org.apache.flink.connector.jdbc.internal.options.JdbcOptions;
+import org.apache.flink.connector.jdbc.internal.options.JdbcReadOptions;
+import org.apache.flink.table.api.TableSchema;
+import org.apache.flink.table.connector.sink.DynamicTableSink;
+import org.apache.flink.table.connector.source.DynamicTableSource;
+import org.apache.flink.table.factories.DynamicTableSinkFactory;
+import org.apache.flink.table.factories.DynamicTableSourceFactory;
+import org.apache.flink.table.factories.FactoryUtil;
+import org.apache.flink.table.types.DataType;
+import org.apache.flink.table.utils.TableSchemaUtils;
+
+import java.time.Duration;
+import java.util.HashSet;
+import java.util.Optional;
+import java.util.Set;
+
+/**
+ * Factory for creating configured instances of {@link JdbcDynamicTableSource}
+ * and {@link JdbcDynamicTableSink}.
+ */
+@Internal
+public class JdbcDynamicTableSourceSinkFactory implements DynamicTableSourceFactory, DynamicTableSinkFactory {
+
+	public static final ConfigOption<String> IDENTIFIER = ConfigOptions
+		.key("connector")
+		.stringType()
+		.defaultValue("jdbc")
+		.withDescription("-- required: specify this table type is jdbc.");
+	public static final ConfigOption<String> URL = ConfigOptions
+		.key("url")
+		.stringType()
+		.noDefaultValue()
+		.withDescription("-- required: the jdbc database url.");
+	public static final ConfigOption<String> TABLE = ConfigOptions
+		.key("table")
+		.stringType()
+		.noDefaultValue()
+		.withDescription("-- required: the jdbc table name.");
+	public static final ConfigOption<String> USERNAME = ConfigOptions
+		.key("username")
+		.stringType()
+		.noDefaultValue()
+		.withDescription("-- optional: the jdbc user name.");
+	public static final ConfigOption<String> PASSWORD = ConfigOptions
+		.key("password")
+		.stringType()
+		.noDefaultValue()
+		.withDescription("-- optional: the jdbc password.");
+	private static final ConfigOption<String> DRIVER = ConfigOptions
+		.key("driver")
+		.stringType()
+		.noDefaultValue()
+		.withDescription("-- optional: the class name of the JDBC driver to use to connect to this URL. " +
+			"If not set, it will automatically be derived from the URL.");
+
+	// read config options
+	private static final ConfigOption<String> READ_PARTITION_COLUMN = ConfigOptions
+		.key("read.partition.column")
+		.stringType()
+		.noDefaultValue()
+		.withDescription("-- optional: the column name used for partitioning the input.");
+	private static final ConfigOption<Integer> READ_PARTITION_NUM = ConfigOptions
+		.key("read.partition.num")
+		.intType()
+		.noDefaultValue()
+		.withDescription("-- optional: the number of partitions.");
+	private static final ConfigOption<Long> READ_PARTITION_LOWER_BOUND = ConfigOptions
+		.key("read.partition.lower-bound")
+		.longType()
+		.noDefaultValue()
+		.withDescription("-- optional: the smallest value of the first partition.");
+	private static final ConfigOption<Long> READ_PARTITION_UPPER_BOUND = ConfigOptions
+		.key("read.partition.upper-bound")
+		.longType()
+		.noDefaultValue()
+		.withDescription("-- optional: the largest value of the last partition.");
+	private static final ConfigOption<Integer> READ_FETCH_SIZE = ConfigOptions
+		.key("read.fetch-size")
+		.intType()
+		.defaultValue(0)
+		.withDescription("-- optional, Gives the reader a hint as to the number of rows that should be fetched, from" +
+			" the database when reading per round trip. If the value specified is zero, then the hint is ignored. The" +
+			" default value is zero.");
+
+	// look up config options
+	private static final ConfigOption<Long> LOOKUP_CACHE_MAX_ROWS = ConfigOptions
+		.key("lookup.cache.max-rows")
+		.longType()
+		.defaultValue(-1L)
+		.withDescription("-- optional, max number of rows of lookup cache, over this value, the oldest rows will " +
+			"be eliminated. \"cache.max-rows\" and \"cache.ttl\" options must all be specified if any of them is " +
+			"specified. Cache is not enabled as default.");
+	private static final ConfigOption<Duration> LOOKUP_CACHE_TTL = ConfigOptions
+		.key("lookup.cache.ttl")
+		.durationType()
+		.defaultValue(Duration.ofSeconds(10))
+		.withDescription("-- optional, the cache time to live.");
+	private static final ConfigOption<Integer> LOOKUP_MAX_RETRIES = ConfigOptions
+		.key("lookup.max-retries")
+		.intType()
+		.defaultValue(3)
+		.withDescription("-- optional, max retry times if lookup database failed.");
+
+	// write config options
+	private static final ConfigOption<Integer> WRITE_FLUSH_MAX_ROWS = ConfigOptions
+		.key("write.flush.max-rows")
+		.intType()
+		.defaultValue(5000)
+		.withDescription("-- optional, flush max size (includes all append, upsert and delete records), over this number" +
+			" of records, will flush data. The default value is 5000.");
+	private static final ConfigOption<Long> WRITE_FLUSH_INTERVAL = ConfigOptions
+		.key("write.flush.interval")
+		.longType()
+		.defaultValue(0L)
+		.withDescription("-- optional, flush interval mills, over this time, asynchronous threads will flush data. The " +
+			"default value is 0, which means no asynchronous flush thread will be scheduled.");
+	private static final ConfigOption<Integer> WRITE_MAX_RETRIES = ConfigOptions
+		.key("write.max-retries")
+		.intType()
+		.defaultValue(3)
+		.withDescription("-- optional, max retry times if writing records to database failed.");
+
+	@Override
+	public DynamicTableSink createDynamicTableSink(Context context) {
+		final FactoryUtil.TableFactoryHelper helper = FactoryUtil.createTableFactoryHelper(this, context);
+		helper.validate();
+		final JdbcOptions jdbcOptions = getJdbcOptions(helper.getOptions());
+		final DataType rowDataType = context.getCatalogTable().getSchema().toPhysicalRowDataType();
+		final TableSchema formatSchema = TableSchemaUtils.getPhysicalSchema(context.getCatalogTable().getSchema());
+		final DataType[] fieldDataTypes = formatSchema.getFieldDataTypes();

Review comment:
       We don't need to declare every local field as `final`.

##########
File path: flink-connectors/flink-connector-jdbc/src/main/java/org/apache/flink/connector/jdbc/table/JdbcDynamicTableSourceSinkFactory.java
##########
@@ -0,0 +1,283 @@
+/*
+ * 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.connector.jdbc.table;
+
+import org.apache.flink.annotation.Internal;
+import org.apache.flink.configuration.ConfigOption;
+import org.apache.flink.configuration.ConfigOptions;
+import org.apache.flink.configuration.ReadableConfig;
+import org.apache.flink.connector.jdbc.JdbcExecutionOptions;
+import org.apache.flink.connector.jdbc.dialect.JdbcDialects;
+import org.apache.flink.connector.jdbc.internal.options.JdbcDmlOptions;
+import org.apache.flink.connector.jdbc.internal.options.JdbcLookupOptions;
+import org.apache.flink.connector.jdbc.internal.options.JdbcOptions;
+import org.apache.flink.connector.jdbc.internal.options.JdbcReadOptions;
+import org.apache.flink.table.api.TableSchema;
+import org.apache.flink.table.connector.sink.DynamicTableSink;
+import org.apache.flink.table.connector.source.DynamicTableSource;
+import org.apache.flink.table.factories.DynamicTableSinkFactory;
+import org.apache.flink.table.factories.DynamicTableSourceFactory;
+import org.apache.flink.table.factories.FactoryUtil;
+import org.apache.flink.table.types.DataType;
+import org.apache.flink.table.utils.TableSchemaUtils;
+
+import java.time.Duration;
+import java.util.HashSet;
+import java.util.Optional;
+import java.util.Set;
+
+/**
+ * Factory for creating configured instances of {@link JdbcDynamicTableSource}
+ * and {@link JdbcDynamicTableSink}.
+ */
+@Internal
+public class JdbcDynamicTableSourceSinkFactory implements DynamicTableSourceFactory, DynamicTableSinkFactory {
+
+	public static final ConfigOption<String> IDENTIFIER = ConfigOptions
+		.key("connector")
+		.stringType()
+		.defaultValue("jdbc")
+		.withDescription("-- required: specify this table type is jdbc.");
+	public static final ConfigOption<String> URL = ConfigOptions
+		.key("url")
+		.stringType()
+		.noDefaultValue()
+		.withDescription("-- required: the jdbc database url.");
+	public static final ConfigOption<String> TABLE = ConfigOptions
+		.key("table")
+		.stringType()
+		.noDefaultValue()
+		.withDescription("-- required: the jdbc table name.");
+	public static final ConfigOption<String> USERNAME = ConfigOptions
+		.key("username")
+		.stringType()
+		.noDefaultValue()
+		.withDescription("-- optional: the jdbc user name.");
+	public static final ConfigOption<String> PASSWORD = ConfigOptions
+		.key("password")
+		.stringType()
+		.noDefaultValue()
+		.withDescription("-- optional: the jdbc password.");
+	private static final ConfigOption<String> DRIVER = ConfigOptions
+		.key("driver")
+		.stringType()
+		.noDefaultValue()
+		.withDescription("-- optional: the class name of the JDBC driver to use to connect to this URL. " +
+			"If not set, it will automatically be derived from the URL.");
+
+	// read config options
+	private static final ConfigOption<String> READ_PARTITION_COLUMN = ConfigOptions
+		.key("read.partition.column")
+		.stringType()
+		.noDefaultValue()
+		.withDescription("-- optional: the column name used for partitioning the input.");
+	private static final ConfigOption<Integer> READ_PARTITION_NUM = ConfigOptions
+		.key("read.partition.num")
+		.intType()
+		.noDefaultValue()
+		.withDescription("-- optional: the number of partitions.");
+	private static final ConfigOption<Long> READ_PARTITION_LOWER_BOUND = ConfigOptions
+		.key("read.partition.lower-bound")
+		.longType()
+		.noDefaultValue()
+		.withDescription("-- optional: the smallest value of the first partition.");
+	private static final ConfigOption<Long> READ_PARTITION_UPPER_BOUND = ConfigOptions
+		.key("read.partition.upper-bound")
+		.longType()
+		.noDefaultValue()
+		.withDescription("-- optional: the largest value of the last partition.");
+	private static final ConfigOption<Integer> READ_FETCH_SIZE = ConfigOptions
+		.key("read.fetch-size")
+		.intType()
+		.defaultValue(0)
+		.withDescription("-- optional, Gives the reader a hint as to the number of rows that should be fetched, from" +
+			" the database when reading per round trip. If the value specified is zero, then the hint is ignored. The" +
+			" default value is zero.");
+
+	// look up config options
+	private static final ConfigOption<Long> LOOKUP_CACHE_MAX_ROWS = ConfigOptions
+		.key("lookup.cache.max-rows")
+		.longType()
+		.defaultValue(-1L)
+		.withDescription("-- optional, max number of rows of lookup cache, over this value, the oldest rows will " +
+			"be eliminated. \"cache.max-rows\" and \"cache.ttl\" options must all be specified if any of them is " +
+			"specified. Cache is not enabled as default.");
+	private static final ConfigOption<Duration> LOOKUP_CACHE_TTL = ConfigOptions
+		.key("lookup.cache.ttl")
+		.durationType()
+		.defaultValue(Duration.ofSeconds(10))
+		.withDescription("-- optional, the cache time to live.");
+	private static final ConfigOption<Integer> LOOKUP_MAX_RETRIES = ConfigOptions
+		.key("lookup.max-retries")
+		.intType()
+		.defaultValue(3)
+		.withDescription("-- optional, max retry times if lookup database failed.");
+
+	// write config options
+	private static final ConfigOption<Integer> WRITE_FLUSH_MAX_ROWS = ConfigOptions
+		.key("write.flush.max-rows")
+		.intType()
+		.defaultValue(5000)
+		.withDescription("-- optional, flush max size (includes all append, upsert and delete records), over this number" +
+			" of records, will flush data. The default value is 5000.");
+	private static final ConfigOption<Long> WRITE_FLUSH_INTERVAL = ConfigOptions
+		.key("write.flush.interval")
+		.longType()
+		.defaultValue(0L)
+		.withDescription("-- optional, flush interval mills, over this time, asynchronous threads will flush data. The " +
+			"default value is 0, which means no asynchronous flush thread will be scheduled.");
+	private static final ConfigOption<Integer> WRITE_MAX_RETRIES = ConfigOptions
+		.key("write.max-retries")
+		.intType()
+		.defaultValue(3)
+		.withDescription("-- optional, max retry times if writing records to database failed.");
+
+	@Override
+	public DynamicTableSink createDynamicTableSink(Context context) {
+		final FactoryUtil.TableFactoryHelper helper = FactoryUtil.createTableFactoryHelper(this, context);
+		helper.validate();
+		final JdbcOptions jdbcOptions = getJdbcOptions(helper.getOptions());
+		final DataType rowDataType = context.getCatalogTable().getSchema().toPhysicalRowDataType();
+		final TableSchema formatSchema = TableSchemaUtils.getPhysicalSchema(context.getCatalogTable().getSchema());
+		final DataType[] fieldDataTypes = formatSchema.getFieldDataTypes();
+
+		return new JdbcDynamicTableSink(
+			jdbcOptions,
+			getJdbcExecutionOptions(helper.getOptions()),
+			getJdbcDmlOptions(jdbcOptions, context.getCatalogTable().getSchema()),
+			rowDataType,
+			fieldDataTypes);
+	}
+
+	@Override
+	public DynamicTableSource createDynamicTableSource(Context context) {
+		final FactoryUtil.TableFactoryHelper helper = FactoryUtil.createTableFactoryHelper(this, context);
+		helper.validate();
+		TableSchema formatSchema = TableSchemaUtils.getPhysicalSchema(context.getCatalogTable().getSchema());
+
+		int[] selectFields = new int[formatSchema.getFieldNames().length];
+		for (int i = 0; i < selectFields.length; i++) {
+			selectFields[i] = i;
+		}
+		return new JdbcDynamicTableSource(
+			getJdbcOptions(helper.getOptions()),
+			getJdbcReadOptions(helper.getOptions()),
+			getJdbcLookupOptions(helper.getOptions()),
+			formatSchema,
+			selectFields);
+	}
+
+	private JdbcOptions getJdbcOptions(ReadableConfig readableConfig) {
+		final String url = readableConfig.get(URL);
+		final JdbcOptions.Builder builder = JdbcOptions.builder()
+			.setDBUrl(url)
+			.setTableName(readableConfig.get(TABLE))
+			.setDialect(JdbcDialects.get(url).get());
+
+		readableConfig.getOptional(DRIVER).ifPresent(builder::setDriverName);
+		readableConfig.getOptional(USERNAME).ifPresent(builder::setUsername);
+		readableConfig.getOptional(PASSWORD).ifPresent(builder::setPassword);
+		return builder.build();
+	}
+
+	private JdbcReadOptions getJdbcReadOptions(ReadableConfig readableConfig) {
+		final Optional<String> partitionColumnName = readableConfig.getOptional(READ_PARTITION_COLUMN);
+		final Optional<Long> partitionLower = readableConfig.getOptional(READ_PARTITION_LOWER_BOUND);
+		final Optional<Long> partitionUpper = readableConfig.getOptional(READ_PARTITION_UPPER_BOUND);
+		final Optional<Integer> numPartitions = readableConfig.getOptional(READ_PARTITION_NUM);

Review comment:
       You can put these 3 in the `partitionColumnName.isPresent()` branch block, and use `config.get(..)` to avoid the warning. 

##########
File path: flink-connectors/flink-connector-jdbc/src/main/java/org/apache/flink/connector/jdbc/table/JdbcDynamicTableSourceSinkFactory.java
##########
@@ -0,0 +1,283 @@
+/*
+ * 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.connector.jdbc.table;
+
+import org.apache.flink.annotation.Internal;
+import org.apache.flink.configuration.ConfigOption;
+import org.apache.flink.configuration.ConfigOptions;
+import org.apache.flink.configuration.ReadableConfig;
+import org.apache.flink.connector.jdbc.JdbcExecutionOptions;
+import org.apache.flink.connector.jdbc.dialect.JdbcDialects;
+import org.apache.flink.connector.jdbc.internal.options.JdbcDmlOptions;
+import org.apache.flink.connector.jdbc.internal.options.JdbcLookupOptions;
+import org.apache.flink.connector.jdbc.internal.options.JdbcOptions;
+import org.apache.flink.connector.jdbc.internal.options.JdbcReadOptions;
+import org.apache.flink.table.api.TableSchema;
+import org.apache.flink.table.connector.sink.DynamicTableSink;
+import org.apache.flink.table.connector.source.DynamicTableSource;
+import org.apache.flink.table.factories.DynamicTableSinkFactory;
+import org.apache.flink.table.factories.DynamicTableSourceFactory;
+import org.apache.flink.table.factories.FactoryUtil;
+import org.apache.flink.table.types.DataType;
+import org.apache.flink.table.utils.TableSchemaUtils;
+
+import java.time.Duration;
+import java.util.HashSet;
+import java.util.Optional;
+import java.util.Set;
+
+/**
+ * Factory for creating configured instances of {@link JdbcDynamicTableSource}
+ * and {@link JdbcDynamicTableSink}.
+ */
+@Internal
+public class JdbcDynamicTableSourceSinkFactory implements DynamicTableSourceFactory, DynamicTableSinkFactory {
+
+	public static final ConfigOption<String> IDENTIFIER = ConfigOptions

Review comment:
       `IDENTIFIER` can be just a simple string.

##########
File path: flink-connectors/flink-connector-jdbc/src/main/java/org/apache/flink/connector/jdbc/table/JdbcDynamicTableSourceSinkFactory.java
##########
@@ -0,0 +1,283 @@
+/*
+ * 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.connector.jdbc.table;
+
+import org.apache.flink.annotation.Internal;
+import org.apache.flink.configuration.ConfigOption;
+import org.apache.flink.configuration.ConfigOptions;
+import org.apache.flink.configuration.ReadableConfig;
+import org.apache.flink.connector.jdbc.JdbcExecutionOptions;
+import org.apache.flink.connector.jdbc.dialect.JdbcDialects;
+import org.apache.flink.connector.jdbc.internal.options.JdbcDmlOptions;
+import org.apache.flink.connector.jdbc.internal.options.JdbcLookupOptions;
+import org.apache.flink.connector.jdbc.internal.options.JdbcOptions;
+import org.apache.flink.connector.jdbc.internal.options.JdbcReadOptions;
+import org.apache.flink.table.api.TableSchema;
+import org.apache.flink.table.connector.sink.DynamicTableSink;
+import org.apache.flink.table.connector.source.DynamicTableSource;
+import org.apache.flink.table.factories.DynamicTableSinkFactory;
+import org.apache.flink.table.factories.DynamicTableSourceFactory;
+import org.apache.flink.table.factories.FactoryUtil;
+import org.apache.flink.table.types.DataType;
+import org.apache.flink.table.utils.TableSchemaUtils;
+
+import java.time.Duration;
+import java.util.HashSet;
+import java.util.Optional;
+import java.util.Set;
+
+/**
+ * Factory for creating configured instances of {@link JdbcDynamicTableSource}
+ * and {@link JdbcDynamicTableSink}.
+ */
+@Internal
+public class JdbcDynamicTableSourceSinkFactory implements DynamicTableSourceFactory, DynamicTableSinkFactory {

Review comment:
       Please add a dedicate validation logic to validate options, e.g. dialect, primary key.
   username and password should set together, partition options should be set together, lowerBound should < upperBound.

##########
File path: flink-connectors/flink-connector-jdbc/src/main/java/org/apache/flink/connector/jdbc/table/JdbcDynamicTableSourceSinkFactory.java
##########
@@ -0,0 +1,283 @@
+/*
+ * 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.connector.jdbc.table;
+
+import org.apache.flink.annotation.Internal;
+import org.apache.flink.configuration.ConfigOption;
+import org.apache.flink.configuration.ConfigOptions;
+import org.apache.flink.configuration.ReadableConfig;
+import org.apache.flink.connector.jdbc.JdbcExecutionOptions;
+import org.apache.flink.connector.jdbc.dialect.JdbcDialects;
+import org.apache.flink.connector.jdbc.internal.options.JdbcDmlOptions;
+import org.apache.flink.connector.jdbc.internal.options.JdbcLookupOptions;
+import org.apache.flink.connector.jdbc.internal.options.JdbcOptions;
+import org.apache.flink.connector.jdbc.internal.options.JdbcReadOptions;
+import org.apache.flink.table.api.TableSchema;
+import org.apache.flink.table.connector.sink.DynamicTableSink;
+import org.apache.flink.table.connector.source.DynamicTableSource;
+import org.apache.flink.table.factories.DynamicTableSinkFactory;
+import org.apache.flink.table.factories.DynamicTableSourceFactory;
+import org.apache.flink.table.factories.FactoryUtil;
+import org.apache.flink.table.types.DataType;
+import org.apache.flink.table.utils.TableSchemaUtils;
+
+import java.time.Duration;
+import java.util.HashSet;
+import java.util.Optional;
+import java.util.Set;
+
+/**
+ * Factory for creating configured instances of {@link JdbcDynamicTableSource}
+ * and {@link JdbcDynamicTableSink}.
+ */
+@Internal
+public class JdbcDynamicTableSourceSinkFactory implements DynamicTableSourceFactory, DynamicTableSinkFactory {
+
+	public static final ConfigOption<String> IDENTIFIER = ConfigOptions
+		.key("connector")
+		.stringType()
+		.defaultValue("jdbc")
+		.withDescription("-- required: specify this table type is jdbc.");
+	public static final ConfigOption<String> URL = ConfigOptions
+		.key("url")
+		.stringType()
+		.noDefaultValue()
+		.withDescription("-- required: the jdbc database url.");
+	public static final ConfigOption<String> TABLE = ConfigOptions
+		.key("table")
+		.stringType()
+		.noDefaultValue()
+		.withDescription("-- required: the jdbc table name.");
+	public static final ConfigOption<String> USERNAME = ConfigOptions
+		.key("username")
+		.stringType()
+		.noDefaultValue()
+		.withDescription("-- optional: the jdbc user name.");
+	public static final ConfigOption<String> PASSWORD = ConfigOptions
+		.key("password")
+		.stringType()
+		.noDefaultValue()
+		.withDescription("-- optional: the jdbc password.");
+	private static final ConfigOption<String> DRIVER = ConfigOptions
+		.key("driver")
+		.stringType()
+		.noDefaultValue()
+		.withDescription("-- optional: the class name of the JDBC driver to use to connect to this URL. " +
+			"If not set, it will automatically be derived from the URL.");
+
+	// read config options
+	private static final ConfigOption<String> READ_PARTITION_COLUMN = ConfigOptions
+		.key("read.partition.column")
+		.stringType()
+		.noDefaultValue()
+		.withDescription("-- optional: the column name used for partitioning the input.");
+	private static final ConfigOption<Integer> READ_PARTITION_NUM = ConfigOptions
+		.key("read.partition.num")
+		.intType()
+		.noDefaultValue()
+		.withDescription("-- optional: the number of partitions.");
+	private static final ConfigOption<Long> READ_PARTITION_LOWER_BOUND = ConfigOptions
+		.key("read.partition.lower-bound")
+		.longType()
+		.noDefaultValue()
+		.withDescription("-- optional: the smallest value of the first partition.");
+	private static final ConfigOption<Long> READ_PARTITION_UPPER_BOUND = ConfigOptions
+		.key("read.partition.upper-bound")
+		.longType()
+		.noDefaultValue()
+		.withDescription("-- optional: the largest value of the last partition.");
+	private static final ConfigOption<Integer> READ_FETCH_SIZE = ConfigOptions
+		.key("read.fetch-size")
+		.intType()
+		.defaultValue(0)
+		.withDescription("-- optional, Gives the reader a hint as to the number of rows that should be fetched, from" +
+			" the database when reading per round trip. If the value specified is zero, then the hint is ignored. The" +
+			" default value is zero.");
+
+	// look up config options
+	private static final ConfigOption<Long> LOOKUP_CACHE_MAX_ROWS = ConfigOptions
+		.key("lookup.cache.max-rows")
+		.longType()
+		.defaultValue(-1L)
+		.withDescription("-- optional, max number of rows of lookup cache, over this value, the oldest rows will " +
+			"be eliminated. \"cache.max-rows\" and \"cache.ttl\" options must all be specified if any of them is " +
+			"specified. Cache is not enabled as default.");
+	private static final ConfigOption<Duration> LOOKUP_CACHE_TTL = ConfigOptions
+		.key("lookup.cache.ttl")
+		.durationType()
+		.defaultValue(Duration.ofSeconds(10))
+		.withDescription("-- optional, the cache time to live.");
+	private static final ConfigOption<Integer> LOOKUP_MAX_RETRIES = ConfigOptions
+		.key("lookup.max-retries")
+		.intType()
+		.defaultValue(3)
+		.withDescription("-- optional, max retry times if lookup database failed.");
+
+	// write config options
+	private static final ConfigOption<Integer> WRITE_FLUSH_MAX_ROWS = ConfigOptions
+		.key("write.flush.max-rows")
+		.intType()
+		.defaultValue(5000)
+		.withDescription("-- optional, flush max size (includes all append, upsert and delete records), over this number" +
+			" of records, will flush data. The default value is 5000.");
+	private static final ConfigOption<Long> WRITE_FLUSH_INTERVAL = ConfigOptions
+		.key("write.flush.interval")
+		.longType()
+		.defaultValue(0L)
+		.withDescription("-- optional, flush interval mills, over this time, asynchronous threads will flush data. The " +
+			"default value is 0, which means no asynchronous flush thread will be scheduled.");
+	private static final ConfigOption<Integer> WRITE_MAX_RETRIES = ConfigOptions
+		.key("write.max-retries")
+		.intType()
+		.defaultValue(3)
+		.withDescription("-- optional, max retry times if writing records to database failed.");
+
+	@Override
+	public DynamicTableSink createDynamicTableSink(Context context) {

Review comment:
       Add a validation that JDBC sink table should declare a primary key.

##########
File path: flink-connectors/flink-connector-jdbc/src/main/java/org/apache/flink/connector/jdbc/table/JdbcDynamicTableSource.java
##########
@@ -0,0 +1,162 @@
+/*
+ * 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.connector.jdbc.table;
+
+import org.apache.flink.annotation.Internal;
+import org.apache.flink.api.java.typeutils.RowTypeInfo;
+import org.apache.flink.connector.jdbc.dialect.JdbcDialect;
+import org.apache.flink.connector.jdbc.internal.options.JdbcLookupOptions;
+import org.apache.flink.connector.jdbc.internal.options.JdbcOptions;
+import org.apache.flink.connector.jdbc.internal.options.JdbcReadOptions;
+import org.apache.flink.connector.jdbc.split.JdbcNumericBetweenParametersProvider;
+import org.apache.flink.table.api.TableSchema;
+import org.apache.flink.table.connector.ChangelogMode;
+import org.apache.flink.table.connector.source.DynamicTableSource;
+import org.apache.flink.table.connector.source.InputFormatProvider;
+import org.apache.flink.table.connector.source.LookupTableSource;
+import org.apache.flink.table.connector.source.ScanTableSource;
+import org.apache.flink.table.connector.source.TableFunctionProvider;
+import org.apache.flink.table.types.DataType;
+import org.apache.flink.table.types.logical.LogicalType;
+import org.apache.flink.table.types.logical.RowType;
+
+import java.util.Arrays;
+import java.util.Objects;
+
+import static org.apache.flink.table.types.utils.TypeConversions.fromDataTypeToLegacyInfo;
+
+/**
+ * A {@link DynamicTableSource} for JDBC.
+ */
+@Internal
+public class JdbcDynamicTableSource implements ScanTableSource, LookupTableSource {
+
+	private static final String name = "JdbcTableSource";
+	private final JdbcOptions options;
+	private final JdbcReadOptions readOptions;
+	private final JdbcLookupOptions lookupOptions;
+	private final TableSchema schema;
+	private final int[] selectFields;
+
+	public JdbcDynamicTableSource(
+		JdbcOptions options,
+		JdbcReadOptions readOptions,
+		JdbcLookupOptions lookupOptions,
+		TableSchema schema,
+		int[] selectFields) {
+		this.options = options;
+		this.readOptions = readOptions;
+		this.lookupOptions = lookupOptions;
+		this.schema = schema;
+		this.selectFields = selectFields;
+	}
+
+	@Override
+	public LookupRuntimeProvider getLookupRuntimeProvider(LookupTableSource.Context context) {
+		// JDBC only support non-nested look up keys
+		String[] keyNames = new String[context.getKeys().length];
+		for (int i = 0; i < keyNames.length; i++) {
+			int index = context.getKeys()[i][0];
+			keyNames[i] = schema.getFieldNames()[index];
+		}
+		return TableFunctionProvider.of(JdbcDynamicLookupFunction.builder()
+			.setFieldNames(schema.getFieldNames())
+			.setFieldTypes(schema.getFieldDataTypes())
+			.setKeyNames(keyNames)
+			.setOptions(options)
+			.setLookupOptions(lookupOptions)
+			.build());
+	}
+
+	@Override
+	public ScanRuntimeProvider getScanRuntimeProvider(ScanTableSource.Context runtimeProviderContext) {
+		final DataType rowDataType = schema.toPhysicalRowDataType();
+		final RowTypeInfo rowTypeInfo = (RowTypeInfo) fromDataTypeToLegacyInfo(rowDataType);
+		final JdbcDynamicInputFormat.Builder builder = JdbcDynamicInputFormat.builder()
+			.setDrivername(options.getDriverName())
+			.setDBUrl(options.getDbURL())
+			.setUsername(options.getUsername().orElse(null))
+			.setPassword(options.getPassword().orElse(null))
+			.setRowTypeInfo(new RowTypeInfo(rowTypeInfo.getFieldTypes(), rowTypeInfo.getFieldNames()));

Review comment:
       This is never used. 




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

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [flink] flinkbot edited a comment on pull request #12176: [FLINK-17029][jdbc]Introduce a new JDBC connector with new property keys

Posted by GitBox <gi...@apache.org>.
flinkbot edited a comment on pull request #12176:
URL: https://github.com/apache/flink/pull/12176#issuecomment-629283127


   <!--
   Meta data
   {
     "version" : 1,
     "metaDataEntries" : [ {
       "hash" : "499722944af576d13757da75f4ad0d08c4538782",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=1453",
       "triggerID" : "499722944af576d13757da75f4ad0d08c4538782",
       "triggerType" : "PUSH"
     }, {
       "hash" : "d07c3a602f13eaf3ea441e11ece1bf671bfe5beb",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=1547",
       "triggerID" : "d07c3a602f13eaf3ea441e11ece1bf671bfe5beb",
       "triggerType" : "PUSH"
     }, {
       "hash" : "226bafeb2086b9c98a29b398c3c34a325390aa2e",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=1558",
       "triggerID" : "226bafeb2086b9c98a29b398c3c34a325390aa2e",
       "triggerType" : "PUSH"
     }, {
       "hash" : "057d8fa644c9a203a753fe184de8e204ea81918e",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=1566",
       "triggerID" : "057d8fa644c9a203a753fe184de8e204ea81918e",
       "triggerType" : "PUSH"
     }, {
       "hash" : "e557b09c5dc9371c42542211651b79ea749cbf03",
       "status" : "FAILURE",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=1584",
       "triggerID" : "e557b09c5dc9371c42542211651b79ea749cbf03",
       "triggerType" : "PUSH"
     }, {
       "hash" : "7f67ad7536d0041ec9fc5f646dc23972822125f8",
       "status" : "UNKNOWN",
       "url" : "TBD",
       "triggerID" : "7f67ad7536d0041ec9fc5f646dc23972822125f8",
       "triggerType" : "PUSH"
     } ]
   }-->
   ## CI report:
   
   * e557b09c5dc9371c42542211651b79ea749cbf03 Azure: [FAILURE](https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=1584) 
   * 7f67ad7536d0041ec9fc5f646dc23972822125f8 UNKNOWN
   
   <details>
   <summary>Bot commands</summary>
     The @flinkbot bot supports the following commands:
   
    - `@flinkbot run travis` re-run the last Travis build
    - `@flinkbot run azure` re-run the last Azure build
   </details>


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

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [flink] wuchong commented on a change in pull request #12176: [FLINK-17029][jdbc]Introduce a new JDBC connector with new property keys

Posted by GitBox <gi...@apache.org>.
wuchong commented on a change in pull request #12176:
URL: https://github.com/apache/flink/pull/12176#discussion_r426125630



##########
File path: flink-connectors/flink-connector-jdbc/src/main/java/org/apache/flink/connector/jdbc/table/JdbcDynamicInputFormat.java
##########
@@ -0,0 +1,402 @@
+/*
+ * 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.connector.jdbc.table;
+
+import org.apache.flink.annotation.Internal;
+import org.apache.flink.api.common.io.DefaultInputSplitAssigner;
+import org.apache.flink.api.common.io.InputFormat;
+import org.apache.flink.api.common.io.RichInputFormat;
+import org.apache.flink.api.common.io.statistics.BaseStatistics;
+import org.apache.flink.api.common.typeinfo.TypeInformation;
+import org.apache.flink.api.java.typeutils.GenericTypeInfo;
+import org.apache.flink.api.java.typeutils.ResultTypeQueryable;
+import org.apache.flink.api.java.typeutils.RowTypeInfo;
+import org.apache.flink.configuration.Configuration;
+import org.apache.flink.connector.jdbc.JdbcConnectionOptions;
+import org.apache.flink.connector.jdbc.internal.connection.SimpleJdbcConnectionProvider;
+import org.apache.flink.connector.jdbc.internal.converter.JdbcToRowConverter;
+import org.apache.flink.connector.jdbc.split.JdbcParameterValuesProvider;
+import org.apache.flink.core.io.GenericInputSplit;
+import org.apache.flink.core.io.InputSplit;
+import org.apache.flink.core.io.InputSplitAssigner;
+import org.apache.flink.table.data.RowData;
+import org.apache.flink.util.Preconditions;
+
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import java.io.IOException;
+import java.math.BigDecimal;
+import java.sql.Array;
+import java.sql.Connection;
+import java.sql.Date;
+import java.sql.PreparedStatement;
+import java.sql.ResultSet;
+import java.sql.SQLException;
+import java.sql.Time;
+import java.sql.Timestamp;
+import java.util.Arrays;
+
+/**
+ * InputFormat for {@link JdbcDynamicTableSource}.
+ */
+@Internal
+public class JdbcDynamicInputFormat extends RichInputFormat<RowData, InputSplit> implements ResultTypeQueryable<RowData> {

Review comment:
       Rename to `JdbcRowDataInputFormat`. `Dynamic` is a logical concept in SQL and we should avoid to use this word in runtime code. 

##########
File path: flink-connectors/flink-connector-jdbc/src/main/java/org/apache/flink/connector/jdbc/table/JdbcDynamicLookupFunction.java
##########
@@ -0,0 +1,293 @@
+/*
+ * 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.connector.jdbc.table;
+
+import org.apache.flink.annotation.Internal;
+import org.apache.flink.connector.jdbc.dialect.JdbcDialect;
+import org.apache.flink.connector.jdbc.dialect.JdbcDialects;
+import org.apache.flink.connector.jdbc.internal.options.JdbcLookupOptions;
+import org.apache.flink.connector.jdbc.internal.options.JdbcOptions;
+import org.apache.flink.connector.jdbc.utils.JdbcTypeUtil;
+import org.apache.flink.connector.jdbc.utils.JdbcUtils;
+import org.apache.flink.table.data.GenericRowData;
+import org.apache.flink.table.data.RowData;
+import org.apache.flink.table.functions.FunctionContext;
+import org.apache.flink.table.functions.TableFunction;
+import org.apache.flink.table.types.DataType;
+import org.apache.flink.table.types.logical.LogicalType;
+import org.apache.flink.table.types.logical.RowType;
+import org.apache.flink.table.types.utils.TypeConversions;
+
+import org.apache.flink.shaded.guava18.com.google.common.cache.Cache;
+import org.apache.flink.shaded.guava18.com.google.common.cache.CacheBuilder;
+
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import java.io.IOException;
+import java.sql.Connection;
+import java.sql.DriverManager;
+import java.sql.PreparedStatement;
+import java.sql.ResultSet;
+import java.sql.SQLException;
+import java.util.ArrayList;
+import java.util.Arrays;
+import java.util.List;
+import java.util.concurrent.TimeUnit;
+
+import static org.apache.flink.util.Preconditions.checkArgument;
+import static org.apache.flink.util.Preconditions.checkNotNull;
+
+/**
+ * A lookup function for {@link JdbcDynamicTableSource}.
+ */
+@Internal
+public class JdbcDynamicLookupFunction extends TableFunction<RowData> {

Review comment:
       Rename to `JdbcRowDataLookupFunction`.

##########
File path: flink-connectors/flink-connector-jdbc/src/main/java/org/apache/flink/connector/jdbc/table/JdbcDynamicOutputFormat.java
##########
@@ -0,0 +1,265 @@
+/*
+ * 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.connector.jdbc.table;
+
+import org.apache.flink.annotation.Internal;
+import org.apache.flink.api.common.functions.RuntimeContext;
+import org.apache.flink.api.common.typeinfo.TypeInformation;
+import org.apache.flink.api.common.typeutils.TypeSerializer;
+import org.apache.flink.connector.jdbc.JdbcExecutionOptions;
+import org.apache.flink.connector.jdbc.JdbcStatementBuilder;
+import org.apache.flink.connector.jdbc.dialect.JdbcDialect;
+import org.apache.flink.connector.jdbc.internal.JdbcBatchingOutputFormat;
+import org.apache.flink.connector.jdbc.internal.connection.JdbcConnectionProvider;
+import org.apache.flink.connector.jdbc.internal.connection.SimpleJdbcConnectionProvider;
+import org.apache.flink.connector.jdbc.internal.executor.InsertOrUpdateJdbcExecutor;
+import org.apache.flink.connector.jdbc.internal.executor.JdbcBatchStatementExecutor;
+import org.apache.flink.connector.jdbc.internal.options.JdbcDmlOptions;
+import org.apache.flink.connector.jdbc.internal.options.JdbcOptions;
+import org.apache.flink.connector.jdbc.utils.JdbcUtils;
+import org.apache.flink.table.data.GenericRowData;
+import org.apache.flink.table.data.RowData;
+import org.apache.flink.table.types.DataType;
+import org.apache.flink.table.types.logical.LogicalType;
+import org.apache.flink.types.Row;
+
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import java.io.IOException;
+import java.sql.SQLException;
+import java.util.Arrays;
+import java.util.function.Function;
+
+import static org.apache.flink.util.Preconditions.checkArgument;
+import static org.apache.flink.util.Preconditions.checkNotNull;
+
+/**
+ * OutputFormat for {@link JdbcDynamicTableSource}.
+ */
+@Internal
+public class JdbcDynamicOutputFormat extends JdbcBatchingOutputFormat<RowData, RowData, JdbcBatchStatementExecutor<RowData>> {

Review comment:
       Rename to `JdbcRowDataOutputFormat`.

##########
File path: flink-connectors/flink-connector-jdbc/src/main/java/org/apache/flink/connector/jdbc/table/JdbcDynamicInputFormat.java
##########
@@ -0,0 +1,402 @@
+/*
+ * 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.connector.jdbc.table;
+
+import org.apache.flink.annotation.Internal;
+import org.apache.flink.api.common.io.DefaultInputSplitAssigner;
+import org.apache.flink.api.common.io.InputFormat;
+import org.apache.flink.api.common.io.RichInputFormat;
+import org.apache.flink.api.common.io.statistics.BaseStatistics;
+import org.apache.flink.api.common.typeinfo.TypeInformation;
+import org.apache.flink.api.java.typeutils.GenericTypeInfo;
+import org.apache.flink.api.java.typeutils.ResultTypeQueryable;
+import org.apache.flink.api.java.typeutils.RowTypeInfo;
+import org.apache.flink.configuration.Configuration;
+import org.apache.flink.connector.jdbc.JdbcConnectionOptions;
+import org.apache.flink.connector.jdbc.internal.connection.SimpleJdbcConnectionProvider;
+import org.apache.flink.connector.jdbc.internal.converter.JdbcToRowConverter;
+import org.apache.flink.connector.jdbc.split.JdbcParameterValuesProvider;
+import org.apache.flink.core.io.GenericInputSplit;
+import org.apache.flink.core.io.InputSplit;
+import org.apache.flink.core.io.InputSplitAssigner;
+import org.apache.flink.table.data.RowData;
+import org.apache.flink.util.Preconditions;
+
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import java.io.IOException;
+import java.math.BigDecimal;
+import java.sql.Array;
+import java.sql.Connection;
+import java.sql.Date;
+import java.sql.PreparedStatement;
+import java.sql.ResultSet;
+import java.sql.SQLException;
+import java.sql.Time;
+import java.sql.Timestamp;
+import java.util.Arrays;
+
+/**
+ * InputFormat for {@link JdbcDynamicTableSource}.
+ */
+@Internal
+public class JdbcDynamicInputFormat extends RichInputFormat<RowData, InputSplit> implements ResultTypeQueryable<RowData> {
+
+	private static final long serialVersionUID = 1L;
+	private static final Logger LOG = LoggerFactory.getLogger(JdbcDynamicInputFormat.class);
+
+	private JdbcConnectionOptions connectionOptions;
+	private int fetchSize;
+	private Boolean autoCommit;
+	private Object[][] parameterValues;
+	private String queryTemplate;
+	private int resultSetType;
+	private int resultSetConcurrency;
+	private JdbcToRowConverter rowConverter;
+
+	private transient Connection dbConn;
+	private transient PreparedStatement statement;
+	private transient ResultSet resultSet;
+	private transient boolean hasNext;
+
+	private JdbcDynamicInputFormat(
+		JdbcConnectionOptions connectionOptions,
+		int fetchSize,
+		Boolean autoCommit,
+		Object[][] parameterValues,
+		String queryTemplate,
+		int resultSetType,
+		int resultSetConcurrency,
+		JdbcToRowConverter rowConverter) {
+		this.connectionOptions = connectionOptions;
+		this.fetchSize = fetchSize;
+		this.autoCommit = autoCommit;
+		this.parameterValues = parameterValues;
+		this.queryTemplate = queryTemplate;
+		this.resultSetType = resultSetType;
+		this.resultSetConcurrency = resultSetConcurrency;
+		this.rowConverter = rowConverter;
+	}
+
+	@Override
+	public void configure(Configuration parameters) {
+		//do nothing here
+	}
+
+	@Override
+	public void openInputFormat() {
+		//called once per inputFormat (on open)
+		try {
+			dbConn = new SimpleJdbcConnectionProvider(connectionOptions).getConnection();
+			// set autoCommit mode only if it was explicitly configured.
+			// keep connection default otherwise.
+			if (autoCommit != null) {
+				dbConn.setAutoCommit(autoCommit);
+			}
+			statement = dbConn.prepareStatement(queryTemplate, resultSetType, resultSetConcurrency);
+			if (fetchSize == Integer.MIN_VALUE || fetchSize > 0) {
+				statement.setFetchSize(fetchSize);
+			}
+		} catch (SQLException se) {
+			throw new IllegalArgumentException("open() failed." + se.getMessage(), se);
+		} catch (ClassNotFoundException cnfe) {
+			throw new IllegalArgumentException("JDBC-Class not found. - " + cnfe.getMessage(), cnfe);
+		}
+	}
+
+	@Override
+	public void closeInputFormat() {
+		//called once per inputFormat (on close)
+		try {
+			if (statement != null) {
+				statement.close();
+			}
+		} catch (SQLException se) {
+			LOG.info("Inputformat Statement couldn't be closed - " + se.getMessage());
+		} finally {
+			statement = null;
+		}
+
+		try {
+			if (dbConn != null) {
+				dbConn.close();
+			}
+		} catch (SQLException se) {
+			LOG.info("Inputformat couldn't be closed - " + se.getMessage());
+		} finally {
+			dbConn = null;
+		}
+
+		parameterValues = null;
+	}
+
+	/**
+	 * Connects to the source database and executes the query in a <b>parallel
+	 * fashion</b> if
+	 * this {@link InputFormat} is built using a parameterized query (i.e. using
+	 * a {@link PreparedStatement})
+	 * and a proper {@link JdbcParameterValuesProvider}, in a <b>non-parallel
+	 * fashion</b> otherwise.
+	 *
+	 * @param inputSplit which is ignored if this InputFormat is executed as a
+	 *                   non-parallel source,
+	 *                   a "hook" to the query parameters otherwise (using its
+	 *                   <i>splitNumber</i>)
+	 * @throws IOException if there's an error during the execution of the query
+	 */
+	@Override
+	public void open(InputSplit inputSplit) throws IOException {
+		try {
+			if (inputSplit != null && parameterValues != null) {
+				for (int i = 0; i < parameterValues[inputSplit.getSplitNumber()].length; i++) {
+					Object param = parameterValues[inputSplit.getSplitNumber()][i];
+					if (param instanceof String) {
+						statement.setString(i + 1, (String) param);
+					} else if (param instanceof Long) {
+						statement.setLong(i + 1, (Long) param);
+					} else if (param instanceof Integer) {
+						statement.setInt(i + 1, (Integer) param);
+					} else if (param instanceof Double) {
+						statement.setDouble(i + 1, (Double) param);
+					} else if (param instanceof Boolean) {
+						statement.setBoolean(i + 1, (Boolean) param);
+					} else if (param instanceof Float) {
+						statement.setFloat(i + 1, (Float) param);
+					} else if (param instanceof BigDecimal) {
+						statement.setBigDecimal(i + 1, (BigDecimal) param);
+					} else if (param instanceof Byte) {
+						statement.setByte(i + 1, (Byte) param);
+					} else if (param instanceof Short) {
+						statement.setShort(i + 1, (Short) param);
+					} else if (param instanceof Date) {
+						statement.setDate(i + 1, (Date) param);
+					} else if (param instanceof Time) {
+						statement.setTime(i + 1, (Time) param);
+					} else if (param instanceof Timestamp) {
+						statement.setTimestamp(i + 1, (Timestamp) param);
+					} else if (param instanceof Array) {
+						statement.setArray(i + 1, (Array) param);
+					} else {
+						//extends with other types if needed
+						throw new IllegalArgumentException("open() failed. Parameter " + i + " of type " + param.getClass() + " is not handled (yet).");
+					}
+				}
+				if (LOG.isDebugEnabled()) {
+					LOG.debug(String.format("Executing '%s' with parameters %s", queryTemplate, Arrays.deepToString(parameterValues[inputSplit.getSplitNumber()])));
+				}
+			}
+			resultSet = statement.executeQuery();
+			hasNext = resultSet.next();
+		} catch (SQLException se) {
+			throw new IllegalArgumentException("open() failed." + se.getMessage(), se);
+		}
+	}
+
+	/**
+	 * Closes all resources used.
+	 *
+	 * @throws IOException Indicates that a resource could not be closed.
+	 */
+	@Override
+	public void close() throws IOException {
+		if (resultSet == null) {
+			return;
+		}
+		try {
+			resultSet.close();
+		} catch (SQLException se) {
+			LOG.info("Inputformat ResultSet couldn't be closed - " + se.getMessage());
+		}
+	}
+
+	@Override
+	public TypeInformation<RowData> getProducedType() {
+		return new GenericTypeInfo<RowData>(RowData.class);

Review comment:
       We can pass in the RowData TypeInformation as construct parameter which can be get from `ScanTableSource#Context#createTypeInformation`. 

##########
File path: flink-connectors/flink-connector-jdbc/src/main/java/org/apache/flink/connector/jdbc/table/JdbcDynamicLookupFunction.java
##########
@@ -0,0 +1,293 @@
+/*
+ * 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.connector.jdbc.table;
+
+import org.apache.flink.annotation.Internal;
+import org.apache.flink.connector.jdbc.dialect.JdbcDialect;
+import org.apache.flink.connector.jdbc.dialect.JdbcDialects;
+import org.apache.flink.connector.jdbc.internal.options.JdbcLookupOptions;
+import org.apache.flink.connector.jdbc.internal.options.JdbcOptions;
+import org.apache.flink.connector.jdbc.utils.JdbcTypeUtil;
+import org.apache.flink.connector.jdbc.utils.JdbcUtils;
+import org.apache.flink.table.data.GenericRowData;
+import org.apache.flink.table.data.RowData;
+import org.apache.flink.table.functions.FunctionContext;
+import org.apache.flink.table.functions.TableFunction;
+import org.apache.flink.table.types.DataType;
+import org.apache.flink.table.types.logical.LogicalType;
+import org.apache.flink.table.types.logical.RowType;
+import org.apache.flink.table.types.utils.TypeConversions;
+
+import org.apache.flink.shaded.guava18.com.google.common.cache.Cache;
+import org.apache.flink.shaded.guava18.com.google.common.cache.CacheBuilder;
+
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import java.io.IOException;
+import java.sql.Connection;
+import java.sql.DriverManager;
+import java.sql.PreparedStatement;
+import java.sql.ResultSet;
+import java.sql.SQLException;
+import java.util.ArrayList;
+import java.util.Arrays;
+import java.util.List;
+import java.util.concurrent.TimeUnit;
+
+import static org.apache.flink.util.Preconditions.checkArgument;
+import static org.apache.flink.util.Preconditions.checkNotNull;
+
+/**
+ * A lookup function for {@link JdbcDynamicTableSource}.
+ */
+@Internal
+public class JdbcDynamicLookupFunction extends TableFunction<RowData> {
+
+	private static final Logger LOG = LoggerFactory.getLogger(JdbcDynamicLookupFunction.class);
+	private static final long serialVersionUID = 1L;
+
+	private final String query;
+	private final String drivername;
+	private final String dbURL;
+	private final String username;
+	private final String password;
+	private final DataType[] keyTypes;
+	private final int[] keySqlTypes;
+	private final long cacheMaxSize;
+	private final long cacheExpireMs;
+	private final int maxRetryTimes;
+	private final JdbcDialect jdbcDialect;
+	private final RowType rowType;
+
+	private transient Connection dbConn;
+	private transient PreparedStatement statement;
+	private transient Cache<RowData, List<RowData>> cache;
+
+	private JdbcDynamicLookupFunction(
+		JdbcOptions options, JdbcLookupOptions lookupOptions,
+		String[] fieldNames, DataType[] fieldTypes, String[] keyNames, RowType rowType) {

Review comment:
       Indent.

##########
File path: flink-connectors/flink-connector-jdbc/src/main/java/org/apache/flink/connector/jdbc/table/JdbcDynamicOutputFormat.java
##########
@@ -0,0 +1,265 @@
+/*
+ * 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.connector.jdbc.table;
+
+import org.apache.flink.annotation.Internal;
+import org.apache.flink.api.common.functions.RuntimeContext;
+import org.apache.flink.api.common.typeinfo.TypeInformation;
+import org.apache.flink.api.common.typeutils.TypeSerializer;
+import org.apache.flink.connector.jdbc.JdbcExecutionOptions;
+import org.apache.flink.connector.jdbc.JdbcStatementBuilder;
+import org.apache.flink.connector.jdbc.dialect.JdbcDialect;
+import org.apache.flink.connector.jdbc.internal.JdbcBatchingOutputFormat;
+import org.apache.flink.connector.jdbc.internal.connection.JdbcConnectionProvider;
+import org.apache.flink.connector.jdbc.internal.connection.SimpleJdbcConnectionProvider;
+import org.apache.flink.connector.jdbc.internal.executor.InsertOrUpdateJdbcExecutor;
+import org.apache.flink.connector.jdbc.internal.executor.JdbcBatchStatementExecutor;
+import org.apache.flink.connector.jdbc.internal.options.JdbcDmlOptions;
+import org.apache.flink.connector.jdbc.internal.options.JdbcOptions;
+import org.apache.flink.connector.jdbc.utils.JdbcUtils;
+import org.apache.flink.table.data.GenericRowData;
+import org.apache.flink.table.data.RowData;
+import org.apache.flink.table.types.DataType;
+import org.apache.flink.table.types.logical.LogicalType;
+import org.apache.flink.types.Row;
+
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import java.io.IOException;
+import java.sql.SQLException;
+import java.util.Arrays;
+import java.util.function.Function;
+
+import static org.apache.flink.util.Preconditions.checkArgument;
+import static org.apache.flink.util.Preconditions.checkNotNull;
+
+/**
+ * OutputFormat for {@link JdbcDynamicTableSource}.
+ */
+@Internal
+public class JdbcDynamicOutputFormat extends JdbcBatchingOutputFormat<RowData, RowData, JdbcBatchStatementExecutor<RowData>> {
+	private static final Logger LOG = LoggerFactory.getLogger(JdbcDynamicOutputFormat.class);
+
+	private JdbcBatchStatementExecutor<RowData> deleteExecutor;
+	private final JdbcDmlOptions dmlOptions;
+	private final LogicalType[] logicalTypes;
+
+	private JdbcDynamicOutputFormat(JdbcConnectionProvider connectionProvider, JdbcDmlOptions dmlOptions, JdbcExecutionOptions batchOptions, TypeInformation<RowData> rowDataTypeInfo, LogicalType[] logicalTypes) {
+		super(connectionProvider, batchOptions, ctx -> createUpsertRowExecutor(dmlOptions, ctx, rowDataTypeInfo, logicalTypes), RecordExtractor.identity());
+		this.dmlOptions = dmlOptions;
+		this.logicalTypes = logicalTypes;
+	}
+
+	private JdbcDynamicOutputFormat(JdbcConnectionProvider connectionProvider, JdbcDmlOptions dmlOptions, JdbcExecutionOptions batchOptions, TypeInformation<RowData> rowDataTypeInfo, LogicalType[] logicalTypes, String sql) {
+		super(connectionProvider, batchOptions,
+			ctx -> createSimpleRowDataExecutor(dmlOptions.getDialect(), sql, logicalTypes, ctx, rowDataTypeInfo),
+			RecordExtractor.identity());
+		this.dmlOptions = dmlOptions;
+		this.logicalTypes = logicalTypes;
+	}
+
+	@Override
+	public void open(int taskNumber, int numTasks) throws IOException {
+		super.open(taskNumber, numTasks);
+		deleteExecutor = createDeleteExecutor();
+		try {
+			deleteExecutor.open(connection);
+		} catch (SQLException e) {
+			throw new IOException(e);
+		}
+	}
+
+	private JdbcBatchStatementExecutor<RowData> createDeleteExecutor() {
+		int[] pkFields = Arrays.stream(dmlOptions.getFieldNames()).mapToInt(Arrays.asList(dmlOptions.getFieldNames())::indexOf).toArray();
+		LogicalType[] pkTypes = Arrays.stream(pkFields).mapToObj(f -> logicalTypes[f]).toArray(LogicalType[]::new);
+		String deleteSql = dmlOptions.getDialect().getDeleteStatement(dmlOptions.getTableName(), dmlOptions.getFieldNames());
+		return createKeyedRowExecutor(dmlOptions.getDialect(), pkFields, pkTypes, deleteSql, logicalTypes);
+	}
+
+	@Override
+	protected void addToBatch(RowData original, RowData extracted) throws SQLException {
+		switch (original.getRowKind()) {
+			case DELETE:
+				deleteExecutor.addToBatch(extracted);
+				break;
+			case INSERT:
+			case UPDATE_AFTER:
+				super.addToBatch(original, extracted);
+				break;
+			default:
+				return;

Review comment:
       Remove this? We don't have other RowKinds for now. There is a warning on the `return`. 

##########
File path: flink-connectors/flink-connector-jdbc/src/main/java/org/apache/flink/connector/jdbc/table/JdbcDynamicOutputFormat.java
##########
@@ -0,0 +1,265 @@
+/*
+ * 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.connector.jdbc.table;
+
+import org.apache.flink.annotation.Internal;
+import org.apache.flink.api.common.functions.RuntimeContext;
+import org.apache.flink.api.common.typeinfo.TypeInformation;
+import org.apache.flink.api.common.typeutils.TypeSerializer;
+import org.apache.flink.connector.jdbc.JdbcExecutionOptions;
+import org.apache.flink.connector.jdbc.JdbcStatementBuilder;
+import org.apache.flink.connector.jdbc.dialect.JdbcDialect;
+import org.apache.flink.connector.jdbc.internal.JdbcBatchingOutputFormat;
+import org.apache.flink.connector.jdbc.internal.connection.JdbcConnectionProvider;
+import org.apache.flink.connector.jdbc.internal.connection.SimpleJdbcConnectionProvider;
+import org.apache.flink.connector.jdbc.internal.executor.InsertOrUpdateJdbcExecutor;
+import org.apache.flink.connector.jdbc.internal.executor.JdbcBatchStatementExecutor;
+import org.apache.flink.connector.jdbc.internal.options.JdbcDmlOptions;
+import org.apache.flink.connector.jdbc.internal.options.JdbcOptions;
+import org.apache.flink.connector.jdbc.utils.JdbcUtils;
+import org.apache.flink.table.data.GenericRowData;
+import org.apache.flink.table.data.RowData;
+import org.apache.flink.table.types.DataType;
+import org.apache.flink.table.types.logical.LogicalType;
+import org.apache.flink.types.Row;
+
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import java.io.IOException;
+import java.sql.SQLException;
+import java.util.Arrays;
+import java.util.function.Function;
+
+import static org.apache.flink.util.Preconditions.checkArgument;
+import static org.apache.flink.util.Preconditions.checkNotNull;
+
+/**
+ * OutputFormat for {@link JdbcDynamicTableSource}.
+ */
+@Internal
+public class JdbcDynamicOutputFormat extends JdbcBatchingOutputFormat<RowData, RowData, JdbcBatchStatementExecutor<RowData>> {

Review comment:
       Add `serialVersionUID`.

##########
File path: flink-connectors/flink-connector-jdbc/src/main/java/org/apache/flink/connector/jdbc/table/JdbcDynamicTableSink.java
##########
@@ -0,0 +1,113 @@
+/*
+ * 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.connector.jdbc.table;
+
+import org.apache.flink.annotation.Internal;
+import org.apache.flink.api.common.typeinfo.TypeInformation;
+import org.apache.flink.connector.jdbc.JdbcExecutionOptions;
+import org.apache.flink.connector.jdbc.internal.options.JdbcDmlOptions;
+import org.apache.flink.connector.jdbc.internal.options.JdbcOptions;
+import org.apache.flink.table.connector.ChangelogMode;
+import org.apache.flink.table.connector.sink.DynamicTableSink;
+import org.apache.flink.table.connector.sink.OutputFormatProvider;
+import org.apache.flink.table.data.RowData;
+import org.apache.flink.table.types.DataType;
+import org.apache.flink.types.RowKind;
+
+import java.util.Arrays;
+import java.util.Objects;
+
+import static org.apache.flink.connector.jdbc.table.JdbcDynamicOutputFormat.DynamicOutputFormatBuilder;
+
+/**
+ * A {@link DynamicTableSink} for JDBC.
+ */
+@Internal
+public class JdbcDynamicTableSink implements DynamicTableSink {
+
+	private static final String name = "JdbcTableSink";
+	private final JdbcOptions jdbcOptions;
+	private final JdbcExecutionOptions executionOptions;
+	private final JdbcDmlOptions dmlOptions;
+	private final DataType rowDataType;
+	private final DataType[] fieldTypes;
+
+	public JdbcDynamicTableSink(JdbcOptions jdbcOptions, JdbcExecutionOptions executionOptions, JdbcDmlOptions dmlOptions, DataType rowDataType, DataType[] fieldTypes) {
+		this.jdbcOptions = jdbcOptions;
+		this.executionOptions = executionOptions;
+		this.dmlOptions = dmlOptions;
+		this.rowDataType = rowDataType;
+		this.fieldTypes = fieldTypes;
+	}
+
+	@Override
+	public ChangelogMode getChangelogMode(ChangelogMode requestedMode) {
+		return ChangelogMode.newBuilder()
+			.addContainedKind(RowKind.INSERT)
+			.addContainedKind(RowKind.DELETE)
+			.addContainedKind(RowKind.UPDATE_AFTER)
+			.build();
+	}
+
+	@Override
+	public SinkRuntimeProvider getSinkRuntimeProvider(Context context) {
+		final TypeInformation<RowData> rowDataTypeInformation = (TypeInformation<RowData>) context.createTypeInformation(rowDataType);
+		final DynamicOutputFormatBuilder builder = JdbcDynamicOutputFormat.dynamicOutputFormatBuilder();
+
+		builder.setJdbcOptions(jdbcOptions);
+		builder.setJdbcDmlOptions(dmlOptions);
+		builder.setJdbcExecutionOptions(executionOptions);
+		builder.setRowDataTypeInfo(rowDataTypeInformation);
+		builder.setFieldDataTypes(fieldTypes);
+		return OutputFormatProvider.of(builder.build());
+	}
+
+	@Override
+	public DynamicTableSink copy() {
+		return new JdbcDynamicTableSink(jdbcOptions, executionOptions, dmlOptions, rowDataType, fieldTypes);
+	}
+
+	@Override
+	public String asSummaryString() {
+		return name;

Review comment:
       Would be better to return `"JDBC:" + dialectName`.

##########
File path: flink-connectors/flink-connector-jdbc/src/main/java/org/apache/flink/connector/jdbc/table/JdbcDynamicOutputFormat.java
##########
@@ -0,0 +1,265 @@
+/*
+ * 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.connector.jdbc.table;
+
+import org.apache.flink.annotation.Internal;
+import org.apache.flink.api.common.functions.RuntimeContext;
+import org.apache.flink.api.common.typeinfo.TypeInformation;
+import org.apache.flink.api.common.typeutils.TypeSerializer;
+import org.apache.flink.connector.jdbc.JdbcExecutionOptions;
+import org.apache.flink.connector.jdbc.JdbcStatementBuilder;
+import org.apache.flink.connector.jdbc.dialect.JdbcDialect;
+import org.apache.flink.connector.jdbc.internal.JdbcBatchingOutputFormat;
+import org.apache.flink.connector.jdbc.internal.connection.JdbcConnectionProvider;
+import org.apache.flink.connector.jdbc.internal.connection.SimpleJdbcConnectionProvider;
+import org.apache.flink.connector.jdbc.internal.executor.InsertOrUpdateJdbcExecutor;
+import org.apache.flink.connector.jdbc.internal.executor.JdbcBatchStatementExecutor;
+import org.apache.flink.connector.jdbc.internal.options.JdbcDmlOptions;
+import org.apache.flink.connector.jdbc.internal.options.JdbcOptions;
+import org.apache.flink.connector.jdbc.utils.JdbcUtils;
+import org.apache.flink.table.data.GenericRowData;
+import org.apache.flink.table.data.RowData;
+import org.apache.flink.table.types.DataType;
+import org.apache.flink.table.types.logical.LogicalType;
+import org.apache.flink.types.Row;
+
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import java.io.IOException;
+import java.sql.SQLException;
+import java.util.Arrays;
+import java.util.function.Function;
+
+import static org.apache.flink.util.Preconditions.checkArgument;
+import static org.apache.flink.util.Preconditions.checkNotNull;
+
+/**
+ * OutputFormat for {@link JdbcDynamicTableSource}.
+ */
+@Internal
+public class JdbcDynamicOutputFormat extends JdbcBatchingOutputFormat<RowData, RowData, JdbcBatchStatementExecutor<RowData>> {
+	private static final Logger LOG = LoggerFactory.getLogger(JdbcDynamicOutputFormat.class);
+
+	private JdbcBatchStatementExecutor<RowData> deleteExecutor;
+	private final JdbcDmlOptions dmlOptions;
+	private final LogicalType[] logicalTypes;
+
+	private JdbcDynamicOutputFormat(JdbcConnectionProvider connectionProvider, JdbcDmlOptions dmlOptions, JdbcExecutionOptions batchOptions, TypeInformation<RowData> rowDataTypeInfo, LogicalType[] logicalTypes) {
+		super(connectionProvider, batchOptions, ctx -> createUpsertRowExecutor(dmlOptions, ctx, rowDataTypeInfo, logicalTypes), RecordExtractor.identity());
+		this.dmlOptions = dmlOptions;
+		this.logicalTypes = logicalTypes;
+	}
+
+	private JdbcDynamicOutputFormat(JdbcConnectionProvider connectionProvider, JdbcDmlOptions dmlOptions, JdbcExecutionOptions batchOptions, TypeInformation<RowData> rowDataTypeInfo, LogicalType[] logicalTypes, String sql) {
+		super(connectionProvider, batchOptions,
+			ctx -> createSimpleRowDataExecutor(dmlOptions.getDialect(), sql, logicalTypes, ctx, rowDataTypeInfo),
+			RecordExtractor.identity());
+		this.dmlOptions = dmlOptions;
+		this.logicalTypes = logicalTypes;
+	}
+
+	@Override
+	public void open(int taskNumber, int numTasks) throws IOException {
+		super.open(taskNumber, numTasks);
+		deleteExecutor = createDeleteExecutor();
+		try {
+			deleteExecutor.open(connection);
+		} catch (SQLException e) {
+			throw new IOException(e);
+		}
+	}
+
+	private JdbcBatchStatementExecutor<RowData> createDeleteExecutor() {
+		int[] pkFields = Arrays.stream(dmlOptions.getFieldNames()).mapToInt(Arrays.asList(dmlOptions.getFieldNames())::indexOf).toArray();
+		LogicalType[] pkTypes = Arrays.stream(pkFields).mapToObj(f -> logicalTypes[f]).toArray(LogicalType[]::new);
+		String deleteSql = dmlOptions.getDialect().getDeleteStatement(dmlOptions.getTableName(), dmlOptions.getFieldNames());
+		return createKeyedRowExecutor(dmlOptions.getDialect(), pkFields, pkTypes, deleteSql, logicalTypes);
+	}
+
+	@Override
+	protected void addToBatch(RowData original, RowData extracted) throws SQLException {
+		switch (original.getRowKind()) {
+			case DELETE:
+				deleteExecutor.addToBatch(extracted);
+				break;
+			case INSERT:
+			case UPDATE_AFTER:
+				super.addToBatch(original, extracted);
+				break;
+			default:
+				return;
+		}
+	}
+
+	@Override
+	public synchronized void close() {
+		try {
+			super.close();
+		} finally {
+			try {
+				if (deleteExecutor != null) {
+					deleteExecutor.close();
+				}
+			} catch (SQLException e) {
+				LOG.warn("unable to close delete statement runner", e);
+			}
+		}
+	}
+
+	@Override
+	protected void attemptFlush() throws SQLException {
+		super.attemptFlush();
+		deleteExecutor.executeBatch();
+	}
+
+	private static JdbcBatchStatementExecutor<RowData> createKeyedRowExecutor(JdbcDialect dialect, int[] pkFields, LogicalType[] pkTypes, String sql, LogicalType[] logicalTypes) {
+		return JdbcBatchStatementExecutor.keyed(
+			sql,
+			createRowKeyExtractor(pkFields, logicalTypes),
+			(st, record) -> dialect
+				.getOutputConverter(logicalTypes)
+				.toExternal(createRowKeyExtractor(pkFields, logicalTypes).apply(record), st));
+	}
+
+	private static JdbcBatchStatementExecutor<RowData> createUpsertRowExecutor(JdbcDmlOptions opt, RuntimeContext ctx, TypeInformation<RowData> rowDataTypeInfo, LogicalType[] logicalTypes) {
+		checkArgument(opt.getKeyFields().isPresent());
+
+		int[] pkFields = Arrays.stream(opt.getKeyFields().get()).mapToInt(Arrays.asList(opt.getFieldNames())::indexOf).toArray();
+		LogicalType[] pkTypes = Arrays.stream(pkFields).mapToObj(f -> logicalTypes[f]).toArray(LogicalType[]::new);
+		JdbcDialect dialect = opt.getDialect();
+		final TypeSerializer<RowData> typeSerializer = rowDataTypeInfo.createSerializer(ctx.getExecutionConfig());
+		return opt.getDialect()
+			.getUpsertStatement(opt.getTableName(), opt.getFieldNames(), opt.getKeyFields().get())
+			.map(sql -> createSimpleRowDataExecutor(dialect, sql, logicalTypes, ctx, rowDataTypeInfo))
+			.orElseGet(() ->
+				new InsertOrUpdateJdbcExecutor<>(
+					opt.getDialect().getRowExistsStatement(opt.getTableName(), opt.getKeyFields().get()),
+					opt.getDialect().getInsertIntoStatement(opt.getTableName(), opt.getFieldNames()),
+					opt.getDialect().getUpdateStatement(opt.getTableName(), opt.getFieldNames(), opt.getKeyFields().get()),
+					createRowDataJdbcStatementBuilder(dialect, pkTypes),
+					createRowDataJdbcStatementBuilder(dialect, logicalTypes),
+					createRowDataJdbcStatementBuilder(dialect, logicalTypes),
+					createRowKeyExtractor(pkFields, logicalTypes),
+					ctx.getExecutionConfig().isObjectReuseEnabled() ? typeSerializer::copy : r -> r));
+	}
+
+	private static Function<RowData, RowData> createRowKeyExtractor(int[] pkFields, LogicalType[] logicalTypes) {
+		return row -> getPrimaryKey(row, pkFields, logicalTypes);
+	}
+
+	private static JdbcBatchStatementExecutor<RowData> createSimpleRowDataExecutor(JdbcDialect dialect, String sql, LogicalType[] fieldTypes, RuntimeContext ctx, TypeInformation<RowData> rowDataTypeInfo) {
+		final TypeSerializer<RowData> typeSerializer = rowDataTypeInfo.createSerializer(ctx.getExecutionConfig());
+		return JdbcBatchStatementExecutor.simple(
+			sql,
+			createRowDataJdbcStatementBuilder(dialect, fieldTypes),
+			ctx.getExecutionConfig().isObjectReuseEnabled() ? typeSerializer::copy : Function.identity());
+	}
+
+	/**
+	 * Creates a {@link JdbcStatementBuilder} for {@link Row} using the provided SQL types array.
+	 * Uses {@link JdbcUtils#setRecordToStatement}
+	 */
+	private static JdbcStatementBuilder<RowData> createRowDataJdbcStatementBuilder(JdbcDialect dialect, LogicalType[] types) {
+		return (st, record) -> dialect.getOutputConverter(types).toExternal(record, st);
+	}
+
+	private static RowData getPrimaryKey(RowData row, int[] pkFields, LogicalType[] logicalTypes) {
+		GenericRowData pkRow = new GenericRowData(pkFields.length);
+		for (int i = 0; i < pkFields.length; i++) {
+			pkRow.setField(i, RowData.get(row, pkFields[i], logicalTypes[pkFields[i]]));

Review comment:
       Please use RowData#FieldGetter.

##########
File path: flink-connectors/flink-connector-jdbc/src/main/java/org/apache/flink/connector/jdbc/table/JdbcDynamicOutputFormat.java
##########
@@ -0,0 +1,265 @@
+/*
+ * 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.connector.jdbc.table;
+
+import org.apache.flink.annotation.Internal;
+import org.apache.flink.api.common.functions.RuntimeContext;
+import org.apache.flink.api.common.typeinfo.TypeInformation;
+import org.apache.flink.api.common.typeutils.TypeSerializer;
+import org.apache.flink.connector.jdbc.JdbcExecutionOptions;
+import org.apache.flink.connector.jdbc.JdbcStatementBuilder;
+import org.apache.flink.connector.jdbc.dialect.JdbcDialect;
+import org.apache.flink.connector.jdbc.internal.JdbcBatchingOutputFormat;
+import org.apache.flink.connector.jdbc.internal.connection.JdbcConnectionProvider;
+import org.apache.flink.connector.jdbc.internal.connection.SimpleJdbcConnectionProvider;
+import org.apache.flink.connector.jdbc.internal.executor.InsertOrUpdateJdbcExecutor;
+import org.apache.flink.connector.jdbc.internal.executor.JdbcBatchStatementExecutor;
+import org.apache.flink.connector.jdbc.internal.options.JdbcDmlOptions;
+import org.apache.flink.connector.jdbc.internal.options.JdbcOptions;
+import org.apache.flink.connector.jdbc.utils.JdbcUtils;
+import org.apache.flink.table.data.GenericRowData;
+import org.apache.flink.table.data.RowData;
+import org.apache.flink.table.types.DataType;
+import org.apache.flink.table.types.logical.LogicalType;
+import org.apache.flink.types.Row;
+
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import java.io.IOException;
+import java.sql.SQLException;
+import java.util.Arrays;
+import java.util.function.Function;
+
+import static org.apache.flink.util.Preconditions.checkArgument;
+import static org.apache.flink.util.Preconditions.checkNotNull;
+
+/**
+ * OutputFormat for {@link JdbcDynamicTableSource}.
+ */
+@Internal
+public class JdbcDynamicOutputFormat extends JdbcBatchingOutputFormat<RowData, RowData, JdbcBatchStatementExecutor<RowData>> {
+	private static final Logger LOG = LoggerFactory.getLogger(JdbcDynamicOutputFormat.class);
+
+	private JdbcBatchStatementExecutor<RowData> deleteExecutor;
+	private final JdbcDmlOptions dmlOptions;
+	private final LogicalType[] logicalTypes;
+
+	private JdbcDynamicOutputFormat(JdbcConnectionProvider connectionProvider, JdbcDmlOptions dmlOptions, JdbcExecutionOptions batchOptions, TypeInformation<RowData> rowDataTypeInfo, LogicalType[] logicalTypes) {
+		super(connectionProvider, batchOptions, ctx -> createUpsertRowExecutor(dmlOptions, ctx, rowDataTypeInfo, logicalTypes), RecordExtractor.identity());
+		this.dmlOptions = dmlOptions;
+		this.logicalTypes = logicalTypes;
+	}
+
+	private JdbcDynamicOutputFormat(JdbcConnectionProvider connectionProvider, JdbcDmlOptions dmlOptions, JdbcExecutionOptions batchOptions, TypeInformation<RowData> rowDataTypeInfo, LogicalType[] logicalTypes, String sql) {
+		super(connectionProvider, batchOptions,
+			ctx -> createSimpleRowDataExecutor(dmlOptions.getDialect(), sql, logicalTypes, ctx, rowDataTypeInfo),
+			RecordExtractor.identity());
+		this.dmlOptions = dmlOptions;
+		this.logicalTypes = logicalTypes;
+	}
+
+	@Override
+	public void open(int taskNumber, int numTasks) throws IOException {
+		super.open(taskNumber, numTasks);
+		deleteExecutor = createDeleteExecutor();
+		try {
+			deleteExecutor.open(connection);
+		} catch (SQLException e) {
+			throw new IOException(e);
+		}
+	}
+
+	private JdbcBatchStatementExecutor<RowData> createDeleteExecutor() {
+		int[] pkFields = Arrays.stream(dmlOptions.getFieldNames()).mapToInt(Arrays.asList(dmlOptions.getFieldNames())::indexOf).toArray();
+		LogicalType[] pkTypes = Arrays.stream(pkFields).mapToObj(f -> logicalTypes[f]).toArray(LogicalType[]::new);
+		String deleteSql = dmlOptions.getDialect().getDeleteStatement(dmlOptions.getTableName(), dmlOptions.getFieldNames());
+		return createKeyedRowExecutor(dmlOptions.getDialect(), pkFields, pkTypes, deleteSql, logicalTypes);
+	}
+
+	@Override
+	protected void addToBatch(RowData original, RowData extracted) throws SQLException {
+		switch (original.getRowKind()) {
+			case DELETE:
+				deleteExecutor.addToBatch(extracted);
+				break;
+			case INSERT:
+			case UPDATE_AFTER:
+				super.addToBatch(original, extracted);
+				break;
+			default:
+				return;
+		}
+	}
+
+	@Override
+	public synchronized void close() {
+		try {
+			super.close();
+		} finally {
+			try {
+				if (deleteExecutor != null) {
+					deleteExecutor.close();
+				}
+			} catch (SQLException e) {
+				LOG.warn("unable to close delete statement runner", e);
+			}
+		}
+	}
+
+	@Override
+	protected void attemptFlush() throws SQLException {
+		super.attemptFlush();
+		deleteExecutor.executeBatch();
+	}
+
+	private static JdbcBatchStatementExecutor<RowData> createKeyedRowExecutor(JdbcDialect dialect, int[] pkFields, LogicalType[] pkTypes, String sql, LogicalType[] logicalTypes) {
+		return JdbcBatchStatementExecutor.keyed(
+			sql,
+			createRowKeyExtractor(pkFields, logicalTypes),
+			(st, record) -> dialect
+				.getOutputConverter(logicalTypes)

Review comment:
       Hot path! 

##########
File path: flink-connectors/flink-connector-jdbc/src/main/java/org/apache/flink/connector/jdbc/table/JdbcDynamicOutputFormat.java
##########
@@ -0,0 +1,265 @@
+/*
+ * 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.connector.jdbc.table;
+
+import org.apache.flink.annotation.Internal;
+import org.apache.flink.api.common.functions.RuntimeContext;
+import org.apache.flink.api.common.typeinfo.TypeInformation;
+import org.apache.flink.api.common.typeutils.TypeSerializer;
+import org.apache.flink.connector.jdbc.JdbcExecutionOptions;
+import org.apache.flink.connector.jdbc.JdbcStatementBuilder;
+import org.apache.flink.connector.jdbc.dialect.JdbcDialect;
+import org.apache.flink.connector.jdbc.internal.JdbcBatchingOutputFormat;
+import org.apache.flink.connector.jdbc.internal.connection.JdbcConnectionProvider;
+import org.apache.flink.connector.jdbc.internal.connection.SimpleJdbcConnectionProvider;
+import org.apache.flink.connector.jdbc.internal.executor.InsertOrUpdateJdbcExecutor;
+import org.apache.flink.connector.jdbc.internal.executor.JdbcBatchStatementExecutor;
+import org.apache.flink.connector.jdbc.internal.options.JdbcDmlOptions;
+import org.apache.flink.connector.jdbc.internal.options.JdbcOptions;
+import org.apache.flink.connector.jdbc.utils.JdbcUtils;
+import org.apache.flink.table.data.GenericRowData;
+import org.apache.flink.table.data.RowData;
+import org.apache.flink.table.types.DataType;
+import org.apache.flink.table.types.logical.LogicalType;
+import org.apache.flink.types.Row;
+
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import java.io.IOException;
+import java.sql.SQLException;
+import java.util.Arrays;
+import java.util.function.Function;
+
+import static org.apache.flink.util.Preconditions.checkArgument;
+import static org.apache.flink.util.Preconditions.checkNotNull;
+
+/**
+ * OutputFormat for {@link JdbcDynamicTableSource}.
+ */
+@Internal
+public class JdbcDynamicOutputFormat extends JdbcBatchingOutputFormat<RowData, RowData, JdbcBatchStatementExecutor<RowData>> {
+	private static final Logger LOG = LoggerFactory.getLogger(JdbcDynamicOutputFormat.class);
+
+	private JdbcBatchStatementExecutor<RowData> deleteExecutor;
+	private final JdbcDmlOptions dmlOptions;
+	private final LogicalType[] logicalTypes;
+
+	private JdbcDynamicOutputFormat(JdbcConnectionProvider connectionProvider, JdbcDmlOptions dmlOptions, JdbcExecutionOptions batchOptions, TypeInformation<RowData> rowDataTypeInfo, LogicalType[] logicalTypes) {
+		super(connectionProvider, batchOptions, ctx -> createUpsertRowExecutor(dmlOptions, ctx, rowDataTypeInfo, logicalTypes), RecordExtractor.identity());
+		this.dmlOptions = dmlOptions;
+		this.logicalTypes = logicalTypes;
+	}
+
+	private JdbcDynamicOutputFormat(JdbcConnectionProvider connectionProvider, JdbcDmlOptions dmlOptions, JdbcExecutionOptions batchOptions, TypeInformation<RowData> rowDataTypeInfo, LogicalType[] logicalTypes, String sql) {
+		super(connectionProvider, batchOptions,
+			ctx -> createSimpleRowDataExecutor(dmlOptions.getDialect(), sql, logicalTypes, ctx, rowDataTypeInfo),
+			RecordExtractor.identity());
+		this.dmlOptions = dmlOptions;
+		this.logicalTypes = logicalTypes;
+	}
+
+	@Override
+	public void open(int taskNumber, int numTasks) throws IOException {
+		super.open(taskNumber, numTasks);
+		deleteExecutor = createDeleteExecutor();
+		try {
+			deleteExecutor.open(connection);
+		} catch (SQLException e) {
+			throw new IOException(e);
+		}
+	}
+
+	private JdbcBatchStatementExecutor<RowData> createDeleteExecutor() {
+		int[] pkFields = Arrays.stream(dmlOptions.getFieldNames()).mapToInt(Arrays.asList(dmlOptions.getFieldNames())::indexOf).toArray();
+		LogicalType[] pkTypes = Arrays.stream(pkFields).mapToObj(f -> logicalTypes[f]).toArray(LogicalType[]::new);
+		String deleteSql = dmlOptions.getDialect().getDeleteStatement(dmlOptions.getTableName(), dmlOptions.getFieldNames());
+		return createKeyedRowExecutor(dmlOptions.getDialect(), pkFields, pkTypes, deleteSql, logicalTypes);
+	}
+
+	@Override
+	protected void addToBatch(RowData original, RowData extracted) throws SQLException {
+		switch (original.getRowKind()) {
+			case DELETE:
+				deleteExecutor.addToBatch(extracted);
+				break;
+			case INSERT:
+			case UPDATE_AFTER:

Review comment:
       Move this before DELETE, because insertions are more than deletions. 

##########
File path: flink-connectors/flink-connector-jdbc/src/main/java/org/apache/flink/connector/jdbc/dialect/JdbcDialect.java
##########
@@ -43,11 +45,18 @@
 	boolean canHandle(String url);
 
 	/**
-	 * Get a row converter for the database according to the given row type.
+	 * Get converter that convert the jdbc object to flink internal object according to the given row type.
 	 * @param rowType the given row type
 	 * @return a row converter for the database
 	 */
-	JdbcRowConverter getRowConverter(RowType rowType);
+	JdbcToRowConverter getInputConverter(RowType rowType);
+
+	/**
+	 * Get converter that convert the flink internal object to jdbc object according to the given jdbc type.
+	 * @param jdbcTypes the given jdbc type
+	 * @return a row converter for the database
+	 */
+	RowToJdbcConverter getOutputConverter(LogicalType[] jdbcTypes);

Review comment:
       Could we merge these two interface into one? IMO, a dialect must implement both of them. 

##########
File path: flink-connectors/flink-connector-jdbc/src/main/java/org/apache/flink/connector/jdbc/table/JdbcDynamicTableSourceSinkFactory.java
##########
@@ -0,0 +1,283 @@
+/*
+ * 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.connector.jdbc.table;
+
+import org.apache.flink.annotation.Internal;
+import org.apache.flink.configuration.ConfigOption;
+import org.apache.flink.configuration.ConfigOptions;
+import org.apache.flink.configuration.ReadableConfig;
+import org.apache.flink.connector.jdbc.JdbcExecutionOptions;
+import org.apache.flink.connector.jdbc.dialect.JdbcDialects;
+import org.apache.flink.connector.jdbc.internal.options.JdbcDmlOptions;
+import org.apache.flink.connector.jdbc.internal.options.JdbcLookupOptions;
+import org.apache.flink.connector.jdbc.internal.options.JdbcOptions;
+import org.apache.flink.connector.jdbc.internal.options.JdbcReadOptions;
+import org.apache.flink.table.api.TableSchema;
+import org.apache.flink.table.connector.sink.DynamicTableSink;
+import org.apache.flink.table.connector.source.DynamicTableSource;
+import org.apache.flink.table.factories.DynamicTableSinkFactory;
+import org.apache.flink.table.factories.DynamicTableSourceFactory;
+import org.apache.flink.table.factories.FactoryUtil;
+import org.apache.flink.table.types.DataType;
+import org.apache.flink.table.utils.TableSchemaUtils;
+
+import java.time.Duration;
+import java.util.HashSet;
+import java.util.Optional;
+import java.util.Set;
+
+/**
+ * Factory for creating configured instances of {@link JdbcDynamicTableSource}
+ * and {@link JdbcDynamicTableSink}.
+ */
+@Internal
+public class JdbcDynamicTableSourceSinkFactory implements DynamicTableSourceFactory, DynamicTableSinkFactory {
+
+	public static final ConfigOption<String> IDENTIFIER = ConfigOptions
+		.key("connector")
+		.stringType()
+		.defaultValue("jdbc")
+		.withDescription("-- required: specify this table type is jdbc.");
+	public static final ConfigOption<String> URL = ConfigOptions
+		.key("url")
+		.stringType()
+		.noDefaultValue()
+		.withDescription("-- required: the jdbc database url.");
+	public static final ConfigOption<String> TABLE = ConfigOptions
+		.key("table")

Review comment:
       Use `table-name`, see https://issues.apache.org/jira/browse/FLINK-17029
   Please update other option keys according to the JIRA description. 

##########
File path: flink-connectors/flink-connector-jdbc/src/main/java/org/apache/flink/connector/jdbc/table/JdbcDynamicLookupFunction.java
##########
@@ -0,0 +1,293 @@
+/*
+ * 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.connector.jdbc.table;
+
+import org.apache.flink.annotation.Internal;
+import org.apache.flink.connector.jdbc.dialect.JdbcDialect;
+import org.apache.flink.connector.jdbc.dialect.JdbcDialects;
+import org.apache.flink.connector.jdbc.internal.options.JdbcLookupOptions;
+import org.apache.flink.connector.jdbc.internal.options.JdbcOptions;
+import org.apache.flink.connector.jdbc.utils.JdbcTypeUtil;
+import org.apache.flink.connector.jdbc.utils.JdbcUtils;
+import org.apache.flink.table.data.GenericRowData;
+import org.apache.flink.table.data.RowData;
+import org.apache.flink.table.functions.FunctionContext;
+import org.apache.flink.table.functions.TableFunction;
+import org.apache.flink.table.types.DataType;
+import org.apache.flink.table.types.logical.LogicalType;
+import org.apache.flink.table.types.logical.RowType;
+import org.apache.flink.table.types.utils.TypeConversions;
+
+import org.apache.flink.shaded.guava18.com.google.common.cache.Cache;
+import org.apache.flink.shaded.guava18.com.google.common.cache.CacheBuilder;
+
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import java.io.IOException;
+import java.sql.Connection;
+import java.sql.DriverManager;
+import java.sql.PreparedStatement;
+import java.sql.ResultSet;
+import java.sql.SQLException;
+import java.util.ArrayList;
+import java.util.Arrays;
+import java.util.List;
+import java.util.concurrent.TimeUnit;
+
+import static org.apache.flink.util.Preconditions.checkArgument;
+import static org.apache.flink.util.Preconditions.checkNotNull;
+
+/**
+ * A lookup function for {@link JdbcDynamicTableSource}.
+ */
+@Internal
+public class JdbcDynamicLookupFunction extends TableFunction<RowData> {
+
+	private static final Logger LOG = LoggerFactory.getLogger(JdbcDynamicLookupFunction.class);
+	private static final long serialVersionUID = 1L;
+
+	private final String query;
+	private final String drivername;
+	private final String dbURL;
+	private final String username;
+	private final String password;
+	private final DataType[] keyTypes;
+	private final int[] keySqlTypes;
+	private final long cacheMaxSize;
+	private final long cacheExpireMs;
+	private final int maxRetryTimes;
+	private final JdbcDialect jdbcDialect;
+	private final RowType rowType;
+
+	private transient Connection dbConn;
+	private transient PreparedStatement statement;
+	private transient Cache<RowData, List<RowData>> cache;
+
+	private JdbcDynamicLookupFunction(
+		JdbcOptions options, JdbcLookupOptions lookupOptions,
+		String[] fieldNames, DataType[] fieldTypes, String[] keyNames, RowType rowType) {
+		this.drivername = options.getDriverName();
+		this.dbURL = options.getDbURL();
+		this.username = options.getUsername().orElse(null);
+		this.password = options.getPassword().orElse(null);
+		List<String> nameList = Arrays.asList(fieldNames);
+		this.keyTypes = Arrays.stream(keyNames)
+			.map(s -> {
+				checkArgument(nameList.contains(s),
+					"keyName %s can't find in fieldNames %s.", s, nameList);
+				return fieldTypes[nameList.indexOf(s)];
+			})
+			.toArray(DataType[]::new);
+		this.cacheMaxSize = lookupOptions.getCacheMaxSize();
+		this.cacheExpireMs = lookupOptions.getCacheExpireMs();
+		this.maxRetryTimes = lookupOptions.getMaxRetryTimes();
+		this.keySqlTypes = Arrays.stream(keyTypes)
+			.map(TypeConversions::fromDataTypeToLegacyInfo)
+			.mapToInt(JdbcTypeUtil::typeInformationToSqlType).toArray();
+		this.query = options.getDialect().getSelectFromStatement(
+			options.getTableName(), fieldNames, keyNames);
+		this.jdbcDialect = JdbcDialects.get(dbURL)
+			.orElseThrow(() -> new UnsupportedOperationException(String.format("Unknown dbUrl:%s", dbURL)));
+		this.rowType = rowType;
+	}
+
+	public static Builder builder() {
+		return new Builder();
+	}
+
+	@Override
+	public void open(FunctionContext context) throws Exception {
+		try {
+			establishConnection();
+			statement = dbConn.prepareStatement(query);
+			this.cache = cacheMaxSize == -1 || cacheExpireMs == -1 ? null : CacheBuilder.newBuilder()
+				.expireAfterWrite(cacheExpireMs, TimeUnit.MILLISECONDS)
+				.maximumSize(cacheMaxSize)
+				.build();
+		} catch (SQLException sqe) {
+			throw new IllegalArgumentException("open() failed.", sqe);
+		} catch (ClassNotFoundException cnfe) {
+			throw new IllegalArgumentException("JDBC driver class not found.", cnfe);
+		}
+	}
+
+	public void eval(Object... keys) {
+		RowData keyRow = GenericRowData.of(keys);
+		if (cache != null) {
+			List<RowData> cachedRows = cache.getIfPresent(keyRow);
+			if (cachedRows != null) {
+				for (RowData cachedRow : cachedRows) {
+					collect(cachedRow);
+				}
+				return;
+			}
+		}
+
+		for (int retry = 1; retry <= maxRetryTimes; retry++) {
+			try {
+				statement.clearParameters();
+				for (int i = 0; i < keys.length; i++) {
+					JdbcUtils.setField(statement, keySqlTypes[i], keys[i], i);
+				}
+				try (ResultSet resultSet = statement.executeQuery()) {
+					if (cache == null) {
+						while (resultSet.next()) {
+							collect(convertToRowFromResultSet(resultSet));
+						}
+					} else {
+						ArrayList<RowData> rows = new ArrayList<>();
+						while (resultSet.next()) {
+							RowData row = convertToRowFromResultSet(resultSet);
+							rows.add(row);
+							collect(row);
+						}
+						rows.trimToSize();
+						cache.put(keyRow, rows);
+					}
+				}
+				break;
+			} catch (SQLException e) {
+				LOG.error(String.format("JDBC executeBatch error, retry times = %d", retry), e);
+				if (retry >= maxRetryTimes) {
+					throw new RuntimeException("Execution of JDBC statement failed.", e);
+				}
+
+				try {
+					Thread.sleep(1000 * retry);
+				} catch (InterruptedException e1) {
+					throw new RuntimeException(e1);
+				}
+			}
+		}
+	}
+
+	private RowData convertToRowFromResultSet(ResultSet resultSet) throws SQLException {
+		return jdbcDialect.getInputConverter(rowType).toInternal(resultSet);

Review comment:
       This is a hot path, we shouldn't call `getInputConverter` for every record which is a very heavy operation. 

##########
File path: flink-connectors/flink-connector-jdbc/src/main/java/org/apache/flink/connector/jdbc/table/JdbcDynamicInputFormat.java
##########
@@ -0,0 +1,402 @@
+/*
+ * 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.connector.jdbc.table;
+
+import org.apache.flink.annotation.Internal;
+import org.apache.flink.api.common.io.DefaultInputSplitAssigner;
+import org.apache.flink.api.common.io.InputFormat;
+import org.apache.flink.api.common.io.RichInputFormat;
+import org.apache.flink.api.common.io.statistics.BaseStatistics;
+import org.apache.flink.api.common.typeinfo.TypeInformation;
+import org.apache.flink.api.java.typeutils.GenericTypeInfo;
+import org.apache.flink.api.java.typeutils.ResultTypeQueryable;
+import org.apache.flink.api.java.typeutils.RowTypeInfo;
+import org.apache.flink.configuration.Configuration;
+import org.apache.flink.connector.jdbc.JdbcConnectionOptions;
+import org.apache.flink.connector.jdbc.internal.connection.SimpleJdbcConnectionProvider;
+import org.apache.flink.connector.jdbc.internal.converter.JdbcToRowConverter;
+import org.apache.flink.connector.jdbc.split.JdbcParameterValuesProvider;
+import org.apache.flink.core.io.GenericInputSplit;
+import org.apache.flink.core.io.InputSplit;
+import org.apache.flink.core.io.InputSplitAssigner;
+import org.apache.flink.table.data.RowData;
+import org.apache.flink.util.Preconditions;
+
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import java.io.IOException;
+import java.math.BigDecimal;
+import java.sql.Array;
+import java.sql.Connection;
+import java.sql.Date;
+import java.sql.PreparedStatement;
+import java.sql.ResultSet;
+import java.sql.SQLException;
+import java.sql.Time;
+import java.sql.Timestamp;
+import java.util.Arrays;
+
+/**
+ * InputFormat for {@link JdbcDynamicTableSource}.
+ */
+@Internal
+public class JdbcDynamicInputFormat extends RichInputFormat<RowData, InputSplit> implements ResultTypeQueryable<RowData> {

Review comment:
       Can we reuse with `JdbcInputFormat`?

##########
File path: flink-connectors/flink-connector-jdbc/src/main/java/org/apache/flink/connector/jdbc/table/JdbcDynamicInputFormat.java
##########
@@ -0,0 +1,402 @@
+/*
+ * 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.connector.jdbc.table;
+
+import org.apache.flink.annotation.Internal;
+import org.apache.flink.api.common.io.DefaultInputSplitAssigner;
+import org.apache.flink.api.common.io.InputFormat;
+import org.apache.flink.api.common.io.RichInputFormat;
+import org.apache.flink.api.common.io.statistics.BaseStatistics;
+import org.apache.flink.api.common.typeinfo.TypeInformation;
+import org.apache.flink.api.java.typeutils.GenericTypeInfo;
+import org.apache.flink.api.java.typeutils.ResultTypeQueryable;
+import org.apache.flink.api.java.typeutils.RowTypeInfo;
+import org.apache.flink.configuration.Configuration;
+import org.apache.flink.connector.jdbc.JdbcConnectionOptions;
+import org.apache.flink.connector.jdbc.internal.connection.SimpleJdbcConnectionProvider;
+import org.apache.flink.connector.jdbc.internal.converter.JdbcToRowConverter;
+import org.apache.flink.connector.jdbc.split.JdbcParameterValuesProvider;
+import org.apache.flink.core.io.GenericInputSplit;
+import org.apache.flink.core.io.InputSplit;
+import org.apache.flink.core.io.InputSplitAssigner;
+import org.apache.flink.table.data.RowData;
+import org.apache.flink.util.Preconditions;
+
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import java.io.IOException;
+import java.math.BigDecimal;
+import java.sql.Array;
+import java.sql.Connection;
+import java.sql.Date;
+import java.sql.PreparedStatement;
+import java.sql.ResultSet;
+import java.sql.SQLException;
+import java.sql.Time;
+import java.sql.Timestamp;
+import java.util.Arrays;
+
+/**
+ * InputFormat for {@link JdbcDynamicTableSource}.
+ */
+@Internal
+public class JdbcDynamicInputFormat extends RichInputFormat<RowData, InputSplit> implements ResultTypeQueryable<RowData> {
+
+	private static final long serialVersionUID = 1L;
+	private static final Logger LOG = LoggerFactory.getLogger(JdbcDynamicInputFormat.class);
+
+	private JdbcConnectionOptions connectionOptions;
+	private int fetchSize;
+	private Boolean autoCommit;
+	private Object[][] parameterValues;
+	private String queryTemplate;
+	private int resultSetType;
+	private int resultSetConcurrency;
+	private JdbcToRowConverter rowConverter;
+
+	private transient Connection dbConn;
+	private transient PreparedStatement statement;
+	private transient ResultSet resultSet;
+	private transient boolean hasNext;
+
+	private JdbcDynamicInputFormat(
+		JdbcConnectionOptions connectionOptions,
+		int fetchSize,
+		Boolean autoCommit,
+		Object[][] parameterValues,
+		String queryTemplate,
+		int resultSetType,
+		int resultSetConcurrency,
+		JdbcToRowConverter rowConverter) {

Review comment:
       Indent. 

##########
File path: flink-connectors/flink-connector-jdbc/src/main/java/org/apache/flink/connector/jdbc/table/JdbcDynamicLookupFunction.java
##########
@@ -0,0 +1,293 @@
+/*
+ * 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.connector.jdbc.table;
+
+import org.apache.flink.annotation.Internal;
+import org.apache.flink.connector.jdbc.dialect.JdbcDialect;
+import org.apache.flink.connector.jdbc.dialect.JdbcDialects;
+import org.apache.flink.connector.jdbc.internal.options.JdbcLookupOptions;
+import org.apache.flink.connector.jdbc.internal.options.JdbcOptions;
+import org.apache.flink.connector.jdbc.utils.JdbcTypeUtil;
+import org.apache.flink.connector.jdbc.utils.JdbcUtils;
+import org.apache.flink.table.data.GenericRowData;
+import org.apache.flink.table.data.RowData;
+import org.apache.flink.table.functions.FunctionContext;
+import org.apache.flink.table.functions.TableFunction;
+import org.apache.flink.table.types.DataType;
+import org.apache.flink.table.types.logical.LogicalType;
+import org.apache.flink.table.types.logical.RowType;
+import org.apache.flink.table.types.utils.TypeConversions;
+
+import org.apache.flink.shaded.guava18.com.google.common.cache.Cache;
+import org.apache.flink.shaded.guava18.com.google.common.cache.CacheBuilder;
+
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import java.io.IOException;
+import java.sql.Connection;
+import java.sql.DriverManager;
+import java.sql.PreparedStatement;
+import java.sql.ResultSet;
+import java.sql.SQLException;
+import java.util.ArrayList;
+import java.util.Arrays;
+import java.util.List;
+import java.util.concurrent.TimeUnit;
+
+import static org.apache.flink.util.Preconditions.checkArgument;
+import static org.apache.flink.util.Preconditions.checkNotNull;
+
+/**
+ * A lookup function for {@link JdbcDynamicTableSource}.
+ */
+@Internal
+public class JdbcDynamicLookupFunction extends TableFunction<RowData> {
+
+	private static final Logger LOG = LoggerFactory.getLogger(JdbcDynamicLookupFunction.class);
+	private static final long serialVersionUID = 1L;
+
+	private final String query;
+	private final String drivername;
+	private final String dbURL;
+	private final String username;
+	private final String password;
+	private final DataType[] keyTypes;
+	private final int[] keySqlTypes;
+	private final long cacheMaxSize;
+	private final long cacheExpireMs;
+	private final int maxRetryTimes;
+	private final JdbcDialect jdbcDialect;
+	private final RowType rowType;
+
+	private transient Connection dbConn;
+	private transient PreparedStatement statement;
+	private transient Cache<RowData, List<RowData>> cache;
+
+	private JdbcDynamicLookupFunction(
+		JdbcOptions options, JdbcLookupOptions lookupOptions,
+		String[] fieldNames, DataType[] fieldTypes, String[] keyNames, RowType rowType) {
+		this.drivername = options.getDriverName();
+		this.dbURL = options.getDbURL();
+		this.username = options.getUsername().orElse(null);
+		this.password = options.getPassword().orElse(null);
+		List<String> nameList = Arrays.asList(fieldNames);
+		this.keyTypes = Arrays.stream(keyNames)
+			.map(s -> {
+				checkArgument(nameList.contains(s),
+					"keyName %s can't find in fieldNames %s.", s, nameList);
+				return fieldTypes[nameList.indexOf(s)];
+			})
+			.toArray(DataType[]::new);
+		this.cacheMaxSize = lookupOptions.getCacheMaxSize();
+		this.cacheExpireMs = lookupOptions.getCacheExpireMs();
+		this.maxRetryTimes = lookupOptions.getMaxRetryTimes();
+		this.keySqlTypes = Arrays.stream(keyTypes)
+			.map(TypeConversions::fromDataTypeToLegacyInfo)
+			.mapToInt(JdbcTypeUtil::typeInformationToSqlType).toArray();
+		this.query = options.getDialect().getSelectFromStatement(
+			options.getTableName(), fieldNames, keyNames);
+		this.jdbcDialect = JdbcDialects.get(dbURL)
+			.orElseThrow(() -> new UnsupportedOperationException(String.format("Unknown dbUrl:%s", dbURL)));
+		this.rowType = rowType;
+	}
+
+	public static Builder builder() {
+		return new Builder();
+	}
+
+	@Override
+	public void open(FunctionContext context) throws Exception {
+		try {
+			establishConnection();
+			statement = dbConn.prepareStatement(query);
+			this.cache = cacheMaxSize == -1 || cacheExpireMs == -1 ? null : CacheBuilder.newBuilder()
+				.expireAfterWrite(cacheExpireMs, TimeUnit.MILLISECONDS)
+				.maximumSize(cacheMaxSize)
+				.build();
+		} catch (SQLException sqe) {
+			throw new IllegalArgumentException("open() failed.", sqe);
+		} catch (ClassNotFoundException cnfe) {
+			throw new IllegalArgumentException("JDBC driver class not found.", cnfe);
+		}
+	}
+
+	public void eval(Object... keys) {
+		RowData keyRow = GenericRowData.of(keys);
+		if (cache != null) {
+			List<RowData> cachedRows = cache.getIfPresent(keyRow);
+			if (cachedRows != null) {
+				for (RowData cachedRow : cachedRows) {
+					collect(cachedRow);
+				}
+				return;
+			}
+		}
+
+		for (int retry = 1; retry <= maxRetryTimes; retry++) {
+			try {
+				statement.clearParameters();
+				for (int i = 0; i < keys.length; i++) {
+					JdbcUtils.setField(statement, keySqlTypes[i], keys[i], i);

Review comment:
       Why not use the jdbc converter from dialect?

##########
File path: flink-connectors/flink-connector-jdbc/src/main/java/org/apache/flink/connector/jdbc/table/JdbcDynamicLookupFunction.java
##########
@@ -0,0 +1,293 @@
+/*
+ * 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.connector.jdbc.table;
+
+import org.apache.flink.annotation.Internal;
+import org.apache.flink.connector.jdbc.dialect.JdbcDialect;
+import org.apache.flink.connector.jdbc.dialect.JdbcDialects;
+import org.apache.flink.connector.jdbc.internal.options.JdbcLookupOptions;
+import org.apache.flink.connector.jdbc.internal.options.JdbcOptions;
+import org.apache.flink.connector.jdbc.utils.JdbcTypeUtil;
+import org.apache.flink.connector.jdbc.utils.JdbcUtils;
+import org.apache.flink.table.data.GenericRowData;
+import org.apache.flink.table.data.RowData;
+import org.apache.flink.table.functions.FunctionContext;
+import org.apache.flink.table.functions.TableFunction;
+import org.apache.flink.table.types.DataType;
+import org.apache.flink.table.types.logical.LogicalType;
+import org.apache.flink.table.types.logical.RowType;
+import org.apache.flink.table.types.utils.TypeConversions;
+
+import org.apache.flink.shaded.guava18.com.google.common.cache.Cache;
+import org.apache.flink.shaded.guava18.com.google.common.cache.CacheBuilder;
+
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import java.io.IOException;
+import java.sql.Connection;
+import java.sql.DriverManager;
+import java.sql.PreparedStatement;
+import java.sql.ResultSet;
+import java.sql.SQLException;
+import java.util.ArrayList;
+import java.util.Arrays;
+import java.util.List;
+import java.util.concurrent.TimeUnit;
+
+import static org.apache.flink.util.Preconditions.checkArgument;
+import static org.apache.flink.util.Preconditions.checkNotNull;
+
+/**
+ * A lookup function for {@link JdbcDynamicTableSource}.
+ */
+@Internal
+public class JdbcDynamicLookupFunction extends TableFunction<RowData> {
+
+	private static final Logger LOG = LoggerFactory.getLogger(JdbcDynamicLookupFunction.class);
+	private static final long serialVersionUID = 1L;
+
+	private final String query;
+	private final String drivername;
+	private final String dbURL;
+	private final String username;
+	private final String password;
+	private final DataType[] keyTypes;
+	private final int[] keySqlTypes;
+	private final long cacheMaxSize;
+	private final long cacheExpireMs;
+	private final int maxRetryTimes;
+	private final JdbcDialect jdbcDialect;
+	private final RowType rowType;
+
+	private transient Connection dbConn;
+	private transient PreparedStatement statement;
+	private transient Cache<RowData, List<RowData>> cache;
+
+	private JdbcDynamicLookupFunction(
+		JdbcOptions options, JdbcLookupOptions lookupOptions,
+		String[] fieldNames, DataType[] fieldTypes, String[] keyNames, RowType rowType) {
+		this.drivername = options.getDriverName();
+		this.dbURL = options.getDbURL();
+		this.username = options.getUsername().orElse(null);
+		this.password = options.getPassword().orElse(null);
+		List<String> nameList = Arrays.asList(fieldNames);
+		this.keyTypes = Arrays.stream(keyNames)
+			.map(s -> {
+				checkArgument(nameList.contains(s),
+					"keyName %s can't find in fieldNames %s.", s, nameList);
+				return fieldTypes[nameList.indexOf(s)];
+			})
+			.toArray(DataType[]::new);
+		this.cacheMaxSize = lookupOptions.getCacheMaxSize();
+		this.cacheExpireMs = lookupOptions.getCacheExpireMs();
+		this.maxRetryTimes = lookupOptions.getMaxRetryTimes();
+		this.keySqlTypes = Arrays.stream(keyTypes)
+			.map(TypeConversions::fromDataTypeToLegacyInfo)
+			.mapToInt(JdbcTypeUtil::typeInformationToSqlType).toArray();
+		this.query = options.getDialect().getSelectFromStatement(
+			options.getTableName(), fieldNames, keyNames);
+		this.jdbcDialect = JdbcDialects.get(dbURL)
+			.orElseThrow(() -> new UnsupportedOperationException(String.format("Unknown dbUrl:%s", dbURL)));
+		this.rowType = rowType;
+	}
+
+	public static Builder builder() {
+		return new Builder();
+	}
+
+	@Override
+	public void open(FunctionContext context) throws Exception {
+		try {
+			establishConnection();
+			statement = dbConn.prepareStatement(query);
+			this.cache = cacheMaxSize == -1 || cacheExpireMs == -1 ? null : CacheBuilder.newBuilder()
+				.expireAfterWrite(cacheExpireMs, TimeUnit.MILLISECONDS)
+				.maximumSize(cacheMaxSize)
+				.build();
+		} catch (SQLException sqe) {
+			throw new IllegalArgumentException("open() failed.", sqe);
+		} catch (ClassNotFoundException cnfe) {
+			throw new IllegalArgumentException("JDBC driver class not found.", cnfe);
+		}
+	}
+
+	public void eval(Object... keys) {

Review comment:
       Add a Javadoc on this method. This is a lookup method which is called by Flink framework in runtime. 

##########
File path: flink-connectors/flink-connector-jdbc/src/main/java/org/apache/flink/connector/jdbc/table/JdbcDynamicLookupFunction.java
##########
@@ -0,0 +1,293 @@
+/*
+ * 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.connector.jdbc.table;
+
+import org.apache.flink.annotation.Internal;
+import org.apache.flink.connector.jdbc.dialect.JdbcDialect;
+import org.apache.flink.connector.jdbc.dialect.JdbcDialects;
+import org.apache.flink.connector.jdbc.internal.options.JdbcLookupOptions;
+import org.apache.flink.connector.jdbc.internal.options.JdbcOptions;
+import org.apache.flink.connector.jdbc.utils.JdbcTypeUtil;
+import org.apache.flink.connector.jdbc.utils.JdbcUtils;
+import org.apache.flink.table.data.GenericRowData;
+import org.apache.flink.table.data.RowData;
+import org.apache.flink.table.functions.FunctionContext;
+import org.apache.flink.table.functions.TableFunction;
+import org.apache.flink.table.types.DataType;
+import org.apache.flink.table.types.logical.LogicalType;
+import org.apache.flink.table.types.logical.RowType;
+import org.apache.flink.table.types.utils.TypeConversions;
+
+import org.apache.flink.shaded.guava18.com.google.common.cache.Cache;
+import org.apache.flink.shaded.guava18.com.google.common.cache.CacheBuilder;
+
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import java.io.IOException;
+import java.sql.Connection;
+import java.sql.DriverManager;
+import java.sql.PreparedStatement;
+import java.sql.ResultSet;
+import java.sql.SQLException;
+import java.util.ArrayList;
+import java.util.Arrays;
+import java.util.List;
+import java.util.concurrent.TimeUnit;
+
+import static org.apache.flink.util.Preconditions.checkArgument;
+import static org.apache.flink.util.Preconditions.checkNotNull;
+
+/**
+ * A lookup function for {@link JdbcDynamicTableSource}.
+ */
+@Internal
+public class JdbcDynamicLookupFunction extends TableFunction<RowData> {
+
+	private static final Logger LOG = LoggerFactory.getLogger(JdbcDynamicLookupFunction.class);
+	private static final long serialVersionUID = 1L;
+
+	private final String query;
+	private final String drivername;
+	private final String dbURL;
+	private final String username;
+	private final String password;
+	private final DataType[] keyTypes;
+	private final int[] keySqlTypes;
+	private final long cacheMaxSize;
+	private final long cacheExpireMs;
+	private final int maxRetryTimes;
+	private final JdbcDialect jdbcDialect;
+	private final RowType rowType;
+
+	private transient Connection dbConn;
+	private transient PreparedStatement statement;
+	private transient Cache<RowData, List<RowData>> cache;
+
+	private JdbcDynamicLookupFunction(
+		JdbcOptions options, JdbcLookupOptions lookupOptions,
+		String[] fieldNames, DataType[] fieldTypes, String[] keyNames, RowType rowType) {
+		this.drivername = options.getDriverName();
+		this.dbURL = options.getDbURL();
+		this.username = options.getUsername().orElse(null);
+		this.password = options.getPassword().orElse(null);
+		List<String> nameList = Arrays.asList(fieldNames);
+		this.keyTypes = Arrays.stream(keyNames)
+			.map(s -> {
+				checkArgument(nameList.contains(s),
+					"keyName %s can't find in fieldNames %s.", s, nameList);
+				return fieldTypes[nameList.indexOf(s)];
+			})
+			.toArray(DataType[]::new);
+		this.cacheMaxSize = lookupOptions.getCacheMaxSize();
+		this.cacheExpireMs = lookupOptions.getCacheExpireMs();
+		this.maxRetryTimes = lookupOptions.getMaxRetryTimes();
+		this.keySqlTypes = Arrays.stream(keyTypes)
+			.map(TypeConversions::fromDataTypeToLegacyInfo)
+			.mapToInt(JdbcTypeUtil::typeInformationToSqlType).toArray();
+		this.query = options.getDialect().getSelectFromStatement(
+			options.getTableName(), fieldNames, keyNames);
+		this.jdbcDialect = JdbcDialects.get(dbURL)
+			.orElseThrow(() -> new UnsupportedOperationException(String.format("Unknown dbUrl:%s", dbURL)));
+		this.rowType = rowType;
+	}
+
+	public static Builder builder() {
+		return new Builder();
+	}
+
+	@Override
+	public void open(FunctionContext context) throws Exception {
+		try {
+			establishConnection();
+			statement = dbConn.prepareStatement(query);
+			this.cache = cacheMaxSize == -1 || cacheExpireMs == -1 ? null : CacheBuilder.newBuilder()
+				.expireAfterWrite(cacheExpireMs, TimeUnit.MILLISECONDS)
+				.maximumSize(cacheMaxSize)
+				.build();
+		} catch (SQLException sqe) {
+			throw new IllegalArgumentException("open() failed.", sqe);
+		} catch (ClassNotFoundException cnfe) {
+			throw new IllegalArgumentException("JDBC driver class not found.", cnfe);
+		}
+	}
+
+	public void eval(Object... keys) {
+		RowData keyRow = GenericRowData.of(keys);
+		if (cache != null) {
+			List<RowData> cachedRows = cache.getIfPresent(keyRow);
+			if (cachedRows != null) {
+				for (RowData cachedRow : cachedRows) {
+					collect(cachedRow);
+				}
+				return;
+			}
+		}
+
+		for (int retry = 1; retry <= maxRetryTimes; retry++) {
+			try {
+				statement.clearParameters();
+				for (int i = 0; i < keys.length; i++) {
+					JdbcUtils.setField(statement, keySqlTypes[i], keys[i], i);
+				}
+				try (ResultSet resultSet = statement.executeQuery()) {
+					if (cache == null) {
+						while (resultSet.next()) {
+							collect(convertToRowFromResultSet(resultSet));
+						}
+					} else {
+						ArrayList<RowData> rows = new ArrayList<>();
+						while (resultSet.next()) {
+							RowData row = convertToRowFromResultSet(resultSet);
+							rows.add(row);
+							collect(row);
+						}
+						rows.trimToSize();
+						cache.put(keyRow, rows);
+					}
+				}
+				break;
+			} catch (SQLException e) {
+				LOG.error(String.format("JDBC executeBatch error, retry times = %d", retry), e);
+				if (retry >= maxRetryTimes) {
+					throw new RuntimeException("Execution of JDBC statement failed.", e);
+				}
+
+				try {
+					Thread.sleep(1000 * retry);
+				} catch (InterruptedException e1) {
+					throw new RuntimeException(e1);
+				}
+			}
+		}
+	}
+
+	private RowData convertToRowFromResultSet(ResultSet resultSet) throws SQLException {
+		return jdbcDialect.getInputConverter(rowType).toInternal(resultSet);
+	}
+
+	private void establishConnection() throws SQLException, ClassNotFoundException {
+		Class.forName(drivername);
+		if (username == null) {
+			dbConn = DriverManager.getConnection(dbURL);
+		} else {
+			dbConn = DriverManager.getConnection(dbURL, username, password);
+		}
+	}
+
+	@Override
+	public void close() throws IOException {
+		if (cache != null) {
+			cache.cleanUp();
+			cache = null;
+		}
+		if (statement != null) {
+			try {
+				statement.close();
+			} catch (SQLException e) {
+				LOG.info("JDBC statement could not be closed: " + e.getMessage());
+			} finally {
+				statement = null;
+			}
+		}
+
+		if (dbConn != null) {
+			try {
+				dbConn.close();
+			} catch (SQLException se) {
+				LOG.info("JDBC connection could not be closed: " + se.getMessage());
+			} finally {
+				dbConn = null;
+			}
+		}
+	}
+
+	/**
+	 * Builder for a {@link JdbcDynamicLookupFunction}.
+	 */
+	public static class Builder {

Review comment:
       We don't need a Builder for this lookup function. Because this is an internal class which is only be called by factory. 

##########
File path: flink-connectors/flink-connector-jdbc/src/main/java/org/apache/flink/connector/jdbc/table/JdbcDynamicOutputFormat.java
##########
@@ -0,0 +1,265 @@
+/*
+ * 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.connector.jdbc.table;
+
+import org.apache.flink.annotation.Internal;
+import org.apache.flink.api.common.functions.RuntimeContext;
+import org.apache.flink.api.common.typeinfo.TypeInformation;
+import org.apache.flink.api.common.typeutils.TypeSerializer;
+import org.apache.flink.connector.jdbc.JdbcExecutionOptions;
+import org.apache.flink.connector.jdbc.JdbcStatementBuilder;
+import org.apache.flink.connector.jdbc.dialect.JdbcDialect;
+import org.apache.flink.connector.jdbc.internal.JdbcBatchingOutputFormat;
+import org.apache.flink.connector.jdbc.internal.connection.JdbcConnectionProvider;
+import org.apache.flink.connector.jdbc.internal.connection.SimpleJdbcConnectionProvider;
+import org.apache.flink.connector.jdbc.internal.executor.InsertOrUpdateJdbcExecutor;
+import org.apache.flink.connector.jdbc.internal.executor.JdbcBatchStatementExecutor;
+import org.apache.flink.connector.jdbc.internal.options.JdbcDmlOptions;
+import org.apache.flink.connector.jdbc.internal.options.JdbcOptions;
+import org.apache.flink.connector.jdbc.utils.JdbcUtils;
+import org.apache.flink.table.data.GenericRowData;
+import org.apache.flink.table.data.RowData;
+import org.apache.flink.table.types.DataType;
+import org.apache.flink.table.types.logical.LogicalType;
+import org.apache.flink.types.Row;
+
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import java.io.IOException;
+import java.sql.SQLException;
+import java.util.Arrays;
+import java.util.function.Function;
+
+import static org.apache.flink.util.Preconditions.checkArgument;
+import static org.apache.flink.util.Preconditions.checkNotNull;
+
+/**
+ * OutputFormat for {@link JdbcDynamicTableSource}.
+ */
+@Internal
+public class JdbcDynamicOutputFormat extends JdbcBatchingOutputFormat<RowData, RowData, JdbcBatchStatementExecutor<RowData>> {
+	private static final Logger LOG = LoggerFactory.getLogger(JdbcDynamicOutputFormat.class);
+
+	private JdbcBatchStatementExecutor<RowData> deleteExecutor;
+	private final JdbcDmlOptions dmlOptions;
+	private final LogicalType[] logicalTypes;
+
+	private JdbcDynamicOutputFormat(JdbcConnectionProvider connectionProvider, JdbcDmlOptions dmlOptions, JdbcExecutionOptions batchOptions, TypeInformation<RowData> rowDataTypeInfo, LogicalType[] logicalTypes) {
+		super(connectionProvider, batchOptions, ctx -> createUpsertRowExecutor(dmlOptions, ctx, rowDataTypeInfo, logicalTypes), RecordExtractor.identity());
+		this.dmlOptions = dmlOptions;
+		this.logicalTypes = logicalTypes;
+	}
+
+	private JdbcDynamicOutputFormat(JdbcConnectionProvider connectionProvider, JdbcDmlOptions dmlOptions, JdbcExecutionOptions batchOptions, TypeInformation<RowData> rowDataTypeInfo, LogicalType[] logicalTypes, String sql) {
+		super(connectionProvider, batchOptions,
+			ctx -> createSimpleRowDataExecutor(dmlOptions.getDialect(), sql, logicalTypes, ctx, rowDataTypeInfo),
+			RecordExtractor.identity());
+		this.dmlOptions = dmlOptions;
+		this.logicalTypes = logicalTypes;
+	}
+
+	@Override
+	public void open(int taskNumber, int numTasks) throws IOException {
+		super.open(taskNumber, numTasks);
+		deleteExecutor = createDeleteExecutor();
+		try {
+			deleteExecutor.open(connection);
+		} catch (SQLException e) {
+			throw new IOException(e);
+		}
+	}
+
+	private JdbcBatchStatementExecutor<RowData> createDeleteExecutor() {
+		int[] pkFields = Arrays.stream(dmlOptions.getFieldNames()).mapToInt(Arrays.asList(dmlOptions.getFieldNames())::indexOf).toArray();
+		LogicalType[] pkTypes = Arrays.stream(pkFields).mapToObj(f -> logicalTypes[f]).toArray(LogicalType[]::new);
+		String deleteSql = dmlOptions.getDialect().getDeleteStatement(dmlOptions.getTableName(), dmlOptions.getFieldNames());
+		return createKeyedRowExecutor(dmlOptions.getDialect(), pkFields, pkTypes, deleteSql, logicalTypes);
+	}
+
+	@Override
+	protected void addToBatch(RowData original, RowData extracted) throws SQLException {
+		switch (original.getRowKind()) {
+			case DELETE:
+				deleteExecutor.addToBatch(extracted);
+				break;
+			case INSERT:
+			case UPDATE_AFTER:
+				super.addToBatch(original, extracted);
+				break;
+			default:
+				return;
+		}
+	}
+
+	@Override
+	public synchronized void close() {
+		try {
+			super.close();
+		} finally {
+			try {
+				if (deleteExecutor != null) {
+					deleteExecutor.close();
+				}
+			} catch (SQLException e) {
+				LOG.warn("unable to close delete statement runner", e);
+			}
+		}
+	}
+
+	@Override
+	protected void attemptFlush() throws SQLException {
+		super.attemptFlush();
+		deleteExecutor.executeBatch();
+	}
+
+	private static JdbcBatchStatementExecutor<RowData> createKeyedRowExecutor(JdbcDialect dialect, int[] pkFields, LogicalType[] pkTypes, String sql, LogicalType[] logicalTypes) {
+		return JdbcBatchStatementExecutor.keyed(
+			sql,
+			createRowKeyExtractor(pkFields, logicalTypes),
+			(st, record) -> dialect
+				.getOutputConverter(logicalTypes)
+				.toExternal(createRowKeyExtractor(pkFields, logicalTypes).apply(record), st));
+	}
+
+	private static JdbcBatchStatementExecutor<RowData> createUpsertRowExecutor(JdbcDmlOptions opt, RuntimeContext ctx, TypeInformation<RowData> rowDataTypeInfo, LogicalType[] logicalTypes) {
+		checkArgument(opt.getKeyFields().isPresent());
+
+		int[] pkFields = Arrays.stream(opt.getKeyFields().get()).mapToInt(Arrays.asList(opt.getFieldNames())::indexOf).toArray();
+		LogicalType[] pkTypes = Arrays.stream(pkFields).mapToObj(f -> logicalTypes[f]).toArray(LogicalType[]::new);
+		JdbcDialect dialect = opt.getDialect();
+		final TypeSerializer<RowData> typeSerializer = rowDataTypeInfo.createSerializer(ctx.getExecutionConfig());
+		return opt.getDialect()
+			.getUpsertStatement(opt.getTableName(), opt.getFieldNames(), opt.getKeyFields().get())
+			.map(sql -> createSimpleRowDataExecutor(dialect, sql, logicalTypes, ctx, rowDataTypeInfo))
+			.orElseGet(() ->
+				new InsertOrUpdateJdbcExecutor<>(
+					opt.getDialect().getRowExistsStatement(opt.getTableName(), opt.getKeyFields().get()),
+					opt.getDialect().getInsertIntoStatement(opt.getTableName(), opt.getFieldNames()),
+					opt.getDialect().getUpdateStatement(opt.getTableName(), opt.getFieldNames(), opt.getKeyFields().get()),
+					createRowDataJdbcStatementBuilder(dialect, pkTypes),
+					createRowDataJdbcStatementBuilder(dialect, logicalTypes),
+					createRowDataJdbcStatementBuilder(dialect, logicalTypes),
+					createRowKeyExtractor(pkFields, logicalTypes),
+					ctx.getExecutionConfig().isObjectReuseEnabled() ? typeSerializer::copy : r -> r));
+	}
+
+	private static Function<RowData, RowData> createRowKeyExtractor(int[] pkFields, LogicalType[] logicalTypes) {
+		return row -> getPrimaryKey(row, pkFields, logicalTypes);
+	}
+
+	private static JdbcBatchStatementExecutor<RowData> createSimpleRowDataExecutor(JdbcDialect dialect, String sql, LogicalType[] fieldTypes, RuntimeContext ctx, TypeInformation<RowData> rowDataTypeInfo) {
+		final TypeSerializer<RowData> typeSerializer = rowDataTypeInfo.createSerializer(ctx.getExecutionConfig());
+		return JdbcBatchStatementExecutor.simple(
+			sql,
+			createRowDataJdbcStatementBuilder(dialect, fieldTypes),
+			ctx.getExecutionConfig().isObjectReuseEnabled() ? typeSerializer::copy : Function.identity());
+	}
+
+	/**
+	 * Creates a {@link JdbcStatementBuilder} for {@link Row} using the provided SQL types array.
+	 * Uses {@link JdbcUtils#setRecordToStatement}
+	 */
+	private static JdbcStatementBuilder<RowData> createRowDataJdbcStatementBuilder(JdbcDialect dialect, LogicalType[] types) {
+		return (st, record) -> dialect.getOutputConverter(types).toExternal(record, st);
+	}
+
+	private static RowData getPrimaryKey(RowData row, int[] pkFields, LogicalType[] logicalTypes) {
+		GenericRowData pkRow = new GenericRowData(pkFields.length);
+		for (int i = 0; i < pkFields.length; i++) {
+			pkRow.setField(i, RowData.get(row, pkFields[i], logicalTypes[pkFields[i]]));
+		}
+		return pkRow;
+	}
+
+	public static DynamicOutputFormatBuilder dynamicOutputFormatBuilder() {
+		return new DynamicOutputFormatBuilder();
+	}
+
+	/**
+	 * Builder for {@link JdbcDynamicOutputFormat}.
+	 */
+	public static class DynamicOutputFormatBuilder {

Review comment:
       We don't need Builder for this class. 

##########
File path: flink-connectors/flink-connector-jdbc/src/main/java/org/apache/flink/connector/jdbc/table/JdbcDynamicOutputFormat.java
##########
@@ -0,0 +1,265 @@
+/*
+ * 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.connector.jdbc.table;
+
+import org.apache.flink.annotation.Internal;
+import org.apache.flink.api.common.functions.RuntimeContext;
+import org.apache.flink.api.common.typeinfo.TypeInformation;
+import org.apache.flink.api.common.typeutils.TypeSerializer;
+import org.apache.flink.connector.jdbc.JdbcExecutionOptions;
+import org.apache.flink.connector.jdbc.JdbcStatementBuilder;
+import org.apache.flink.connector.jdbc.dialect.JdbcDialect;
+import org.apache.flink.connector.jdbc.internal.JdbcBatchingOutputFormat;
+import org.apache.flink.connector.jdbc.internal.connection.JdbcConnectionProvider;
+import org.apache.flink.connector.jdbc.internal.connection.SimpleJdbcConnectionProvider;
+import org.apache.flink.connector.jdbc.internal.executor.InsertOrUpdateJdbcExecutor;
+import org.apache.flink.connector.jdbc.internal.executor.JdbcBatchStatementExecutor;
+import org.apache.flink.connector.jdbc.internal.options.JdbcDmlOptions;
+import org.apache.flink.connector.jdbc.internal.options.JdbcOptions;
+import org.apache.flink.connector.jdbc.utils.JdbcUtils;
+import org.apache.flink.table.data.GenericRowData;
+import org.apache.flink.table.data.RowData;
+import org.apache.flink.table.types.DataType;
+import org.apache.flink.table.types.logical.LogicalType;
+import org.apache.flink.types.Row;
+
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import java.io.IOException;
+import java.sql.SQLException;
+import java.util.Arrays;
+import java.util.function.Function;
+
+import static org.apache.flink.util.Preconditions.checkArgument;
+import static org.apache.flink.util.Preconditions.checkNotNull;
+
+/**
+ * OutputFormat for {@link JdbcDynamicTableSource}.
+ */
+@Internal
+public class JdbcDynamicOutputFormat extends JdbcBatchingOutputFormat<RowData, RowData, JdbcBatchStatementExecutor<RowData>> {
+	private static final Logger LOG = LoggerFactory.getLogger(JdbcDynamicOutputFormat.class);
+
+	private JdbcBatchStatementExecutor<RowData> deleteExecutor;
+	private final JdbcDmlOptions dmlOptions;
+	private final LogicalType[] logicalTypes;
+
+	private JdbcDynamicOutputFormat(JdbcConnectionProvider connectionProvider, JdbcDmlOptions dmlOptions, JdbcExecutionOptions batchOptions, TypeInformation<RowData> rowDataTypeInfo, LogicalType[] logicalTypes) {
+		super(connectionProvider, batchOptions, ctx -> createUpsertRowExecutor(dmlOptions, ctx, rowDataTypeInfo, logicalTypes), RecordExtractor.identity());
+		this.dmlOptions = dmlOptions;
+		this.logicalTypes = logicalTypes;
+	}
+
+	private JdbcDynamicOutputFormat(JdbcConnectionProvider connectionProvider, JdbcDmlOptions dmlOptions, JdbcExecutionOptions batchOptions, TypeInformation<RowData> rowDataTypeInfo, LogicalType[] logicalTypes, String sql) {

Review comment:
       Break the parameters into separate lines if long. 

##########
File path: flink-connectors/flink-connector-jdbc/src/main/java/org/apache/flink/connector/jdbc/table/JdbcDynamicOutputFormat.java
##########
@@ -0,0 +1,265 @@
+/*
+ * 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.connector.jdbc.table;
+
+import org.apache.flink.annotation.Internal;
+import org.apache.flink.api.common.functions.RuntimeContext;
+import org.apache.flink.api.common.typeinfo.TypeInformation;
+import org.apache.flink.api.common.typeutils.TypeSerializer;
+import org.apache.flink.connector.jdbc.JdbcExecutionOptions;
+import org.apache.flink.connector.jdbc.JdbcStatementBuilder;
+import org.apache.flink.connector.jdbc.dialect.JdbcDialect;
+import org.apache.flink.connector.jdbc.internal.JdbcBatchingOutputFormat;
+import org.apache.flink.connector.jdbc.internal.connection.JdbcConnectionProvider;
+import org.apache.flink.connector.jdbc.internal.connection.SimpleJdbcConnectionProvider;
+import org.apache.flink.connector.jdbc.internal.executor.InsertOrUpdateJdbcExecutor;
+import org.apache.flink.connector.jdbc.internal.executor.JdbcBatchStatementExecutor;
+import org.apache.flink.connector.jdbc.internal.options.JdbcDmlOptions;
+import org.apache.flink.connector.jdbc.internal.options.JdbcOptions;
+import org.apache.flink.connector.jdbc.utils.JdbcUtils;
+import org.apache.flink.table.data.GenericRowData;
+import org.apache.flink.table.data.RowData;
+import org.apache.flink.table.types.DataType;
+import org.apache.flink.table.types.logical.LogicalType;
+import org.apache.flink.types.Row;
+
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import java.io.IOException;
+import java.sql.SQLException;
+import java.util.Arrays;
+import java.util.function.Function;
+
+import static org.apache.flink.util.Preconditions.checkArgument;
+import static org.apache.flink.util.Preconditions.checkNotNull;
+
+/**
+ * OutputFormat for {@link JdbcDynamicTableSource}.
+ */
+@Internal
+public class JdbcDynamicOutputFormat extends JdbcBatchingOutputFormat<RowData, RowData, JdbcBatchStatementExecutor<RowData>> {
+	private static final Logger LOG = LoggerFactory.getLogger(JdbcDynamicOutputFormat.class);
+
+	private JdbcBatchStatementExecutor<RowData> deleteExecutor;
+	private final JdbcDmlOptions dmlOptions;
+	private final LogicalType[] logicalTypes;
+
+	private JdbcDynamicOutputFormat(JdbcConnectionProvider connectionProvider, JdbcDmlOptions dmlOptions, JdbcExecutionOptions batchOptions, TypeInformation<RowData> rowDataTypeInfo, LogicalType[] logicalTypes) {
+		super(connectionProvider, batchOptions, ctx -> createUpsertRowExecutor(dmlOptions, ctx, rowDataTypeInfo, logicalTypes), RecordExtractor.identity());
+		this.dmlOptions = dmlOptions;
+		this.logicalTypes = logicalTypes;
+	}
+
+	private JdbcDynamicOutputFormat(JdbcConnectionProvider connectionProvider, JdbcDmlOptions dmlOptions, JdbcExecutionOptions batchOptions, TypeInformation<RowData> rowDataTypeInfo, LogicalType[] logicalTypes, String sql) {
+		super(connectionProvider, batchOptions,
+			ctx -> createSimpleRowDataExecutor(dmlOptions.getDialect(), sql, logicalTypes, ctx, rowDataTypeInfo),
+			RecordExtractor.identity());
+		this.dmlOptions = dmlOptions;
+		this.logicalTypes = logicalTypes;
+	}
+
+	@Override
+	public void open(int taskNumber, int numTasks) throws IOException {
+		super.open(taskNumber, numTasks);
+		deleteExecutor = createDeleteExecutor();
+		try {
+			deleteExecutor.open(connection);
+		} catch (SQLException e) {
+			throw new IOException(e);
+		}
+	}
+
+	private JdbcBatchStatementExecutor<RowData> createDeleteExecutor() {
+		int[] pkFields = Arrays.stream(dmlOptions.getFieldNames()).mapToInt(Arrays.asList(dmlOptions.getFieldNames())::indexOf).toArray();
+		LogicalType[] pkTypes = Arrays.stream(pkFields).mapToObj(f -> logicalTypes[f]).toArray(LogicalType[]::new);
+		String deleteSql = dmlOptions.getDialect().getDeleteStatement(dmlOptions.getTableName(), dmlOptions.getFieldNames());
+		return createKeyedRowExecutor(dmlOptions.getDialect(), pkFields, pkTypes, deleteSql, logicalTypes);
+	}
+
+	@Override
+	protected void addToBatch(RowData original, RowData extracted) throws SQLException {
+		switch (original.getRowKind()) {
+			case DELETE:

Review comment:
       add `case UPDATE_BEFORE:`, it is still possible to receive a `UPDATE_BEFORE` because ignore before is just an optimization. 

##########
File path: flink-connectors/flink-connector-jdbc/src/main/java/org/apache/flink/connector/jdbc/table/JdbcDynamicOutputFormat.java
##########
@@ -0,0 +1,265 @@
+/*
+ * 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.connector.jdbc.table;
+
+import org.apache.flink.annotation.Internal;
+import org.apache.flink.api.common.functions.RuntimeContext;
+import org.apache.flink.api.common.typeinfo.TypeInformation;
+import org.apache.flink.api.common.typeutils.TypeSerializer;
+import org.apache.flink.connector.jdbc.JdbcExecutionOptions;
+import org.apache.flink.connector.jdbc.JdbcStatementBuilder;
+import org.apache.flink.connector.jdbc.dialect.JdbcDialect;
+import org.apache.flink.connector.jdbc.internal.JdbcBatchingOutputFormat;
+import org.apache.flink.connector.jdbc.internal.connection.JdbcConnectionProvider;
+import org.apache.flink.connector.jdbc.internal.connection.SimpleJdbcConnectionProvider;
+import org.apache.flink.connector.jdbc.internal.executor.InsertOrUpdateJdbcExecutor;
+import org.apache.flink.connector.jdbc.internal.executor.JdbcBatchStatementExecutor;
+import org.apache.flink.connector.jdbc.internal.options.JdbcDmlOptions;
+import org.apache.flink.connector.jdbc.internal.options.JdbcOptions;
+import org.apache.flink.connector.jdbc.utils.JdbcUtils;
+import org.apache.flink.table.data.GenericRowData;
+import org.apache.flink.table.data.RowData;
+import org.apache.flink.table.types.DataType;
+import org.apache.flink.table.types.logical.LogicalType;
+import org.apache.flink.types.Row;
+
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import java.io.IOException;
+import java.sql.SQLException;
+import java.util.Arrays;
+import java.util.function.Function;
+
+import static org.apache.flink.util.Preconditions.checkArgument;
+import static org.apache.flink.util.Preconditions.checkNotNull;
+
+/**
+ * OutputFormat for {@link JdbcDynamicTableSource}.
+ */
+@Internal
+public class JdbcDynamicOutputFormat extends JdbcBatchingOutputFormat<RowData, RowData, JdbcBatchStatementExecutor<RowData>> {
+	private static final Logger LOG = LoggerFactory.getLogger(JdbcDynamicOutputFormat.class);
+
+	private JdbcBatchStatementExecutor<RowData> deleteExecutor;
+	private final JdbcDmlOptions dmlOptions;
+	private final LogicalType[] logicalTypes;
+
+	private JdbcDynamicOutputFormat(JdbcConnectionProvider connectionProvider, JdbcDmlOptions dmlOptions, JdbcExecutionOptions batchOptions, TypeInformation<RowData> rowDataTypeInfo, LogicalType[] logicalTypes) {

Review comment:
       Break the parameters into separate lines if long. 




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

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [flink] wuchong commented on a change in pull request #12176: [FLINK-17029][jdbc]Introduce a new JDBC connector with new property keys

Posted by GitBox <gi...@apache.org>.
wuchong commented on a change in pull request #12176:
URL: https://github.com/apache/flink/pull/12176#discussion_r426165412



##########
File path: flink-connectors/flink-connector-jdbc/src/main/java/org/apache/flink/connector/jdbc/table/JdbcDynamicTableSource.java
##########
@@ -0,0 +1,169 @@
+/*
+ * 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.connector.jdbc.table;
+
+import org.apache.flink.annotation.Internal;
+import org.apache.flink.api.common.typeinfo.TypeInformation;
+import org.apache.flink.connector.jdbc.dialect.JdbcDialect;
+import org.apache.flink.connector.jdbc.internal.options.JdbcLookupOptions;
+import org.apache.flink.connector.jdbc.internal.options.JdbcOptions;
+import org.apache.flink.connector.jdbc.internal.options.JdbcReadOptions;
+import org.apache.flink.connector.jdbc.split.JdbcNumericBetweenParametersProvider;
+import org.apache.flink.table.api.TableSchema;
+import org.apache.flink.table.connector.ChangelogMode;
+import org.apache.flink.table.connector.source.DynamicTableSource;
+import org.apache.flink.table.connector.source.InputFormatProvider;
+import org.apache.flink.table.connector.source.LookupTableSource;
+import org.apache.flink.table.connector.source.ScanTableSource;
+import org.apache.flink.table.connector.source.TableFunctionProvider;
+import org.apache.flink.table.data.RowData;
+import org.apache.flink.table.types.DataType;
+import org.apache.flink.table.types.logical.LogicalType;
+import org.apache.flink.table.types.logical.RowType;
+import org.apache.flink.util.Preconditions;
+
+import java.util.Arrays;
+import java.util.Objects;
+
+/**
+ * A {@link DynamicTableSource} for JDBC.
+ */
+@Internal
+public class JdbcDynamicTableSource implements ScanTableSource, LookupTableSource {
+
+	private final JdbcOptions options;
+	private final JdbcReadOptions readOptions;
+	private final JdbcLookupOptions lookupOptions;
+	private final TableSchema schema;
+	private final int[] selectFields;
+	private final String dialectName;
+
+	public JdbcDynamicTableSource(
+			JdbcOptions options,
+			JdbcReadOptions readOptions,
+			JdbcLookupOptions lookupOptions,
+			TableSchema schema,
+			int[] selectFields) {
+		this.options = options;
+		this.readOptions = readOptions;
+		this.lookupOptions = lookupOptions;
+		this.schema = schema;
+		this.selectFields = selectFields;
+		this.dialectName = options.getDialect().dialectName();
+	}
+
+	@Override
+	public LookupRuntimeProvider getLookupRuntimeProvider(LookupTableSource.Context context) {
+		// JDBC only support non-nested look up keys
+		String[] keyNames = new String[context.getKeys().length];
+		for (int i = 0; i < keyNames.length; i++) {
+			int[] innerKeyArr = context.getKeys()[i];
+			Preconditions.checkArgument(innerKeyArr.length == 1,
+				"JDBC only support non-nested look up keys");
+			keyNames[i] = schema.getFieldNames()[innerKeyArr[0]];
+		}
+		final RowType rowType = (RowType) schema.toRowDataType().getLogicalType();
+
+		return TableFunctionProvider.of(new JdbcRowDataLookupFunction(
+			options,
+			lookupOptions,
+			schema.getFieldNames(),
+			schema.getFieldDataTypes(),
+			keyNames,
+			rowType));
+	}
+
+	@Override
+	public ScanRuntimeProvider getScanRuntimeProvider(ScanTableSource.Context runtimeProviderContext) {
+		final JdbcRowDataInputFormat.Builder builder = JdbcRowDataInputFormat.builder()
+			.setDrivername(options.getDriverName())
+			.setDBUrl(options.getDbURL())
+			.setUsername(options.getUsername().orElse(null))
+			.setPassword(options.getPassword().orElse(null));
+
+		if (readOptions.getFetchSize() != 0) {
+			builder.setFetchSize(readOptions.getFetchSize());
+		}
+		final JdbcDialect dialect = options.getDialect();
+		String query = dialect.getSelectFromStatement(
+			options.getTableName(), schema.getFieldNames(), new String[0]);
+		if (readOptions.getPartitionColumnName().isPresent()) {
+			long lowerBound = readOptions.getPartitionLowerBound().get();
+			long upperBound = readOptions.getPartitionUpperBound().get();
+			int numPartitions = readOptions.getNumPartitions().get();
+			builder.setParametersProvider(
+				new JdbcNumericBetweenParametersProvider(lowerBound, upperBound).ofBatchNum(numPartitions));
+			query += " WHERE " +
+				dialect.quoteIdentifier(readOptions.getPartitionColumnName().get()) +
+				" BETWEEN ? AND ?";
+		}
+		builder.setQuery(query);
+		final RowType rowType = RowType.of(
+			Arrays.stream(schema.getFieldDataTypes())
+				.map(DataType::getLogicalType)
+				.toArray(LogicalType[]::new),
+			schema.getFieldNames());
+		builder.setRowConverter(dialect.getRowConverter(rowType));
+		@SuppressWarnings("unchecked")

Review comment:
       move to method

##########
File path: flink-connectors/flink-connector-jdbc/src/main/java/org/apache/flink/connector/jdbc/internal/converter/AbstractJdbcRowConverter.java
##########
@@ -18,54 +18,273 @@
 
 package org.apache.flink.connector.jdbc.internal.converter;
 
+import org.apache.flink.connector.jdbc.utils.JdbcTypeUtil;
+import org.apache.flink.table.data.DecimalData;
+import org.apache.flink.table.data.GenericArrayData;
+import org.apache.flink.table.data.GenericRowData;
+import org.apache.flink.table.data.RowData;
+import org.apache.flink.table.data.StringData;
+import org.apache.flink.table.data.TimestampData;
+import org.apache.flink.table.types.logical.ArrayType;
+import org.apache.flink.table.types.logical.DecimalType;
 import org.apache.flink.table.types.logical.LogicalType;
-import org.apache.flink.table.types.logical.LogicalTypeRoot;
 import org.apache.flink.table.types.logical.RowType;
-import org.apache.flink.types.Row;
+import org.apache.flink.table.types.logical.TimestampType;
+import org.apache.flink.table.types.logical.utils.LogicalTypeUtils;
+import org.apache.flink.table.types.utils.TypeConversions;
 
+import java.io.Serializable;
+import java.lang.reflect.Array;
+import java.math.BigDecimal;
+import java.sql.Date;
+import java.sql.PreparedStatement;
 import java.sql.ResultSet;
 import java.sql.SQLException;
+import java.sql.Time;
+import java.sql.Timestamp;
+import java.time.LocalDate;
+import java.time.LocalTime;
 
 import static org.apache.flink.util.Preconditions.checkNotNull;
 
 /**
- * Base for all row converters.
+ * Base class for all converters that convert between JDBC object and Flink internal object.
  */
 public abstract class AbstractJdbcRowConverter implements JdbcRowConverter {
 
 	protected final RowType rowType;
-	protected final JdbcFieldConverter[] converters;
+	protected final JdbcFieldConverter[] toInternalConverters;
+	protected final RowFieldConverter[] toExternalConverters;
+	protected final LogicalType[] fieldTypes;
 
 	public AbstractJdbcRowConverter(RowType rowType) {
 		this.rowType = checkNotNull(rowType);
-		converters = new JdbcFieldConverter[rowType.getFieldCount()];
+		this.fieldTypes = rowType.getFields().stream()
+			.map(RowType.RowField::getType)
+			.toArray(LogicalType[]::new);
+		this.toInternalConverters = new JdbcFieldConverter[rowType.getFieldCount()];
+		this.toExternalConverters = new RowFieldConverter[rowType.getFieldCount()];
+		for (int i = 0; i < rowType.getFieldCount(); i++) {
+			toInternalConverters[i] = createNullableInternalConverter(rowType.getTypeAt(i));
+			toExternalConverters[i] = createNullableExternalConverter(fieldTypes[i]);
+		}
+	}
 
-		for (int i = 0; i < converters.length; i++) {
-			converters[i] = createConverter(rowType.getTypeAt(i));
+	@Override
+	public RowData toInternal(ResultSet resultSet) throws SQLException {
+		GenericRowData genericRowData = new GenericRowData(rowType.getFieldCount());
+		try {
+			for (int pos = 0; pos < rowType.getFieldCount(); pos++) {
+				genericRowData.setField(pos, toInternalConverters[pos].convert(resultSet.getObject(pos + 1)));
+			}
+		} catch (Exception e) {
+			e.printStackTrace();

Review comment:
       remove.

##########
File path: flink-connectors/flink-connector-jdbc/src/main/java/org/apache/flink/connector/jdbc/internal/converter/AbstractJdbcRowConverter.java
##########
@@ -18,54 +18,273 @@
 
 package org.apache.flink.connector.jdbc.internal.converter;
 
+import org.apache.flink.connector.jdbc.utils.JdbcTypeUtil;
+import org.apache.flink.table.data.DecimalData;
+import org.apache.flink.table.data.GenericArrayData;
+import org.apache.flink.table.data.GenericRowData;
+import org.apache.flink.table.data.RowData;
+import org.apache.flink.table.data.StringData;
+import org.apache.flink.table.data.TimestampData;
+import org.apache.flink.table.types.logical.ArrayType;
+import org.apache.flink.table.types.logical.DecimalType;
 import org.apache.flink.table.types.logical.LogicalType;
-import org.apache.flink.table.types.logical.LogicalTypeRoot;
 import org.apache.flink.table.types.logical.RowType;
-import org.apache.flink.types.Row;
+import org.apache.flink.table.types.logical.TimestampType;
+import org.apache.flink.table.types.logical.utils.LogicalTypeUtils;
+import org.apache.flink.table.types.utils.TypeConversions;
 
+import java.io.Serializable;
+import java.lang.reflect.Array;
+import java.math.BigDecimal;
+import java.sql.Date;
+import java.sql.PreparedStatement;
 import java.sql.ResultSet;
 import java.sql.SQLException;
+import java.sql.Time;
+import java.sql.Timestamp;
+import java.time.LocalDate;
+import java.time.LocalTime;
 
 import static org.apache.flink.util.Preconditions.checkNotNull;
 
 /**
- * Base for all row converters.
+ * Base class for all converters that convert between JDBC object and Flink internal object.
  */
 public abstract class AbstractJdbcRowConverter implements JdbcRowConverter {
 
 	protected final RowType rowType;
-	protected final JdbcFieldConverter[] converters;
+	protected final JdbcFieldConverter[] toInternalConverters;
+	protected final RowFieldConverter[] toExternalConverters;
+	protected final LogicalType[] fieldTypes;
 
 	public AbstractJdbcRowConverter(RowType rowType) {
 		this.rowType = checkNotNull(rowType);
-		converters = new JdbcFieldConverter[rowType.getFieldCount()];
+		this.fieldTypes = rowType.getFields().stream()
+			.map(RowType.RowField::getType)
+			.toArray(LogicalType[]::new);
+		this.toInternalConverters = new JdbcFieldConverter[rowType.getFieldCount()];
+		this.toExternalConverters = new RowFieldConverter[rowType.getFieldCount()];
+		for (int i = 0; i < rowType.getFieldCount(); i++) {
+			toInternalConverters[i] = createNullableInternalConverter(rowType.getTypeAt(i));
+			toExternalConverters[i] = createNullableExternalConverter(fieldTypes[i]);
+		}
+	}
 
-		for (int i = 0; i < converters.length; i++) {
-			converters[i] = createConverter(rowType.getTypeAt(i));
+	@Override
+	public RowData toInternal(ResultSet resultSet) throws SQLException {
+		GenericRowData genericRowData = new GenericRowData(rowType.getFieldCount());
+		try {
+			for (int pos = 0; pos < rowType.getFieldCount(); pos++) {
+				genericRowData.setField(pos, toInternalConverters[pos].convert(resultSet.getObject(pos + 1)));
+			}
+		} catch (Exception e) {
+			e.printStackTrace();
 		}
+
+		return genericRowData;
 	}
 
 	@Override
-	public Row convert(ResultSet resultSet, Row reuse) throws SQLException {
-		for (int pos = 0; pos < rowType.getFieldCount(); pos++) {
-			reuse.setField(pos, converters[pos].convert(resultSet.getObject(pos + 1)));
+	public PreparedStatement toExternal(RowData rowData, PreparedStatement statement) throws SQLException {
+		for (int index = 0; index < rowData.getArity(); index++) {
+			toExternalConverters[index].convert(statement, index, fieldTypes[index], rowData);
 		}
+		return statement;	}

Review comment:
       style.

##########
File path: flink-connectors/flink-connector-jdbc/src/main/java/org/apache/flink/connector/jdbc/table/JdbcDynamicTableSourceSinkFactory.java
##########
@@ -0,0 +1,323 @@
+/*
+ * 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.connector.jdbc.table;
+
+import org.apache.flink.annotation.Internal;
+import org.apache.flink.configuration.ConfigOption;
+import org.apache.flink.configuration.ConfigOptions;
+import org.apache.flink.configuration.ReadableConfig;
+import org.apache.flink.connector.jdbc.JdbcExecutionOptions;
+import org.apache.flink.connector.jdbc.dialect.JdbcDialect;
+import org.apache.flink.connector.jdbc.dialect.JdbcDialects;
+import org.apache.flink.connector.jdbc.internal.options.JdbcDmlOptions;
+import org.apache.flink.connector.jdbc.internal.options.JdbcLookupOptions;
+import org.apache.flink.connector.jdbc.internal.options.JdbcOptions;
+import org.apache.flink.connector.jdbc.internal.options.JdbcReadOptions;
+import org.apache.flink.table.api.TableSchema;
+import org.apache.flink.table.connector.sink.DynamicTableSink;
+import org.apache.flink.table.connector.source.DynamicTableSource;
+import org.apache.flink.table.factories.DynamicTableSinkFactory;
+import org.apache.flink.table.factories.DynamicTableSourceFactory;
+import org.apache.flink.table.factories.FactoryUtil;
+import org.apache.flink.table.utils.TableSchemaUtils;
+import org.apache.flink.util.Preconditions;
+
+import java.time.Duration;
+import java.util.Arrays;
+import java.util.HashSet;
+import java.util.Optional;
+import java.util.Set;
+
+import static org.apache.flink.util.Preconditions.checkState;
+
+/**
+ * Factory for creating configured instances of {@link JdbcDynamicTableSource}
+ * and {@link JdbcDynamicTableSink}.
+ */
+@Internal
+public class JdbcDynamicTableSourceSinkFactory implements DynamicTableSourceFactory, DynamicTableSinkFactory {
+
+	public static final String IDENTIFIER = "jdbc";
+	public static final ConfigOption<String> URL = ConfigOptions
+		.key("url")
+		.stringType()
+		.noDefaultValue()
+		.withDescription("the jdbc database url.");
+	public static final ConfigOption<String> TABLE_NAME = ConfigOptions
+		.key("table-name")
+		.stringType()
+		.noDefaultValue()
+		.withDescription("the jdbc table name.");
+	public static final ConfigOption<String> USERNAME = ConfigOptions
+		.key("username")
+		.stringType()
+		.noDefaultValue()
+		.withDescription("the jdbc user name.");
+	public static final ConfigOption<String> PASSWORD = ConfigOptions
+		.key("password")
+		.stringType()
+		.noDefaultValue()
+		.withDescription("the jdbc password.");
+	private static final ConfigOption<String> DRIVER = ConfigOptions
+		.key("driver")
+		.stringType()
+		.noDefaultValue()
+		.withDescription("the class name of the JDBC driver to use to connect to this URL. " +
+			"If not set, it will automatically be derived from the URL.");
+
+	// read config options
+	private static final ConfigOption<String> SCAN_PARTITION_COLUMN = ConfigOptions
+		.key("read.partition.column")
+		.stringType()
+		.noDefaultValue()
+		.withDescription("the column name used for partitioning the input.");
+	private static final ConfigOption<Integer> SCAN_PARTITION_NUM = ConfigOptions
+		.key("scan.partition.num")
+		.intType()
+		.noDefaultValue()
+		.withDescription("the number of partitions.");
+	private static final ConfigOption<Long> SCAN_PARTITION_LOWER_BOUND = ConfigOptions
+		.key("scan.partition.lower-bound")
+		.longType()
+		.noDefaultValue()
+		.withDescription("the smallest value of the first partition.");
+	private static final ConfigOption<Long> SCAN_PARTITION_UPPER_BOUND = ConfigOptions
+		.key("scan.partition.upper-bound")
+		.longType()
+		.noDefaultValue()
+		.withDescription("the largest value of the last partition.");
+	private static final ConfigOption<Integer> SCAN_FETCH_SIZE = ConfigOptions
+		.key("scan.fetch-size")
+		.intType()
+		.defaultValue(0)
+		.withDescription("gives the reader a hint as to the number of rows that should be fetched, from" +
+			" the database when reading per round trip. If the value specified is zero, then the hint is ignored. The" +
+			" default value is zero.");
+
+	// look up config options
+	private static final ConfigOption<Long> LOOKUP_CACHE_MAX_ROWS = ConfigOptions
+		.key("lookup.cache.max-rows")
+		.longType()
+		.defaultValue(-1L)
+		.withDescription("the max number of rows of lookup cache, over this value, the oldest rows will " +
+			"be eliminated. \"cache.max-rows\" and \"cache.ttl\" options must all be specified if any of them is " +
+			"specified. Cache is not enabled as default.");
+	private static final ConfigOption<Duration> LOOKUP_CACHE_TTL = ConfigOptions
+		.key("lookup.cache.ttl")
+		.durationType()
+		.defaultValue(Duration.ofSeconds(10))
+		.withDescription("the cache time to live.");
+	private static final ConfigOption<Integer> LOOKUP_MAX_RETRIES = ConfigOptions
+		.key("lookup.max-retries")
+		.intType()
+		.defaultValue(3)
+		.withDescription("the max retry times if lookup database failed.");
+
+	// write config options
+	private static final ConfigOption<Integer> SINK_FLUSH_MAX_ROWS = ConfigOptions
+		.key("sink.flush.max-rows")
+		.intType()
+		.defaultValue(5000)
+		.withDescription("the flush max size (includes all append, upsert and delete records), over this number" +
+			" of records, will flush data. The default value is 5000.");
+	private static final ConfigOption<Long> SINK_FLUSH_INTERVAL = ConfigOptions
+		.key("sink.flush.interval")
+		.longType()
+		.defaultValue(0L)
+		.withDescription("the flush interval mills, over this time, asynchronous threads will flush data. The " +
+			"default value is 0, which means no asynchronous flush thread will be scheduled.");
+	private static final ConfigOption<Integer> SINK_MAX_RETRIES = ConfigOptions
+		.key("sink.max-retries")
+		.intType()
+		.defaultValue(3)
+		.withDescription("the max retry times if writing records to database failed.");
+
+	@Override
+	public DynamicTableSink createDynamicTableSink(Context context) {
+		final FactoryUtil.TableFactoryHelper helper = FactoryUtil.createTableFactoryHelper(this, context);
+		final ReadableConfig config = helper.getOptions();
+
+		helper.validate();
+		validateConfigOptions(config);
+		JdbcOptions jdbcOptions = getJdbcOptions(config);
+		TableSchema formatSchema = TableSchemaUtils.getPhysicalSchema(context.getCatalogTable().getSchema());

Review comment:
       physicalSchema

##########
File path: flink-connectors/flink-connector-jdbc/src/main/java/org/apache/flink/connector/jdbc/table/JdbcDynamicTableSourceSinkFactory.java
##########
@@ -0,0 +1,323 @@
+/*
+ * 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.connector.jdbc.table;
+
+import org.apache.flink.annotation.Internal;
+import org.apache.flink.configuration.ConfigOption;
+import org.apache.flink.configuration.ConfigOptions;
+import org.apache.flink.configuration.ReadableConfig;
+import org.apache.flink.connector.jdbc.JdbcExecutionOptions;
+import org.apache.flink.connector.jdbc.dialect.JdbcDialect;
+import org.apache.flink.connector.jdbc.dialect.JdbcDialects;
+import org.apache.flink.connector.jdbc.internal.options.JdbcDmlOptions;
+import org.apache.flink.connector.jdbc.internal.options.JdbcLookupOptions;
+import org.apache.flink.connector.jdbc.internal.options.JdbcOptions;
+import org.apache.flink.connector.jdbc.internal.options.JdbcReadOptions;
+import org.apache.flink.table.api.TableSchema;
+import org.apache.flink.table.connector.sink.DynamicTableSink;
+import org.apache.flink.table.connector.source.DynamicTableSource;
+import org.apache.flink.table.factories.DynamicTableSinkFactory;
+import org.apache.flink.table.factories.DynamicTableSourceFactory;
+import org.apache.flink.table.factories.FactoryUtil;
+import org.apache.flink.table.utils.TableSchemaUtils;
+import org.apache.flink.util.Preconditions;
+
+import java.time.Duration;
+import java.util.Arrays;
+import java.util.HashSet;
+import java.util.Optional;
+import java.util.Set;
+
+import static org.apache.flink.util.Preconditions.checkState;
+
+/**
+ * Factory for creating configured instances of {@link JdbcDynamicTableSource}
+ * and {@link JdbcDynamicTableSink}.
+ */
+@Internal
+public class JdbcDynamicTableSourceSinkFactory implements DynamicTableSourceFactory, DynamicTableSinkFactory {
+
+	public static final String IDENTIFIER = "jdbc";
+	public static final ConfigOption<String> URL = ConfigOptions
+		.key("url")
+		.stringType()
+		.noDefaultValue()
+		.withDescription("the jdbc database url.");
+	public static final ConfigOption<String> TABLE_NAME = ConfigOptions
+		.key("table-name")
+		.stringType()
+		.noDefaultValue()
+		.withDescription("the jdbc table name.");
+	public static final ConfigOption<String> USERNAME = ConfigOptions
+		.key("username")
+		.stringType()
+		.noDefaultValue()
+		.withDescription("the jdbc user name.");
+	public static final ConfigOption<String> PASSWORD = ConfigOptions
+		.key("password")
+		.stringType()
+		.noDefaultValue()
+		.withDescription("the jdbc password.");
+	private static final ConfigOption<String> DRIVER = ConfigOptions
+		.key("driver")
+		.stringType()
+		.noDefaultValue()
+		.withDescription("the class name of the JDBC driver to use to connect to this URL. " +
+			"If not set, it will automatically be derived from the URL.");
+
+	// read config options
+	private static final ConfigOption<String> SCAN_PARTITION_COLUMN = ConfigOptions
+		.key("read.partition.column")
+		.stringType()
+		.noDefaultValue()
+		.withDescription("the column name used for partitioning the input.");
+	private static final ConfigOption<Integer> SCAN_PARTITION_NUM = ConfigOptions
+		.key("scan.partition.num")
+		.intType()
+		.noDefaultValue()
+		.withDescription("the number of partitions.");
+	private static final ConfigOption<Long> SCAN_PARTITION_LOWER_BOUND = ConfigOptions
+		.key("scan.partition.lower-bound")
+		.longType()
+		.noDefaultValue()
+		.withDescription("the smallest value of the first partition.");
+	private static final ConfigOption<Long> SCAN_PARTITION_UPPER_BOUND = ConfigOptions
+		.key("scan.partition.upper-bound")
+		.longType()
+		.noDefaultValue()
+		.withDescription("the largest value of the last partition.");
+	private static final ConfigOption<Integer> SCAN_FETCH_SIZE = ConfigOptions
+		.key("scan.fetch-size")
+		.intType()
+		.defaultValue(0)
+		.withDescription("gives the reader a hint as to the number of rows that should be fetched, from" +
+			" the database when reading per round trip. If the value specified is zero, then the hint is ignored. The" +
+			" default value is zero.");
+
+	// look up config options
+	private static final ConfigOption<Long> LOOKUP_CACHE_MAX_ROWS = ConfigOptions
+		.key("lookup.cache.max-rows")
+		.longType()
+		.defaultValue(-1L)
+		.withDescription("the max number of rows of lookup cache, over this value, the oldest rows will " +
+			"be eliminated. \"cache.max-rows\" and \"cache.ttl\" options must all be specified if any of them is " +
+			"specified. Cache is not enabled as default.");
+	private static final ConfigOption<Duration> LOOKUP_CACHE_TTL = ConfigOptions
+		.key("lookup.cache.ttl")
+		.durationType()
+		.defaultValue(Duration.ofSeconds(10))
+		.withDescription("the cache time to live.");
+	private static final ConfigOption<Integer> LOOKUP_MAX_RETRIES = ConfigOptions
+		.key("lookup.max-retries")
+		.intType()
+		.defaultValue(3)
+		.withDescription("the max retry times if lookup database failed.");
+
+	// write config options
+	private static final ConfigOption<Integer> SINK_FLUSH_MAX_ROWS = ConfigOptions
+		.key("sink.flush.max-rows")
+		.intType()
+		.defaultValue(5000)
+		.withDescription("the flush max size (includes all append, upsert and delete records), over this number" +
+			" of records, will flush data. The default value is 5000.");
+	private static final ConfigOption<Long> SINK_FLUSH_INTERVAL = ConfigOptions
+		.key("sink.flush.interval")
+		.longType()
+		.defaultValue(0L)
+		.withDescription("the flush interval mills, over this time, asynchronous threads will flush data. The " +
+			"default value is 0, which means no asynchronous flush thread will be scheduled.");
+	private static final ConfigOption<Integer> SINK_MAX_RETRIES = ConfigOptions
+		.key("sink.max-retries")
+		.intType()
+		.defaultValue(3)
+		.withDescription("the max retry times if writing records to database failed.");
+
+	@Override
+	public DynamicTableSink createDynamicTableSink(Context context) {
+		final FactoryUtil.TableFactoryHelper helper = FactoryUtil.createTableFactoryHelper(this, context);
+		final ReadableConfig config = helper.getOptions();
+
+		helper.validate();
+		validateConfigOptions(config);
+		JdbcOptions jdbcOptions = getJdbcOptions(config);
+		TableSchema formatSchema = TableSchemaUtils.getPhysicalSchema(context.getCatalogTable().getSchema());
+
+		return new JdbcDynamicTableSink(
+			jdbcOptions,
+			getJdbcExecutionOptions(config),
+			getJdbcDmlOptions(jdbcOptions, context.getCatalogTable().getSchema()),
+			formatSchema);
+	}
+
+	@Override
+	public DynamicTableSource createDynamicTableSource(Context context) {
+		final FactoryUtil.TableFactoryHelper helper = FactoryUtil.createTableFactoryHelper(this, context);
+		final ReadableConfig config = helper.getOptions();
+
+		helper.validate();
+		validateConfigOptions(config);
+		TableSchema formatSchema = TableSchemaUtils.getPhysicalSchema(context.getCatalogTable().getSchema());
+		int[] selectFields = new int[formatSchema.getFieldNames().length];
+		for (int i = 0; i < selectFields.length; i++) {
+			selectFields[i] = i;
+		}
+		return new JdbcDynamicTableSource(
+			getJdbcOptions(helper.getOptions()),
+			getJdbcReadOptions(helper.getOptions()),
+			getJdbcLookupOptions(helper.getOptions()),
+			formatSchema,
+			selectFields);
+	}
+
+	private JdbcOptions getJdbcOptions(ReadableConfig readableConfig) {
+		final String url = readableConfig.get(URL);
+		final JdbcOptions.Builder builder = JdbcOptions.builder()
+			.setDBUrl(url)
+			.setTableName(readableConfig.get(TABLE_NAME))
+			.setDialect(JdbcDialects.get(url).get());
+
+		readableConfig.getOptional(DRIVER).ifPresent(builder::setDriverName);
+		readableConfig.getOptional(USERNAME).ifPresent(builder::setUsername);
+		readableConfig.getOptional(PASSWORD).ifPresent(builder::setPassword);
+		return builder.build();
+	}
+
+	private JdbcReadOptions getJdbcReadOptions(ReadableConfig readableConfig) {
+		final Optional<String> partitionColumnName = readableConfig.getOptional(SCAN_PARTITION_COLUMN);
+		final JdbcReadOptions.Builder builder = JdbcReadOptions.builder();
+		if (partitionColumnName.isPresent()) {
+			builder.setPartitionColumnName(partitionColumnName.get());
+			builder.setPartitionLowerBound(readableConfig.get(SCAN_PARTITION_LOWER_BOUND));
+			builder.setPartitionUpperBound(readableConfig.get(SCAN_PARTITION_UPPER_BOUND));
+			builder.setNumPartitions(readableConfig.get(SCAN_PARTITION_NUM));
+		}
+		readableConfig.getOptional(SCAN_FETCH_SIZE).ifPresent(builder::setFetchSize);
+		return builder.build();
+	}
+
+	private JdbcLookupOptions getJdbcLookupOptions(ReadableConfig readableConfig) {
+		return new JdbcLookupOptions(
+			readableConfig.getOptional(LOOKUP_CACHE_MAX_ROWS).orElse(LOOKUP_CACHE_MAX_ROWS.defaultValue()),
+			readableConfig.getOptional(LOOKUP_CACHE_TTL).orElse(LOOKUP_CACHE_TTL.defaultValue()).toMillis(),
+			readableConfig.getOptional(LOOKUP_MAX_RETRIES).orElse(LOOKUP_MAX_RETRIES.defaultValue()));
+	}
+
+	private JdbcExecutionOptions getJdbcExecutionOptions(ReadableConfig readableConfig) {
+		final JdbcExecutionOptions.Builder builder = new JdbcExecutionOptions.Builder();
+		readableConfig.getOptional(SINK_FLUSH_MAX_ROWS)
+			.ifPresent(builder::withBatchSize);

Review comment:
       do not need `ifPresent`

##########
File path: flink-connectors/flink-connector-jdbc/src/main/java/org/apache/flink/connector/jdbc/table/JdbcDynamicTableSourceSinkFactory.java
##########
@@ -0,0 +1,323 @@
+/*
+ * 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.connector.jdbc.table;
+
+import org.apache.flink.annotation.Internal;
+import org.apache.flink.configuration.ConfigOption;
+import org.apache.flink.configuration.ConfigOptions;
+import org.apache.flink.configuration.ReadableConfig;
+import org.apache.flink.connector.jdbc.JdbcExecutionOptions;
+import org.apache.flink.connector.jdbc.dialect.JdbcDialect;
+import org.apache.flink.connector.jdbc.dialect.JdbcDialects;
+import org.apache.flink.connector.jdbc.internal.options.JdbcDmlOptions;
+import org.apache.flink.connector.jdbc.internal.options.JdbcLookupOptions;
+import org.apache.flink.connector.jdbc.internal.options.JdbcOptions;
+import org.apache.flink.connector.jdbc.internal.options.JdbcReadOptions;
+import org.apache.flink.table.api.TableSchema;
+import org.apache.flink.table.connector.sink.DynamicTableSink;
+import org.apache.flink.table.connector.source.DynamicTableSource;
+import org.apache.flink.table.factories.DynamicTableSinkFactory;
+import org.apache.flink.table.factories.DynamicTableSourceFactory;
+import org.apache.flink.table.factories.FactoryUtil;
+import org.apache.flink.table.utils.TableSchemaUtils;
+import org.apache.flink.util.Preconditions;
+
+import java.time.Duration;
+import java.util.Arrays;
+import java.util.HashSet;
+import java.util.Optional;
+import java.util.Set;
+
+import static org.apache.flink.util.Preconditions.checkState;
+
+/**
+ * Factory for creating configured instances of {@link JdbcDynamicTableSource}
+ * and {@link JdbcDynamicTableSink}.
+ */
+@Internal
+public class JdbcDynamicTableSourceSinkFactory implements DynamicTableSourceFactory, DynamicTableSinkFactory {
+
+	public static final String IDENTIFIER = "jdbc";
+	public static final ConfigOption<String> URL = ConfigOptions
+		.key("url")
+		.stringType()
+		.noDefaultValue()
+		.withDescription("the jdbc database url.");
+	public static final ConfigOption<String> TABLE_NAME = ConfigOptions
+		.key("table-name")
+		.stringType()
+		.noDefaultValue()
+		.withDescription("the jdbc table name.");
+	public static final ConfigOption<String> USERNAME = ConfigOptions
+		.key("username")
+		.stringType()
+		.noDefaultValue()
+		.withDescription("the jdbc user name.");
+	public static final ConfigOption<String> PASSWORD = ConfigOptions
+		.key("password")
+		.stringType()
+		.noDefaultValue()
+		.withDescription("the jdbc password.");
+	private static final ConfigOption<String> DRIVER = ConfigOptions
+		.key("driver")
+		.stringType()
+		.noDefaultValue()
+		.withDescription("the class name of the JDBC driver to use to connect to this URL. " +
+			"If not set, it will automatically be derived from the URL.");
+
+	// read config options
+	private static final ConfigOption<String> SCAN_PARTITION_COLUMN = ConfigOptions
+		.key("read.partition.column")
+		.stringType()
+		.noDefaultValue()
+		.withDescription("the column name used for partitioning the input.");
+	private static final ConfigOption<Integer> SCAN_PARTITION_NUM = ConfigOptions
+		.key("scan.partition.num")
+		.intType()
+		.noDefaultValue()
+		.withDescription("the number of partitions.");
+	private static final ConfigOption<Long> SCAN_PARTITION_LOWER_BOUND = ConfigOptions
+		.key("scan.partition.lower-bound")
+		.longType()
+		.noDefaultValue()
+		.withDescription("the smallest value of the first partition.");
+	private static final ConfigOption<Long> SCAN_PARTITION_UPPER_BOUND = ConfigOptions
+		.key("scan.partition.upper-bound")
+		.longType()
+		.noDefaultValue()
+		.withDescription("the largest value of the last partition.");
+	private static final ConfigOption<Integer> SCAN_FETCH_SIZE = ConfigOptions
+		.key("scan.fetch-size")
+		.intType()
+		.defaultValue(0)
+		.withDescription("gives the reader a hint as to the number of rows that should be fetched, from" +
+			" the database when reading per round trip. If the value specified is zero, then the hint is ignored. The" +
+			" default value is zero.");
+
+	// look up config options
+	private static final ConfigOption<Long> LOOKUP_CACHE_MAX_ROWS = ConfigOptions
+		.key("lookup.cache.max-rows")
+		.longType()
+		.defaultValue(-1L)
+		.withDescription("the max number of rows of lookup cache, over this value, the oldest rows will " +
+			"be eliminated. \"cache.max-rows\" and \"cache.ttl\" options must all be specified if any of them is " +
+			"specified. Cache is not enabled as default.");
+	private static final ConfigOption<Duration> LOOKUP_CACHE_TTL = ConfigOptions
+		.key("lookup.cache.ttl")
+		.durationType()
+		.defaultValue(Duration.ofSeconds(10))
+		.withDescription("the cache time to live.");
+	private static final ConfigOption<Integer> LOOKUP_MAX_RETRIES = ConfigOptions
+		.key("lookup.max-retries")
+		.intType()
+		.defaultValue(3)
+		.withDescription("the max retry times if lookup database failed.");
+
+	// write config options
+	private static final ConfigOption<Integer> SINK_FLUSH_MAX_ROWS = ConfigOptions
+		.key("sink.flush.max-rows")
+		.intType()
+		.defaultValue(5000)
+		.withDescription("the flush max size (includes all append, upsert and delete records), over this number" +
+			" of records, will flush data. The default value is 5000.");
+	private static final ConfigOption<Long> SINK_FLUSH_INTERVAL = ConfigOptions
+		.key("sink.flush.interval")
+		.longType()
+		.defaultValue(0L)
+		.withDescription("the flush interval mills, over this time, asynchronous threads will flush data. The " +
+			"default value is 0, which means no asynchronous flush thread will be scheduled.");
+	private static final ConfigOption<Integer> SINK_MAX_RETRIES = ConfigOptions
+		.key("sink.max-retries")
+		.intType()
+		.defaultValue(3)
+		.withDescription("the max retry times if writing records to database failed.");
+
+	@Override
+	public DynamicTableSink createDynamicTableSink(Context context) {
+		final FactoryUtil.TableFactoryHelper helper = FactoryUtil.createTableFactoryHelper(this, context);
+		final ReadableConfig config = helper.getOptions();
+
+		helper.validate();
+		validateConfigOptions(config);
+		JdbcOptions jdbcOptions = getJdbcOptions(config);
+		TableSchema formatSchema = TableSchemaUtils.getPhysicalSchema(context.getCatalogTable().getSchema());
+
+		return new JdbcDynamicTableSink(
+			jdbcOptions,
+			getJdbcExecutionOptions(config),
+			getJdbcDmlOptions(jdbcOptions, context.getCatalogTable().getSchema()),
+			formatSchema);
+	}
+
+	@Override
+	public DynamicTableSource createDynamicTableSource(Context context) {
+		final FactoryUtil.TableFactoryHelper helper = FactoryUtil.createTableFactoryHelper(this, context);
+		final ReadableConfig config = helper.getOptions();
+
+		helper.validate();
+		validateConfigOptions(config);
+		TableSchema formatSchema = TableSchemaUtils.getPhysicalSchema(context.getCatalogTable().getSchema());
+		int[] selectFields = new int[formatSchema.getFieldNames().length];
+		for (int i = 0; i < selectFields.length; i++) {
+			selectFields[i] = i;
+		}
+		return new JdbcDynamicTableSource(
+			getJdbcOptions(helper.getOptions()),
+			getJdbcReadOptions(helper.getOptions()),
+			getJdbcLookupOptions(helper.getOptions()),
+			formatSchema,
+			selectFields);
+	}
+
+	private JdbcOptions getJdbcOptions(ReadableConfig readableConfig) {
+		final String url = readableConfig.get(URL);
+		final JdbcOptions.Builder builder = JdbcOptions.builder()
+			.setDBUrl(url)
+			.setTableName(readableConfig.get(TABLE_NAME))
+			.setDialect(JdbcDialects.get(url).get());
+
+		readableConfig.getOptional(DRIVER).ifPresent(builder::setDriverName);
+		readableConfig.getOptional(USERNAME).ifPresent(builder::setUsername);
+		readableConfig.getOptional(PASSWORD).ifPresent(builder::setPassword);
+		return builder.build();
+	}
+
+	private JdbcReadOptions getJdbcReadOptions(ReadableConfig readableConfig) {
+		final Optional<String> partitionColumnName = readableConfig.getOptional(SCAN_PARTITION_COLUMN);
+		final JdbcReadOptions.Builder builder = JdbcReadOptions.builder();
+		if (partitionColumnName.isPresent()) {
+			builder.setPartitionColumnName(partitionColumnName.get());
+			builder.setPartitionLowerBound(readableConfig.get(SCAN_PARTITION_LOWER_BOUND));
+			builder.setPartitionUpperBound(readableConfig.get(SCAN_PARTITION_UPPER_BOUND));
+			builder.setNumPartitions(readableConfig.get(SCAN_PARTITION_NUM));
+		}
+		readableConfig.getOptional(SCAN_FETCH_SIZE).ifPresent(builder::setFetchSize);
+		return builder.build();
+	}
+
+	private JdbcLookupOptions getJdbcLookupOptions(ReadableConfig readableConfig) {
+		return new JdbcLookupOptions(
+			readableConfig.getOptional(LOOKUP_CACHE_MAX_ROWS).orElse(LOOKUP_CACHE_MAX_ROWS.defaultValue()),
+			readableConfig.getOptional(LOOKUP_CACHE_TTL).orElse(LOOKUP_CACHE_TTL.defaultValue()).toMillis(),
+			readableConfig.getOptional(LOOKUP_MAX_RETRIES).orElse(LOOKUP_MAX_RETRIES.defaultValue()));
+	}
+
+	private JdbcExecutionOptions getJdbcExecutionOptions(ReadableConfig readableConfig) {
+		final JdbcExecutionOptions.Builder builder = new JdbcExecutionOptions.Builder();
+		readableConfig.getOptional(SINK_FLUSH_MAX_ROWS)
+			.ifPresent(builder::withBatchSize);
+		readableConfig.getOptional(SINK_FLUSH_INTERVAL)
+			.ifPresent(builder::withBatchIntervalMs);
+		readableConfig.getOptional(SINK_MAX_RETRIES)
+			.ifPresent(builder::withMaxRetries);
+		return builder.build();
+	}
+
+	private JdbcDmlOptions getJdbcDmlOptions(JdbcOptions jdbcOptions, TableSchema schema) {
+		String[] keyFields = schema.getPrimaryKey()
+			.map(pk -> pk.getColumns().toArray(new String[0]))
+			.orElse(null);
+
+		return JdbcDmlOptions.builder()
+			.withTableName(jdbcOptions.getTableName())
+			.withDialect(jdbcOptions.getDialect())
+			.withFieldNames(schema.getFieldNames())
+			.withKeyFields(keyFields)
+			.build();
+	}
+
+	@Override
+	public String factoryIdentifier() {
+		return IDENTIFIER;
+	}
+
+	@Override
+	public Set<ConfigOption<?>> requiredOptions() {
+		Set<ConfigOption<?>> requiredOptions = new HashSet<>();
+		requiredOptions.add(URL);
+		requiredOptions.add(TABLE_NAME);
+		return requiredOptions;
+	}
+
+	@Override
+	public Set<ConfigOption<?>> optionalOptions() {
+		Set<ConfigOption<?>> optionalOptions = new HashSet<>();
+		optionalOptions.add(DRIVER);
+		optionalOptions.add(USERNAME);
+		optionalOptions.add(PASSWORD);
+		optionalOptions.add(SCAN_PARTITION_COLUMN);
+		optionalOptions.add(SCAN_PARTITION_LOWER_BOUND);
+		optionalOptions.add(SCAN_PARTITION_UPPER_BOUND);
+		optionalOptions.add(SCAN_PARTITION_NUM);
+		optionalOptions.add(SCAN_FETCH_SIZE);
+		optionalOptions.add(LOOKUP_CACHE_MAX_ROWS);
+		optionalOptions.add(LOOKUP_CACHE_TTL);
+		optionalOptions.add(LOOKUP_MAX_RETRIES);
+		optionalOptions.add(SINK_FLUSH_MAX_ROWS);
+		optionalOptions.add(SINK_FLUSH_INTERVAL);
+		optionalOptions.add(SINK_MAX_RETRIES);
+		return optionalOptions;
+	}
+
+	private void validateConfigOptions(ReadableConfig config) {
+		config.getOptional(URL).orElseThrow(() -> new IllegalArgumentException(
+			String.format("Could not find required property: %s", URL.key())));
+		config.getOptional(TABLE_NAME).orElseThrow(() -> new IllegalArgumentException(
+			String.format("Could not find required property: %s", TABLE_NAME.key())));

Review comment:
       ```suggestion
   			String.format("Could not find required option: %s", TABLE_NAME.key())));
   ```

##########
File path: flink-connectors/flink-connector-jdbc/src/main/java/org/apache/flink/connector/jdbc/table/JdbcDynamicTableSourceSinkFactory.java
##########
@@ -0,0 +1,323 @@
+/*
+ * 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.connector.jdbc.table;
+
+import org.apache.flink.annotation.Internal;
+import org.apache.flink.configuration.ConfigOption;
+import org.apache.flink.configuration.ConfigOptions;
+import org.apache.flink.configuration.ReadableConfig;
+import org.apache.flink.connector.jdbc.JdbcExecutionOptions;
+import org.apache.flink.connector.jdbc.dialect.JdbcDialect;
+import org.apache.flink.connector.jdbc.dialect.JdbcDialects;
+import org.apache.flink.connector.jdbc.internal.options.JdbcDmlOptions;
+import org.apache.flink.connector.jdbc.internal.options.JdbcLookupOptions;
+import org.apache.flink.connector.jdbc.internal.options.JdbcOptions;
+import org.apache.flink.connector.jdbc.internal.options.JdbcReadOptions;
+import org.apache.flink.table.api.TableSchema;
+import org.apache.flink.table.connector.sink.DynamicTableSink;
+import org.apache.flink.table.connector.source.DynamicTableSource;
+import org.apache.flink.table.factories.DynamicTableSinkFactory;
+import org.apache.flink.table.factories.DynamicTableSourceFactory;
+import org.apache.flink.table.factories.FactoryUtil;
+import org.apache.flink.table.utils.TableSchemaUtils;
+import org.apache.flink.util.Preconditions;
+
+import java.time.Duration;
+import java.util.Arrays;
+import java.util.HashSet;
+import java.util.Optional;
+import java.util.Set;
+
+import static org.apache.flink.util.Preconditions.checkState;
+
+/**
+ * Factory for creating configured instances of {@link JdbcDynamicTableSource}
+ * and {@link JdbcDynamicTableSink}.
+ */
+@Internal
+public class JdbcDynamicTableSourceSinkFactory implements DynamicTableSourceFactory, DynamicTableSinkFactory {
+
+	public static final String IDENTIFIER = "jdbc";
+	public static final ConfigOption<String> URL = ConfigOptions
+		.key("url")
+		.stringType()
+		.noDefaultValue()
+		.withDescription("the jdbc database url.");
+	public static final ConfigOption<String> TABLE_NAME = ConfigOptions
+		.key("table-name")
+		.stringType()
+		.noDefaultValue()
+		.withDescription("the jdbc table name.");
+	public static final ConfigOption<String> USERNAME = ConfigOptions
+		.key("username")
+		.stringType()
+		.noDefaultValue()
+		.withDescription("the jdbc user name.");
+	public static final ConfigOption<String> PASSWORD = ConfigOptions
+		.key("password")
+		.stringType()
+		.noDefaultValue()
+		.withDescription("the jdbc password.");
+	private static final ConfigOption<String> DRIVER = ConfigOptions
+		.key("driver")
+		.stringType()
+		.noDefaultValue()
+		.withDescription("the class name of the JDBC driver to use to connect to this URL. " +
+			"If not set, it will automatically be derived from the URL.");
+
+	// read config options
+	private static final ConfigOption<String> SCAN_PARTITION_COLUMN = ConfigOptions
+		.key("read.partition.column")
+		.stringType()
+		.noDefaultValue()
+		.withDescription("the column name used for partitioning the input.");
+	private static final ConfigOption<Integer> SCAN_PARTITION_NUM = ConfigOptions
+		.key("scan.partition.num")
+		.intType()
+		.noDefaultValue()
+		.withDescription("the number of partitions.");
+	private static final ConfigOption<Long> SCAN_PARTITION_LOWER_BOUND = ConfigOptions
+		.key("scan.partition.lower-bound")
+		.longType()
+		.noDefaultValue()
+		.withDescription("the smallest value of the first partition.");
+	private static final ConfigOption<Long> SCAN_PARTITION_UPPER_BOUND = ConfigOptions
+		.key("scan.partition.upper-bound")
+		.longType()
+		.noDefaultValue()
+		.withDescription("the largest value of the last partition.");
+	private static final ConfigOption<Integer> SCAN_FETCH_SIZE = ConfigOptions
+		.key("scan.fetch-size")
+		.intType()
+		.defaultValue(0)
+		.withDescription("gives the reader a hint as to the number of rows that should be fetched, from" +
+			" the database when reading per round trip. If the value specified is zero, then the hint is ignored. The" +
+			" default value is zero.");
+
+	// look up config options
+	private static final ConfigOption<Long> LOOKUP_CACHE_MAX_ROWS = ConfigOptions
+		.key("lookup.cache.max-rows")
+		.longType()
+		.defaultValue(-1L)
+		.withDescription("the max number of rows of lookup cache, over this value, the oldest rows will " +
+			"be eliminated. \"cache.max-rows\" and \"cache.ttl\" options must all be specified if any of them is " +
+			"specified. Cache is not enabled as default.");
+	private static final ConfigOption<Duration> LOOKUP_CACHE_TTL = ConfigOptions
+		.key("lookup.cache.ttl")
+		.durationType()
+		.defaultValue(Duration.ofSeconds(10))
+		.withDescription("the cache time to live.");
+	private static final ConfigOption<Integer> LOOKUP_MAX_RETRIES = ConfigOptions
+		.key("lookup.max-retries")
+		.intType()
+		.defaultValue(3)
+		.withDescription("the max retry times if lookup database failed.");
+
+	// write config options
+	private static final ConfigOption<Integer> SINK_FLUSH_MAX_ROWS = ConfigOptions
+		.key("sink.flush.max-rows")
+		.intType()
+		.defaultValue(5000)
+		.withDescription("the flush max size (includes all append, upsert and delete records), over this number" +
+			" of records, will flush data. The default value is 5000.");
+	private static final ConfigOption<Long> SINK_FLUSH_INTERVAL = ConfigOptions
+		.key("sink.flush.interval")
+		.longType()
+		.defaultValue(0L)
+		.withDescription("the flush interval mills, over this time, asynchronous threads will flush data. The " +
+			"default value is 0, which means no asynchronous flush thread will be scheduled.");
+	private static final ConfigOption<Integer> SINK_MAX_RETRIES = ConfigOptions
+		.key("sink.max-retries")
+		.intType()
+		.defaultValue(3)
+		.withDescription("the max retry times if writing records to database failed.");
+
+	@Override
+	public DynamicTableSink createDynamicTableSink(Context context) {
+		final FactoryUtil.TableFactoryHelper helper = FactoryUtil.createTableFactoryHelper(this, context);
+		final ReadableConfig config = helper.getOptions();
+
+		helper.validate();
+		validateConfigOptions(config);
+		JdbcOptions jdbcOptions = getJdbcOptions(config);
+		TableSchema formatSchema = TableSchemaUtils.getPhysicalSchema(context.getCatalogTable().getSchema());
+
+		return new JdbcDynamicTableSink(
+			jdbcOptions,
+			getJdbcExecutionOptions(config),
+			getJdbcDmlOptions(jdbcOptions, context.getCatalogTable().getSchema()),
+			formatSchema);
+	}
+
+	@Override
+	public DynamicTableSource createDynamicTableSource(Context context) {
+		final FactoryUtil.TableFactoryHelper helper = FactoryUtil.createTableFactoryHelper(this, context);
+		final ReadableConfig config = helper.getOptions();
+
+		helper.validate();
+		validateConfigOptions(config);
+		TableSchema formatSchema = TableSchemaUtils.getPhysicalSchema(context.getCatalogTable().getSchema());
+		int[] selectFields = new int[formatSchema.getFieldNames().length];
+		for (int i = 0; i < selectFields.length; i++) {
+			selectFields[i] = i;
+		}
+		return new JdbcDynamicTableSource(
+			getJdbcOptions(helper.getOptions()),
+			getJdbcReadOptions(helper.getOptions()),
+			getJdbcLookupOptions(helper.getOptions()),
+			formatSchema,
+			selectFields);
+	}
+
+	private JdbcOptions getJdbcOptions(ReadableConfig readableConfig) {
+		final String url = readableConfig.get(URL);
+		final JdbcOptions.Builder builder = JdbcOptions.builder()
+			.setDBUrl(url)
+			.setTableName(readableConfig.get(TABLE_NAME))
+			.setDialect(JdbcDialects.get(url).get());
+
+		readableConfig.getOptional(DRIVER).ifPresent(builder::setDriverName);
+		readableConfig.getOptional(USERNAME).ifPresent(builder::setUsername);
+		readableConfig.getOptional(PASSWORD).ifPresent(builder::setPassword);
+		return builder.build();
+	}
+
+	private JdbcReadOptions getJdbcReadOptions(ReadableConfig readableConfig) {
+		final Optional<String> partitionColumnName = readableConfig.getOptional(SCAN_PARTITION_COLUMN);
+		final JdbcReadOptions.Builder builder = JdbcReadOptions.builder();
+		if (partitionColumnName.isPresent()) {
+			builder.setPartitionColumnName(partitionColumnName.get());
+			builder.setPartitionLowerBound(readableConfig.get(SCAN_PARTITION_LOWER_BOUND));
+			builder.setPartitionUpperBound(readableConfig.get(SCAN_PARTITION_UPPER_BOUND));
+			builder.setNumPartitions(readableConfig.get(SCAN_PARTITION_NUM));
+		}
+		readableConfig.getOptional(SCAN_FETCH_SIZE).ifPresent(builder::setFetchSize);
+		return builder.build();
+	}
+
+	private JdbcLookupOptions getJdbcLookupOptions(ReadableConfig readableConfig) {
+		return new JdbcLookupOptions(
+			readableConfig.getOptional(LOOKUP_CACHE_MAX_ROWS).orElse(LOOKUP_CACHE_MAX_ROWS.defaultValue()),
+			readableConfig.getOptional(LOOKUP_CACHE_TTL).orElse(LOOKUP_CACHE_TTL.defaultValue()).toMillis(),
+			readableConfig.getOptional(LOOKUP_MAX_RETRIES).orElse(LOOKUP_MAX_RETRIES.defaultValue()));

Review comment:
       readableConfig.get(LOOKUP_CACHE_MAX_ROWS),




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

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [flink] flinkbot commented on pull request #12176: [FLINK-17029][jdbc]Introduce a new JDBC connector with new property keys

Posted by GitBox <gi...@apache.org>.
flinkbot commented on pull request #12176:
URL: https://github.com/apache/flink/pull/12176#issuecomment-629283127


   <!--
   Meta data
   {
     "version" : 1,
     "metaDataEntries" : [ {
       "hash" : "499722944af576d13757da75f4ad0d08c4538782",
       "status" : "UNKNOWN",
       "url" : "TBD",
       "triggerID" : "499722944af576d13757da75f4ad0d08c4538782",
       "triggerType" : "PUSH"
     } ]
   }-->
   ## CI report:
   
   * 499722944af576d13757da75f4ad0d08c4538782 UNKNOWN
   
   <details>
   <summary>Bot commands</summary>
     The @flinkbot bot supports the following commands:
   
    - `@flinkbot run travis` re-run the last Travis build
    - `@flinkbot run azure` re-run the last Azure build
   </details>


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

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [flink] flinkbot edited a comment on pull request #12176: [FLINK-17029][jdbc]Introduce a new JDBC connector with new property keys

Posted by GitBox <gi...@apache.org>.
flinkbot edited a comment on pull request #12176:
URL: https://github.com/apache/flink/pull/12176#issuecomment-629283127


   <!--
   Meta data
   {
     "version" : 1,
     "metaDataEntries" : [ {
       "hash" : "499722944af576d13757da75f4ad0d08c4538782",
       "status" : "FAILURE",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=1453",
       "triggerID" : "499722944af576d13757da75f4ad0d08c4538782",
       "triggerType" : "PUSH"
     }, {
       "hash" : "d07c3a602f13eaf3ea441e11ece1bf671bfe5beb",
       "status" : "UNKNOWN",
       "url" : "TBD",
       "triggerID" : "d07c3a602f13eaf3ea441e11ece1bf671bfe5beb",
       "triggerType" : "PUSH"
     } ]
   }-->
   ## CI report:
   
   * 499722944af576d13757da75f4ad0d08c4538782 Azure: [FAILURE](https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=1453) 
   * d07c3a602f13eaf3ea441e11ece1bf671bfe5beb UNKNOWN
   
   <details>
   <summary>Bot commands</summary>
     The @flinkbot bot supports the following commands:
   
    - `@flinkbot run travis` re-run the last Travis build
    - `@flinkbot run azure` re-run the last Azure build
   </details>


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

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [flink] flinkbot edited a comment on pull request #12176: [FLINK-17029][jdbc]Introduce a new JDBC connector with new property keys

Posted by GitBox <gi...@apache.org>.
flinkbot edited a comment on pull request #12176:
URL: https://github.com/apache/flink/pull/12176#issuecomment-629283127


   <!--
   Meta data
   {
     "version" : 1,
     "metaDataEntries" : [ {
       "hash" : "499722944af576d13757da75f4ad0d08c4538782",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=1453",
       "triggerID" : "499722944af576d13757da75f4ad0d08c4538782",
       "triggerType" : "PUSH"
     }, {
       "hash" : "d07c3a602f13eaf3ea441e11ece1bf671bfe5beb",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=1547",
       "triggerID" : "d07c3a602f13eaf3ea441e11ece1bf671bfe5beb",
       "triggerType" : "PUSH"
     }, {
       "hash" : "226bafeb2086b9c98a29b398c3c34a325390aa2e",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=1558",
       "triggerID" : "226bafeb2086b9c98a29b398c3c34a325390aa2e",
       "triggerType" : "PUSH"
     }, {
       "hash" : "057d8fa644c9a203a753fe184de8e204ea81918e",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=1566",
       "triggerID" : "057d8fa644c9a203a753fe184de8e204ea81918e",
       "triggerType" : "PUSH"
     }, {
       "hash" : "e557b09c5dc9371c42542211651b79ea749cbf03",
       "status" : "FAILURE",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=1584",
       "triggerID" : "e557b09c5dc9371c42542211651b79ea749cbf03",
       "triggerType" : "PUSH"
     }, {
       "hash" : "7f67ad7536d0041ec9fc5f646dc23972822125f8",
       "status" : "PENDING",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=1607",
       "triggerID" : "7f67ad7536d0041ec9fc5f646dc23972822125f8",
       "triggerType" : "PUSH"
     } ]
   }-->
   ## CI report:
   
   * e557b09c5dc9371c42542211651b79ea749cbf03 Azure: [FAILURE](https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=1584) 
   * 7f67ad7536d0041ec9fc5f646dc23972822125f8 Azure: [PENDING](https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=1607) 
   
   <details>
   <summary>Bot commands</summary>
     The @flinkbot bot supports the following commands:
   
    - `@flinkbot run travis` re-run the last Travis build
    - `@flinkbot run azure` re-run the last Azure build
   </details>


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

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [flink] flinkbot edited a comment on pull request #12176: [FLINK-17029][jdbc]Introduce a new JDBC connector with new property keys

Posted by GitBox <gi...@apache.org>.
flinkbot edited a comment on pull request #12176:
URL: https://github.com/apache/flink/pull/12176#issuecomment-629283127


   <!--
   Meta data
   {
     "version" : 1,
     "metaDataEntries" : [ {
       "hash" : "499722944af576d13757da75f4ad0d08c4538782",
       "status" : "FAILURE",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=1453",
       "triggerID" : "499722944af576d13757da75f4ad0d08c4538782",
       "triggerType" : "PUSH"
     }, {
       "hash" : "d07c3a602f13eaf3ea441e11ece1bf671bfe5beb",
       "status" : "PENDING",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=1547",
       "triggerID" : "d07c3a602f13eaf3ea441e11ece1bf671bfe5beb",
       "triggerType" : "PUSH"
     }, {
       "hash" : "226bafeb2086b9c98a29b398c3c34a325390aa2e",
       "status" : "UNKNOWN",
       "url" : "TBD",
       "triggerID" : "226bafeb2086b9c98a29b398c3c34a325390aa2e",
       "triggerType" : "PUSH"
     } ]
   }-->
   ## CI report:
   
   * 499722944af576d13757da75f4ad0d08c4538782 Azure: [FAILURE](https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=1453) 
   * d07c3a602f13eaf3ea441e11ece1bf671bfe5beb Azure: [PENDING](https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=1547) 
   * 226bafeb2086b9c98a29b398c3c34a325390aa2e UNKNOWN
   
   <details>
   <summary>Bot commands</summary>
     The @flinkbot bot supports the following commands:
   
    - `@flinkbot run travis` re-run the last Travis build
    - `@flinkbot run azure` re-run the last Azure build
   </details>


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

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [flink] flinkbot edited a comment on pull request #12176: [FLINK-17029][jdbc]Introduce a new JDBC connector with new property keys

Posted by GitBox <gi...@apache.org>.
flinkbot edited a comment on pull request #12176:
URL: https://github.com/apache/flink/pull/12176#issuecomment-629256514


   Thanks a lot for your contribution to the Apache Flink project. I'm the @flinkbot. I help the community
   to review your pull request. We will use this comment to track the progress of the review.
   
   
   ## Automated Checks
   Last check on commit b498b54282c579d560905a03ca5d526e5cfcad12 (Fri Oct 16 10:36:10 UTC 2020)
   
   **Warnings:**
    * No documentation files were touched! Remember to keep the Flink docs up to date!
   
   
   <sub>Mention the bot in a comment to re-run the automated checks.</sub>
   ## Review Progress
   
   * ❓ 1. The [description] looks good.
   * ❓ 2. There is [consensus] that the contribution should go into to Flink.
   * ❓ 3. Needs [attention] from.
   * ❓ 4. The change fits into the overall [architecture].
   * ❓ 5. Overall code [quality] is good.
   
   Please see the [Pull Request Review Guide](https://flink.apache.org/contributing/reviewing-prs.html) for a full explanation of the review process.<details>
    The Bot is tracking the review progress through labels. Labels are applied according to the order of the review items. For consensus, approval by a Flink committer of PMC member is required <summary>Bot commands</summary>
     The @flinkbot bot supports the following commands:
   
    - `@flinkbot approve description` to approve one or more aspects (aspects: `description`, `consensus`, `architecture` and `quality`)
    - `@flinkbot approve all` to approve all aspects
    - `@flinkbot approve-until architecture` to approve everything until `architecture`
    - `@flinkbot attention @username1 [@username2 ..]` to require somebody's attention
    - `@flinkbot disapprove architecture` to remove an approval you gave earlier
   </details>


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

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [flink] flinkbot edited a comment on pull request #12176: [FLINK-17029][jdbc]Introduce a new JDBC connector with new property keys

Posted by GitBox <gi...@apache.org>.
flinkbot edited a comment on pull request #12176:
URL: https://github.com/apache/flink/pull/12176#issuecomment-629283127


   <!--
   Meta data
   {
     "version" : 1,
     "metaDataEntries" : [ {
       "hash" : "499722944af576d13757da75f4ad0d08c4538782",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=1453",
       "triggerID" : "499722944af576d13757da75f4ad0d08c4538782",
       "triggerType" : "PUSH"
     }, {
       "hash" : "d07c3a602f13eaf3ea441e11ece1bf671bfe5beb",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=1547",
       "triggerID" : "d07c3a602f13eaf3ea441e11ece1bf671bfe5beb",
       "triggerType" : "PUSH"
     }, {
       "hash" : "226bafeb2086b9c98a29b398c3c34a325390aa2e",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=1558",
       "triggerID" : "226bafeb2086b9c98a29b398c3c34a325390aa2e",
       "triggerType" : "PUSH"
     }, {
       "hash" : "057d8fa644c9a203a753fe184de8e204ea81918e",
       "status" : "FAILURE",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=1566",
       "triggerID" : "057d8fa644c9a203a753fe184de8e204ea81918e",
       "triggerType" : "PUSH"
     }, {
       "hash" : "e557b09c5dc9371c42542211651b79ea749cbf03",
       "status" : "UNKNOWN",
       "url" : "TBD",
       "triggerID" : "e557b09c5dc9371c42542211651b79ea749cbf03",
       "triggerType" : "PUSH"
     } ]
   }-->
   ## CI report:
   
   * 057d8fa644c9a203a753fe184de8e204ea81918e Azure: [FAILURE](https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=1566) 
   * e557b09c5dc9371c42542211651b79ea749cbf03 UNKNOWN
   
   <details>
   <summary>Bot commands</summary>
     The @flinkbot bot supports the following commands:
   
    - `@flinkbot run travis` re-run the last Travis build
    - `@flinkbot run azure` re-run the last Azure build
   </details>


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

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [flink] flinkbot edited a comment on pull request #12176: [FLINK-17029][jdbc]Introduce a new JDBC connector with new property keys

Posted by GitBox <gi...@apache.org>.
flinkbot edited a comment on pull request #12176:
URL: https://github.com/apache/flink/pull/12176#issuecomment-629283127


   <!--
   Meta data
   {
     "version" : 1,
     "metaDataEntries" : [ {
       "hash" : "499722944af576d13757da75f4ad0d08c4538782",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=1453",
       "triggerID" : "499722944af576d13757da75f4ad0d08c4538782",
       "triggerType" : "PUSH"
     }, {
       "hash" : "d07c3a602f13eaf3ea441e11ece1bf671bfe5beb",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=1547",
       "triggerID" : "d07c3a602f13eaf3ea441e11ece1bf671bfe5beb",
       "triggerType" : "PUSH"
     }, {
       "hash" : "226bafeb2086b9c98a29b398c3c34a325390aa2e",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=1558",
       "triggerID" : "226bafeb2086b9c98a29b398c3c34a325390aa2e",
       "triggerType" : "PUSH"
     }, {
       "hash" : "057d8fa644c9a203a753fe184de8e204ea81918e",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=1566",
       "triggerID" : "057d8fa644c9a203a753fe184de8e204ea81918e",
       "triggerType" : "PUSH"
     }, {
       "hash" : "e557b09c5dc9371c42542211651b79ea749cbf03",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=1584",
       "triggerID" : "e557b09c5dc9371c42542211651b79ea749cbf03",
       "triggerType" : "PUSH"
     }, {
       "hash" : "7f67ad7536d0041ec9fc5f646dc23972822125f8",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=1607",
       "triggerID" : "7f67ad7536d0041ec9fc5f646dc23972822125f8",
       "triggerType" : "PUSH"
     }, {
       "hash" : "e006fa3ad7437b5fc9e4089630f7a17c21367971",
       "status" : "SUCCESS",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=1643",
       "triggerID" : "e006fa3ad7437b5fc9e4089630f7a17c21367971",
       "triggerType" : "PUSH"
     }, {
       "hash" : "b498b54282c579d560905a03ca5d526e5cfcad12",
       "status" : "PENDING",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=1663",
       "triggerID" : "b498b54282c579d560905a03ca5d526e5cfcad12",
       "triggerType" : "PUSH"
     } ]
   }-->
   ## CI report:
   
   * e006fa3ad7437b5fc9e4089630f7a17c21367971 Azure: [SUCCESS](https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=1643) 
   * b498b54282c579d560905a03ca5d526e5cfcad12 Azure: [PENDING](https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=1663) 
   
   <details>
   <summary>Bot commands</summary>
     The @flinkbot bot supports the following commands:
   
    - `@flinkbot run travis` re-run the last Travis build
    - `@flinkbot run azure` re-run the last Azure build
   </details>


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

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [flink] flinkbot edited a comment on pull request #12176: [FLINK-17029][jdbc]Introduce a new JDBC connector with new property keys

Posted by GitBox <gi...@apache.org>.
flinkbot edited a comment on pull request #12176:
URL: https://github.com/apache/flink/pull/12176#issuecomment-629283127


   <!--
   Meta data
   {
     "version" : 1,
     "metaDataEntries" : [ {
       "hash" : "499722944af576d13757da75f4ad0d08c4538782",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=1453",
       "triggerID" : "499722944af576d13757da75f4ad0d08c4538782",
       "triggerType" : "PUSH"
     }, {
       "hash" : "d07c3a602f13eaf3ea441e11ece1bf671bfe5beb",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=1547",
       "triggerID" : "d07c3a602f13eaf3ea441e11ece1bf671bfe5beb",
       "triggerType" : "PUSH"
     }, {
       "hash" : "226bafeb2086b9c98a29b398c3c34a325390aa2e",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=1558",
       "triggerID" : "226bafeb2086b9c98a29b398c3c34a325390aa2e",
       "triggerType" : "PUSH"
     }, {
       "hash" : "057d8fa644c9a203a753fe184de8e204ea81918e",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=1566",
       "triggerID" : "057d8fa644c9a203a753fe184de8e204ea81918e",
       "triggerType" : "PUSH"
     }, {
       "hash" : "e557b09c5dc9371c42542211651b79ea749cbf03",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=1584",
       "triggerID" : "e557b09c5dc9371c42542211651b79ea749cbf03",
       "triggerType" : "PUSH"
     }, {
       "hash" : "7f67ad7536d0041ec9fc5f646dc23972822125f8",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=1607",
       "triggerID" : "7f67ad7536d0041ec9fc5f646dc23972822125f8",
       "triggerType" : "PUSH"
     }, {
       "hash" : "e006fa3ad7437b5fc9e4089630f7a17c21367971",
       "status" : "SUCCESS",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=1643",
       "triggerID" : "e006fa3ad7437b5fc9e4089630f7a17c21367971",
       "triggerType" : "PUSH"
     }, {
       "hash" : "b498b54282c579d560905a03ca5d526e5cfcad12",
       "status" : "UNKNOWN",
       "url" : "TBD",
       "triggerID" : "b498b54282c579d560905a03ca5d526e5cfcad12",
       "triggerType" : "PUSH"
     } ]
   }-->
   ## CI report:
   
   * e006fa3ad7437b5fc9e4089630f7a17c21367971 Azure: [SUCCESS](https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=1643) 
   * b498b54282c579d560905a03ca5d526e5cfcad12 UNKNOWN
   
   <details>
   <summary>Bot commands</summary>
     The @flinkbot bot supports the following commands:
   
    - `@flinkbot run travis` re-run the last Travis build
    - `@flinkbot run azure` re-run the last Azure build
   </details>


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

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [flink] wuchong commented on a change in pull request #12176: [FLINK-17029][jdbc]Introduce a new JDBC connector with new property keys

Posted by GitBox <gi...@apache.org>.
wuchong commented on a change in pull request #12176:
URL: https://github.com/apache/flink/pull/12176#discussion_r426267397



##########
File path: flink-connectors/flink-connector-jdbc/src/main/java/org/apache/flink/connector/jdbc/dialect/PostgresDialect.java
##########
@@ -0,0 +1,134 @@
+/*
+ * 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.connector.jdbc.dialect;
+
+import org.apache.flink.connector.jdbc.internal.converter.JdbcRowConverter;
+import org.apache.flink.connector.jdbc.internal.converter.PostgresRowConverter;
+import org.apache.flink.table.types.logical.LogicalTypeRoot;
+import org.apache.flink.table.types.logical.RowType;
+
+import java.util.Arrays;
+import java.util.List;
+import java.util.Optional;
+import java.util.stream.Collectors;
+
+/**
+ * JDBC dialect for Postgres.
+ */
+public class PostgresDialect extends AbstractDialect {
+
+	private static final long serialVersionUID = 1L;
+
+	// Define MAX/MIN precision of TIMESTAMP type according to PostgreSQL docs:
+	// https://www.postgresql.org/docs/12/datatype-datetime.html
+	private static final int MAX_TIMESTAMP_PRECISION = 6;
+	private static final int MIN_TIMESTAMP_PRECISION = 1;
+
+	// Define MAX/MIN precision of TIMESTAMP type according to PostgreSQL docs:
+	// https://www.postgresql.org/docs/12/datatype-numeric.html#DATATYPE-NUMERIC-DECIMAL
+	private static final int MAX_DECIMAL_PRECISION = 1000;
+	private static final int MIN_DECIMAL_PRECISION = 1;
+
+	@Override
+	public boolean canHandle(String url) {
+		return url.startsWith("jdbc:postgresql:");
+	}
+
+	@Override
+	public JdbcRowConverter getRowConverter(RowType rowType) {
+		return new PostgresRowConverter(rowType);
+	}
+
+	@Override
+	public Optional<String> defaultDriverName() {
+		return Optional.of("org.postgresql.Driver");
+	}
+
+	/**
+	 * Postgres upsert query. It use ON CONFLICT ... DO UPDATE SET.. to replace into Postgres.
+	 */
+	@Override
+	public Optional<String> getUpsertStatement(String tableName, String[] fieldNames, String[] uniqueKeyFields) {
+		String uniqueColumns = Arrays.stream(uniqueKeyFields)
+			.map(this::quoteIdentifier)
+			.collect(Collectors.joining(", "));
+		String updateClause = Arrays.stream(fieldNames)
+			.map(f -> quoteIdentifier(f) + "=EXCLUDED." + quoteIdentifier(f))
+			.collect(Collectors.joining(", "));
+		return Optional.of(getInsertIntoStatement(tableName, fieldNames) +
+			" ON CONFLICT (" + uniqueColumns + ")" +
+			" DO UPDATE SET " + updateClause
+		);
+	}
+
+	@Override
+	public String quoteIdentifier(String identifier) {
+		return identifier;
+	}
+
+	@Override
+	public String dialectName() {
+		return "Postgres";

Review comment:
       Nit: "PostgreSQL";

##########
File path: flink-connectors/flink-connector-jdbc/src/main/java/org/apache/flink/connector/jdbc/table/JdbcDynamicTableSourceSinkFactory.java
##########
@@ -0,0 +1,320 @@
+/*
+ * 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.connector.jdbc.table;
+
+import org.apache.flink.annotation.Internal;
+import org.apache.flink.configuration.ConfigOption;
+import org.apache.flink.configuration.ConfigOptions;
+import org.apache.flink.configuration.ReadableConfig;
+import org.apache.flink.connector.jdbc.JdbcExecutionOptions;
+import org.apache.flink.connector.jdbc.dialect.JdbcDialect;
+import org.apache.flink.connector.jdbc.dialect.JdbcDialects;
+import org.apache.flink.connector.jdbc.internal.options.JdbcDmlOptions;
+import org.apache.flink.connector.jdbc.internal.options.JdbcLookupOptions;
+import org.apache.flink.connector.jdbc.internal.options.JdbcOptions;
+import org.apache.flink.connector.jdbc.internal.options.JdbcReadOptions;
+import org.apache.flink.table.api.TableSchema;
+import org.apache.flink.table.connector.sink.DynamicTableSink;
+import org.apache.flink.table.connector.source.DynamicTableSource;
+import org.apache.flink.table.factories.DynamicTableSinkFactory;
+import org.apache.flink.table.factories.DynamicTableSourceFactory;
+import org.apache.flink.table.factories.FactoryUtil;
+import org.apache.flink.table.utils.TableSchemaUtils;
+import org.apache.flink.util.Preconditions;
+
+import java.time.Duration;
+import java.util.Arrays;
+import java.util.HashSet;
+import java.util.Optional;
+import java.util.Set;
+
+import static org.apache.flink.util.Preconditions.checkState;
+
+/**
+ * Factory for creating configured instances of {@link JdbcDynamicTableSource}
+ * and {@link JdbcDynamicTableSink}.
+ */
+@Internal
+public class JdbcDynamicTableSourceSinkFactory implements DynamicTableSourceFactory, DynamicTableSinkFactory {
+
+	public static final String IDENTIFIER = "jdbc";
+	public static final ConfigOption<String> URL = ConfigOptions
+		.key("url")
+		.stringType()
+		.noDefaultValue()
+		.withDescription("the jdbc database url.");
+	public static final ConfigOption<String> TABLE_NAME = ConfigOptions
+		.key("table-name")
+		.stringType()
+		.noDefaultValue()
+		.withDescription("the jdbc table name.");
+	public static final ConfigOption<String> USERNAME = ConfigOptions
+		.key("username")
+		.stringType()
+		.noDefaultValue()
+		.withDescription("the jdbc user name.");
+	public static final ConfigOption<String> PASSWORD = ConfigOptions
+		.key("password")
+		.stringType()
+		.noDefaultValue()
+		.withDescription("the jdbc password.");
+	private static final ConfigOption<String> DRIVER = ConfigOptions
+		.key("driver")
+		.stringType()
+		.noDefaultValue()
+		.withDescription("the class name of the JDBC driver to use to connect to this URL. " +
+			"If not set, it will automatically be derived from the URL.");
+
+	// read config options
+	private static final ConfigOption<String> SCAN_PARTITION_COLUMN = ConfigOptions
+		.key("scan.partition.column")
+		.stringType()
+		.noDefaultValue()
+		.withDescription("the column name used for partitioning the input.");
+	private static final ConfigOption<Integer> SCAN_PARTITION_NUM = ConfigOptions
+		.key("scan.partition.num")
+		.intType()
+		.noDefaultValue()
+		.withDescription("the number of partitions.");
+	private static final ConfigOption<Long> SCAN_PARTITION_LOWER_BOUND = ConfigOptions
+		.key("scan.partition.lower-bound")
+		.longType()
+		.noDefaultValue()
+		.withDescription("the smallest value of the first partition.");
+	private static final ConfigOption<Long> SCAN_PARTITION_UPPER_BOUND = ConfigOptions
+		.key("scan.partition.upper-bound")
+		.longType()
+		.noDefaultValue()
+		.withDescription("the largest value of the last partition.");
+	private static final ConfigOption<Integer> SCAN_FETCH_SIZE = ConfigOptions
+		.key("scan.fetch-size")
+		.intType()
+		.defaultValue(0)
+		.withDescription("gives the reader a hint as to the number of rows that should be fetched, from" +
+			" the database when reading per round trip. If the value specified is zero, then the hint is ignored. The" +
+			" default value is zero.");
+
+	// look up config options
+	private static final ConfigOption<Long> LOOKUP_CACHE_MAX_ROWS = ConfigOptions
+		.key("lookup.cache.max-rows")
+		.longType()
+		.defaultValue(-1L)
+		.withDescription("the max number of rows of lookup cache, over this value, the oldest rows will " +
+			"be eliminated. \"cache.max-rows\" and \"cache.ttl\" options must all be specified if any of them is " +
+			"specified. Cache is not enabled as default.");
+	private static final ConfigOption<Duration> LOOKUP_CACHE_TTL = ConfigOptions
+		.key("lookup.cache.ttl")
+		.durationType()
+		.defaultValue(Duration.ofSeconds(10))
+		.withDescription("the cache time to live.");
+	private static final ConfigOption<Integer> LOOKUP_MAX_RETRIES = ConfigOptions
+		.key("lookup.max-retries")
+		.intType()
+		.defaultValue(3)
+		.withDescription("the max retry times if lookup database failed.");
+
+	// write config options
+	private static final ConfigOption<Integer> SINK_FLUSH_MAX_ROWS = ConfigOptions
+		.key("sink.flush.max-rows")
+		.intType()
+		.defaultValue(5000)
+		.withDescription("the flush max size (includes all append, upsert and delete records), over this number" +
+			" of records, will flush data. The default value is 5000.");
+	private static final ConfigOption<Long> SINK_FLUSH_INTERVAL = ConfigOptions
+		.key("sink.flush.interval")
+		.longType()
+		.defaultValue(0L)
+		.withDescription("the flush interval mills, over this time, asynchronous threads will flush data. The " +
+			"default value is 0, which means no asynchronous flush thread will be scheduled.");
+	private static final ConfigOption<Integer> SINK_MAX_RETRIES = ConfigOptions
+		.key("sink.max-retries")
+		.intType()
+		.defaultValue(3)
+		.withDescription("the max retry times if writing records to database failed.");
+
+	@Override
+	public DynamicTableSink createDynamicTableSink(Context context) {
+		final FactoryUtil.TableFactoryHelper helper = FactoryUtil.createTableFactoryHelper(this, context);
+		final ReadableConfig config = helper.getOptions();
+
+		helper.validate();
+		validateConfigOptions(config);
+		JdbcOptions jdbcOptions = getJdbcOptions(config);
+		TableSchema physicalSchema = TableSchemaUtils.getPhysicalSchema(context.getCatalogTable().getSchema());
+
+		return new JdbcDynamicTableSink(
+			jdbcOptions,
+			getJdbcExecutionOptions(config),
+			getJdbcDmlOptions(jdbcOptions, context.getCatalogTable().getSchema()),

Review comment:
       Should use `physicalSchema` here. 
   We should keep primary key in `TableSchemaUtils.getPhysicalSchema`. ES connector also did this: https://github.com/apache/flink/pull/12184/files#diff-8fe9a415913cbd47651e39f66af0114e

##########
File path: flink-connectors/flink-connector-jdbc/src/main/java/org/apache/flink/connector/jdbc/table/JdbcRowDataOutputFormat.java
##########
@@ -0,0 +1,295 @@
+/*
+ * 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.connector.jdbc.table;
+
+import org.apache.flink.annotation.Internal;
+import org.apache.flink.api.common.functions.RuntimeContext;
+import org.apache.flink.api.common.typeinfo.TypeInformation;
+import org.apache.flink.api.common.typeutils.TypeSerializer;
+import org.apache.flink.connector.jdbc.JdbcExecutionOptions;
+import org.apache.flink.connector.jdbc.JdbcStatementBuilder;
+import org.apache.flink.connector.jdbc.dialect.JdbcDialect;
+import org.apache.flink.connector.jdbc.internal.JdbcBatchingOutputFormat;
+import org.apache.flink.connector.jdbc.internal.connection.JdbcConnectionProvider;
+import org.apache.flink.connector.jdbc.internal.connection.SimpleJdbcConnectionProvider;
+import org.apache.flink.connector.jdbc.internal.converter.JdbcRowConverter;
+import org.apache.flink.connector.jdbc.internal.executor.InsertOrUpdateJdbcExecutor;
+import org.apache.flink.connector.jdbc.internal.executor.JdbcBatchStatementExecutor;
+import org.apache.flink.connector.jdbc.internal.options.JdbcDmlOptions;
+import org.apache.flink.connector.jdbc.internal.options.JdbcOptions;
+import org.apache.flink.connector.jdbc.utils.JdbcUtils;
+import org.apache.flink.table.data.GenericRowData;
+import org.apache.flink.table.data.RowData;
+import org.apache.flink.table.data.RowData.FieldGetter;
+import org.apache.flink.table.types.DataType;
+import org.apache.flink.table.types.logical.LogicalType;
+import org.apache.flink.table.types.logical.RowType;
+import org.apache.flink.types.Row;
+
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import java.io.IOException;
+import java.sql.SQLException;
+import java.util.Arrays;
+import java.util.function.Function;
+
+import static org.apache.flink.table.data.RowData.createFieldGetter;
+import static org.apache.flink.util.Preconditions.checkArgument;
+import static org.apache.flink.util.Preconditions.checkNotNull;
+
+/**
+ * OutputFormat for {@link JdbcDynamicTableSource}.
+ */
+@Internal
+public class JdbcRowDataOutputFormat extends JdbcBatchingOutputFormat<RowData, RowData, JdbcBatchStatementExecutor<RowData>> {
+
+	private static final long serialVersionUID = 1L;
+	private static final Logger LOG = LoggerFactory.getLogger(JdbcRowDataOutputFormat.class);
+
+	private JdbcBatchStatementExecutor<RowData> deleteExecutor;
+	private final JdbcDmlOptions dmlOptions;
+	private final LogicalType[] logicalTypes;
+
+	private JdbcRowDataOutputFormat(
+			JdbcConnectionProvider connectionProvider,
+			JdbcDmlOptions dmlOptions,
+			JdbcExecutionOptions batchOptions,
+			TypeInformation<RowData> rowDataTypeInfo,
+			LogicalType[] logicalTypes) {
+		super(
+			connectionProvider,
+			batchOptions,
+			ctx -> createUpsertRowExecutor(dmlOptions, ctx, rowDataTypeInfo, logicalTypes),
+			RecordExtractor.identity());
+		this.dmlOptions = dmlOptions;
+		this.logicalTypes = logicalTypes;
+	}
+
+	private JdbcRowDataOutputFormat(
+			JdbcConnectionProvider connectionProvider,
+			JdbcDmlOptions dmlOptions,
+			JdbcExecutionOptions batchOptions,
+			TypeInformation<RowData> rowDataTypeInfo,
+			LogicalType[] logicalTypes,
+			String sql) {
+		super(connectionProvider,
+			batchOptions,
+			ctx -> createSimpleRowDataExecutor(dmlOptions.getDialect(), sql, logicalTypes, ctx, rowDataTypeInfo),
+			RecordExtractor.identity());
+		this.dmlOptions = dmlOptions;
+		this.logicalTypes = logicalTypes;
+	}
+
+	@Override
+	public void open(int taskNumber, int numTasks) throws IOException {
+		deleteExecutor = createDeleteExecutor();
+		super.open(taskNumber, numTasks);
+		try {
+			deleteExecutor.open(connection);
+		} catch (SQLException e) {
+			throw new IOException(e);
+		}
+	}
+
+	private JdbcBatchStatementExecutor<RowData> createDeleteExecutor() {
+		int[] pkFields = Arrays.stream(dmlOptions.getFieldNames()).mapToInt(Arrays.asList(dmlOptions.getFieldNames())::indexOf).toArray();
+		LogicalType[] pkTypes = Arrays.stream(pkFields).mapToObj(f -> logicalTypes[f]).toArray(LogicalType[]::new);
+		String deleteSql = dmlOptions.getDialect().getDeleteStatement(dmlOptions.getTableName(), dmlOptions.getFieldNames());
+		return createKeyedRowExecutor(dmlOptions.getDialect(), pkFields, pkTypes, deleteSql, logicalTypes);
+	}
+
+	@Override
+	protected void addToBatch(RowData original, RowData extracted) throws SQLException {
+		switch (original.getRowKind()) {
+			case INSERT:
+			case UPDATE_AFTER:
+				super.addToBatch(original, extracted);
+				break;
+			case DELETE:
+			case UPDATE_BEFORE:
+				deleteExecutor.addToBatch(extracted);
+				break;
+			default:
+				throw new UnsupportedOperationException(
+					String.format("unknown row kind, the supported row kinds is: INSERT, UPDATE_BEFORE, UPDATE_AFTER," +
+						" DELETE, but get: %s.", original.getRowKind()));
+		}
+	}
+
+	@Override
+	public synchronized void close() {
+		try {
+			super.close();
+		} finally {
+			try {
+				if (deleteExecutor != null) {
+					deleteExecutor.close();
+				}
+			} catch (SQLException e) {
+				LOG.warn("unable to close delete statement runner", e);
+			}
+		}
+	}
+
+	@Override
+	protected void attemptFlush() throws SQLException {
+		super.attemptFlush();
+		deleteExecutor.executeBatch();
+	}
+
+	private static JdbcBatchStatementExecutor<RowData> createKeyedRowExecutor(JdbcDialect dialect, int[] pkFields, LogicalType[] pkTypes, String sql, LogicalType[] logicalTypes) {
+		final JdbcRowConverter rowConverter = dialect.getRowConverter(RowType.of(logicalTypes));
+		final Function<RowData, RowData>  keyExtractor = createRowKeyExtractor(logicalTypes, pkFields);
+		return JdbcBatchStatementExecutor.keyed(
+			sql,
+			keyExtractor,
+			(st, record) -> rowConverter
+				.toExternal(keyExtractor.apply(record), st));
+	}
+
+	private static JdbcBatchStatementExecutor<RowData> createUpsertRowExecutor(JdbcDmlOptions opt, RuntimeContext ctx, TypeInformation<RowData> rowDataTypeInfo, LogicalType[] logicalTypes) {
+		checkArgument(opt.getKeyFields().isPresent());
+
+		int[] pkFields = Arrays.stream(opt.getKeyFields().get()).mapToInt(Arrays.asList(opt.getFieldNames())::indexOf).toArray();
+		LogicalType[] pkTypes = Arrays.stream(pkFields).mapToObj(f -> logicalTypes[f]).toArray(LogicalType[]::new);
+		JdbcDialect dialect = opt.getDialect();
+		final TypeSerializer<RowData> typeSerializer = rowDataTypeInfo.createSerializer(ctx.getExecutionConfig());
+		return opt.getDialect()
+			.getUpsertStatement(opt.getTableName(), opt.getFieldNames(), opt.getKeyFields().get())
+			.map(sql -> createSimpleRowDataExecutor(dialect, sql, logicalTypes, ctx, rowDataTypeInfo))
+			.orElseGet(() ->
+				new InsertOrUpdateJdbcExecutor<>(
+					opt.getDialect().getRowExistsStatement(opt.getTableName(), opt.getKeyFields().get()),
+					opt.getDialect().getInsertIntoStatement(opt.getTableName(), opt.getFieldNames()),
+					opt.getDialect().getUpdateStatement(opt.getTableName(), opt.getFieldNames(), opt.getKeyFields().get()),
+					createRowDataJdbcStatementBuilder(dialect, pkTypes),
+					createRowDataJdbcStatementBuilder(dialect, logicalTypes),
+					createRowDataJdbcStatementBuilder(dialect, logicalTypes),
+					createRowKeyExtractor(logicalTypes, pkFields),
+					ctx.getExecutionConfig().isObjectReuseEnabled() ? typeSerializer::copy : r -> r));
+	}
+
+	private static Function<RowData, RowData> createRowKeyExtractor(LogicalType[] logicalTypes, int[] pkFields) {
+		final FieldGetter[] fieldGetters = new FieldGetter[pkFields.length];
+		for (int i = 0; i < pkFields.length; i++) {
+			fieldGetters[i] = createFieldGetter(logicalTypes[pkFields[i]], pkFields[i]);
+		}
+		return row -> getPrimaryKey(row, fieldGetters);
+	}
+
+	private static JdbcBatchStatementExecutor<RowData> createSimpleRowDataExecutor(JdbcDialect dialect, String sql, LogicalType[] fieldTypes, RuntimeContext ctx, TypeInformation<RowData> rowDataTypeInfo) {
+		final TypeSerializer<RowData> typeSerializer = rowDataTypeInfo.createSerializer(ctx.getExecutionConfig());
+		return JdbcBatchStatementExecutor.simple(
+			sql,
+			createRowDataJdbcStatementBuilder(dialect, fieldTypes),
+			ctx.getExecutionConfig().isObjectReuseEnabled() ? typeSerializer::copy : Function.identity());
+	}
+
+	/**
+	 * Creates a {@link JdbcStatementBuilder} for {@link Row} using the provided SQL types array.
+	 * Uses {@link JdbcUtils#setRecordToStatement}
+	 */
+	private static JdbcStatementBuilder<RowData> createRowDataJdbcStatementBuilder(JdbcDialect dialect, LogicalType[] types) {
+		return (st, record) -> dialect.getRowConverter(RowType.of(types)).toExternal(record, st);

Review comment:
       Do not call `dialect.getRowConverter` during runtime.

##########
File path: flink-connectors/flink-connector-jdbc/src/main/java/org/apache/flink/connector/jdbc/table/JdbcDynamicTableSource.java
##########
@@ -0,0 +1,162 @@
+/*
+ * 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.connector.jdbc.table;
+
+import org.apache.flink.annotation.Internal;
+import org.apache.flink.api.java.typeutils.RowTypeInfo;
+import org.apache.flink.connector.jdbc.dialect.JdbcDialect;
+import org.apache.flink.connector.jdbc.internal.options.JdbcLookupOptions;
+import org.apache.flink.connector.jdbc.internal.options.JdbcOptions;
+import org.apache.flink.connector.jdbc.internal.options.JdbcReadOptions;
+import org.apache.flink.connector.jdbc.split.JdbcNumericBetweenParametersProvider;
+import org.apache.flink.table.api.TableSchema;
+import org.apache.flink.table.connector.ChangelogMode;
+import org.apache.flink.table.connector.source.DynamicTableSource;
+import org.apache.flink.table.connector.source.InputFormatProvider;
+import org.apache.flink.table.connector.source.LookupTableSource;
+import org.apache.flink.table.connector.source.ScanTableSource;
+import org.apache.flink.table.connector.source.TableFunctionProvider;
+import org.apache.flink.table.types.DataType;
+import org.apache.flink.table.types.logical.LogicalType;
+import org.apache.flink.table.types.logical.RowType;
+
+import java.util.Arrays;
+import java.util.Objects;
+
+import static org.apache.flink.table.types.utils.TypeConversions.fromDataTypeToLegacyInfo;
+
+/**
+ * A {@link DynamicTableSource} for JDBC.
+ */
+@Internal
+public class JdbcDynamicTableSource implements ScanTableSource, LookupTableSource {
+
+	private static final String name = "JdbcTableSource";
+	private final JdbcOptions options;
+	private final JdbcReadOptions readOptions;
+	private final JdbcLookupOptions lookupOptions;
+	private final TableSchema schema;
+	private final int[] selectFields;
+
+	public JdbcDynamicTableSource(
+		JdbcOptions options,
+		JdbcReadOptions readOptions,
+		JdbcLookupOptions lookupOptions,
+		TableSchema schema,
+		int[] selectFields) {
+		this.options = options;
+		this.readOptions = readOptions;
+		this.lookupOptions = lookupOptions;
+		this.schema = schema;
+		this.selectFields = selectFields;
+	}
+
+	@Override
+	public LookupRuntimeProvider getLookupRuntimeProvider(LookupTableSource.Context context) {
+		// JDBC only support non-nested look up keys
+		String[] keyNames = new String[context.getKeys().length];
+		for (int i = 0; i < keyNames.length; i++) {
+			int index = context.getKeys()[i][0];
+			keyNames[i] = schema.getFieldNames()[index];
+		}
+		return TableFunctionProvider.of(JdbcDynamicLookupFunction.builder()
+			.setFieldNames(schema.getFieldNames())
+			.setFieldTypes(schema.getFieldDataTypes())
+			.setKeyNames(keyNames)
+			.setOptions(options)
+			.setLookupOptions(lookupOptions)
+			.build());
+	}
+
+	@Override
+	public ScanRuntimeProvider getScanRuntimeProvider(ScanTableSource.Context runtimeProviderContext) {
+		final DataType rowDataType = schema.toPhysicalRowDataType();
+		final RowTypeInfo rowTypeInfo = (RowTypeInfo) fromDataTypeToLegacyInfo(rowDataType);
+		final JdbcDynamicInputFormat.Builder builder = JdbcDynamicInputFormat.builder()
+			.setDrivername(options.getDriverName())
+			.setDBUrl(options.getDbURL())
+			.setUsername(options.getUsername().orElse(null))
+			.setPassword(options.getPassword().orElse(null))
+			.setRowTypeInfo(new RowTypeInfo(rowTypeInfo.getFieldTypes(), rowTypeInfo.getFieldNames()));
+
+		if (readOptions.getFetchSize() != 0) {
+			builder.setFetchSize(readOptions.getFetchSize());
+		}
+		final JdbcDialect dialect = options.getDialect();
+		String query = dialect.getSelectFromStatement(
+			options.getTableName(), rowTypeInfo.getFieldNames(), new String[0]);
+		if (readOptions.getPartitionColumnName().isPresent()) {
+			long lowerBound = readOptions.getPartitionLowerBound().get();
+			long upperBound = readOptions.getPartitionUpperBound().get();
+			int numPartitions = readOptions.getNumPartitions().get();
+			builder.setParametersProvider(
+				new JdbcNumericBetweenParametersProvider(lowerBound, upperBound).ofBatchNum(numPartitions));
+			query += " WHERE " +
+				dialect.quoteIdentifier(readOptions.getPartitionColumnName().get()) +
+				" BETWEEN ? AND ?";
+		}
+		builder.setQuery(query);
+		final RowType rowType = RowType.of(
+			Arrays.stream(schema.getFieldDataTypes())
+				.map(DataType::getLogicalType)
+				.toArray(LogicalType[]::new),
+			schema.getFieldNames());

Review comment:
       Not resolved yet.

##########
File path: flink-connectors/flink-connector-jdbc/src/main/java/org/apache/flink/connector/jdbc/table/JdbcDynamicTableSourceSinkFactory.java
##########
@@ -0,0 +1,320 @@
+/*
+ * 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.connector.jdbc.table;
+
+import org.apache.flink.annotation.Internal;
+import org.apache.flink.configuration.ConfigOption;
+import org.apache.flink.configuration.ConfigOptions;
+import org.apache.flink.configuration.ReadableConfig;
+import org.apache.flink.connector.jdbc.JdbcExecutionOptions;
+import org.apache.flink.connector.jdbc.dialect.JdbcDialect;
+import org.apache.flink.connector.jdbc.dialect.JdbcDialects;
+import org.apache.flink.connector.jdbc.internal.options.JdbcDmlOptions;
+import org.apache.flink.connector.jdbc.internal.options.JdbcLookupOptions;
+import org.apache.flink.connector.jdbc.internal.options.JdbcOptions;
+import org.apache.flink.connector.jdbc.internal.options.JdbcReadOptions;
+import org.apache.flink.table.api.TableSchema;
+import org.apache.flink.table.connector.sink.DynamicTableSink;
+import org.apache.flink.table.connector.source.DynamicTableSource;
+import org.apache.flink.table.factories.DynamicTableSinkFactory;
+import org.apache.flink.table.factories.DynamicTableSourceFactory;
+import org.apache.flink.table.factories.FactoryUtil;
+import org.apache.flink.table.utils.TableSchemaUtils;
+import org.apache.flink.util.Preconditions;
+
+import java.time.Duration;
+import java.util.Arrays;
+import java.util.HashSet;
+import java.util.Optional;
+import java.util.Set;
+
+import static org.apache.flink.util.Preconditions.checkState;
+
+/**
+ * Factory for creating configured instances of {@link JdbcDynamicTableSource}
+ * and {@link JdbcDynamicTableSink}.
+ */
+@Internal
+public class JdbcDynamicTableSourceSinkFactory implements DynamicTableSourceFactory, DynamicTableSinkFactory {
+
+	public static final String IDENTIFIER = "jdbc";
+	public static final ConfigOption<String> URL = ConfigOptions
+		.key("url")
+		.stringType()
+		.noDefaultValue()
+		.withDescription("the jdbc database url.");
+	public static final ConfigOption<String> TABLE_NAME = ConfigOptions
+		.key("table-name")
+		.stringType()
+		.noDefaultValue()
+		.withDescription("the jdbc table name.");
+	public static final ConfigOption<String> USERNAME = ConfigOptions
+		.key("username")
+		.stringType()
+		.noDefaultValue()
+		.withDescription("the jdbc user name.");
+	public static final ConfigOption<String> PASSWORD = ConfigOptions
+		.key("password")
+		.stringType()
+		.noDefaultValue()
+		.withDescription("the jdbc password.");
+	private static final ConfigOption<String> DRIVER = ConfigOptions
+		.key("driver")
+		.stringType()
+		.noDefaultValue()
+		.withDescription("the class name of the JDBC driver to use to connect to this URL. " +
+			"If not set, it will automatically be derived from the URL.");
+
+	// read config options
+	private static final ConfigOption<String> SCAN_PARTITION_COLUMN = ConfigOptions
+		.key("scan.partition.column")
+		.stringType()
+		.noDefaultValue()
+		.withDescription("the column name used for partitioning the input.");
+	private static final ConfigOption<Integer> SCAN_PARTITION_NUM = ConfigOptions
+		.key("scan.partition.num")
+		.intType()
+		.noDefaultValue()
+		.withDescription("the number of partitions.");
+	private static final ConfigOption<Long> SCAN_PARTITION_LOWER_BOUND = ConfigOptions
+		.key("scan.partition.lower-bound")
+		.longType()
+		.noDefaultValue()
+		.withDescription("the smallest value of the first partition.");
+	private static final ConfigOption<Long> SCAN_PARTITION_UPPER_BOUND = ConfigOptions
+		.key("scan.partition.upper-bound")
+		.longType()
+		.noDefaultValue()
+		.withDescription("the largest value of the last partition.");
+	private static final ConfigOption<Integer> SCAN_FETCH_SIZE = ConfigOptions
+		.key("scan.fetch-size")
+		.intType()
+		.defaultValue(0)
+		.withDescription("gives the reader a hint as to the number of rows that should be fetched, from" +
+			" the database when reading per round trip. If the value specified is zero, then the hint is ignored. The" +
+			" default value is zero.");
+
+	// look up config options
+	private static final ConfigOption<Long> LOOKUP_CACHE_MAX_ROWS = ConfigOptions
+		.key("lookup.cache.max-rows")
+		.longType()
+		.defaultValue(-1L)
+		.withDescription("the max number of rows of lookup cache, over this value, the oldest rows will " +
+			"be eliminated. \"cache.max-rows\" and \"cache.ttl\" options must all be specified if any of them is " +
+			"specified. Cache is not enabled as default.");
+	private static final ConfigOption<Duration> LOOKUP_CACHE_TTL = ConfigOptions
+		.key("lookup.cache.ttl")
+		.durationType()
+		.defaultValue(Duration.ofSeconds(10))
+		.withDescription("the cache time to live.");
+	private static final ConfigOption<Integer> LOOKUP_MAX_RETRIES = ConfigOptions
+		.key("lookup.max-retries")
+		.intType()
+		.defaultValue(3)
+		.withDescription("the max retry times if lookup database failed.");
+
+	// write config options
+	private static final ConfigOption<Integer> SINK_FLUSH_MAX_ROWS = ConfigOptions
+		.key("sink.flush.max-rows")

Review comment:
       "sink.buffer-flush.max-rows"

##########
File path: flink-connectors/flink-connector-jdbc/src/main/java/org/apache/flink/connector/jdbc/table/JdbcDynamicTableSourceSinkFactory.java
##########
@@ -0,0 +1,320 @@
+/*
+ * 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.connector.jdbc.table;
+
+import org.apache.flink.annotation.Internal;
+import org.apache.flink.configuration.ConfigOption;
+import org.apache.flink.configuration.ConfigOptions;
+import org.apache.flink.configuration.ReadableConfig;
+import org.apache.flink.connector.jdbc.JdbcExecutionOptions;
+import org.apache.flink.connector.jdbc.dialect.JdbcDialect;
+import org.apache.flink.connector.jdbc.dialect.JdbcDialects;
+import org.apache.flink.connector.jdbc.internal.options.JdbcDmlOptions;
+import org.apache.flink.connector.jdbc.internal.options.JdbcLookupOptions;
+import org.apache.flink.connector.jdbc.internal.options.JdbcOptions;
+import org.apache.flink.connector.jdbc.internal.options.JdbcReadOptions;
+import org.apache.flink.table.api.TableSchema;
+import org.apache.flink.table.connector.sink.DynamicTableSink;
+import org.apache.flink.table.connector.source.DynamicTableSource;
+import org.apache.flink.table.factories.DynamicTableSinkFactory;
+import org.apache.flink.table.factories.DynamicTableSourceFactory;
+import org.apache.flink.table.factories.FactoryUtil;
+import org.apache.flink.table.utils.TableSchemaUtils;
+import org.apache.flink.util.Preconditions;
+
+import java.time.Duration;
+import java.util.Arrays;
+import java.util.HashSet;
+import java.util.Optional;
+import java.util.Set;
+
+import static org.apache.flink.util.Preconditions.checkState;
+
+/**
+ * Factory for creating configured instances of {@link JdbcDynamicTableSource}
+ * and {@link JdbcDynamicTableSink}.
+ */
+@Internal
+public class JdbcDynamicTableSourceSinkFactory implements DynamicTableSourceFactory, DynamicTableSinkFactory {
+
+	public static final String IDENTIFIER = "jdbc";
+	public static final ConfigOption<String> URL = ConfigOptions
+		.key("url")
+		.stringType()
+		.noDefaultValue()
+		.withDescription("the jdbc database url.");
+	public static final ConfigOption<String> TABLE_NAME = ConfigOptions
+		.key("table-name")
+		.stringType()
+		.noDefaultValue()
+		.withDescription("the jdbc table name.");
+	public static final ConfigOption<String> USERNAME = ConfigOptions
+		.key("username")
+		.stringType()
+		.noDefaultValue()
+		.withDescription("the jdbc user name.");
+	public static final ConfigOption<String> PASSWORD = ConfigOptions
+		.key("password")
+		.stringType()
+		.noDefaultValue()
+		.withDescription("the jdbc password.");
+	private static final ConfigOption<String> DRIVER = ConfigOptions
+		.key("driver")
+		.stringType()
+		.noDefaultValue()
+		.withDescription("the class name of the JDBC driver to use to connect to this URL. " +
+			"If not set, it will automatically be derived from the URL.");
+
+	// read config options
+	private static final ConfigOption<String> SCAN_PARTITION_COLUMN = ConfigOptions
+		.key("scan.partition.column")
+		.stringType()
+		.noDefaultValue()
+		.withDescription("the column name used for partitioning the input.");
+	private static final ConfigOption<Integer> SCAN_PARTITION_NUM = ConfigOptions
+		.key("scan.partition.num")
+		.intType()
+		.noDefaultValue()
+		.withDescription("the number of partitions.");
+	private static final ConfigOption<Long> SCAN_PARTITION_LOWER_BOUND = ConfigOptions
+		.key("scan.partition.lower-bound")
+		.longType()
+		.noDefaultValue()
+		.withDescription("the smallest value of the first partition.");
+	private static final ConfigOption<Long> SCAN_PARTITION_UPPER_BOUND = ConfigOptions
+		.key("scan.partition.upper-bound")
+		.longType()
+		.noDefaultValue()
+		.withDescription("the largest value of the last partition.");
+	private static final ConfigOption<Integer> SCAN_FETCH_SIZE = ConfigOptions
+		.key("scan.fetch-size")
+		.intType()
+		.defaultValue(0)
+		.withDescription("gives the reader a hint as to the number of rows that should be fetched, from" +
+			" the database when reading per round trip. If the value specified is zero, then the hint is ignored. The" +
+			" default value is zero.");
+
+	// look up config options
+	private static final ConfigOption<Long> LOOKUP_CACHE_MAX_ROWS = ConfigOptions
+		.key("lookup.cache.max-rows")
+		.longType()
+		.defaultValue(-1L)
+		.withDescription("the max number of rows of lookup cache, over this value, the oldest rows will " +
+			"be eliminated. \"cache.max-rows\" and \"cache.ttl\" options must all be specified if any of them is " +
+			"specified. Cache is not enabled as default.");
+	private static final ConfigOption<Duration> LOOKUP_CACHE_TTL = ConfigOptions
+		.key("lookup.cache.ttl")
+		.durationType()
+		.defaultValue(Duration.ofSeconds(10))
+		.withDescription("the cache time to live.");
+	private static final ConfigOption<Integer> LOOKUP_MAX_RETRIES = ConfigOptions
+		.key("lookup.max-retries")
+		.intType()
+		.defaultValue(3)
+		.withDescription("the max retry times if lookup database failed.");
+
+	// write config options
+	private static final ConfigOption<Integer> SINK_FLUSH_MAX_ROWS = ConfigOptions
+		.key("sink.flush.max-rows")
+		.intType()
+		.defaultValue(5000)
+		.withDescription("the flush max size (includes all append, upsert and delete records), over this number" +
+			" of records, will flush data. The default value is 5000.");
+	private static final ConfigOption<Long> SINK_FLUSH_INTERVAL = ConfigOptions
+		.key("sink.flush.interval")

Review comment:
       "sink.buffer-flush.interval"

##########
File path: flink-connectors/flink-connector-jdbc/src/main/java/org/apache/flink/connector/jdbc/table/JdbcRowDataLookupFunction.java
##########
@@ -0,0 +1,228 @@
+/*
+ * 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.connector.jdbc.table;
+
+import org.apache.flink.annotation.Internal;
+import org.apache.flink.connector.jdbc.dialect.JdbcDialect;
+import org.apache.flink.connector.jdbc.dialect.JdbcDialects;
+import org.apache.flink.connector.jdbc.internal.converter.JdbcRowConverter;
+import org.apache.flink.connector.jdbc.internal.options.JdbcLookupOptions;
+import org.apache.flink.connector.jdbc.internal.options.JdbcOptions;
+import org.apache.flink.connector.jdbc.utils.JdbcTypeUtil;
+import org.apache.flink.connector.jdbc.utils.JdbcUtils;
+import org.apache.flink.table.data.GenericRowData;
+import org.apache.flink.table.data.RowData;
+import org.apache.flink.table.functions.FunctionContext;
+import org.apache.flink.table.functions.TableFunction;
+import org.apache.flink.table.types.DataType;
+import org.apache.flink.table.types.logical.RowType;
+import org.apache.flink.table.types.utils.TypeConversions;
+
+import org.apache.flink.shaded.guava18.com.google.common.cache.Cache;
+import org.apache.flink.shaded.guava18.com.google.common.cache.CacheBuilder;
+
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import java.io.IOException;
+import java.sql.Connection;
+import java.sql.DriverManager;
+import java.sql.PreparedStatement;
+import java.sql.ResultSet;
+import java.sql.SQLException;
+import java.util.ArrayList;
+import java.util.Arrays;
+import java.util.List;
+import java.util.concurrent.TimeUnit;
+
+import static org.apache.flink.util.Preconditions.checkArgument;
+import static org.apache.flink.util.Preconditions.checkNotNull;
+
+/**
+ * A lookup function for {@link JdbcDynamicTableSource}.
+ */
+@Internal
+public class JdbcRowDataLookupFunction extends TableFunction<RowData> {
+
+	private static final Logger LOG = LoggerFactory.getLogger(JdbcRowDataLookupFunction.class);
+	private static final long serialVersionUID = 1L;
+
+	private final String query;
+	private final String drivername;
+	private final String dbURL;
+	private final String username;
+	private final String password;
+	private final DataType[] keyTypes;
+	private final int[] keySqlTypes;
+	private final long cacheMaxSize;
+	private final long cacheExpireMs;
+	private final int maxRetryTimes;
+	private final JdbcDialect jdbcDialect;
+	private final JdbcRowConverter jdbcRowConverter;
+
+	private transient Connection dbConn;
+	private transient PreparedStatement statement;
+	private transient Cache<RowData, List<RowData>> cache;
+
+	public JdbcRowDataLookupFunction(
+			JdbcOptions options,
+			JdbcLookupOptions lookupOptions,
+			String[] fieldNames,
+			DataType[] fieldTypes,
+			String[] keyNames,
+			RowType rowType) {
+		checkNotNull(options, "No JdbcOptions supplied.");
+		checkNotNull(fieldNames, "No fieldNames supplied.");
+		checkNotNull(fieldTypes, "No fieldTypes supplied.");
+		checkNotNull(keyNames, "No keyNames supplied.");
+		this.drivername = options.getDriverName();
+		this.dbURL = options.getDbURL();
+		this.username = options.getUsername().orElse(null);
+		this.password = options.getPassword().orElse(null);
+		List<String> nameList = Arrays.asList(fieldNames);
+		this.keyTypes = Arrays.stream(keyNames)
+			.map(s -> {
+				checkArgument(nameList.contains(s),
+					"keyName %s can't find in fieldNames %s.", s, nameList);
+				return fieldTypes[nameList.indexOf(s)];
+			})
+			.toArray(DataType[]::new);
+		this.cacheMaxSize = lookupOptions.getCacheMaxSize();
+		this.cacheExpireMs = lookupOptions.getCacheExpireMs();
+		this.maxRetryTimes = lookupOptions.getMaxRetryTimes();
+		this.keySqlTypes = Arrays.stream(keyTypes)
+			.map(TypeConversions::fromDataTypeToLegacyInfo)
+			.mapToInt(JdbcTypeUtil::typeInformationToSqlType).toArray();
+		this.query = options.getDialect().getSelectFromStatement(
+			options.getTableName(), fieldNames, keyNames);
+		this.jdbcDialect = JdbcDialects.get(dbURL)
+			.orElseThrow(() -> new UnsupportedOperationException(String.format("Unknown dbUrl:%s", dbURL)));
+		this.jdbcRowConverter = jdbcDialect.getRowConverter(rowType);
+	}
+
+	@Override
+	public void open(FunctionContext context) throws Exception {
+		try {
+			establishConnection();
+			statement = dbConn.prepareStatement(query);
+			this.cache = cacheMaxSize == -1 || cacheExpireMs == -1 ? null : CacheBuilder.newBuilder()
+				.expireAfterWrite(cacheExpireMs, TimeUnit.MILLISECONDS)
+				.maximumSize(cacheMaxSize)
+				.build();
+		} catch (SQLException sqe) {
+			throw new IllegalArgumentException("open() failed.", sqe);
+		} catch (ClassNotFoundException cnfe) {
+			throw new IllegalArgumentException("JDBC driver class not found.", cnfe);
+		}
+	}
+
+	/**
+	 * This is a lookup method which is called by Flink framework in runtime.
+	 * @param keys lookup keys
+	 */
+	public void eval(Object... keys) {
+		RowData keyRow = GenericRowData.of(keys);
+		if (cache != null) {
+			List<RowData> cachedRows = cache.getIfPresent(keyRow);
+			if (cachedRows != null) {
+				for (RowData cachedRow : cachedRows) {
+					collect(cachedRow);
+				}
+				return;
+			}
+		}
+
+		for (int retry = 1; retry <= maxRetryTimes; retry++) {
+			try {
+				statement.clearParameters();
+				for (int i = 0; i < keys.length; i++) {
+					JdbcUtils.setField(statement, keySqlTypes[i], keys[i], i);
+				}
+				try (ResultSet resultSet = statement.executeQuery()) {
+					if (cache == null) {
+						while (resultSet.next()) {
+							collect(convertToRowFromResultSet(resultSet));
+						}
+					} else {
+						ArrayList<RowData> rows = new ArrayList<>();
+						while (resultSet.next()) {
+							RowData row = convertToRowFromResultSet(resultSet);
+							rows.add(row);
+							collect(row);
+						}
+						rows.trimToSize();
+						cache.put(keyRow, rows);
+					}
+				}
+				break;
+			} catch (SQLException e) {
+				LOG.error(String.format("JDBC executeBatch error, retry times = %d", retry), e);
+				if (retry >= maxRetryTimes) {
+					throw new RuntimeException("Execution of JDBC statement failed.", e);
+				}
+
+				try {
+					Thread.sleep(1000 * retry);
+				} catch (InterruptedException e1) {
+					throw new RuntimeException(e1);
+				}
+			}
+		}
+	}
+
+	private RowData convertToRowFromResultSet(ResultSet resultSet) throws SQLException {
+		return jdbcRowConverter.toInternal(resultSet);

Review comment:
       Why wrap such a simple call into a function?

##########
File path: flink-connectors/flink-connector-jdbc/src/main/java/org/apache/flink/connector/jdbc/table/JdbcRowDataLookupFunction.java
##########
@@ -0,0 +1,228 @@
+/*
+ * 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.connector.jdbc.table;
+
+import org.apache.flink.annotation.Internal;
+import org.apache.flink.connector.jdbc.dialect.JdbcDialect;
+import org.apache.flink.connector.jdbc.dialect.JdbcDialects;
+import org.apache.flink.connector.jdbc.internal.converter.JdbcRowConverter;
+import org.apache.flink.connector.jdbc.internal.options.JdbcLookupOptions;
+import org.apache.flink.connector.jdbc.internal.options.JdbcOptions;
+import org.apache.flink.connector.jdbc.utils.JdbcTypeUtil;
+import org.apache.flink.connector.jdbc.utils.JdbcUtils;
+import org.apache.flink.table.data.GenericRowData;
+import org.apache.flink.table.data.RowData;
+import org.apache.flink.table.functions.FunctionContext;
+import org.apache.flink.table.functions.TableFunction;
+import org.apache.flink.table.types.DataType;
+import org.apache.flink.table.types.logical.RowType;
+import org.apache.flink.table.types.utils.TypeConversions;
+
+import org.apache.flink.shaded.guava18.com.google.common.cache.Cache;
+import org.apache.flink.shaded.guava18.com.google.common.cache.CacheBuilder;
+
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import java.io.IOException;
+import java.sql.Connection;
+import java.sql.DriverManager;
+import java.sql.PreparedStatement;
+import java.sql.ResultSet;
+import java.sql.SQLException;
+import java.util.ArrayList;
+import java.util.Arrays;
+import java.util.List;
+import java.util.concurrent.TimeUnit;
+
+import static org.apache.flink.util.Preconditions.checkArgument;
+import static org.apache.flink.util.Preconditions.checkNotNull;
+
+/**
+ * A lookup function for {@link JdbcDynamicTableSource}.
+ */
+@Internal
+public class JdbcRowDataLookupFunction extends TableFunction<RowData> {
+
+	private static final Logger LOG = LoggerFactory.getLogger(JdbcRowDataLookupFunction.class);
+	private static final long serialVersionUID = 1L;
+
+	private final String query;
+	private final String drivername;
+	private final String dbURL;
+	private final String username;
+	private final String password;
+	private final DataType[] keyTypes;
+	private final int[] keySqlTypes;
+	private final long cacheMaxSize;
+	private final long cacheExpireMs;
+	private final int maxRetryTimes;
+	private final JdbcDialect jdbcDialect;
+	private final JdbcRowConverter jdbcRowConverter;
+
+	private transient Connection dbConn;
+	private transient PreparedStatement statement;
+	private transient Cache<RowData, List<RowData>> cache;
+
+	public JdbcRowDataLookupFunction(
+			JdbcOptions options,
+			JdbcLookupOptions lookupOptions,
+			String[] fieldNames,
+			DataType[] fieldTypes,
+			String[] keyNames,
+			RowType rowType) {
+		checkNotNull(options, "No JdbcOptions supplied.");
+		checkNotNull(fieldNames, "No fieldNames supplied.");
+		checkNotNull(fieldTypes, "No fieldTypes supplied.");
+		checkNotNull(keyNames, "No keyNames supplied.");
+		this.drivername = options.getDriverName();
+		this.dbURL = options.getDbURL();
+		this.username = options.getUsername().orElse(null);
+		this.password = options.getPassword().orElse(null);
+		List<String> nameList = Arrays.asList(fieldNames);
+		this.keyTypes = Arrays.stream(keyNames)
+			.map(s -> {
+				checkArgument(nameList.contains(s),
+					"keyName %s can't find in fieldNames %s.", s, nameList);
+				return fieldTypes[nameList.indexOf(s)];
+			})
+			.toArray(DataType[]::new);
+		this.cacheMaxSize = lookupOptions.getCacheMaxSize();
+		this.cacheExpireMs = lookupOptions.getCacheExpireMs();
+		this.maxRetryTimes = lookupOptions.getMaxRetryTimes();
+		this.keySqlTypes = Arrays.stream(keyTypes)
+			.map(TypeConversions::fromDataTypeToLegacyInfo)
+			.mapToInt(JdbcTypeUtil::typeInformationToSqlType).toArray();
+		this.query = options.getDialect().getSelectFromStatement(
+			options.getTableName(), fieldNames, keyNames);
+		this.jdbcDialect = JdbcDialects.get(dbURL)
+			.orElseThrow(() -> new UnsupportedOperationException(String.format("Unknown dbUrl:%s", dbURL)));
+		this.jdbcRowConverter = jdbcDialect.getRowConverter(rowType);
+	}
+
+	@Override
+	public void open(FunctionContext context) throws Exception {
+		try {
+			establishConnection();
+			statement = dbConn.prepareStatement(query);
+			this.cache = cacheMaxSize == -1 || cacheExpireMs == -1 ? null : CacheBuilder.newBuilder()
+				.expireAfterWrite(cacheExpireMs, TimeUnit.MILLISECONDS)
+				.maximumSize(cacheMaxSize)
+				.build();
+		} catch (SQLException sqe) {
+			throw new IllegalArgumentException("open() failed.", sqe);
+		} catch (ClassNotFoundException cnfe) {
+			throw new IllegalArgumentException("JDBC driver class not found.", cnfe);
+		}
+	}
+
+	/**
+	 * This is a lookup method which is called by Flink framework in runtime.
+	 * @param keys lookup keys
+	 */
+	public void eval(Object... keys) {
+		RowData keyRow = GenericRowData.of(keys);
+		if (cache != null) {
+			List<RowData> cachedRows = cache.getIfPresent(keyRow);
+			if (cachedRows != null) {
+				for (RowData cachedRow : cachedRows) {
+					collect(cachedRow);
+				}
+				return;
+			}
+		}
+
+		for (int retry = 1; retry <= maxRetryTimes; retry++) {
+			try {
+				statement.clearParameters();
+				for (int i = 0; i < keys.length; i++) {
+					JdbcUtils.setField(statement, keySqlTypes[i], keys[i], i);

Review comment:
       This is wrong. You should use `JdbcRowConverter` to set the statements, otherwise, if the lookup key is a string type or timestamp type, an exception will be thrown. You can also add a test for this.




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

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [flink] leonardBang commented on a change in pull request #12176: [FLINK-17029][jdbc]Introduce a new JDBC connector with new property keys

Posted by GitBox <gi...@apache.org>.
leonardBang commented on a change in pull request #12176:
URL: https://github.com/apache/flink/pull/12176#discussion_r426135776



##########
File path: flink-connectors/flink-connector-jdbc/src/main/java/org/apache/flink/connector/jdbc/table/JdbcDynamicInputFormat.java
##########
@@ -0,0 +1,402 @@
+/*
+ * 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.connector.jdbc.table;
+
+import org.apache.flink.annotation.Internal;
+import org.apache.flink.api.common.io.DefaultInputSplitAssigner;
+import org.apache.flink.api.common.io.InputFormat;
+import org.apache.flink.api.common.io.RichInputFormat;
+import org.apache.flink.api.common.io.statistics.BaseStatistics;
+import org.apache.flink.api.common.typeinfo.TypeInformation;
+import org.apache.flink.api.java.typeutils.GenericTypeInfo;
+import org.apache.flink.api.java.typeutils.ResultTypeQueryable;
+import org.apache.flink.api.java.typeutils.RowTypeInfo;
+import org.apache.flink.configuration.Configuration;
+import org.apache.flink.connector.jdbc.JdbcConnectionOptions;
+import org.apache.flink.connector.jdbc.internal.connection.SimpleJdbcConnectionProvider;
+import org.apache.flink.connector.jdbc.internal.converter.JdbcToRowConverter;
+import org.apache.flink.connector.jdbc.split.JdbcParameterValuesProvider;
+import org.apache.flink.core.io.GenericInputSplit;
+import org.apache.flink.core.io.InputSplit;
+import org.apache.flink.core.io.InputSplitAssigner;
+import org.apache.flink.table.data.RowData;
+import org.apache.flink.util.Preconditions;
+
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import java.io.IOException;
+import java.math.BigDecimal;
+import java.sql.Array;
+import java.sql.Connection;
+import java.sql.Date;
+import java.sql.PreparedStatement;
+import java.sql.ResultSet;
+import java.sql.SQLException;
+import java.sql.Time;
+import java.sql.Timestamp;
+import java.util.Arrays;
+
+/**
+ * InputFormat for {@link JdbcDynamicTableSource}.
+ */
+@Internal
+public class JdbcDynamicInputFormat extends RichInputFormat<RowData, InputSplit> implements ResultTypeQueryable<RowData> {

Review comment:
       `JdbcInputFormat` has a children `JDBCInputFormat` for compatibility, and they deals with Row rather than RowData, if we want reuse it, we need to add A JdbcInputFormat, will here have too many  InputFormat ?




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

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [flink] flinkbot edited a comment on pull request #12176: [FLINK-17029][jdbc]Introduce a new JDBC connector with new property keys

Posted by GitBox <gi...@apache.org>.
flinkbot edited a comment on pull request #12176:
URL: https://github.com/apache/flink/pull/12176#issuecomment-629283127


   <!--
   Meta data
   {
     "version" : 1,
     "metaDataEntries" : [ {
       "hash" : "499722944af576d13757da75f4ad0d08c4538782",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=1453",
       "triggerID" : "499722944af576d13757da75f4ad0d08c4538782",
       "triggerType" : "PUSH"
     }, {
       "hash" : "d07c3a602f13eaf3ea441e11ece1bf671bfe5beb",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=1547",
       "triggerID" : "d07c3a602f13eaf3ea441e11ece1bf671bfe5beb",
       "triggerType" : "PUSH"
     }, {
       "hash" : "226bafeb2086b9c98a29b398c3c34a325390aa2e",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=1558",
       "triggerID" : "226bafeb2086b9c98a29b398c3c34a325390aa2e",
       "triggerType" : "PUSH"
     }, {
       "hash" : "057d8fa644c9a203a753fe184de8e204ea81918e",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=1566",
       "triggerID" : "057d8fa644c9a203a753fe184de8e204ea81918e",
       "triggerType" : "PUSH"
     }, {
       "hash" : "e557b09c5dc9371c42542211651b79ea749cbf03",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=1584",
       "triggerID" : "e557b09c5dc9371c42542211651b79ea749cbf03",
       "triggerType" : "PUSH"
     }, {
       "hash" : "7f67ad7536d0041ec9fc5f646dc23972822125f8",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=1607",
       "triggerID" : "7f67ad7536d0041ec9fc5f646dc23972822125f8",
       "triggerType" : "PUSH"
     }, {
       "hash" : "e006fa3ad7437b5fc9e4089630f7a17c21367971",
       "status" : "SUCCESS",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=1643",
       "triggerID" : "e006fa3ad7437b5fc9e4089630f7a17c21367971",
       "triggerType" : "PUSH"
     } ]
   }-->
   ## CI report:
   
   * e006fa3ad7437b5fc9e4089630f7a17c21367971 Azure: [SUCCESS](https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=1643) 
   
   <details>
   <summary>Bot commands</summary>
     The @flinkbot bot supports the following commands:
   
    - `@flinkbot run travis` re-run the last Travis build
    - `@flinkbot run azure` re-run the last Azure build
   </details>


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

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [flink] leonardBang commented on a change in pull request #12176: [FLINK-17029][jdbc]Introduce a new JDBC connector with new property keys

Posted by GitBox <gi...@apache.org>.
leonardBang commented on a change in pull request #12176:
URL: https://github.com/apache/flink/pull/12176#discussion_r426277234



##########
File path: flink-connectors/flink-connector-jdbc/src/main/java/org/apache/flink/connector/jdbc/internal/converter/AbstractJdbcRowConverter.java
##########
@@ -18,54 +18,255 @@
 
 package org.apache.flink.connector.jdbc.internal.converter;
 
+import org.apache.flink.connector.jdbc.utils.JdbcTypeUtil;
+import org.apache.flink.table.data.DecimalData;
+import org.apache.flink.table.data.GenericArrayData;
+import org.apache.flink.table.data.GenericRowData;
+import org.apache.flink.table.data.RowData;
+import org.apache.flink.table.data.StringData;
+import org.apache.flink.table.data.TimestampData;
+import org.apache.flink.table.types.logical.ArrayType;
+import org.apache.flink.table.types.logical.DecimalType;
 import org.apache.flink.table.types.logical.LogicalType;
 import org.apache.flink.table.types.logical.LogicalTypeRoot;
 import org.apache.flink.table.types.logical.RowType;
-import org.apache.flink.types.Row;
+import org.apache.flink.table.types.logical.TimestampType;
+import org.apache.flink.table.types.logical.utils.LogicalTypeUtils;
+import org.apache.flink.table.types.utils.TypeConversions;
 
+import java.io.Serializable;
+import java.lang.reflect.Array;
+import java.math.BigDecimal;
+import java.sql.Date;
+import java.sql.PreparedStatement;
 import java.sql.ResultSet;
 import java.sql.SQLException;
+import java.sql.Time;
+import java.sql.Timestamp;
+import java.time.LocalDate;
+import java.time.LocalTime;
 
 import static org.apache.flink.util.Preconditions.checkNotNull;
 
 /**
- * Base for all row converters.
+ * Base class for all converters that convert between JDBC object and Flink internal object.
  */
 public abstract class AbstractJdbcRowConverter implements JdbcRowConverter {
 
 	protected final RowType rowType;
-	protected final JdbcFieldConverter[] converters;
+	protected final JdbcDeserializationConverter[] toInternalConverters;
+	protected final JdbcSerializationConverter[] toExternalConverters;
+	protected final LogicalType[] fieldTypes;
+
+	public abstract String converterName();
 
 	public AbstractJdbcRowConverter(RowType rowType) {
 		this.rowType = checkNotNull(rowType);
-		converters = new JdbcFieldConverter[rowType.getFieldCount()];
-
-		for (int i = 0; i < converters.length; i++) {
-			converters[i] = createConverter(rowType.getTypeAt(i));
+		this.fieldTypes = rowType.getFields().stream()
+			.map(RowType.RowField::getType)
+			.toArray(LogicalType[]::new);
+		this.toInternalConverters = new JdbcDeserializationConverter[rowType.getFieldCount()];
+		this.toExternalConverters = new JdbcSerializationConverter[rowType.getFieldCount()];
+		for (int i = 0; i < rowType.getFieldCount(); i++) {
+			toInternalConverters[i] = createNullableInternalConverter(rowType.getTypeAt(i));
+			toExternalConverters[i] = createNullableExternalConverter(fieldTypes[i]);
 		}
 	}
 
 	@Override
-	public Row convert(ResultSet resultSet, Row reuse) throws SQLException {
+	public RowData toInternal(ResultSet resultSet) throws SQLException {
+		GenericRowData genericRowData = new GenericRowData(rowType.getFieldCount());
 		for (int pos = 0; pos < rowType.getFieldCount(); pos++) {
-			reuse.setField(pos, converters[pos].convert(resultSet.getObject(pos + 1)));
+			genericRowData.setField(pos, toInternalConverters[pos].deserialize(resultSet.getObject(pos + 1)));
+		}
+		return genericRowData;
+	}
+
+	@Override
+	public PreparedStatement toExternal(RowData rowData, PreparedStatement statement) throws SQLException {
+		for (int index = 0; index < rowData.getArity(); index++) {
+			toExternalConverters[index].serialize(rowData, index, statement);
+		}
+		return statement;
+	}
+
+	/**
+	 * Runtime converter to convert JDBC field to {@link RowData} type object.
+	 */
+	@FunctionalInterface
+	interface JdbcDeserializationConverter extends Serializable {
+		/**
+		 * convert a jdbc field to java object, the field could be a simple type or array type.
+		 * @param jdbcField
+		 */
+		Object deserialize(Object jdbcField) throws SQLException;
+	}
+
+	/**
+	 * Runtime converter to convert {@link RowData} field to java object and fill into the {@link PreparedStatement}.
+	 */
+	@FunctionalInterface
+	interface JdbcSerializationConverter extends Serializable {
+		void serialize(RowData rowData, int index, PreparedStatement statement) throws SQLException;
+	}
+
+	/**
+	 * Create a nullable runtime {@link JdbcDeserializationConverter} from given {@link LogicalType}.
+	 */
+	protected JdbcDeserializationConverter createNullableInternalConverter(LogicalType type) {
+		return wrapIntoNullableInternalConverter(createInternalConverter(type));
+	}
+
+	protected JdbcDeserializationConverter wrapIntoNullableInternalConverter(JdbcDeserializationConverter jdbcDeserializationConverter) {
+		return v -> {
+			if (v == null) {
+				return null;
+			} else {
+				return jdbcDeserializationConverter.deserialize(v);
+			}
+		};
+	}
+
+	protected JdbcDeserializationConverter createInternalConverter(LogicalType type) {
+		switch (type.getTypeRoot()) {
+			case NULL:
+				return v -> null;
+			case BOOLEAN:
+				return v -> (boolean) v;
+			case TINYINT:
+				return v -> (byte) v;
+			case SMALLINT:
+				// Converter for small type that casts value to int and then return short value, since
+				// JDBC 1.0 use int type for small values.
+				return v -> (Integer.valueOf(v.toString())).shortValue();
+			case INTEGER:
+			case INTERVAL_YEAR_MONTH:
+				return v -> (int) v;
+			case BIGINT:
+			case INTERVAL_DAY_TIME:
+				return v -> (long) v;
+			case DATE:
+				return v -> (int) (((Date) v).toLocalDate().toEpochDay());
+			case TIME_WITHOUT_TIME_ZONE:
+				return v -> ((Time) v).toLocalTime().toSecondOfDay() * 1000;
+			case TIMESTAMP_WITH_TIME_ZONE:
+			case TIMESTAMP_WITHOUT_TIME_ZONE:
+				return v -> TimestampData.fromTimestamp((Timestamp) v);
+			case FLOAT:
+				return v -> (float) v;
+			case DOUBLE:
+				return v -> (double) v;
+			case CHAR:
+			case VARCHAR:
+				return v -> StringData.fromString((String) v);
+			case BINARY:
+			case VARBINARY:
+				return v -> (byte[]) v;
+			case DECIMAL:
+				final int precision = ((DecimalType) type).getPrecision();
+				final int scale = ((DecimalType) type).getScale();
+				return v -> DecimalData.fromBigDecimal((BigDecimal) v, precision, scale);
+			case ARRAY:
+				final JdbcDeserializationConverter arrayConverter = createToInternalArrayConverter((ArrayType) type);
+				return v -> arrayConverter.deserialize(v);
+			case ROW:
+			case MAP:
+			case MULTISET:
+			case RAW:
+			default:
+				throw new UnsupportedOperationException("Not support to parse type: " + type);
 		}
+	}
 
-		return reuse;
+	protected JdbcDeserializationConverter createToInternalArrayConverter(ArrayType arrayType) {
+		final JdbcDeserializationConverter elementConverter = createNullableInternalConverter(arrayType.getElementType());
+		final Class<?> elementClass = LogicalTypeUtils.toInternalConversionClass(arrayType.getElementType());
+		return v -> {
+			final Object[] objects = (Object[]) v;
+			final Object[] array = (Object[]) Array.newInstance(elementClass, objects.length);
+			for (int i = 0; i < objects.length; i++) {
+				array[i] = elementConverter.deserialize(objects[i]);
+			}
+			return new GenericArrayData(array);
+		};
 	}
 
 	/**
-	 * Create a runtime JDBC field converter from given {@link LogicalType}.
+	 * Create a nullable JDBC f{@link JdbcSerializationConverter} from given sql type.
 	 */
-	public JdbcFieldConverter createConverter(LogicalType type) {
-		LogicalTypeRoot root = type.getTypeRoot();
-
-		if (root == LogicalTypeRoot.SMALLINT) {
-			// Converter for small type that casts value to int and then return short value, since
-	        // JDBC 1.0 use int type for small values.
-			return v -> ((Integer) v).shortValue();
-		} else {
-			return v -> v;
+	protected JdbcSerializationConverter createNullableExternalConverter(LogicalType type) {
+		return wrapIntoNullableExternalConverter(createExternalConverter(type), type);
+	}
+
+	protected JdbcSerializationConverter wrapIntoNullableExternalConverter(JdbcSerializationConverter jdbcSerializationConverter, LogicalType type) {
+		final int sqlType = JdbcTypeUtil.typeInformationToSqlType(TypeConversions.fromDataTypeToLegacyInfo(
+			TypeConversions.fromLogicalToDataType(type)));
+		return (val, index, statement)  -> {
+			if (val == null || val.isNullAt(index) || LogicalTypeRoot.NULL.equals(type.getTypeRoot())) {
+				statement.setNull(index + 1, sqlType);
+			} else {
+				jdbcSerializationConverter.serialize(val, index, statement);
+			}
+		};
+	}
+
+	protected JdbcSerializationConverter createExternalConverter(LogicalType type) {
+		switch (type.getTypeRoot()) {
+			case BOOLEAN:
+				return (val, index, statement) -> statement.setBoolean(index + 1, val.getBoolean(index));
+			case TINYINT:
+				return (val, index, statement) -> statement.setByte(index + 1, val.getByte(index));
+			case SMALLINT:
+				return (val, index, statement) -> statement.setShort(index + 1, val.getShort(index));
+			case INTEGER:
+			case INTERVAL_YEAR_MONTH:
+				return (val, index, statement) -> statement.setInt(index + 1, val.getInt(index));
+			case BIGINT:
+			case INTERVAL_DAY_TIME:
+				return (val, index, statement) -> statement.setLong(index + 1, val.getLong(index));
+			case FLOAT:
+				return (val, index, statement) -> statement.setFloat(index + 1, val.getFloat(index));
+			case DOUBLE:
+				return (val, index, statement) -> statement.setDouble(index + 1, val.getDouble(index));
+			case CHAR:
+			case VARCHAR:
+				// value is BinaryString
+				return (val, index, statement) -> statement.setString(index + 1, val.getString(index).toString());
+			case BINARY:
+			case VARBINARY:
+				return (val, index, statement) -> statement.setBytes(index + 1, val.getBinary(index));
+			case DATE:
+				return (val, index, statement) ->
+					statement.setDate(index + 1, Date.valueOf(LocalDate.ofEpochDay(val.getInt(index))));
+			case TIME_WITHOUT_TIME_ZONE:
+				return (val, index, statement) ->
+					statement.setTime(index + 1, Time.valueOf(LocalTime.ofSecondOfDay(val.getInt(index) / 1000L)));
+			case TIMESTAMP_WITH_TIME_ZONE:
+			case TIMESTAMP_WITHOUT_TIME_ZONE:
+				final int timestampPrecision = ((TimestampType) type).getPrecision();
+				return (val, index, statement) ->
+					statement.setTimestamp(index + 1, val
+						.getTimestamp(index, timestampPrecision)
+						.toTimestamp());
+			case DECIMAL:
+				final int decimalPrecision = ((DecimalType) type).getPrecision();
+				final int decimalScale = ((DecimalType) type).getScale();
+				return (val, index, statement) ->
+					statement.setBigDecimal(index + 1, val
+						.getDecimal(index, decimalPrecision, decimalScale)
+						.toBigDecimal());
+			case ARRAY:
+				//note: dialect need implements the conversion from ArrayData to JDBC Array if the dialect supports array.
+				return (val, index, statement) -> {
+					throw new IllegalStateException(
+						String.format("JDBC:%s do not support write ARRAY type.", converterName()));
+				};

Review comment:
       From now,  we use same RowType to build a JdbcRowConverter that contains serialize and deserialize converter, but Postgres dialect support PgArray in source but not support in sink(that's the reason), if we throw the exception at compile stage, the deserialize converter can not use too.
   




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

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [flink] wuchong closed pull request #12176: [FLINK-17029][jdbc]Introduce a new JDBC connector with new property keys

Posted by GitBox <gi...@apache.org>.
wuchong closed pull request #12176:
URL: https://github.com/apache/flink/pull/12176


   


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

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [flink] leonardBang commented on pull request #12176: [FLINK-17029][jdbc]Introduce a new JDBC connector with new property keys

Posted by GitBox <gi...@apache.org>.
leonardBang commented on pull request #12176:
URL: https://github.com/apache/flink/pull/12176#issuecomment-629969292


   Thanks for the awesome review!@wuchong


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

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [flink] leonardBang commented on pull request #12176: [FLINK-17029][jdbc]Introduce a new JDBC connector with new property keys

Posted by GitBox <gi...@apache.org>.
leonardBang commented on pull request #12176:
URL: https://github.com/apache/flink/pull/12176#issuecomment-629914312


   @flinkbot run azure


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

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [flink] flinkbot edited a comment on pull request #12176: [FLINK-17029][jdbc]Introduce a new JDBC connector with new property keys

Posted by GitBox <gi...@apache.org>.
flinkbot edited a comment on pull request #12176:
URL: https://github.com/apache/flink/pull/12176#issuecomment-629283127


   <!--
   Meta data
   {
     "version" : 1,
     "metaDataEntries" : [ {
       "hash" : "499722944af576d13757da75f4ad0d08c4538782",
       "status" : "PENDING",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=1453",
       "triggerID" : "499722944af576d13757da75f4ad0d08c4538782",
       "triggerType" : "PUSH"
     } ]
   }-->
   ## CI report:
   
   * 499722944af576d13757da75f4ad0d08c4538782 Azure: [PENDING](https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=1453) 
   
   <details>
   <summary>Bot commands</summary>
     The @flinkbot bot supports the following commands:
   
    - `@flinkbot run travis` re-run the last Travis build
    - `@flinkbot run azure` re-run the last Azure build
   </details>


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

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [flink] flinkbot edited a comment on pull request #12176: [FLINK-17029][jdbc]Introduce a new JDBC connector with new property keys

Posted by GitBox <gi...@apache.org>.
flinkbot edited a comment on pull request #12176:
URL: https://github.com/apache/flink/pull/12176#issuecomment-629283127


   <!--
   Meta data
   {
     "version" : 1,
     "metaDataEntries" : [ {
       "hash" : "499722944af576d13757da75f4ad0d08c4538782",
       "status" : "FAILURE",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=1453",
       "triggerID" : "499722944af576d13757da75f4ad0d08c4538782",
       "triggerType" : "PUSH"
     } ]
   }-->
   ## CI report:
   
   * 499722944af576d13757da75f4ad0d08c4538782 Azure: [FAILURE](https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=1453) 
   
   <details>
   <summary>Bot commands</summary>
     The @flinkbot bot supports the following commands:
   
    - `@flinkbot run travis` re-run the last Travis build
    - `@flinkbot run azure` re-run the last Azure build
   </details>


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

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [flink] flinkbot edited a comment on pull request #12176: [FLINK-17029][jdbc]Introduce a new JDBC connector with new property keys

Posted by GitBox <gi...@apache.org>.
flinkbot edited a comment on pull request #12176:
URL: https://github.com/apache/flink/pull/12176#issuecomment-629283127


   <!--
   Meta data
   {
     "version" : 1,
     "metaDataEntries" : [ {
       "hash" : "499722944af576d13757da75f4ad0d08c4538782",
       "status" : "FAILURE",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=1453",
       "triggerID" : "499722944af576d13757da75f4ad0d08c4538782",
       "triggerType" : "PUSH"
     }, {
       "hash" : "d07c3a602f13eaf3ea441e11ece1bf671bfe5beb",
       "status" : "PENDING",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=1547",
       "triggerID" : "d07c3a602f13eaf3ea441e11ece1bf671bfe5beb",
       "triggerType" : "PUSH"
     } ]
   }-->
   ## CI report:
   
   * 499722944af576d13757da75f4ad0d08c4538782 Azure: [FAILURE](https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=1453) 
   * d07c3a602f13eaf3ea441e11ece1bf671bfe5beb Azure: [PENDING](https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=1547) 
   
   <details>
   <summary>Bot commands</summary>
     The @flinkbot bot supports the following commands:
   
    - `@flinkbot run travis` re-run the last Travis build
    - `@flinkbot run azure` re-run the last Azure build
   </details>


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

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [flink] flinkbot edited a comment on pull request #12176: [FLINK-17029][jdbc]Introduce a new JDBC connector with new property keys

Posted by GitBox <gi...@apache.org>.
flinkbot edited a comment on pull request #12176:
URL: https://github.com/apache/flink/pull/12176#issuecomment-629283127


   <!--
   Meta data
   {
     "version" : 1,
     "metaDataEntries" : [ {
       "hash" : "499722944af576d13757da75f4ad0d08c4538782",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=1453",
       "triggerID" : "499722944af576d13757da75f4ad0d08c4538782",
       "triggerType" : "PUSH"
     }, {
       "hash" : "d07c3a602f13eaf3ea441e11ece1bf671bfe5beb",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=1547",
       "triggerID" : "d07c3a602f13eaf3ea441e11ece1bf671bfe5beb",
       "triggerType" : "PUSH"
     }, {
       "hash" : "226bafeb2086b9c98a29b398c3c34a325390aa2e",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=1558",
       "triggerID" : "226bafeb2086b9c98a29b398c3c34a325390aa2e",
       "triggerType" : "PUSH"
     }, {
       "hash" : "057d8fa644c9a203a753fe184de8e204ea81918e",
       "status" : "FAILURE",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=1566",
       "triggerID" : "057d8fa644c9a203a753fe184de8e204ea81918e",
       "triggerType" : "PUSH"
     } ]
   }-->
   ## CI report:
   
   * 057d8fa644c9a203a753fe184de8e204ea81918e Azure: [FAILURE](https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=1566) 
   
   <details>
   <summary>Bot commands</summary>
     The @flinkbot bot supports the following commands:
   
    - `@flinkbot run travis` re-run the last Travis build
    - `@flinkbot run azure` re-run the last Azure build
   </details>


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

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [flink] flinkbot edited a comment on pull request #12176: [FLINK-17029][jdbc]Introduce a new JDBC connector with new property keys

Posted by GitBox <gi...@apache.org>.
flinkbot edited a comment on pull request #12176:
URL: https://github.com/apache/flink/pull/12176#issuecomment-629283127


   <!--
   Meta data
   {
     "version" : 1,
     "metaDataEntries" : [ {
       "hash" : "499722944af576d13757da75f4ad0d08c4538782",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=1453",
       "triggerID" : "499722944af576d13757da75f4ad0d08c4538782",
       "triggerType" : "PUSH"
     }, {
       "hash" : "d07c3a602f13eaf3ea441e11ece1bf671bfe5beb",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=1547",
       "triggerID" : "d07c3a602f13eaf3ea441e11ece1bf671bfe5beb",
       "triggerType" : "PUSH"
     }, {
       "hash" : "226bafeb2086b9c98a29b398c3c34a325390aa2e",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=1558",
       "triggerID" : "226bafeb2086b9c98a29b398c3c34a325390aa2e",
       "triggerType" : "PUSH"
     }, {
       "hash" : "057d8fa644c9a203a753fe184de8e204ea81918e",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=1566",
       "triggerID" : "057d8fa644c9a203a753fe184de8e204ea81918e",
       "triggerType" : "PUSH"
     }, {
       "hash" : "e557b09c5dc9371c42542211651b79ea749cbf03",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=1584",
       "triggerID" : "e557b09c5dc9371c42542211651b79ea749cbf03",
       "triggerType" : "PUSH"
     }, {
       "hash" : "7f67ad7536d0041ec9fc5f646dc23972822125f8",
       "status" : "FAILURE",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=1607",
       "triggerID" : "7f67ad7536d0041ec9fc5f646dc23972822125f8",
       "triggerType" : "PUSH"
     }, {
       "hash" : "e006fa3ad7437b5fc9e4089630f7a17c21367971",
       "status" : "PENDING",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=1643",
       "triggerID" : "e006fa3ad7437b5fc9e4089630f7a17c21367971",
       "triggerType" : "PUSH"
     } ]
   }-->
   ## CI report:
   
   * 7f67ad7536d0041ec9fc5f646dc23972822125f8 Azure: [FAILURE](https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=1607) 
   * e006fa3ad7437b5fc9e4089630f7a17c21367971 Azure: [PENDING](https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=1643) 
   
   <details>
   <summary>Bot commands</summary>
     The @flinkbot bot supports the following commands:
   
    - `@flinkbot run travis` re-run the last Travis build
    - `@flinkbot run azure` re-run the last Azure build
   </details>


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

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [flink] flinkbot edited a comment on pull request #12176: [FLINK-17029][jdbc]Introduce a new JDBC connector with new property keys

Posted by GitBox <gi...@apache.org>.
flinkbot edited a comment on pull request #12176:
URL: https://github.com/apache/flink/pull/12176#issuecomment-629283127


   <!--
   Meta data
   {
     "version" : 1,
     "metaDataEntries" : [ {
       "hash" : "499722944af576d13757da75f4ad0d08c4538782",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=1453",
       "triggerID" : "499722944af576d13757da75f4ad0d08c4538782",
       "triggerType" : "PUSH"
     }, {
       "hash" : "d07c3a602f13eaf3ea441e11ece1bf671bfe5beb",
       "status" : "PENDING",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=1547",
       "triggerID" : "d07c3a602f13eaf3ea441e11ece1bf671bfe5beb",
       "triggerType" : "PUSH"
     }, {
       "hash" : "226bafeb2086b9c98a29b398c3c34a325390aa2e",
       "status" : "FAILURE",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=1558",
       "triggerID" : "226bafeb2086b9c98a29b398c3c34a325390aa2e",
       "triggerType" : "PUSH"
     }, {
       "hash" : "057d8fa644c9a203a753fe184de8e204ea81918e",
       "status" : "PENDING",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=1566",
       "triggerID" : "057d8fa644c9a203a753fe184de8e204ea81918e",
       "triggerType" : "PUSH"
     } ]
   }-->
   ## CI report:
   
   * d07c3a602f13eaf3ea441e11ece1bf671bfe5beb Azure: [PENDING](https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=1547) 
   * 226bafeb2086b9c98a29b398c3c34a325390aa2e Azure: [FAILURE](https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=1558) 
   * 057d8fa644c9a203a753fe184de8e204ea81918e Azure: [PENDING](https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=1566) 
   
   <details>
   <summary>Bot commands</summary>
     The @flinkbot bot supports the following commands:
   
    - `@flinkbot run travis` re-run the last Travis build
    - `@flinkbot run azure` re-run the last Azure build
   </details>


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

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [flink] flinkbot edited a comment on pull request #12176: [FLINK-17029][jdbc]Introduce a new JDBC connector with new property keys

Posted by GitBox <gi...@apache.org>.
flinkbot edited a comment on pull request #12176:
URL: https://github.com/apache/flink/pull/12176#issuecomment-629283127


   <!--
   Meta data
   {
     "version" : 1,
     "metaDataEntries" : [ {
       "hash" : "499722944af576d13757da75f4ad0d08c4538782",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=1453",
       "triggerID" : "499722944af576d13757da75f4ad0d08c4538782",
       "triggerType" : "PUSH"
     }, {
       "hash" : "d07c3a602f13eaf3ea441e11ece1bf671bfe5beb",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=1547",
       "triggerID" : "d07c3a602f13eaf3ea441e11ece1bf671bfe5beb",
       "triggerType" : "PUSH"
     }, {
       "hash" : "226bafeb2086b9c98a29b398c3c34a325390aa2e",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=1558",
       "triggerID" : "226bafeb2086b9c98a29b398c3c34a325390aa2e",
       "triggerType" : "PUSH"
     }, {
       "hash" : "057d8fa644c9a203a753fe184de8e204ea81918e",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=1566",
       "triggerID" : "057d8fa644c9a203a753fe184de8e204ea81918e",
       "triggerType" : "PUSH"
     }, {
       "hash" : "e557b09c5dc9371c42542211651b79ea749cbf03",
       "status" : "FAILURE",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=1584",
       "triggerID" : "e557b09c5dc9371c42542211651b79ea749cbf03",
       "triggerType" : "PUSH"
     } ]
   }-->
   ## CI report:
   
   * e557b09c5dc9371c42542211651b79ea749cbf03 Azure: [FAILURE](https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=1584) 
   
   <details>
   <summary>Bot commands</summary>
     The @flinkbot bot supports the following commands:
   
    - `@flinkbot run travis` re-run the last Travis build
    - `@flinkbot run azure` re-run the last Azure build
   </details>


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

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [flink] flinkbot edited a comment on pull request #12176: [FLINK-17029][jdbc]Introduce a new JDBC connector with new property keys

Posted by GitBox <gi...@apache.org>.
flinkbot edited a comment on pull request #12176:
URL: https://github.com/apache/flink/pull/12176#issuecomment-629283127


   <!--
   Meta data
   {
     "version" : 1,
     "metaDataEntries" : [ {
       "hash" : "499722944af576d13757da75f4ad0d08c4538782",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=1453",
       "triggerID" : "499722944af576d13757da75f4ad0d08c4538782",
       "triggerType" : "PUSH"
     }, {
       "hash" : "d07c3a602f13eaf3ea441e11ece1bf671bfe5beb",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=1547",
       "triggerID" : "d07c3a602f13eaf3ea441e11ece1bf671bfe5beb",
       "triggerType" : "PUSH"
     }, {
       "hash" : "226bafeb2086b9c98a29b398c3c34a325390aa2e",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=1558",
       "triggerID" : "226bafeb2086b9c98a29b398c3c34a325390aa2e",
       "triggerType" : "PUSH"
     }, {
       "hash" : "057d8fa644c9a203a753fe184de8e204ea81918e",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=1566",
       "triggerID" : "057d8fa644c9a203a753fe184de8e204ea81918e",
       "triggerType" : "PUSH"
     }, {
       "hash" : "e557b09c5dc9371c42542211651b79ea749cbf03",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=1584",
       "triggerID" : "e557b09c5dc9371c42542211651b79ea749cbf03",
       "triggerType" : "PUSH"
     }, {
       "hash" : "7f67ad7536d0041ec9fc5f646dc23972822125f8",
       "status" : "FAILURE",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=1607",
       "triggerID" : "7f67ad7536d0041ec9fc5f646dc23972822125f8",
       "triggerType" : "PUSH"
     }, {
       "hash" : "e006fa3ad7437b5fc9e4089630f7a17c21367971",
       "status" : "UNKNOWN",
       "url" : "TBD",
       "triggerID" : "e006fa3ad7437b5fc9e4089630f7a17c21367971",
       "triggerType" : "PUSH"
     } ]
   }-->
   ## CI report:
   
   * 7f67ad7536d0041ec9fc5f646dc23972822125f8 Azure: [FAILURE](https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=1607) 
   * e006fa3ad7437b5fc9e4089630f7a17c21367971 UNKNOWN
   
   <details>
   <summary>Bot commands</summary>
     The @flinkbot bot supports the following commands:
   
    - `@flinkbot run travis` re-run the last Travis build
    - `@flinkbot run azure` re-run the last Azure build
   </details>


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

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [flink] wuchong commented on a change in pull request #12176: [FLINK-17029][jdbc]Introduce a new JDBC connector with new property keys

Posted by GitBox <gi...@apache.org>.
wuchong commented on a change in pull request #12176:
URL: https://github.com/apache/flink/pull/12176#discussion_r426264070



##########
File path: flink-connectors/flink-connector-jdbc/src/main/java/org/apache/flink/connector/jdbc/internal/converter/AbstractJdbcRowConverter.java
##########
@@ -18,54 +18,255 @@
 
 package org.apache.flink.connector.jdbc.internal.converter;
 
+import org.apache.flink.connector.jdbc.utils.JdbcTypeUtil;
+import org.apache.flink.table.data.DecimalData;
+import org.apache.flink.table.data.GenericArrayData;
+import org.apache.flink.table.data.GenericRowData;
+import org.apache.flink.table.data.RowData;
+import org.apache.flink.table.data.StringData;
+import org.apache.flink.table.data.TimestampData;
+import org.apache.flink.table.types.logical.ArrayType;
+import org.apache.flink.table.types.logical.DecimalType;
 import org.apache.flink.table.types.logical.LogicalType;
 import org.apache.flink.table.types.logical.LogicalTypeRoot;
 import org.apache.flink.table.types.logical.RowType;
-import org.apache.flink.types.Row;
+import org.apache.flink.table.types.logical.TimestampType;
+import org.apache.flink.table.types.logical.utils.LogicalTypeUtils;
+import org.apache.flink.table.types.utils.TypeConversions;
 
+import java.io.Serializable;
+import java.lang.reflect.Array;
+import java.math.BigDecimal;
+import java.sql.Date;
+import java.sql.PreparedStatement;
 import java.sql.ResultSet;
 import java.sql.SQLException;
+import java.sql.Time;
+import java.sql.Timestamp;
+import java.time.LocalDate;
+import java.time.LocalTime;
 
 import static org.apache.flink.util.Preconditions.checkNotNull;
 
 /**
- * Base for all row converters.
+ * Base class for all converters that convert between JDBC object and Flink internal object.
  */
 public abstract class AbstractJdbcRowConverter implements JdbcRowConverter {
 
 	protected final RowType rowType;
-	protected final JdbcFieldConverter[] converters;
+	protected final JdbcDeserializationConverter[] toInternalConverters;
+	protected final JdbcSerializationConverter[] toExternalConverters;
+	protected final LogicalType[] fieldTypes;
+
+	public abstract String converterName();
 
 	public AbstractJdbcRowConverter(RowType rowType) {
 		this.rowType = checkNotNull(rowType);
-		converters = new JdbcFieldConverter[rowType.getFieldCount()];
-
-		for (int i = 0; i < converters.length; i++) {
-			converters[i] = createConverter(rowType.getTypeAt(i));
+		this.fieldTypes = rowType.getFields().stream()
+			.map(RowType.RowField::getType)
+			.toArray(LogicalType[]::new);
+		this.toInternalConverters = new JdbcDeserializationConverter[rowType.getFieldCount()];
+		this.toExternalConverters = new JdbcSerializationConverter[rowType.getFieldCount()];
+		for (int i = 0; i < rowType.getFieldCount(); i++) {
+			toInternalConverters[i] = createNullableInternalConverter(rowType.getTypeAt(i));
+			toExternalConverters[i] = createNullableExternalConverter(fieldTypes[i]);
 		}
 	}
 
 	@Override
-	public Row convert(ResultSet resultSet, Row reuse) throws SQLException {
+	public RowData toInternal(ResultSet resultSet) throws SQLException {
+		GenericRowData genericRowData = new GenericRowData(rowType.getFieldCount());
 		for (int pos = 0; pos < rowType.getFieldCount(); pos++) {
-			reuse.setField(pos, converters[pos].convert(resultSet.getObject(pos + 1)));
+			genericRowData.setField(pos, toInternalConverters[pos].deserialize(resultSet.getObject(pos + 1)));
+		}
+		return genericRowData;
+	}
+
+	@Override
+	public PreparedStatement toExternal(RowData rowData, PreparedStatement statement) throws SQLException {
+		for (int index = 0; index < rowData.getArity(); index++) {
+			toExternalConverters[index].serialize(rowData, index, statement);
+		}
+		return statement;
+	}
+
+	/**
+	 * Runtime converter to convert JDBC field to {@link RowData} type object.
+	 */
+	@FunctionalInterface
+	interface JdbcDeserializationConverter extends Serializable {
+		/**
+		 * convert a jdbc field to java object, the field could be a simple type or array type.
+		 * @param jdbcField

Review comment:
       ```suggestion
   		 * Convert a jdbc field object of {@link ResultSet} to the internal data structure object.
   ```

##########
File path: flink-connectors/flink-connector-jdbc/src/main/java/org/apache/flink/connector/jdbc/internal/converter/AbstractJdbcRowConverter.java
##########
@@ -18,54 +18,255 @@
 
 package org.apache.flink.connector.jdbc.internal.converter;
 
+import org.apache.flink.connector.jdbc.utils.JdbcTypeUtil;
+import org.apache.flink.table.data.DecimalData;
+import org.apache.flink.table.data.GenericArrayData;
+import org.apache.flink.table.data.GenericRowData;
+import org.apache.flink.table.data.RowData;
+import org.apache.flink.table.data.StringData;
+import org.apache.flink.table.data.TimestampData;
+import org.apache.flink.table.types.logical.ArrayType;
+import org.apache.flink.table.types.logical.DecimalType;
 import org.apache.flink.table.types.logical.LogicalType;
 import org.apache.flink.table.types.logical.LogicalTypeRoot;
 import org.apache.flink.table.types.logical.RowType;
-import org.apache.flink.types.Row;
+import org.apache.flink.table.types.logical.TimestampType;
+import org.apache.flink.table.types.logical.utils.LogicalTypeUtils;
+import org.apache.flink.table.types.utils.TypeConversions;
 
+import java.io.Serializable;
+import java.lang.reflect.Array;
+import java.math.BigDecimal;
+import java.sql.Date;
+import java.sql.PreparedStatement;
 import java.sql.ResultSet;
 import java.sql.SQLException;
+import java.sql.Time;
+import java.sql.Timestamp;
+import java.time.LocalDate;
+import java.time.LocalTime;
 
 import static org.apache.flink.util.Preconditions.checkNotNull;
 
 /**
- * Base for all row converters.
+ * Base class for all converters that convert between JDBC object and Flink internal object.
  */
 public abstract class AbstractJdbcRowConverter implements JdbcRowConverter {
 
 	protected final RowType rowType;
-	protected final JdbcFieldConverter[] converters;
+	protected final JdbcDeserializationConverter[] toInternalConverters;
+	protected final JdbcSerializationConverter[] toExternalConverters;
+	protected final LogicalType[] fieldTypes;
+
+	public abstract String converterName();
 
 	public AbstractJdbcRowConverter(RowType rowType) {
 		this.rowType = checkNotNull(rowType);
-		converters = new JdbcFieldConverter[rowType.getFieldCount()];
-
-		for (int i = 0; i < converters.length; i++) {
-			converters[i] = createConverter(rowType.getTypeAt(i));
+		this.fieldTypes = rowType.getFields().stream()
+			.map(RowType.RowField::getType)
+			.toArray(LogicalType[]::new);
+		this.toInternalConverters = new JdbcDeserializationConverter[rowType.getFieldCount()];
+		this.toExternalConverters = new JdbcSerializationConverter[rowType.getFieldCount()];
+		for (int i = 0; i < rowType.getFieldCount(); i++) {
+			toInternalConverters[i] = createNullableInternalConverter(rowType.getTypeAt(i));
+			toExternalConverters[i] = createNullableExternalConverter(fieldTypes[i]);
 		}
 	}
 
 	@Override
-	public Row convert(ResultSet resultSet, Row reuse) throws SQLException {
+	public RowData toInternal(ResultSet resultSet) throws SQLException {
+		GenericRowData genericRowData = new GenericRowData(rowType.getFieldCount());
 		for (int pos = 0; pos < rowType.getFieldCount(); pos++) {
-			reuse.setField(pos, converters[pos].convert(resultSet.getObject(pos + 1)));
+			genericRowData.setField(pos, toInternalConverters[pos].deserialize(resultSet.getObject(pos + 1)));
+		}
+		return genericRowData;
+	}
+
+	@Override
+	public PreparedStatement toExternal(RowData rowData, PreparedStatement statement) throws SQLException {
+		for (int index = 0; index < rowData.getArity(); index++) {
+			toExternalConverters[index].serialize(rowData, index, statement);
+		}
+		return statement;
+	}
+
+	/**
+	 * Runtime converter to convert JDBC field to {@link RowData} type object.
+	 */
+	@FunctionalInterface
+	interface JdbcDeserializationConverter extends Serializable {
+		/**
+		 * convert a jdbc field to java object, the field could be a simple type or array type.
+		 * @param jdbcField
+		 */
+		Object deserialize(Object jdbcField) throws SQLException;
+	}
+
+	/**
+	 * Runtime converter to convert {@link RowData} field to java object and fill into the {@link PreparedStatement}.
+	 */
+	@FunctionalInterface
+	interface JdbcSerializationConverter extends Serializable {
+		void serialize(RowData rowData, int index, PreparedStatement statement) throws SQLException;
+	}
+
+	/**
+	 * Create a nullable runtime {@link JdbcDeserializationConverter} from given {@link LogicalType}.
+	 */
+	protected JdbcDeserializationConverter createNullableInternalConverter(LogicalType type) {
+		return wrapIntoNullableInternalConverter(createInternalConverter(type));
+	}
+
+	protected JdbcDeserializationConverter wrapIntoNullableInternalConverter(JdbcDeserializationConverter jdbcDeserializationConverter) {
+		return v -> {
+			if (v == null) {
+				return null;
+			} else {
+				return jdbcDeserializationConverter.deserialize(v);
+			}
+		};
+	}
+
+	protected JdbcDeserializationConverter createInternalConverter(LogicalType type) {
+		switch (type.getTypeRoot()) {
+			case NULL:
+				return v -> null;
+			case BOOLEAN:
+				return v -> (boolean) v;
+			case TINYINT:
+				return v -> (byte) v;
+			case SMALLINT:
+				// Converter for small type that casts value to int and then return short value, since
+				// JDBC 1.0 use int type for small values.
+				return v -> (Integer.valueOf(v.toString())).shortValue();
+			case INTEGER:
+			case INTERVAL_YEAR_MONTH:
+				return v -> (int) v;
+			case BIGINT:
+			case INTERVAL_DAY_TIME:
+				return v -> (long) v;
+			case DATE:
+				return v -> (int) (((Date) v).toLocalDate().toEpochDay());
+			case TIME_WITHOUT_TIME_ZONE:
+				return v -> ((Time) v).toLocalTime().toSecondOfDay() * 1000;
+			case TIMESTAMP_WITH_TIME_ZONE:
+			case TIMESTAMP_WITHOUT_TIME_ZONE:
+				return v -> TimestampData.fromTimestamp((Timestamp) v);
+			case FLOAT:
+				return v -> (float) v;
+			case DOUBLE:
+				return v -> (double) v;
+			case CHAR:
+			case VARCHAR:
+				return v -> StringData.fromString((String) v);
+			case BINARY:
+			case VARBINARY:
+				return v -> (byte[]) v;
+			case DECIMAL:
+				final int precision = ((DecimalType) type).getPrecision();
+				final int scale = ((DecimalType) type).getScale();
+				return v -> DecimalData.fromBigDecimal((BigDecimal) v, precision, scale);
+			case ARRAY:
+				final JdbcDeserializationConverter arrayConverter = createToInternalArrayConverter((ArrayType) type);
+				return v -> arrayConverter.deserialize(v);
+			case ROW:
+			case MAP:
+			case MULTISET:
+			case RAW:
+			default:
+				throw new UnsupportedOperationException("Not support to parse type: " + type);
 		}
+	}
 
-		return reuse;
+	protected JdbcDeserializationConverter createToInternalArrayConverter(ArrayType arrayType) {
+		final JdbcDeserializationConverter elementConverter = createNullableInternalConverter(arrayType.getElementType());
+		final Class<?> elementClass = LogicalTypeUtils.toInternalConversionClass(arrayType.getElementType());
+		return v -> {
+			final Object[] objects = (Object[]) v;
+			final Object[] array = (Object[]) Array.newInstance(elementClass, objects.length);
+			for (int i = 0; i < objects.length; i++) {
+				array[i] = elementConverter.deserialize(objects[i]);
+			}
+			return new GenericArrayData(array);
+		};
 	}
 
 	/**
-	 * Create a runtime JDBC field converter from given {@link LogicalType}.
+	 * Create a nullable JDBC f{@link JdbcSerializationConverter} from given sql type.
 	 */
-	public JdbcFieldConverter createConverter(LogicalType type) {
-		LogicalTypeRoot root = type.getTypeRoot();
-
-		if (root == LogicalTypeRoot.SMALLINT) {
-			// Converter for small type that casts value to int and then return short value, since
-	        // JDBC 1.0 use int type for small values.
-			return v -> ((Integer) v).shortValue();
-		} else {
-			return v -> v;
+	protected JdbcSerializationConverter createNullableExternalConverter(LogicalType type) {
+		return wrapIntoNullableExternalConverter(createExternalConverter(type), type);
+	}
+
+	protected JdbcSerializationConverter wrapIntoNullableExternalConverter(JdbcSerializationConverter jdbcSerializationConverter, LogicalType type) {
+		final int sqlType = JdbcTypeUtil.typeInformationToSqlType(TypeConversions.fromDataTypeToLegacyInfo(
+			TypeConversions.fromLogicalToDataType(type)));
+		return (val, index, statement)  -> {
+			if (val == null || val.isNullAt(index) || LogicalTypeRoot.NULL.equals(type.getTypeRoot())) {
+				statement.setNull(index + 1, sqlType);
+			} else {
+				jdbcSerializationConverter.serialize(val, index, statement);
+			}
+		};
+	}
+
+	protected JdbcSerializationConverter createExternalConverter(LogicalType type) {
+		switch (type.getTypeRoot()) {
+			case BOOLEAN:
+				return (val, index, statement) -> statement.setBoolean(index + 1, val.getBoolean(index));
+			case TINYINT:
+				return (val, index, statement) -> statement.setByte(index + 1, val.getByte(index));
+			case SMALLINT:
+				return (val, index, statement) -> statement.setShort(index + 1, val.getShort(index));
+			case INTEGER:
+			case INTERVAL_YEAR_MONTH:
+				return (val, index, statement) -> statement.setInt(index + 1, val.getInt(index));
+			case BIGINT:
+			case INTERVAL_DAY_TIME:
+				return (val, index, statement) -> statement.setLong(index + 1, val.getLong(index));
+			case FLOAT:
+				return (val, index, statement) -> statement.setFloat(index + 1, val.getFloat(index));
+			case DOUBLE:
+				return (val, index, statement) -> statement.setDouble(index + 1, val.getDouble(index));
+			case CHAR:
+			case VARCHAR:
+				// value is BinaryString
+				return (val, index, statement) -> statement.setString(index + 1, val.getString(index).toString());
+			case BINARY:
+			case VARBINARY:
+				return (val, index, statement) -> statement.setBytes(index + 1, val.getBinary(index));
+			case DATE:
+				return (val, index, statement) ->
+					statement.setDate(index + 1, Date.valueOf(LocalDate.ofEpochDay(val.getInt(index))));
+			case TIME_WITHOUT_TIME_ZONE:
+				return (val, index, statement) ->
+					statement.setTime(index + 1, Time.valueOf(LocalTime.ofSecondOfDay(val.getInt(index) / 1000L)));

Review comment:
       Would be better to use `LocalTime.ofNanoOfDay(val.getInt(index) * 1_000_000L)` to support millisecond.

##########
File path: flink-connectors/flink-connector-jdbc/src/main/java/org/apache/flink/connector/jdbc/internal/converter/AbstractJdbcRowConverter.java
##########
@@ -18,54 +18,255 @@
 
 package org.apache.flink.connector.jdbc.internal.converter;
 
+import org.apache.flink.connector.jdbc.utils.JdbcTypeUtil;
+import org.apache.flink.table.data.DecimalData;
+import org.apache.flink.table.data.GenericArrayData;
+import org.apache.flink.table.data.GenericRowData;
+import org.apache.flink.table.data.RowData;
+import org.apache.flink.table.data.StringData;
+import org.apache.flink.table.data.TimestampData;
+import org.apache.flink.table.types.logical.ArrayType;
+import org.apache.flink.table.types.logical.DecimalType;
 import org.apache.flink.table.types.logical.LogicalType;
 import org.apache.flink.table.types.logical.LogicalTypeRoot;
 import org.apache.flink.table.types.logical.RowType;
-import org.apache.flink.types.Row;
+import org.apache.flink.table.types.logical.TimestampType;
+import org.apache.flink.table.types.logical.utils.LogicalTypeUtils;
+import org.apache.flink.table.types.utils.TypeConversions;
 
+import java.io.Serializable;
+import java.lang.reflect.Array;
+import java.math.BigDecimal;
+import java.sql.Date;
+import java.sql.PreparedStatement;
 import java.sql.ResultSet;
 import java.sql.SQLException;
+import java.sql.Time;
+import java.sql.Timestamp;
+import java.time.LocalDate;
+import java.time.LocalTime;
 
 import static org.apache.flink.util.Preconditions.checkNotNull;
 
 /**
- * Base for all row converters.
+ * Base class for all converters that convert between JDBC object and Flink internal object.
  */
 public abstract class AbstractJdbcRowConverter implements JdbcRowConverter {
 
 	protected final RowType rowType;
-	protected final JdbcFieldConverter[] converters;
+	protected final JdbcDeserializationConverter[] toInternalConverters;
+	protected final JdbcSerializationConverter[] toExternalConverters;
+	protected final LogicalType[] fieldTypes;
+
+	public abstract String converterName();
 
 	public AbstractJdbcRowConverter(RowType rowType) {
 		this.rowType = checkNotNull(rowType);
-		converters = new JdbcFieldConverter[rowType.getFieldCount()];
-
-		for (int i = 0; i < converters.length; i++) {
-			converters[i] = createConverter(rowType.getTypeAt(i));
+		this.fieldTypes = rowType.getFields().stream()
+			.map(RowType.RowField::getType)
+			.toArray(LogicalType[]::new);
+		this.toInternalConverters = new JdbcDeserializationConverter[rowType.getFieldCount()];
+		this.toExternalConverters = new JdbcSerializationConverter[rowType.getFieldCount()];
+		for (int i = 0; i < rowType.getFieldCount(); i++) {
+			toInternalConverters[i] = createNullableInternalConverter(rowType.getTypeAt(i));
+			toExternalConverters[i] = createNullableExternalConverter(fieldTypes[i]);
 		}
 	}
 
 	@Override
-	public Row convert(ResultSet resultSet, Row reuse) throws SQLException {
+	public RowData toInternal(ResultSet resultSet) throws SQLException {
+		GenericRowData genericRowData = new GenericRowData(rowType.getFieldCount());
 		for (int pos = 0; pos < rowType.getFieldCount(); pos++) {
-			reuse.setField(pos, converters[pos].convert(resultSet.getObject(pos + 1)));
+			genericRowData.setField(pos, toInternalConverters[pos].deserialize(resultSet.getObject(pos + 1)));
+		}
+		return genericRowData;
+	}
+
+	@Override
+	public PreparedStatement toExternal(RowData rowData, PreparedStatement statement) throws SQLException {
+		for (int index = 0; index < rowData.getArity(); index++) {
+			toExternalConverters[index].serialize(rowData, index, statement);
+		}
+		return statement;
+	}
+
+	/**
+	 * Runtime converter to convert JDBC field to {@link RowData} type object.
+	 */
+	@FunctionalInterface
+	interface JdbcDeserializationConverter extends Serializable {
+		/**
+		 * convert a jdbc field to java object, the field could be a simple type or array type.
+		 * @param jdbcField
+		 */
+		Object deserialize(Object jdbcField) throws SQLException;
+	}
+
+	/**
+	 * Runtime converter to convert {@link RowData} field to java object and fill into the {@link PreparedStatement}.
+	 */
+	@FunctionalInterface
+	interface JdbcSerializationConverter extends Serializable {
+		void serialize(RowData rowData, int index, PreparedStatement statement) throws SQLException;
+	}
+
+	/**
+	 * Create a nullable runtime {@link JdbcDeserializationConverter} from given {@link LogicalType}.
+	 */
+	protected JdbcDeserializationConverter createNullableInternalConverter(LogicalType type) {
+		return wrapIntoNullableInternalConverter(createInternalConverter(type));
+	}
+
+	protected JdbcDeserializationConverter wrapIntoNullableInternalConverter(JdbcDeserializationConverter jdbcDeserializationConverter) {
+		return v -> {
+			if (v == null) {
+				return null;
+			} else {
+				return jdbcDeserializationConverter.deserialize(v);
+			}
+		};
+	}
+
+	protected JdbcDeserializationConverter createInternalConverter(LogicalType type) {
+		switch (type.getTypeRoot()) {
+			case NULL:
+				return v -> null;
+			case BOOLEAN:
+				return v -> (boolean) v;

Review comment:
       We don't need hard unboxing here. Because we will boxing it into `GenericRowData` later. 

##########
File path: flink-connectors/flink-connector-jdbc/src/main/java/org/apache/flink/connector/jdbc/internal/converter/AbstractJdbcRowConverter.java
##########
@@ -18,54 +18,255 @@
 
 package org.apache.flink.connector.jdbc.internal.converter;
 
+import org.apache.flink.connector.jdbc.utils.JdbcTypeUtil;
+import org.apache.flink.table.data.DecimalData;
+import org.apache.flink.table.data.GenericArrayData;
+import org.apache.flink.table.data.GenericRowData;
+import org.apache.flink.table.data.RowData;
+import org.apache.flink.table.data.StringData;
+import org.apache.flink.table.data.TimestampData;
+import org.apache.flink.table.types.logical.ArrayType;
+import org.apache.flink.table.types.logical.DecimalType;
 import org.apache.flink.table.types.logical.LogicalType;
 import org.apache.flink.table.types.logical.LogicalTypeRoot;
 import org.apache.flink.table.types.logical.RowType;
-import org.apache.flink.types.Row;
+import org.apache.flink.table.types.logical.TimestampType;
+import org.apache.flink.table.types.logical.utils.LogicalTypeUtils;
+import org.apache.flink.table.types.utils.TypeConversions;
 
+import java.io.Serializable;
+import java.lang.reflect.Array;
+import java.math.BigDecimal;
+import java.sql.Date;
+import java.sql.PreparedStatement;
 import java.sql.ResultSet;
 import java.sql.SQLException;
+import java.sql.Time;
+import java.sql.Timestamp;
+import java.time.LocalDate;
+import java.time.LocalTime;
 
 import static org.apache.flink.util.Preconditions.checkNotNull;
 
 /**
- * Base for all row converters.
+ * Base class for all converters that convert between JDBC object and Flink internal object.
  */
 public abstract class AbstractJdbcRowConverter implements JdbcRowConverter {
 
 	protected final RowType rowType;
-	protected final JdbcFieldConverter[] converters;
+	protected final JdbcDeserializationConverter[] toInternalConverters;
+	protected final JdbcSerializationConverter[] toExternalConverters;
+	protected final LogicalType[] fieldTypes;
+
+	public abstract String converterName();
 
 	public AbstractJdbcRowConverter(RowType rowType) {
 		this.rowType = checkNotNull(rowType);
-		converters = new JdbcFieldConverter[rowType.getFieldCount()];
-
-		for (int i = 0; i < converters.length; i++) {
-			converters[i] = createConverter(rowType.getTypeAt(i));
+		this.fieldTypes = rowType.getFields().stream()
+			.map(RowType.RowField::getType)
+			.toArray(LogicalType[]::new);
+		this.toInternalConverters = new JdbcDeserializationConverter[rowType.getFieldCount()];
+		this.toExternalConverters = new JdbcSerializationConverter[rowType.getFieldCount()];
+		for (int i = 0; i < rowType.getFieldCount(); i++) {
+			toInternalConverters[i] = createNullableInternalConverter(rowType.getTypeAt(i));
+			toExternalConverters[i] = createNullableExternalConverter(fieldTypes[i]);
 		}
 	}
 
 	@Override
-	public Row convert(ResultSet resultSet, Row reuse) throws SQLException {
+	public RowData toInternal(ResultSet resultSet) throws SQLException {
+		GenericRowData genericRowData = new GenericRowData(rowType.getFieldCount());
 		for (int pos = 0; pos < rowType.getFieldCount(); pos++) {
-			reuse.setField(pos, converters[pos].convert(resultSet.getObject(pos + 1)));
+			genericRowData.setField(pos, toInternalConverters[pos].deserialize(resultSet.getObject(pos + 1)));
+		}
+		return genericRowData;
+	}
+
+	@Override
+	public PreparedStatement toExternal(RowData rowData, PreparedStatement statement) throws SQLException {
+		for (int index = 0; index < rowData.getArity(); index++) {
+			toExternalConverters[index].serialize(rowData, index, statement);
+		}
+		return statement;
+	}
+
+	/**
+	 * Runtime converter to convert JDBC field to {@link RowData} type object.
+	 */
+	@FunctionalInterface
+	interface JdbcDeserializationConverter extends Serializable {
+		/**
+		 * convert a jdbc field to java object, the field could be a simple type or array type.
+		 * @param jdbcField
+		 */
+		Object deserialize(Object jdbcField) throws SQLException;
+	}
+
+	/**
+	 * Runtime converter to convert {@link RowData} field to java object and fill into the {@link PreparedStatement}.
+	 */
+	@FunctionalInterface
+	interface JdbcSerializationConverter extends Serializable {
+		void serialize(RowData rowData, int index, PreparedStatement statement) throws SQLException;
+	}
+
+	/**
+	 * Create a nullable runtime {@link JdbcDeserializationConverter} from given {@link LogicalType}.
+	 */
+	protected JdbcDeserializationConverter createNullableInternalConverter(LogicalType type) {
+		return wrapIntoNullableInternalConverter(createInternalConverter(type));
+	}
+
+	protected JdbcDeserializationConverter wrapIntoNullableInternalConverter(JdbcDeserializationConverter jdbcDeserializationConverter) {
+		return v -> {
+			if (v == null) {
+				return null;
+			} else {
+				return jdbcDeserializationConverter.deserialize(v);
+			}
+		};
+	}
+
+	protected JdbcDeserializationConverter createInternalConverter(LogicalType type) {
+		switch (type.getTypeRoot()) {
+			case NULL:
+				return v -> null;
+			case BOOLEAN:
+				return v -> (boolean) v;
+			case TINYINT:
+				return v -> (byte) v;
+			case SMALLINT:
+				// Converter for small type that casts value to int and then return short value, since
+				// JDBC 1.0 use int type for small values.
+				return v -> (Integer.valueOf(v.toString())).shortValue();
+			case INTEGER:
+			case INTERVAL_YEAR_MONTH:
+				return v -> (int) v;
+			case BIGINT:
+			case INTERVAL_DAY_TIME:
+				return v -> (long) v;
+			case DATE:
+				return v -> (int) (((Date) v).toLocalDate().toEpochDay());
+			case TIME_WITHOUT_TIME_ZONE:
+				return v -> ((Time) v).toLocalTime().toSecondOfDay() * 1000;
+			case TIMESTAMP_WITH_TIME_ZONE:
+			case TIMESTAMP_WITHOUT_TIME_ZONE:
+				return v -> TimestampData.fromTimestamp((Timestamp) v);
+			case FLOAT:
+				return v -> (float) v;
+			case DOUBLE:
+				return v -> (double) v;
+			case CHAR:
+			case VARCHAR:
+				return v -> StringData.fromString((String) v);
+			case BINARY:
+			case VARBINARY:
+				return v -> (byte[]) v;
+			case DECIMAL:
+				final int precision = ((DecimalType) type).getPrecision();
+				final int scale = ((DecimalType) type).getScale();
+				return v -> DecimalData.fromBigDecimal((BigDecimal) v, precision, scale);
+			case ARRAY:
+				final JdbcDeserializationConverter arrayConverter = createToInternalArrayConverter((ArrayType) type);
+				return v -> arrayConverter.deserialize(v);

Review comment:
       Simplify to `return createToInternalArrayConverter((ArrayType) type);`.

##########
File path: flink-connectors/flink-connector-jdbc/src/main/java/org/apache/flink/connector/jdbc/internal/converter/AbstractJdbcRowConverter.java
##########
@@ -18,54 +18,255 @@
 
 package org.apache.flink.connector.jdbc.internal.converter;
 
+import org.apache.flink.connector.jdbc.utils.JdbcTypeUtil;
+import org.apache.flink.table.data.DecimalData;
+import org.apache.flink.table.data.GenericArrayData;
+import org.apache.flink.table.data.GenericRowData;
+import org.apache.flink.table.data.RowData;
+import org.apache.flink.table.data.StringData;
+import org.apache.flink.table.data.TimestampData;
+import org.apache.flink.table.types.logical.ArrayType;
+import org.apache.flink.table.types.logical.DecimalType;
 import org.apache.flink.table.types.logical.LogicalType;
 import org.apache.flink.table.types.logical.LogicalTypeRoot;
 import org.apache.flink.table.types.logical.RowType;
-import org.apache.flink.types.Row;
+import org.apache.flink.table.types.logical.TimestampType;
+import org.apache.flink.table.types.logical.utils.LogicalTypeUtils;
+import org.apache.flink.table.types.utils.TypeConversions;
 
+import java.io.Serializable;
+import java.lang.reflect.Array;
+import java.math.BigDecimal;
+import java.sql.Date;
+import java.sql.PreparedStatement;
 import java.sql.ResultSet;
 import java.sql.SQLException;
+import java.sql.Time;
+import java.sql.Timestamp;
+import java.time.LocalDate;
+import java.time.LocalTime;
 
 import static org.apache.flink.util.Preconditions.checkNotNull;
 
 /**
- * Base for all row converters.
+ * Base class for all converters that convert between JDBC object and Flink internal object.
  */
 public abstract class AbstractJdbcRowConverter implements JdbcRowConverter {
 
 	protected final RowType rowType;
-	protected final JdbcFieldConverter[] converters;
+	protected final JdbcDeserializationConverter[] toInternalConverters;
+	protected final JdbcSerializationConverter[] toExternalConverters;
+	protected final LogicalType[] fieldTypes;
+
+	public abstract String converterName();
 
 	public AbstractJdbcRowConverter(RowType rowType) {
 		this.rowType = checkNotNull(rowType);
-		converters = new JdbcFieldConverter[rowType.getFieldCount()];
-
-		for (int i = 0; i < converters.length; i++) {
-			converters[i] = createConverter(rowType.getTypeAt(i));
+		this.fieldTypes = rowType.getFields().stream()
+			.map(RowType.RowField::getType)
+			.toArray(LogicalType[]::new);
+		this.toInternalConverters = new JdbcDeserializationConverter[rowType.getFieldCount()];
+		this.toExternalConverters = new JdbcSerializationConverter[rowType.getFieldCount()];
+		for (int i = 0; i < rowType.getFieldCount(); i++) {
+			toInternalConverters[i] = createNullableInternalConverter(rowType.getTypeAt(i));
+			toExternalConverters[i] = createNullableExternalConverter(fieldTypes[i]);
 		}
 	}
 
 	@Override
-	public Row convert(ResultSet resultSet, Row reuse) throws SQLException {
+	public RowData toInternal(ResultSet resultSet) throws SQLException {
+		GenericRowData genericRowData = new GenericRowData(rowType.getFieldCount());
 		for (int pos = 0; pos < rowType.getFieldCount(); pos++) {
-			reuse.setField(pos, converters[pos].convert(resultSet.getObject(pos + 1)));
+			genericRowData.setField(pos, toInternalConverters[pos].deserialize(resultSet.getObject(pos + 1)));
+		}
+		return genericRowData;
+	}
+
+	@Override
+	public PreparedStatement toExternal(RowData rowData, PreparedStatement statement) throws SQLException {
+		for (int index = 0; index < rowData.getArity(); index++) {
+			toExternalConverters[index].serialize(rowData, index, statement);
+		}
+		return statement;
+	}
+
+	/**
+	 * Runtime converter to convert JDBC field to {@link RowData} type object.
+	 */
+	@FunctionalInterface
+	interface JdbcDeserializationConverter extends Serializable {
+		/**
+		 * convert a jdbc field to java object, the field could be a simple type or array type.
+		 * @param jdbcField
+		 */
+		Object deserialize(Object jdbcField) throws SQLException;
+	}
+
+	/**
+	 * Runtime converter to convert {@link RowData} field to java object and fill into the {@link PreparedStatement}.
+	 */
+	@FunctionalInterface
+	interface JdbcSerializationConverter extends Serializable {
+		void serialize(RowData rowData, int index, PreparedStatement statement) throws SQLException;
+	}
+
+	/**
+	 * Create a nullable runtime {@link JdbcDeserializationConverter} from given {@link LogicalType}.
+	 */
+	protected JdbcDeserializationConverter createNullableInternalConverter(LogicalType type) {
+		return wrapIntoNullableInternalConverter(createInternalConverter(type));
+	}
+
+	protected JdbcDeserializationConverter wrapIntoNullableInternalConverter(JdbcDeserializationConverter jdbcDeserializationConverter) {
+		return v -> {
+			if (v == null) {
+				return null;
+			} else {
+				return jdbcDeserializationConverter.deserialize(v);
+			}
+		};
+	}
+
+	protected JdbcDeserializationConverter createInternalConverter(LogicalType type) {
+		switch (type.getTypeRoot()) {
+			case NULL:
+				return v -> null;
+			case BOOLEAN:
+				return v -> (boolean) v;
+			case TINYINT:
+				return v -> (byte) v;
+			case SMALLINT:
+				// Converter for small type that casts value to int and then return short value, since
+				// JDBC 1.0 use int type for small values.
+				return v -> (Integer.valueOf(v.toString())).shortValue();
+			case INTEGER:
+			case INTERVAL_YEAR_MONTH:
+				return v -> (int) v;
+			case BIGINT:
+			case INTERVAL_DAY_TIME:
+				return v -> (long) v;
+			case DATE:
+				return v -> (int) (((Date) v).toLocalDate().toEpochDay());
+			case TIME_WITHOUT_TIME_ZONE:
+				return v -> ((Time) v).toLocalTime().toSecondOfDay() * 1000;
+			case TIMESTAMP_WITH_TIME_ZONE:
+			case TIMESTAMP_WITHOUT_TIME_ZONE:
+				return v -> TimestampData.fromTimestamp((Timestamp) v);
+			case FLOAT:
+				return v -> (float) v;
+			case DOUBLE:
+				return v -> (double) v;
+			case CHAR:
+			case VARCHAR:
+				return v -> StringData.fromString((String) v);
+			case BINARY:
+			case VARBINARY:
+				return v -> (byte[]) v;
+			case DECIMAL:
+				final int precision = ((DecimalType) type).getPrecision();
+				final int scale = ((DecimalType) type).getScale();
+				return v -> DecimalData.fromBigDecimal((BigDecimal) v, precision, scale);
+			case ARRAY:
+				final JdbcDeserializationConverter arrayConverter = createToInternalArrayConverter((ArrayType) type);
+				return v -> arrayConverter.deserialize(v);
+			case ROW:
+			case MAP:
+			case MULTISET:
+			case RAW:
+			default:
+				throw new UnsupportedOperationException("Not support to parse type: " + type);
 		}
+	}
 
-		return reuse;
+	protected JdbcDeserializationConverter createToInternalArrayConverter(ArrayType arrayType) {
+		final JdbcDeserializationConverter elementConverter = createNullableInternalConverter(arrayType.getElementType());
+		final Class<?> elementClass = LogicalTypeUtils.toInternalConversionClass(arrayType.getElementType());
+		return v -> {
+			final Object[] objects = (Object[]) v;
+			final Object[] array = (Object[]) Array.newInstance(elementClass, objects.length);
+			for (int i = 0; i < objects.length; i++) {
+				array[i] = elementConverter.deserialize(objects[i]);
+			}
+			return new GenericArrayData(array);
+		};
 	}
 
 	/**
-	 * Create a runtime JDBC field converter from given {@link LogicalType}.
+	 * Create a nullable JDBC f{@link JdbcSerializationConverter} from given sql type.
 	 */
-	public JdbcFieldConverter createConverter(LogicalType type) {
-		LogicalTypeRoot root = type.getTypeRoot();
-
-		if (root == LogicalTypeRoot.SMALLINT) {
-			// Converter for small type that casts value to int and then return short value, since
-	        // JDBC 1.0 use int type for small values.
-			return v -> ((Integer) v).shortValue();
-		} else {
-			return v -> v;
+	protected JdbcSerializationConverter createNullableExternalConverter(LogicalType type) {
+		return wrapIntoNullableExternalConverter(createExternalConverter(type), type);
+	}
+
+	protected JdbcSerializationConverter wrapIntoNullableExternalConverter(JdbcSerializationConverter jdbcSerializationConverter, LogicalType type) {
+		final int sqlType = JdbcTypeUtil.typeInformationToSqlType(TypeConversions.fromDataTypeToLegacyInfo(
+			TypeConversions.fromLogicalToDataType(type)));
+		return (val, index, statement)  -> {
+			if (val == null || val.isNullAt(index) || LogicalTypeRoot.NULL.equals(type.getTypeRoot())) {
+				statement.setNull(index + 1, sqlType);
+			} else {
+				jdbcSerializationConverter.serialize(val, index, statement);
+			}
+		};
+	}
+
+	protected JdbcSerializationConverter createExternalConverter(LogicalType type) {
+		switch (type.getTypeRoot()) {
+			case BOOLEAN:
+				return (val, index, statement) -> statement.setBoolean(index + 1, val.getBoolean(index));
+			case TINYINT:
+				return (val, index, statement) -> statement.setByte(index + 1, val.getByte(index));
+			case SMALLINT:
+				return (val, index, statement) -> statement.setShort(index + 1, val.getShort(index));
+			case INTEGER:
+			case INTERVAL_YEAR_MONTH:
+				return (val, index, statement) -> statement.setInt(index + 1, val.getInt(index));
+			case BIGINT:
+			case INTERVAL_DAY_TIME:
+				return (val, index, statement) -> statement.setLong(index + 1, val.getLong(index));
+			case FLOAT:
+				return (val, index, statement) -> statement.setFloat(index + 1, val.getFloat(index));
+			case DOUBLE:
+				return (val, index, statement) -> statement.setDouble(index + 1, val.getDouble(index));
+			case CHAR:
+			case VARCHAR:
+				// value is BinaryString
+				return (val, index, statement) -> statement.setString(index + 1, val.getString(index).toString());
+			case BINARY:
+			case VARBINARY:
+				return (val, index, statement) -> statement.setBytes(index + 1, val.getBinary(index));
+			case DATE:
+				return (val, index, statement) ->
+					statement.setDate(index + 1, Date.valueOf(LocalDate.ofEpochDay(val.getInt(index))));
+			case TIME_WITHOUT_TIME_ZONE:
+				return (val, index, statement) ->
+					statement.setTime(index + 1, Time.valueOf(LocalTime.ofSecondOfDay(val.getInt(index) / 1000L)));
+			case TIMESTAMP_WITH_TIME_ZONE:
+			case TIMESTAMP_WITHOUT_TIME_ZONE:
+				final int timestampPrecision = ((TimestampType) type).getPrecision();
+				return (val, index, statement) ->
+					statement.setTimestamp(index + 1, val
+						.getTimestamp(index, timestampPrecision)
+						.toTimestamp());
+			case DECIMAL:
+				final int decimalPrecision = ((DecimalType) type).getPrecision();
+				final int decimalScale = ((DecimalType) type).getScale();
+				return (val, index, statement) ->
+					statement.setBigDecimal(index + 1, val
+						.getDecimal(index, decimalPrecision, decimalScale)
+						.toBigDecimal());
+			case ARRAY:
+				//note: dialect need implements the conversion from ArrayData to JDBC Array if the dialect supports array.
+				return (val, index, statement) -> {
+					throw new IllegalStateException(
+						String.format("JDBC:%s do not support write ARRAY type.", converterName()));
+				};
+			case MAP:
+			case MULTISET:
+			case ROW:
+			case RAW:
+			default:
+				throw new UnsupportedOperationException("Not support to parse type: " + type);

Review comment:
       "Unsupported type: " + type

##########
File path: flink-connectors/flink-connector-jdbc/src/main/java/org/apache/flink/connector/jdbc/internal/converter/AbstractJdbcRowConverter.java
##########
@@ -18,54 +18,255 @@
 
 package org.apache.flink.connector.jdbc.internal.converter;
 
+import org.apache.flink.connector.jdbc.utils.JdbcTypeUtil;
+import org.apache.flink.table.data.DecimalData;
+import org.apache.flink.table.data.GenericArrayData;
+import org.apache.flink.table.data.GenericRowData;
+import org.apache.flink.table.data.RowData;
+import org.apache.flink.table.data.StringData;
+import org.apache.flink.table.data.TimestampData;
+import org.apache.flink.table.types.logical.ArrayType;
+import org.apache.flink.table.types.logical.DecimalType;
 import org.apache.flink.table.types.logical.LogicalType;
 import org.apache.flink.table.types.logical.LogicalTypeRoot;
 import org.apache.flink.table.types.logical.RowType;
-import org.apache.flink.types.Row;
+import org.apache.flink.table.types.logical.TimestampType;
+import org.apache.flink.table.types.logical.utils.LogicalTypeUtils;
+import org.apache.flink.table.types.utils.TypeConversions;
 
+import java.io.Serializable;
+import java.lang.reflect.Array;
+import java.math.BigDecimal;
+import java.sql.Date;
+import java.sql.PreparedStatement;
 import java.sql.ResultSet;
 import java.sql.SQLException;
+import java.sql.Time;
+import java.sql.Timestamp;
+import java.time.LocalDate;
+import java.time.LocalTime;
 
 import static org.apache.flink.util.Preconditions.checkNotNull;
 
 /**
- * Base for all row converters.
+ * Base class for all converters that convert between JDBC object and Flink internal object.
  */
 public abstract class AbstractJdbcRowConverter implements JdbcRowConverter {
 
 	protected final RowType rowType;
-	protected final JdbcFieldConverter[] converters;
+	protected final JdbcDeserializationConverter[] toInternalConverters;
+	protected final JdbcSerializationConverter[] toExternalConverters;
+	protected final LogicalType[] fieldTypes;
+
+	public abstract String converterName();
 
 	public AbstractJdbcRowConverter(RowType rowType) {
 		this.rowType = checkNotNull(rowType);
-		converters = new JdbcFieldConverter[rowType.getFieldCount()];
-
-		for (int i = 0; i < converters.length; i++) {
-			converters[i] = createConverter(rowType.getTypeAt(i));
+		this.fieldTypes = rowType.getFields().stream()
+			.map(RowType.RowField::getType)
+			.toArray(LogicalType[]::new);
+		this.toInternalConverters = new JdbcDeserializationConverter[rowType.getFieldCount()];
+		this.toExternalConverters = new JdbcSerializationConverter[rowType.getFieldCount()];
+		for (int i = 0; i < rowType.getFieldCount(); i++) {
+			toInternalConverters[i] = createNullableInternalConverter(rowType.getTypeAt(i));
+			toExternalConverters[i] = createNullableExternalConverter(fieldTypes[i]);
 		}
 	}
 
 	@Override
-	public Row convert(ResultSet resultSet, Row reuse) throws SQLException {
+	public RowData toInternal(ResultSet resultSet) throws SQLException {
+		GenericRowData genericRowData = new GenericRowData(rowType.getFieldCount());
 		for (int pos = 0; pos < rowType.getFieldCount(); pos++) {
-			reuse.setField(pos, converters[pos].convert(resultSet.getObject(pos + 1)));
+			genericRowData.setField(pos, toInternalConverters[pos].deserialize(resultSet.getObject(pos + 1)));
+		}
+		return genericRowData;
+	}
+
+	@Override
+	public PreparedStatement toExternal(RowData rowData, PreparedStatement statement) throws SQLException {
+		for (int index = 0; index < rowData.getArity(); index++) {
+			toExternalConverters[index].serialize(rowData, index, statement);
+		}
+		return statement;
+	}
+
+	/**
+	 * Runtime converter to convert JDBC field to {@link RowData} type object.
+	 */
+	@FunctionalInterface
+	interface JdbcDeserializationConverter extends Serializable {
+		/**
+		 * convert a jdbc field to java object, the field could be a simple type or array type.
+		 * @param jdbcField
+		 */
+		Object deserialize(Object jdbcField) throws SQLException;
+	}
+
+	/**
+	 * Runtime converter to convert {@link RowData} field to java object and fill into the {@link PreparedStatement}.
+	 */
+	@FunctionalInterface
+	interface JdbcSerializationConverter extends Serializable {
+		void serialize(RowData rowData, int index, PreparedStatement statement) throws SQLException;
+	}
+
+	/**
+	 * Create a nullable runtime {@link JdbcDeserializationConverter} from given {@link LogicalType}.
+	 */
+	protected JdbcDeserializationConverter createNullableInternalConverter(LogicalType type) {
+		return wrapIntoNullableInternalConverter(createInternalConverter(type));
+	}
+
+	protected JdbcDeserializationConverter wrapIntoNullableInternalConverter(JdbcDeserializationConverter jdbcDeserializationConverter) {
+		return v -> {
+			if (v == null) {
+				return null;
+			} else {
+				return jdbcDeserializationConverter.deserialize(v);
+			}
+		};
+	}
+
+	protected JdbcDeserializationConverter createInternalConverter(LogicalType type) {
+		switch (type.getTypeRoot()) {
+			case NULL:
+				return v -> null;
+			case BOOLEAN:
+				return v -> (boolean) v;
+			case TINYINT:
+				return v -> (byte) v;
+			case SMALLINT:
+				// Converter for small type that casts value to int and then return short value, since
+				// JDBC 1.0 use int type for small values.
+				return v -> (Integer.valueOf(v.toString())).shortValue();
+			case INTEGER:
+			case INTERVAL_YEAR_MONTH:
+				return v -> (int) v;
+			case BIGINT:
+			case INTERVAL_DAY_TIME:
+				return v -> (long) v;
+			case DATE:
+				return v -> (int) (((Date) v).toLocalDate().toEpochDay());
+			case TIME_WITHOUT_TIME_ZONE:
+				return v -> ((Time) v).toLocalTime().toSecondOfDay() * 1000;
+			case TIMESTAMP_WITH_TIME_ZONE:
+			case TIMESTAMP_WITHOUT_TIME_ZONE:
+				return v -> TimestampData.fromTimestamp((Timestamp) v);
+			case FLOAT:
+				return v -> (float) v;
+			case DOUBLE:
+				return v -> (double) v;
+			case CHAR:
+			case VARCHAR:
+				return v -> StringData.fromString((String) v);
+			case BINARY:
+			case VARBINARY:
+				return v -> (byte[]) v;
+			case DECIMAL:
+				final int precision = ((DecimalType) type).getPrecision();
+				final int scale = ((DecimalType) type).getScale();
+				return v -> DecimalData.fromBigDecimal((BigDecimal) v, precision, scale);
+			case ARRAY:
+				final JdbcDeserializationConverter arrayConverter = createToInternalArrayConverter((ArrayType) type);
+				return v -> arrayConverter.deserialize(v);
+			case ROW:
+			case MAP:
+			case MULTISET:
+			case RAW:
+			default:
+				throw new UnsupportedOperationException("Not support to parse type: " + type);
 		}
+	}
 
-		return reuse;
+	protected JdbcDeserializationConverter createToInternalArrayConverter(ArrayType arrayType) {
+		final JdbcDeserializationConverter elementConverter = createNullableInternalConverter(arrayType.getElementType());
+		final Class<?> elementClass = LogicalTypeUtils.toInternalConversionClass(arrayType.getElementType());
+		return v -> {
+			final Object[] objects = (Object[]) v;
+			final Object[] array = (Object[]) Array.newInstance(elementClass, objects.length);
+			for (int i = 0; i < objects.length; i++) {
+				array[i] = elementConverter.deserialize(objects[i]);
+			}
+			return new GenericArrayData(array);
+		};
 	}
 
 	/**
-	 * Create a runtime JDBC field converter from given {@link LogicalType}.
+	 * Create a nullable JDBC f{@link JdbcSerializationConverter} from given sql type.
 	 */
-	public JdbcFieldConverter createConverter(LogicalType type) {
-		LogicalTypeRoot root = type.getTypeRoot();
-
-		if (root == LogicalTypeRoot.SMALLINT) {
-			// Converter for small type that casts value to int and then return short value, since
-	        // JDBC 1.0 use int type for small values.
-			return v -> ((Integer) v).shortValue();
-		} else {
-			return v -> v;
+	protected JdbcSerializationConverter createNullableExternalConverter(LogicalType type) {
+		return wrapIntoNullableExternalConverter(createExternalConverter(type), type);
+	}
+
+	protected JdbcSerializationConverter wrapIntoNullableExternalConverter(JdbcSerializationConverter jdbcSerializationConverter, LogicalType type) {
+		final int sqlType = JdbcTypeUtil.typeInformationToSqlType(TypeConversions.fromDataTypeToLegacyInfo(
+			TypeConversions.fromLogicalToDataType(type)));
+		return (val, index, statement)  -> {
+			if (val == null || val.isNullAt(index) || LogicalTypeRoot.NULL.equals(type.getTypeRoot())) {
+				statement.setNull(index + 1, sqlType);
+			} else {
+				jdbcSerializationConverter.serialize(val, index, statement);
+			}
+		};
+	}
+
+	protected JdbcSerializationConverter createExternalConverter(LogicalType type) {
+		switch (type.getTypeRoot()) {
+			case BOOLEAN:
+				return (val, index, statement) -> statement.setBoolean(index + 1, val.getBoolean(index));
+			case TINYINT:
+				return (val, index, statement) -> statement.setByte(index + 1, val.getByte(index));
+			case SMALLINT:
+				return (val, index, statement) -> statement.setShort(index + 1, val.getShort(index));
+			case INTEGER:
+			case INTERVAL_YEAR_MONTH:
+				return (val, index, statement) -> statement.setInt(index + 1, val.getInt(index));
+			case BIGINT:
+			case INTERVAL_DAY_TIME:
+				return (val, index, statement) -> statement.setLong(index + 1, val.getLong(index));
+			case FLOAT:
+				return (val, index, statement) -> statement.setFloat(index + 1, val.getFloat(index));
+			case DOUBLE:
+				return (val, index, statement) -> statement.setDouble(index + 1, val.getDouble(index));
+			case CHAR:
+			case VARCHAR:
+				// value is BinaryString
+				return (val, index, statement) -> statement.setString(index + 1, val.getString(index).toString());
+			case BINARY:
+			case VARBINARY:
+				return (val, index, statement) -> statement.setBytes(index + 1, val.getBinary(index));
+			case DATE:
+				return (val, index, statement) ->
+					statement.setDate(index + 1, Date.valueOf(LocalDate.ofEpochDay(val.getInt(index))));
+			case TIME_WITHOUT_TIME_ZONE:
+				return (val, index, statement) ->
+					statement.setTime(index + 1, Time.valueOf(LocalTime.ofSecondOfDay(val.getInt(index) / 1000L)));
+			case TIMESTAMP_WITH_TIME_ZONE:
+			case TIMESTAMP_WITHOUT_TIME_ZONE:
+				final int timestampPrecision = ((TimestampType) type).getPrecision();
+				return (val, index, statement) ->
+					statement.setTimestamp(index + 1, val
+						.getTimestamp(index, timestampPrecision)
+						.toTimestamp());

Review comment:
       Nit, I think a better style is:
   
   ```java
   statement.setTimestamp(
   	index + 1,
   	val.getTimestamp(index, timestampPrecision).toTimestamp()
   );
   ```

##########
File path: flink-connectors/flink-connector-jdbc/src/main/java/org/apache/flink/connector/jdbc/internal/converter/AbstractJdbcRowConverter.java
##########
@@ -18,54 +18,255 @@
 
 package org.apache.flink.connector.jdbc.internal.converter;
 
+import org.apache.flink.connector.jdbc.utils.JdbcTypeUtil;
+import org.apache.flink.table.data.DecimalData;
+import org.apache.flink.table.data.GenericArrayData;
+import org.apache.flink.table.data.GenericRowData;
+import org.apache.flink.table.data.RowData;
+import org.apache.flink.table.data.StringData;
+import org.apache.flink.table.data.TimestampData;
+import org.apache.flink.table.types.logical.ArrayType;
+import org.apache.flink.table.types.logical.DecimalType;
 import org.apache.flink.table.types.logical.LogicalType;
 import org.apache.flink.table.types.logical.LogicalTypeRoot;
 import org.apache.flink.table.types.logical.RowType;
-import org.apache.flink.types.Row;
+import org.apache.flink.table.types.logical.TimestampType;
+import org.apache.flink.table.types.logical.utils.LogicalTypeUtils;
+import org.apache.flink.table.types.utils.TypeConversions;
 
+import java.io.Serializable;
+import java.lang.reflect.Array;
+import java.math.BigDecimal;
+import java.sql.Date;
+import java.sql.PreparedStatement;
 import java.sql.ResultSet;
 import java.sql.SQLException;
+import java.sql.Time;
+import java.sql.Timestamp;
+import java.time.LocalDate;
+import java.time.LocalTime;
 
 import static org.apache.flink.util.Preconditions.checkNotNull;
 
 /**
- * Base for all row converters.
+ * Base class for all converters that convert between JDBC object and Flink internal object.
  */
 public abstract class AbstractJdbcRowConverter implements JdbcRowConverter {
 
 	protected final RowType rowType;
-	protected final JdbcFieldConverter[] converters;
+	protected final JdbcDeserializationConverter[] toInternalConverters;
+	protected final JdbcSerializationConverter[] toExternalConverters;
+	protected final LogicalType[] fieldTypes;
+
+	public abstract String converterName();
 
 	public AbstractJdbcRowConverter(RowType rowType) {
 		this.rowType = checkNotNull(rowType);
-		converters = new JdbcFieldConverter[rowType.getFieldCount()];
-
-		for (int i = 0; i < converters.length; i++) {
-			converters[i] = createConverter(rowType.getTypeAt(i));
+		this.fieldTypes = rowType.getFields().stream()
+			.map(RowType.RowField::getType)
+			.toArray(LogicalType[]::new);
+		this.toInternalConverters = new JdbcDeserializationConverter[rowType.getFieldCount()];
+		this.toExternalConverters = new JdbcSerializationConverter[rowType.getFieldCount()];
+		for (int i = 0; i < rowType.getFieldCount(); i++) {
+			toInternalConverters[i] = createNullableInternalConverter(rowType.getTypeAt(i));
+			toExternalConverters[i] = createNullableExternalConverter(fieldTypes[i]);
 		}
 	}
 
 	@Override
-	public Row convert(ResultSet resultSet, Row reuse) throws SQLException {
+	public RowData toInternal(ResultSet resultSet) throws SQLException {
+		GenericRowData genericRowData = new GenericRowData(rowType.getFieldCount());
 		for (int pos = 0; pos < rowType.getFieldCount(); pos++) {
-			reuse.setField(pos, converters[pos].convert(resultSet.getObject(pos + 1)));
+			genericRowData.setField(pos, toInternalConverters[pos].deserialize(resultSet.getObject(pos + 1)));
+		}
+		return genericRowData;
+	}
+
+	@Override
+	public PreparedStatement toExternal(RowData rowData, PreparedStatement statement) throws SQLException {
+		for (int index = 0; index < rowData.getArity(); index++) {
+			toExternalConverters[index].serialize(rowData, index, statement);
+		}
+		return statement;
+	}
+
+	/**
+	 * Runtime converter to convert JDBC field to {@link RowData} type object.
+	 */
+	@FunctionalInterface
+	interface JdbcDeserializationConverter extends Serializable {
+		/**
+		 * convert a jdbc field to java object, the field could be a simple type or array type.
+		 * @param jdbcField
+		 */
+		Object deserialize(Object jdbcField) throws SQLException;
+	}
+
+	/**
+	 * Runtime converter to convert {@link RowData} field to java object and fill into the {@link PreparedStatement}.
+	 */
+	@FunctionalInterface
+	interface JdbcSerializationConverter extends Serializable {
+		void serialize(RowData rowData, int index, PreparedStatement statement) throws SQLException;
+	}
+
+	/**
+	 * Create a nullable runtime {@link JdbcDeserializationConverter} from given {@link LogicalType}.
+	 */
+	protected JdbcDeserializationConverter createNullableInternalConverter(LogicalType type) {
+		return wrapIntoNullableInternalConverter(createInternalConverter(type));
+	}
+
+	protected JdbcDeserializationConverter wrapIntoNullableInternalConverter(JdbcDeserializationConverter jdbcDeserializationConverter) {
+		return v -> {
+			if (v == null) {
+				return null;
+			} else {
+				return jdbcDeserializationConverter.deserialize(v);
+			}
+		};
+	}
+
+	protected JdbcDeserializationConverter createInternalConverter(LogicalType type) {
+		switch (type.getTypeRoot()) {
+			case NULL:
+				return v -> null;
+			case BOOLEAN:
+				return v -> (boolean) v;
+			case TINYINT:
+				return v -> (byte) v;
+			case SMALLINT:
+				// Converter for small type that casts value to int and then return short value, since
+				// JDBC 1.0 use int type for small values.
+				return v -> (Integer.valueOf(v.toString())).shortValue();
+			case INTEGER:
+			case INTERVAL_YEAR_MONTH:
+				return v -> (int) v;
+			case BIGINT:
+			case INTERVAL_DAY_TIME:
+				return v -> (long) v;
+			case DATE:
+				return v -> (int) (((Date) v).toLocalDate().toEpochDay());
+			case TIME_WITHOUT_TIME_ZONE:
+				return v -> ((Time) v).toLocalTime().toSecondOfDay() * 1000;
+			case TIMESTAMP_WITH_TIME_ZONE:
+			case TIMESTAMP_WITHOUT_TIME_ZONE:
+				return v -> TimestampData.fromTimestamp((Timestamp) v);
+			case FLOAT:
+				return v -> (float) v;
+			case DOUBLE:
+				return v -> (double) v;
+			case CHAR:
+			case VARCHAR:
+				return v -> StringData.fromString((String) v);
+			case BINARY:
+			case VARBINARY:
+				return v -> (byte[]) v;
+			case DECIMAL:
+				final int precision = ((DecimalType) type).getPrecision();
+				final int scale = ((DecimalType) type).getScale();
+				return v -> DecimalData.fromBigDecimal((BigDecimal) v, precision, scale);
+			case ARRAY:
+				final JdbcDeserializationConverter arrayConverter = createToInternalArrayConverter((ArrayType) type);
+				return v -> arrayConverter.deserialize(v);
+			case ROW:
+			case MAP:
+			case MULTISET:
+			case RAW:
+			default:
+				throw new UnsupportedOperationException("Not support to parse type: " + type);

Review comment:
       "Unsupported type: " + type

##########
File path: flink-connectors/flink-connector-jdbc/src/main/java/org/apache/flink/connector/jdbc/internal/converter/AbstractJdbcRowConverter.java
##########
@@ -18,54 +18,255 @@
 
 package org.apache.flink.connector.jdbc.internal.converter;
 
+import org.apache.flink.connector.jdbc.utils.JdbcTypeUtil;
+import org.apache.flink.table.data.DecimalData;
+import org.apache.flink.table.data.GenericArrayData;
+import org.apache.flink.table.data.GenericRowData;
+import org.apache.flink.table.data.RowData;
+import org.apache.flink.table.data.StringData;
+import org.apache.flink.table.data.TimestampData;
+import org.apache.flink.table.types.logical.ArrayType;
+import org.apache.flink.table.types.logical.DecimalType;
 import org.apache.flink.table.types.logical.LogicalType;
 import org.apache.flink.table.types.logical.LogicalTypeRoot;
 import org.apache.flink.table.types.logical.RowType;
-import org.apache.flink.types.Row;
+import org.apache.flink.table.types.logical.TimestampType;
+import org.apache.flink.table.types.logical.utils.LogicalTypeUtils;
+import org.apache.flink.table.types.utils.TypeConversions;
 
+import java.io.Serializable;
+import java.lang.reflect.Array;
+import java.math.BigDecimal;
+import java.sql.Date;
+import java.sql.PreparedStatement;
 import java.sql.ResultSet;
 import java.sql.SQLException;
+import java.sql.Time;
+import java.sql.Timestamp;
+import java.time.LocalDate;
+import java.time.LocalTime;
 
 import static org.apache.flink.util.Preconditions.checkNotNull;
 
 /**
- * Base for all row converters.
+ * Base class for all converters that convert between JDBC object and Flink internal object.
  */
 public abstract class AbstractJdbcRowConverter implements JdbcRowConverter {
 
 	protected final RowType rowType;
-	protected final JdbcFieldConverter[] converters;
+	protected final JdbcDeserializationConverter[] toInternalConverters;
+	protected final JdbcSerializationConverter[] toExternalConverters;
+	protected final LogicalType[] fieldTypes;
+
+	public abstract String converterName();
 
 	public AbstractJdbcRowConverter(RowType rowType) {
 		this.rowType = checkNotNull(rowType);
-		converters = new JdbcFieldConverter[rowType.getFieldCount()];
-
-		for (int i = 0; i < converters.length; i++) {
-			converters[i] = createConverter(rowType.getTypeAt(i));
+		this.fieldTypes = rowType.getFields().stream()
+			.map(RowType.RowField::getType)
+			.toArray(LogicalType[]::new);
+		this.toInternalConverters = new JdbcDeserializationConverter[rowType.getFieldCount()];
+		this.toExternalConverters = new JdbcSerializationConverter[rowType.getFieldCount()];
+		for (int i = 0; i < rowType.getFieldCount(); i++) {
+			toInternalConverters[i] = createNullableInternalConverter(rowType.getTypeAt(i));
+			toExternalConverters[i] = createNullableExternalConverter(fieldTypes[i]);
 		}
 	}
 
 	@Override
-	public Row convert(ResultSet resultSet, Row reuse) throws SQLException {
+	public RowData toInternal(ResultSet resultSet) throws SQLException {
+		GenericRowData genericRowData = new GenericRowData(rowType.getFieldCount());
 		for (int pos = 0; pos < rowType.getFieldCount(); pos++) {
-			reuse.setField(pos, converters[pos].convert(resultSet.getObject(pos + 1)));
+			genericRowData.setField(pos, toInternalConverters[pos].deserialize(resultSet.getObject(pos + 1)));
+		}
+		return genericRowData;
+	}
+
+	@Override
+	public PreparedStatement toExternal(RowData rowData, PreparedStatement statement) throws SQLException {
+		for (int index = 0; index < rowData.getArity(); index++) {
+			toExternalConverters[index].serialize(rowData, index, statement);
+		}
+		return statement;
+	}
+
+	/**
+	 * Runtime converter to convert JDBC field to {@link RowData} type object.
+	 */
+	@FunctionalInterface
+	interface JdbcDeserializationConverter extends Serializable {
+		/**
+		 * convert a jdbc field to java object, the field could be a simple type or array type.
+		 * @param jdbcField
+		 */
+		Object deserialize(Object jdbcField) throws SQLException;
+	}
+
+	/**
+	 * Runtime converter to convert {@link RowData} field to java object and fill into the {@link PreparedStatement}.
+	 */
+	@FunctionalInterface
+	interface JdbcSerializationConverter extends Serializable {
+		void serialize(RowData rowData, int index, PreparedStatement statement) throws SQLException;
+	}
+
+	/**
+	 * Create a nullable runtime {@link JdbcDeserializationConverter} from given {@link LogicalType}.
+	 */
+	protected JdbcDeserializationConverter createNullableInternalConverter(LogicalType type) {
+		return wrapIntoNullableInternalConverter(createInternalConverter(type));
+	}
+
+	protected JdbcDeserializationConverter wrapIntoNullableInternalConverter(JdbcDeserializationConverter jdbcDeserializationConverter) {
+		return v -> {
+			if (v == null) {
+				return null;
+			} else {
+				return jdbcDeserializationConverter.deserialize(v);
+			}
+		};
+	}
+
+	protected JdbcDeserializationConverter createInternalConverter(LogicalType type) {
+		switch (type.getTypeRoot()) {
+			case NULL:
+				return v -> null;
+			case BOOLEAN:
+				return v -> (boolean) v;
+			case TINYINT:
+				return v -> (byte) v;
+			case SMALLINT:
+				// Converter for small type that casts value to int and then return short value, since
+				// JDBC 1.0 use int type for small values.
+				return v -> (Integer.valueOf(v.toString())).shortValue();
+			case INTEGER:
+			case INTERVAL_YEAR_MONTH:
+				return v -> (int) v;
+			case BIGINT:
+			case INTERVAL_DAY_TIME:
+				return v -> (long) v;
+			case DATE:
+				return v -> (int) (((Date) v).toLocalDate().toEpochDay());
+			case TIME_WITHOUT_TIME_ZONE:
+				return v -> ((Time) v).toLocalTime().toSecondOfDay() * 1000;

Review comment:
       Would be better to use `return v -> ((Time) v).toLocalTime().toNanoOfDay() / 1_000_000L;` to support millisecond.

##########
File path: flink-connectors/flink-connector-jdbc/src/main/java/org/apache/flink/connector/jdbc/internal/converter/AbstractJdbcRowConverter.java
##########
@@ -18,54 +18,255 @@
 
 package org.apache.flink.connector.jdbc.internal.converter;
 
+import org.apache.flink.connector.jdbc.utils.JdbcTypeUtil;
+import org.apache.flink.table.data.DecimalData;
+import org.apache.flink.table.data.GenericArrayData;
+import org.apache.flink.table.data.GenericRowData;
+import org.apache.flink.table.data.RowData;
+import org.apache.flink.table.data.StringData;
+import org.apache.flink.table.data.TimestampData;
+import org.apache.flink.table.types.logical.ArrayType;
+import org.apache.flink.table.types.logical.DecimalType;
 import org.apache.flink.table.types.logical.LogicalType;
 import org.apache.flink.table.types.logical.LogicalTypeRoot;
 import org.apache.flink.table.types.logical.RowType;
-import org.apache.flink.types.Row;
+import org.apache.flink.table.types.logical.TimestampType;
+import org.apache.flink.table.types.logical.utils.LogicalTypeUtils;
+import org.apache.flink.table.types.utils.TypeConversions;
 
+import java.io.Serializable;
+import java.lang.reflect.Array;
+import java.math.BigDecimal;
+import java.sql.Date;
+import java.sql.PreparedStatement;
 import java.sql.ResultSet;
 import java.sql.SQLException;
+import java.sql.Time;
+import java.sql.Timestamp;
+import java.time.LocalDate;
+import java.time.LocalTime;
 
 import static org.apache.flink.util.Preconditions.checkNotNull;
 
 /**
- * Base for all row converters.
+ * Base class for all converters that convert between JDBC object and Flink internal object.
  */
 public abstract class AbstractJdbcRowConverter implements JdbcRowConverter {
 
 	protected final RowType rowType;
-	protected final JdbcFieldConverter[] converters;
+	protected final JdbcDeserializationConverter[] toInternalConverters;
+	protected final JdbcSerializationConverter[] toExternalConverters;
+	protected final LogicalType[] fieldTypes;
+
+	public abstract String converterName();
 
 	public AbstractJdbcRowConverter(RowType rowType) {
 		this.rowType = checkNotNull(rowType);
-		converters = new JdbcFieldConverter[rowType.getFieldCount()];
-
-		for (int i = 0; i < converters.length; i++) {
-			converters[i] = createConverter(rowType.getTypeAt(i));
+		this.fieldTypes = rowType.getFields().stream()
+			.map(RowType.RowField::getType)
+			.toArray(LogicalType[]::new);
+		this.toInternalConverters = new JdbcDeserializationConverter[rowType.getFieldCount()];
+		this.toExternalConverters = new JdbcSerializationConverter[rowType.getFieldCount()];
+		for (int i = 0; i < rowType.getFieldCount(); i++) {
+			toInternalConverters[i] = createNullableInternalConverter(rowType.getTypeAt(i));
+			toExternalConverters[i] = createNullableExternalConverter(fieldTypes[i]);
 		}
 	}
 
 	@Override
-	public Row convert(ResultSet resultSet, Row reuse) throws SQLException {
+	public RowData toInternal(ResultSet resultSet) throws SQLException {
+		GenericRowData genericRowData = new GenericRowData(rowType.getFieldCount());
 		for (int pos = 0; pos < rowType.getFieldCount(); pos++) {
-			reuse.setField(pos, converters[pos].convert(resultSet.getObject(pos + 1)));
+			genericRowData.setField(pos, toInternalConverters[pos].deserialize(resultSet.getObject(pos + 1)));
+		}
+		return genericRowData;
+	}
+
+	@Override
+	public PreparedStatement toExternal(RowData rowData, PreparedStatement statement) throws SQLException {
+		for (int index = 0; index < rowData.getArity(); index++) {
+			toExternalConverters[index].serialize(rowData, index, statement);
+		}
+		return statement;
+	}
+
+	/**
+	 * Runtime converter to convert JDBC field to {@link RowData} type object.
+	 */
+	@FunctionalInterface
+	interface JdbcDeserializationConverter extends Serializable {
+		/**
+		 * convert a jdbc field to java object, the field could be a simple type or array type.
+		 * @param jdbcField
+		 */
+		Object deserialize(Object jdbcField) throws SQLException;
+	}
+
+	/**
+	 * Runtime converter to convert {@link RowData} field to java object and fill into the {@link PreparedStatement}.
+	 */
+	@FunctionalInterface
+	interface JdbcSerializationConverter extends Serializable {
+		void serialize(RowData rowData, int index, PreparedStatement statement) throws SQLException;
+	}
+
+	/**
+	 * Create a nullable runtime {@link JdbcDeserializationConverter} from given {@link LogicalType}.
+	 */
+	protected JdbcDeserializationConverter createNullableInternalConverter(LogicalType type) {
+		return wrapIntoNullableInternalConverter(createInternalConverter(type));
+	}
+
+	protected JdbcDeserializationConverter wrapIntoNullableInternalConverter(JdbcDeserializationConverter jdbcDeserializationConverter) {
+		return v -> {
+			if (v == null) {
+				return null;
+			} else {
+				return jdbcDeserializationConverter.deserialize(v);
+			}
+		};
+	}
+
+	protected JdbcDeserializationConverter createInternalConverter(LogicalType type) {
+		switch (type.getTypeRoot()) {
+			case NULL:
+				return v -> null;
+			case BOOLEAN:
+				return v -> (boolean) v;
+			case TINYINT:
+				return v -> (byte) v;
+			case SMALLINT:
+				// Converter for small type that casts value to int and then return short value, since
+				// JDBC 1.0 use int type for small values.
+				return v -> (Integer.valueOf(v.toString())).shortValue();
+			case INTEGER:
+			case INTERVAL_YEAR_MONTH:
+				return v -> (int) v;
+			case BIGINT:
+			case INTERVAL_DAY_TIME:
+				return v -> (long) v;
+			case DATE:
+				return v -> (int) (((Date) v).toLocalDate().toEpochDay());
+			case TIME_WITHOUT_TIME_ZONE:
+				return v -> ((Time) v).toLocalTime().toSecondOfDay() * 1000;
+			case TIMESTAMP_WITH_TIME_ZONE:
+			case TIMESTAMP_WITHOUT_TIME_ZONE:
+				return v -> TimestampData.fromTimestamp((Timestamp) v);
+			case FLOAT:
+				return v -> (float) v;
+			case DOUBLE:
+				return v -> (double) v;
+			case CHAR:
+			case VARCHAR:
+				return v -> StringData.fromString((String) v);
+			case BINARY:
+			case VARBINARY:
+				return v -> (byte[]) v;
+			case DECIMAL:
+				final int precision = ((DecimalType) type).getPrecision();
+				final int scale = ((DecimalType) type).getScale();
+				return v -> DecimalData.fromBigDecimal((BigDecimal) v, precision, scale);
+			case ARRAY:
+				final JdbcDeserializationConverter arrayConverter = createToInternalArrayConverter((ArrayType) type);
+				return v -> arrayConverter.deserialize(v);
+			case ROW:
+			case MAP:
+			case MULTISET:
+			case RAW:
+			default:
+				throw new UnsupportedOperationException("Not support to parse type: " + type);
 		}
+	}
 
-		return reuse;
+	protected JdbcDeserializationConverter createToInternalArrayConverter(ArrayType arrayType) {
+		final JdbcDeserializationConverter elementConverter = createNullableInternalConverter(arrayType.getElementType());
+		final Class<?> elementClass = LogicalTypeUtils.toInternalConversionClass(arrayType.getElementType());
+		return v -> {
+			final Object[] objects = (Object[]) v;
+			final Object[] array = (Object[]) Array.newInstance(elementClass, objects.length);
+			for (int i = 0; i < objects.length; i++) {
+				array[i] = elementConverter.deserialize(objects[i]);
+			}
+			return new GenericArrayData(array);
+		};
 	}
 
 	/**
-	 * Create a runtime JDBC field converter from given {@link LogicalType}.
+	 * Create a nullable JDBC f{@link JdbcSerializationConverter} from given sql type.
 	 */
-	public JdbcFieldConverter createConverter(LogicalType type) {
-		LogicalTypeRoot root = type.getTypeRoot();
-
-		if (root == LogicalTypeRoot.SMALLINT) {
-			// Converter for small type that casts value to int and then return short value, since
-	        // JDBC 1.0 use int type for small values.
-			return v -> ((Integer) v).shortValue();
-		} else {
-			return v -> v;
+	protected JdbcSerializationConverter createNullableExternalConverter(LogicalType type) {
+		return wrapIntoNullableExternalConverter(createExternalConverter(type), type);
+	}
+
+	protected JdbcSerializationConverter wrapIntoNullableExternalConverter(JdbcSerializationConverter jdbcSerializationConverter, LogicalType type) {
+		final int sqlType = JdbcTypeUtil.typeInformationToSqlType(TypeConversions.fromDataTypeToLegacyInfo(
+			TypeConversions.fromLogicalToDataType(type)));
+		return (val, index, statement)  -> {
+			if (val == null || val.isNullAt(index) || LogicalTypeRoot.NULL.equals(type.getTypeRoot())) {
+				statement.setNull(index + 1, sqlType);
+			} else {
+				jdbcSerializationConverter.serialize(val, index, statement);
+			}
+		};
+	}
+
+	protected JdbcSerializationConverter createExternalConverter(LogicalType type) {
+		switch (type.getTypeRoot()) {
+			case BOOLEAN:
+				return (val, index, statement) -> statement.setBoolean(index + 1, val.getBoolean(index));
+			case TINYINT:
+				return (val, index, statement) -> statement.setByte(index + 1, val.getByte(index));
+			case SMALLINT:
+				return (val, index, statement) -> statement.setShort(index + 1, val.getShort(index));
+			case INTEGER:
+			case INTERVAL_YEAR_MONTH:
+				return (val, index, statement) -> statement.setInt(index + 1, val.getInt(index));
+			case BIGINT:
+			case INTERVAL_DAY_TIME:
+				return (val, index, statement) -> statement.setLong(index + 1, val.getLong(index));
+			case FLOAT:
+				return (val, index, statement) -> statement.setFloat(index + 1, val.getFloat(index));
+			case DOUBLE:
+				return (val, index, statement) -> statement.setDouble(index + 1, val.getDouble(index));
+			case CHAR:
+			case VARCHAR:
+				// value is BinaryString
+				return (val, index, statement) -> statement.setString(index + 1, val.getString(index).toString());
+			case BINARY:
+			case VARBINARY:
+				return (val, index, statement) -> statement.setBytes(index + 1, val.getBinary(index));
+			case DATE:
+				return (val, index, statement) ->
+					statement.setDate(index + 1, Date.valueOf(LocalDate.ofEpochDay(val.getInt(index))));
+			case TIME_WITHOUT_TIME_ZONE:
+				return (val, index, statement) ->
+					statement.setTime(index + 1, Time.valueOf(LocalTime.ofSecondOfDay(val.getInt(index) / 1000L)));
+			case TIMESTAMP_WITH_TIME_ZONE:
+			case TIMESTAMP_WITHOUT_TIME_ZONE:
+				final int timestampPrecision = ((TimestampType) type).getPrecision();
+				return (val, index, statement) ->
+					statement.setTimestamp(index + 1, val
+						.getTimestamp(index, timestampPrecision)
+						.toTimestamp());
+			case DECIMAL:
+				final int decimalPrecision = ((DecimalType) type).getPrecision();
+				final int decimalScale = ((DecimalType) type).getScale();
+				return (val, index, statement) ->
+					statement.setBigDecimal(index + 1, val
+						.getDecimal(index, decimalPrecision, decimalScale)
+						.toBigDecimal());
+			case ARRAY:
+				//note: dialect need implements the conversion from ArrayData to JDBC Array if the dialect supports array.
+				return (val, index, statement) -> {
+					throw new IllegalStateException(
+						String.format("JDBC:%s do not support write ARRAY type.", converterName()));
+				};

Review comment:
       If the dialect doesn't support ARRAY, please throw the exception at compile stage, rather than at runtime.

##########
File path: flink-connectors/flink-connector-jdbc/src/main/java/org/apache/flink/connector/jdbc/internal/converter/AbstractJdbcRowConverter.java
##########
@@ -18,54 +18,255 @@
 
 package org.apache.flink.connector.jdbc.internal.converter;
 
+import org.apache.flink.connector.jdbc.utils.JdbcTypeUtil;
+import org.apache.flink.table.data.DecimalData;
+import org.apache.flink.table.data.GenericArrayData;
+import org.apache.flink.table.data.GenericRowData;
+import org.apache.flink.table.data.RowData;
+import org.apache.flink.table.data.StringData;
+import org.apache.flink.table.data.TimestampData;
+import org.apache.flink.table.types.logical.ArrayType;
+import org.apache.flink.table.types.logical.DecimalType;
 import org.apache.flink.table.types.logical.LogicalType;
 import org.apache.flink.table.types.logical.LogicalTypeRoot;
 import org.apache.flink.table.types.logical.RowType;
-import org.apache.flink.types.Row;
+import org.apache.flink.table.types.logical.TimestampType;
+import org.apache.flink.table.types.logical.utils.LogicalTypeUtils;
+import org.apache.flink.table.types.utils.TypeConversions;
 
+import java.io.Serializable;
+import java.lang.reflect.Array;
+import java.math.BigDecimal;
+import java.sql.Date;
+import java.sql.PreparedStatement;
 import java.sql.ResultSet;
 import java.sql.SQLException;
+import java.sql.Time;
+import java.sql.Timestamp;
+import java.time.LocalDate;
+import java.time.LocalTime;
 
 import static org.apache.flink.util.Preconditions.checkNotNull;
 
 /**
- * Base for all row converters.
+ * Base class for all converters that convert between JDBC object and Flink internal object.
  */
 public abstract class AbstractJdbcRowConverter implements JdbcRowConverter {
 
 	protected final RowType rowType;
-	protected final JdbcFieldConverter[] converters;
+	protected final JdbcDeserializationConverter[] toInternalConverters;
+	protected final JdbcSerializationConverter[] toExternalConverters;
+	protected final LogicalType[] fieldTypes;
+
+	public abstract String converterName();
 
 	public AbstractJdbcRowConverter(RowType rowType) {
 		this.rowType = checkNotNull(rowType);
-		converters = new JdbcFieldConverter[rowType.getFieldCount()];
-
-		for (int i = 0; i < converters.length; i++) {
-			converters[i] = createConverter(rowType.getTypeAt(i));
+		this.fieldTypes = rowType.getFields().stream()
+			.map(RowType.RowField::getType)
+			.toArray(LogicalType[]::new);
+		this.toInternalConverters = new JdbcDeserializationConverter[rowType.getFieldCount()];
+		this.toExternalConverters = new JdbcSerializationConverter[rowType.getFieldCount()];
+		for (int i = 0; i < rowType.getFieldCount(); i++) {
+			toInternalConverters[i] = createNullableInternalConverter(rowType.getTypeAt(i));
+			toExternalConverters[i] = createNullableExternalConverter(fieldTypes[i]);
 		}
 	}
 
 	@Override
-	public Row convert(ResultSet resultSet, Row reuse) throws SQLException {
+	public RowData toInternal(ResultSet resultSet) throws SQLException {
+		GenericRowData genericRowData = new GenericRowData(rowType.getFieldCount());
 		for (int pos = 0; pos < rowType.getFieldCount(); pos++) {
-			reuse.setField(pos, converters[pos].convert(resultSet.getObject(pos + 1)));
+			genericRowData.setField(pos, toInternalConverters[pos].deserialize(resultSet.getObject(pos + 1)));
+		}
+		return genericRowData;
+	}
+
+	@Override
+	public PreparedStatement toExternal(RowData rowData, PreparedStatement statement) throws SQLException {
+		for (int index = 0; index < rowData.getArity(); index++) {
+			toExternalConverters[index].serialize(rowData, index, statement);
+		}
+		return statement;
+	}
+
+	/**
+	 * Runtime converter to convert JDBC field to {@link RowData} type object.
+	 */
+	@FunctionalInterface
+	interface JdbcDeserializationConverter extends Serializable {
+		/**
+		 * convert a jdbc field to java object, the field could be a simple type or array type.
+		 * @param jdbcField
+		 */
+		Object deserialize(Object jdbcField) throws SQLException;
+	}
+
+	/**
+	 * Runtime converter to convert {@link RowData} field to java object and fill into the {@link PreparedStatement}.
+	 */
+	@FunctionalInterface
+	interface JdbcSerializationConverter extends Serializable {
+		void serialize(RowData rowData, int index, PreparedStatement statement) throws SQLException;
+	}
+
+	/**
+	 * Create a nullable runtime {@link JdbcDeserializationConverter} from given {@link LogicalType}.
+	 */
+	protected JdbcDeserializationConverter createNullableInternalConverter(LogicalType type) {
+		return wrapIntoNullableInternalConverter(createInternalConverter(type));
+	}
+
+	protected JdbcDeserializationConverter wrapIntoNullableInternalConverter(JdbcDeserializationConverter jdbcDeserializationConverter) {
+		return v -> {
+			if (v == null) {
+				return null;
+			} else {
+				return jdbcDeserializationConverter.deserialize(v);
+			}
+		};
+	}
+
+	protected JdbcDeserializationConverter createInternalConverter(LogicalType type) {
+		switch (type.getTypeRoot()) {
+			case NULL:
+				return v -> null;
+			case BOOLEAN:
+				return v -> (boolean) v;
+			case TINYINT:
+				return v -> (byte) v;
+			case SMALLINT:
+				// Converter for small type that casts value to int and then return short value, since
+				// JDBC 1.0 use int type for small values.
+				return v -> (Integer.valueOf(v.toString())).shortValue();
+			case INTEGER:
+			case INTERVAL_YEAR_MONTH:
+				return v -> (int) v;
+			case BIGINT:
+			case INTERVAL_DAY_TIME:
+				return v -> (long) v;
+			case DATE:
+				return v -> (int) (((Date) v).toLocalDate().toEpochDay());
+			case TIME_WITHOUT_TIME_ZONE:
+				return v -> ((Time) v).toLocalTime().toSecondOfDay() * 1000;
+			case TIMESTAMP_WITH_TIME_ZONE:
+			case TIMESTAMP_WITHOUT_TIME_ZONE:
+				return v -> TimestampData.fromTimestamp((Timestamp) v);
+			case FLOAT:
+				return v -> (float) v;
+			case DOUBLE:
+				return v -> (double) v;
+			case CHAR:
+			case VARCHAR:
+				return v -> StringData.fromString((String) v);
+			case BINARY:
+			case VARBINARY:
+				return v -> (byte[]) v;
+			case DECIMAL:
+				final int precision = ((DecimalType) type).getPrecision();
+				final int scale = ((DecimalType) type).getScale();
+				return v -> DecimalData.fromBigDecimal((BigDecimal) v, precision, scale);
+			case ARRAY:
+				final JdbcDeserializationConverter arrayConverter = createToInternalArrayConverter((ArrayType) type);
+				return v -> arrayConverter.deserialize(v);
+			case ROW:
+			case MAP:
+			case MULTISET:
+			case RAW:
+			default:
+				throw new UnsupportedOperationException("Not support to parse type: " + type);
 		}
+	}
 
-		return reuse;
+	protected JdbcDeserializationConverter createToInternalArrayConverter(ArrayType arrayType) {
+		final JdbcDeserializationConverter elementConverter = createNullableInternalConverter(arrayType.getElementType());
+		final Class<?> elementClass = LogicalTypeUtils.toInternalConversionClass(arrayType.getElementType());
+		return v -> {
+			final Object[] objects = (Object[]) v;
+			final Object[] array = (Object[]) Array.newInstance(elementClass, objects.length);
+			for (int i = 0; i < objects.length; i++) {
+				array[i] = elementConverter.deserialize(objects[i]);
+			}
+			return new GenericArrayData(array);
+		};
 	}
 
 	/**
-	 * Create a runtime JDBC field converter from given {@link LogicalType}.
+	 * Create a nullable JDBC f{@link JdbcSerializationConverter} from given sql type.
 	 */
-	public JdbcFieldConverter createConverter(LogicalType type) {
-		LogicalTypeRoot root = type.getTypeRoot();
-
-		if (root == LogicalTypeRoot.SMALLINT) {
-			// Converter for small type that casts value to int and then return short value, since
-	        // JDBC 1.0 use int type for small values.
-			return v -> ((Integer) v).shortValue();
-		} else {
-			return v -> v;
+	protected JdbcSerializationConverter createNullableExternalConverter(LogicalType type) {
+		return wrapIntoNullableExternalConverter(createExternalConverter(type), type);
+	}
+
+	protected JdbcSerializationConverter wrapIntoNullableExternalConverter(JdbcSerializationConverter jdbcSerializationConverter, LogicalType type) {
+		final int sqlType = JdbcTypeUtil.typeInformationToSqlType(TypeConversions.fromDataTypeToLegacyInfo(
+			TypeConversions.fromLogicalToDataType(type)));
+		return (val, index, statement)  -> {
+			if (val == null || val.isNullAt(index) || LogicalTypeRoot.NULL.equals(type.getTypeRoot())) {
+				statement.setNull(index + 1, sqlType);
+			} else {
+				jdbcSerializationConverter.serialize(val, index, statement);
+			}
+		};
+	}
+
+	protected JdbcSerializationConverter createExternalConverter(LogicalType type) {
+		switch (type.getTypeRoot()) {
+			case BOOLEAN:
+				return (val, index, statement) -> statement.setBoolean(index + 1, val.getBoolean(index));
+			case TINYINT:
+				return (val, index, statement) -> statement.setByte(index + 1, val.getByte(index));
+			case SMALLINT:
+				return (val, index, statement) -> statement.setShort(index + 1, val.getShort(index));
+			case INTEGER:
+			case INTERVAL_YEAR_MONTH:
+				return (val, index, statement) -> statement.setInt(index + 1, val.getInt(index));
+			case BIGINT:
+			case INTERVAL_DAY_TIME:
+				return (val, index, statement) -> statement.setLong(index + 1, val.getLong(index));
+			case FLOAT:
+				return (val, index, statement) -> statement.setFloat(index + 1, val.getFloat(index));
+			case DOUBLE:
+				return (val, index, statement) -> statement.setDouble(index + 1, val.getDouble(index));
+			case CHAR:
+			case VARCHAR:
+				// value is BinaryString
+				return (val, index, statement) -> statement.setString(index + 1, val.getString(index).toString());
+			case BINARY:
+			case VARBINARY:
+				return (val, index, statement) -> statement.setBytes(index + 1, val.getBinary(index));
+			case DATE:
+				return (val, index, statement) ->
+					statement.setDate(index + 1, Date.valueOf(LocalDate.ofEpochDay(val.getInt(index))));
+			case TIME_WITHOUT_TIME_ZONE:
+				return (val, index, statement) ->
+					statement.setTime(index + 1, Time.valueOf(LocalTime.ofSecondOfDay(val.getInt(index) / 1000L)));
+			case TIMESTAMP_WITH_TIME_ZONE:
+			case TIMESTAMP_WITHOUT_TIME_ZONE:
+				final int timestampPrecision = ((TimestampType) type).getPrecision();
+				return (val, index, statement) ->
+					statement.setTimestamp(index + 1, val
+						.getTimestamp(index, timestampPrecision)
+						.toTimestamp());
+			case DECIMAL:
+				final int decimalPrecision = ((DecimalType) type).getPrecision();
+				final int decimalScale = ((DecimalType) type).getScale();
+				return (val, index, statement) ->
+					statement.setBigDecimal(index + 1, val
+						.getDecimal(index, decimalPrecision, decimalScale)
+						.toBigDecimal());

Review comment:
       ditto.

##########
File path: flink-connectors/flink-connector-jdbc/src/main/java/org/apache/flink/connector/jdbc/internal/converter/PostgresRowConverter.java
##########
@@ -18,57 +18,81 @@
 
 package org.apache.flink.connector.jdbc.internal.converter;
 
+import org.apache.flink.table.data.GenericArrayData;
 import org.apache.flink.table.types.logical.ArrayType;
 import org.apache.flink.table.types.logical.LogicalType;
 import org.apache.flink.table.types.logical.LogicalTypeFamily;
 import org.apache.flink.table.types.logical.LogicalTypeRoot;
 import org.apache.flink.table.types.logical.RowType;
 import org.apache.flink.table.types.logical.utils.LogicalTypeChecks;
+import org.apache.flink.table.types.logical.utils.LogicalTypeUtils;
 
 import org.postgresql.jdbc.PgArray;
 import org.postgresql.util.PGobject;
 
+import java.lang.reflect.Array;
+
 /**
- * Row converter for Postgres.
+ * Runtime converter that responsible to convert between JDBC object and Flink internal object for Postgres.
  */
 public class PostgresRowConverter extends AbstractJdbcRowConverter {
 
+	private static final long serialVersionUID = 1L;
+
+	@Override
+	public String converterName() {
+		return "Postgres";

Review comment:
       Nit. `PostgreSQL`




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

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [flink] flinkbot edited a comment on pull request #12176: [FLINK-17029][jdbc]Introduce a new JDBC connector with new property keys

Posted by GitBox <gi...@apache.org>.
flinkbot edited a comment on pull request #12176:
URL: https://github.com/apache/flink/pull/12176#issuecomment-629283127


   <!--
   Meta data
   {
     "version" : 1,
     "metaDataEntries" : [ {
       "hash" : "499722944af576d13757da75f4ad0d08c4538782",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=1453",
       "triggerID" : "499722944af576d13757da75f4ad0d08c4538782",
       "triggerType" : "PUSH"
     }, {
       "hash" : "d07c3a602f13eaf3ea441e11ece1bf671bfe5beb",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=1547",
       "triggerID" : "d07c3a602f13eaf3ea441e11ece1bf671bfe5beb",
       "triggerType" : "PUSH"
     }, {
       "hash" : "226bafeb2086b9c98a29b398c3c34a325390aa2e",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=1558",
       "triggerID" : "226bafeb2086b9c98a29b398c3c34a325390aa2e",
       "triggerType" : "PUSH"
     }, {
       "hash" : "057d8fa644c9a203a753fe184de8e204ea81918e",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=1566",
       "triggerID" : "057d8fa644c9a203a753fe184de8e204ea81918e",
       "triggerType" : "PUSH"
     }, {
       "hash" : "e557b09c5dc9371c42542211651b79ea749cbf03",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=1584",
       "triggerID" : "e557b09c5dc9371c42542211651b79ea749cbf03",
       "triggerType" : "PUSH"
     }, {
       "hash" : "7f67ad7536d0041ec9fc5f646dc23972822125f8",
       "status" : "FAILURE",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=1607",
       "triggerID" : "7f67ad7536d0041ec9fc5f646dc23972822125f8",
       "triggerType" : "PUSH"
     } ]
   }-->
   ## CI report:
   
   * 7f67ad7536d0041ec9fc5f646dc23972822125f8 Azure: [FAILURE](https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=1607) 
   
   <details>
   <summary>Bot commands</summary>
     The @flinkbot bot supports the following commands:
   
    - `@flinkbot run travis` re-run the last Travis build
    - `@flinkbot run azure` re-run the last Azure build
   </details>


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

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [flink] flinkbot commented on pull request #12176: [FLINK-17029][jdbc]Introduce a new JDBC connector with new property keys

Posted by GitBox <gi...@apache.org>.
flinkbot commented on pull request #12176:
URL: https://github.com/apache/flink/pull/12176#issuecomment-629256514


   Thanks a lot for your contribution to the Apache Flink project. I'm the @flinkbot. I help the community
   to review your pull request. We will use this comment to track the progress of the review.
   
   
   ## Automated Checks
   Last check on commit 499722944af576d13757da75f4ad0d08c4538782 (Fri May 15 14:08:25 UTC 2020)
   
   **Warnings:**
    * No documentation files were touched! Remember to keep the Flink docs up to date!
   
   
   <sub>Mention the bot in a comment to re-run the automated checks.</sub>
   ## Review Progress
   
   * ❓ 1. The [description] looks good.
   * ❓ 2. There is [consensus] that the contribution should go into to Flink.
   * ❓ 3. Needs [attention] from.
   * ❓ 4. The change fits into the overall [architecture].
   * ❓ 5. Overall code [quality] is good.
   
   Please see the [Pull Request Review Guide](https://flink.apache.org/contributing/reviewing-prs.html) for a full explanation of the review process.<details>
    The Bot is tracking the review progress through labels. Labels are applied according to the order of the review items. For consensus, approval by a Flink committer of PMC member is required <summary>Bot commands</summary>
     The @flinkbot bot supports the following commands:
   
    - `@flinkbot approve description` to approve one or more aspects (aspects: `description`, `consensus`, `architecture` and `quality`)
    - `@flinkbot approve all` to approve all aspects
    - `@flinkbot approve-until architecture` to approve everything until `architecture`
    - `@flinkbot attention @username1 [@username2 ..]` to require somebody's attention
    - `@flinkbot disapprove architecture` to remove an approval you gave earlier
   </details>


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

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [flink] flinkbot edited a comment on pull request #12176: [FLINK-17029][jdbc]Introduce a new JDBC connector with new property keys

Posted by GitBox <gi...@apache.org>.
flinkbot edited a comment on pull request #12176:
URL: https://github.com/apache/flink/pull/12176#issuecomment-629283127


   <!--
   Meta data
   {
     "version" : 1,
     "metaDataEntries" : [ {
       "hash" : "499722944af576d13757da75f4ad0d08c4538782",
       "status" : "FAILURE",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=1453",
       "triggerID" : "499722944af576d13757da75f4ad0d08c4538782",
       "triggerType" : "PUSH"
     }, {
       "hash" : "d07c3a602f13eaf3ea441e11ece1bf671bfe5beb",
       "status" : "PENDING",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=1547",
       "triggerID" : "d07c3a602f13eaf3ea441e11ece1bf671bfe5beb",
       "triggerType" : "PUSH"
     }, {
       "hash" : "226bafeb2086b9c98a29b398c3c34a325390aa2e",
       "status" : "PENDING",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=1558",
       "triggerID" : "226bafeb2086b9c98a29b398c3c34a325390aa2e",
       "triggerType" : "PUSH"
     }, {
       "hash" : "057d8fa644c9a203a753fe184de8e204ea81918e",
       "status" : "UNKNOWN",
       "url" : "TBD",
       "triggerID" : "057d8fa644c9a203a753fe184de8e204ea81918e",
       "triggerType" : "PUSH"
     } ]
   }-->
   ## CI report:
   
   * 499722944af576d13757da75f4ad0d08c4538782 Azure: [FAILURE](https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=1453) 
   * d07c3a602f13eaf3ea441e11ece1bf671bfe5beb Azure: [PENDING](https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=1547) 
   * 226bafeb2086b9c98a29b398c3c34a325390aa2e Azure: [PENDING](https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=1558) 
   * 057d8fa644c9a203a753fe184de8e204ea81918e UNKNOWN
   
   <details>
   <summary>Bot commands</summary>
     The @flinkbot bot supports the following commands:
   
    - `@flinkbot run travis` re-run the last Travis build
    - `@flinkbot run azure` re-run the last Azure build
   </details>


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

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [flink] flinkbot edited a comment on pull request #12176: [FLINK-17029][jdbc]Introduce a new JDBC connector with new property keys

Posted by GitBox <gi...@apache.org>.
flinkbot edited a comment on pull request #12176:
URL: https://github.com/apache/flink/pull/12176#issuecomment-629283127


   <!--
   Meta data
   {
     "version" : 1,
     "metaDataEntries" : [ {
       "hash" : "499722944af576d13757da75f4ad0d08c4538782",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=1453",
       "triggerID" : "499722944af576d13757da75f4ad0d08c4538782",
       "triggerType" : "PUSH"
     }, {
       "hash" : "d07c3a602f13eaf3ea441e11ece1bf671bfe5beb",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=1547",
       "triggerID" : "d07c3a602f13eaf3ea441e11ece1bf671bfe5beb",
       "triggerType" : "PUSH"
     }, {
       "hash" : "226bafeb2086b9c98a29b398c3c34a325390aa2e",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=1558",
       "triggerID" : "226bafeb2086b9c98a29b398c3c34a325390aa2e",
       "triggerType" : "PUSH"
     }, {
       "hash" : "057d8fa644c9a203a753fe184de8e204ea81918e",
       "status" : "FAILURE",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=1566",
       "triggerID" : "057d8fa644c9a203a753fe184de8e204ea81918e",
       "triggerType" : "PUSH"
     }, {
       "hash" : "e557b09c5dc9371c42542211651b79ea749cbf03",
       "status" : "PENDING",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=1584",
       "triggerID" : "e557b09c5dc9371c42542211651b79ea749cbf03",
       "triggerType" : "PUSH"
     } ]
   }-->
   ## CI report:
   
   * 057d8fa644c9a203a753fe184de8e204ea81918e Azure: [FAILURE](https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=1566) 
   * e557b09c5dc9371c42542211651b79ea749cbf03 Azure: [PENDING](https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=1584) 
   
   <details>
   <summary>Bot commands</summary>
     The @flinkbot bot supports the following commands:
   
    - `@flinkbot run travis` re-run the last Travis build
    - `@flinkbot run azure` re-run the last Azure build
   </details>


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

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [flink] flinkbot edited a comment on pull request #12176: [FLINK-17029][jdbc]Introduce a new JDBC connector with new property keys

Posted by GitBox <gi...@apache.org>.
flinkbot edited a comment on pull request #12176:
URL: https://github.com/apache/flink/pull/12176#issuecomment-629283127


   <!--
   Meta data
   {
     "version" : 1,
     "metaDataEntries" : [ {
       "hash" : "499722944af576d13757da75f4ad0d08c4538782",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=1453",
       "triggerID" : "499722944af576d13757da75f4ad0d08c4538782",
       "triggerType" : "PUSH"
     }, {
       "hash" : "d07c3a602f13eaf3ea441e11ece1bf671bfe5beb",
       "status" : "PENDING",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=1547",
       "triggerID" : "d07c3a602f13eaf3ea441e11ece1bf671bfe5beb",
       "triggerType" : "PUSH"
     }, {
       "hash" : "226bafeb2086b9c98a29b398c3c34a325390aa2e",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=1558",
       "triggerID" : "226bafeb2086b9c98a29b398c3c34a325390aa2e",
       "triggerType" : "PUSH"
     }, {
       "hash" : "057d8fa644c9a203a753fe184de8e204ea81918e",
       "status" : "FAILURE",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=1566",
       "triggerID" : "057d8fa644c9a203a753fe184de8e204ea81918e",
       "triggerType" : "PUSH"
     } ]
   }-->
   ## CI report:
   
   * d07c3a602f13eaf3ea441e11ece1bf671bfe5beb Azure: [PENDING](https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=1547) 
   * 057d8fa644c9a203a753fe184de8e204ea81918e Azure: [FAILURE](https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=1566) 
   
   <details>
   <summary>Bot commands</summary>
     The @flinkbot bot supports the following commands:
   
    - `@flinkbot run travis` re-run the last Travis build
    - `@flinkbot run azure` re-run the last Azure build
   </details>


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

For queries about this service, please contact Infrastructure at:
users@infra.apache.org