You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@rocketmq.apache.org by zh...@apache.org on 2022/05/16 05:34:12 UTC

[rocketmq-connect] branch master updated: Upgrade rocketmq connect jdbc (#123)

This is an automated email from the ASF dual-hosted git repository.

zhoubo pushed a commit to branch master
in repository https://gitbox.apache.org/repos/asf/rocketmq-connect.git


The following commit(s) were added to refs/heads/master by this push:
     new e1f725c  Upgrade rocketmq connect jdbc (#123)
e1f725c is described below

commit e1f725c07cc43f345dc0efe88506aa990b9886ba
Author: xiaoyi <su...@163.com>
AuthorDate: Mon May 16 13:34:07 2022 +0800

    Upgrade rocketmq connect jdbc (#123)
    
    * Upgrade rocketmq connect JDBC and be compatible with openmldb JDBC write mode
    
    * add OpenMLDBDatabaseDialect
    
    * Upgrade rocketmq connect API to increase scalability; Add support for openmldb #114
    
    * Update README.md
    
    * delete author information
    
    * remove deprecated method
    
    * reference package expand
    
    Co-authored-by: “sunxiaojian” <“sunxiaojian926@163.com”>
---
 connectors/rocketmq-connect-jdbc/README.md         |  113 +-
 connectors/rocketmq-connect-jdbc/pom.xml           |   80 +-
 .../rocketmq/connect/jdbc/common/CloneUtils.java   |   45 -
 .../rocketmq/connect/jdbc/common/DBUtils.java      |  212 ---
 .../TimeColumnParser.java => common/Dialect.java}  |   41 +-
 .../DialectType.java}                              |   12 +-
 .../JdbcSourceConfigConstants.java}                |   39 +-
 .../common/{ConstDefine.java => OperatorType.java} |   11 +-
 .../apache/rocketmq/connect/jdbc/common/Utils.java |   74 -
 .../connect/jdbc/config/AbstractConfig.java        |  133 ++
 .../rocketmq/connect/jdbc/config/Config.java       |  357 ----
 .../rocketmq/connect/jdbc/config/ConfigDef.java    |  573 ++++++
 .../rocketmq/connect/jdbc/config/ConfigUtil.java   |   69 -
 .../connect/jdbc/config/DbConnectorConfig.java     |  101 --
 .../connect/jdbc/config/SinkDbConnectorConfig.java |  115 --
 .../jdbc/config/SourceDbConnectorConfig.java       |   90 -
 .../connect/jdbc/config/TaskDivideConfig.java      |  112 --
 .../connect/jdbc/connector/JdbcSinkConfig.java     |  302 ++++
 .../connect/jdbc/connector/JdbcSinkConnector.java  |  237 +--
 .../connect/jdbc/connector/JdbcSinkTask.java       |  181 +-
 .../connect/jdbc/connector/JdbcSourceConfig.java   |  335 ++++
 .../jdbc/connector/JdbcSourceConnector.java        |  108 +-
 .../connect/jdbc/connector/JdbcSourceTask.java     |  384 ++--
 .../JdbcSourceTaskConfig.java}                     |   25 +-
 .../connect/jdbc/dialect/DatabaseDialect.java      |  319 ++++
 .../jdbc/dialect/DatabaseDialectFactory.java       |  192 ++
 .../rocketmq/connect/jdbc/dialect/DropOptions.java |  100 ++
 .../jdbc/dialect/PreparedStatementBinder.java      |  130 ++
 .../jdbc/dialect/impl/GenericDatabaseDialect.java  | 1888 ++++++++++++++++++++
 .../jdbc/dialect/impl/MySqlDatabaseDialect.java    |  149 ++
 .../jdbc/dialect/impl/OpenMLDBDatabaseDialect.java |   68 +
 .../dialect/provider/CachedConnectionProvider.java |  128 ++
 .../jdbc/dialect/provider/ConnectionProvider.java  |   65 +
 .../provider/DatabaseDialectProvider.java}         |   54 +-
 .../provider/JdbcUrlInfo.java}                     |   49 +-
 .../ConfigException.java}                          |   26 +-
 .../TableAlterOrCreateException.java}              |   29 +-
 .../rocketmq/connect/jdbc/schema/Database.java     |  109 --
 .../rocketmq/connect/jdbc/schema/Schema.java       |  123 --
 .../apache/rocketmq/connect/jdbc/schema/Table.java |  101 --
 .../jdbc/schema/column/ColumnDefAdjuster.java      |   72 +
 .../jdbc/schema/column/ColumnDefinition.java       |  339 ++++
 .../connect/jdbc/schema/column/ColumnId.java       |  105 ++
 .../connect/jdbc/schema/column/ColumnParser.java   |  104 --
 .../jdbc/schema/column/DateTimeColumnParser.java   |   53 -
 .../jdbc/schema/column/IntColumnParser.java        |   66 -
 .../jdbc/schema/column/SetColumnParser.java        |   54 -
 .../jdbc/schema/column/StringColumnParser.java     |   57 -
 .../schema/column/parser/DateColumnParser.java     |   61 +
 .../schema/column/parser/TimeColumnParser.java     |   63 +
 .../column/parser/TimestampColumnParser.java       |   54 +
 .../connect/jdbc/schema/db/DbStructure.java        |  310 ++++
 .../connect/jdbc/schema/table/TableDefinition.java |  123 ++
 .../jdbc/schema/table/TableDefinitions.java        |   93 +
 .../connect/jdbc/schema/table/TableId.java         |  144 ++
 .../connect/jdbc/sink/BufferedRecords.java         |  330 ++++
 .../apache/rocketmq/connect/jdbc/sink/Updater.java |  314 +---
 .../connect/jdbc/sink/metadata/FieldsMetadata.java |  239 +++
 .../connect/jdbc/sink/metadata/SchemaPair.java     |   62 +
 .../jdbc/sink/metadata/SinkRecordField.java        |   94 +
 .../rocketmq/connect/jdbc/source/Querier.java      |  190 --
 .../jdbc/source/TimestampIncrementingCriteria.java |  346 ++++
 .../jdbc/source/TimestampIncrementingQuerier.java  |  328 ----
 .../jdbc/source/metadata/ColumnMapping.java        |  100 ++
 .../jdbc/source/metadata/SchemaMapping.java        |  158 ++
 .../jdbc/source/offset/SourceOffsetCompute.java    |  333 ++++
 .../source/offset/TimestampIncrementingOffset.java |  112 ++
 .../connect/jdbc/source/querier/BulkQuerier.java   |  139 ++
 .../connect/jdbc/source/querier/Querier.java       |  164 ++
 .../querier/TimestampIncrementingQuerier.java      |  246 +++
 .../connect/jdbc/strategy/DivideTaskByQueue.java   |   72 -
 .../connect/jdbc/strategy/DivideTaskByTopic.java   |  113 --
 .../{config/DataType.java => util/BytesUtil.java}  |   19 +-
 .../connect/jdbc/util/ConnectorGroupUtils.java     |   53 +
 .../rocketmq/connect/jdbc/util/DateTimeUtils.java  |   71 +
 .../connect/jdbc/util/ExpressionBuilder.java       |  598 +++++++
 .../connect/jdbc/util/IdentifierRules.java         |  186 ++
 .../rocketmq/connect/jdbc/util/JdbcDriverInfo.java |  136 ++
 .../rocketmq/connect/jdbc/util/NumericMapping.java |   53 +
 .../QuoteMethod.java}                              |   38 +-
 .../rocketmq/connect/jdbc/util/TableType.java      |   86 +
 ...t.jdbc.dialect.provider.DatabaseDialectProvider |    3 +
 .../jdbc/connector/JdbcSourceConnectorTest.java    |  116 +-
 .../dialect/DatabaseDialectFactoryTest.java        |   15 +
 .../jdbc/connector/source/JdbcSourceTaskTest.java  |   26 +
 85 files changed, 10172 insertions(+), 3545 deletions(-)

diff --git a/connectors/rocketmq-connect-jdbc/README.md b/connectors/rocketmq-connect-jdbc/README.md
index 54c04ea..fb8b8f5 100644
--- a/connectors/rocketmq-connect-jdbc/README.md
+++ b/connectors/rocketmq-connect-jdbc/README.md
@@ -1,40 +1,48 @@
 # rocketmq-connect-jdbc
-
+```  
+注: 目前支持的数据库类型为 mysql ,openMLDB ,其它数据库的jdbc模式持续扩展中
+```  
 ## rocketmq-connect-jdbc 打包
 ```
-mvn clean install -Dmaven.test.skip=true
+mvn clean package -Dmaven.test.skip=true
 ```
 
 ## rocketmq-connect-jdbc 启动
 
 * **jdbc-source-connector** 启动
-
 ```
-http://${runtime-ip}:${runtime-port}/connectors/${rocketmq-jdbc-source-connector-name}
-?config={"source-rocketmq":"${runtime-ip}:${runtime-port}","source-cluster":"${broker-cluster}","connector-class":"org.apache.rocketmq.connect.jdbc.connector.JdbcSourceConnector",“dbUrl”:"${source-db-ip}",dbPort”:"${source-db-port}",dbUsername”:"${source-db-username}",dbPassword”:"${source-db-password}","rocketmqTopic":"${source-table-name}","mode":"bulk","whiteDataBase":{"${source-db-name}":{"${source-table-name}":{"${source-column-name}":"${source-column-value}"}}},"source-record-conve [...]
+POST  http://${runtime-ip}:${runtime-port}/connectors/${rocketmq-jdbc-source-connector-name}
+{
+    "connector-class":"org.apache.rocketmq.connect.jdbc.connector.JdbcSourceConnector",
+    "max-task":"2",
+    "connection.url":"jdbc:mysql://XXXXXXXXX:3306",
+    "connection.user":"*****",
+    "connection.password":"*****",
+    "table.whitelist":"db.table",
+    "mode": "incrementing",
+    "incrementing.column.name":"id",
+    "timestamp.initial": -1,
+    "source-record-converter":"org.apache.rocketmq.connect.runtime.converter.JsonConverter"
+}
 ```
 
-例子
-
-```
-http://localhost:8081/connectors/jdbcConnectorSource?config={"source-rocketmq":"localhost:9876","source-cluster":"DefaultCluster",
-"connector-class":"org.apache.rocketmq.connect.jdbc.connector.JdbcSourceConnector","dbUrl":"192.168.1.3","dbPort":"3306","dbUsername":"root","dbPassword":"mysqldb123456",
-"rocketmqTopic":"test_table","mode":"bulk","whiteDataBase":{"test_database":{"test_table":{"test_column":"8"}}},
-"source-record-converter":"org.apache.rocketmq.connect.runtime.converter.JsonConverter"}
-```
 
 * **jdbc-sink-connector** 启动
 
 ```
-http://${runtime-ip}:${runtime-port}/connectors/${rocketmq-jdbc-sink-connector-name}
-?config={"source-rocketmq":"${runtime-ip}:${runtime-port}","source-cluster":"${broker-cluster}","connector-class":"org.apache.rocketmq.connect.jdbc.connector.JdbcSinkConnector",“dbUrl”:"${sink-db-ip}",dbPort”:"${sink-db-port}",dbUsername”:"${sink-db-username}",dbPassword”:"${sink-db-password}","mode":"bulk","topicNames":"${source-table-name}","source-record-converter":"org.apache.rocketmq.connect.runtime.converter.JsonConverter"}
-```
-
-例子 
-```
-http://localhost:8081/connectors/jdbcConnectorSink?config={"source-rocketmq":"localhost:9876","source-cluster":"DefaultCluster",
-"connector-class":"org.apache.rocketmq.connect.jdbc.connector.JdbcSinkConnector","dbUrl":"192.168.1.2","dbPort":"3306","dbUsername":"root",
-"dbPassword":"mysqldb123456","topicNames":"test_table","mode":"bulk","source-record-converter":"org.apache.rocketmq.connect.runtime.converter.JsonConverter"}
+POST  http://${runtime-ip}:${runtime-port}/connectors/${rocketmq-jdbc-sink-connector-name}
+{
+    "connector-class":"org.apache.rocketmq.connect.jdbc.connector.JdbcSinkConnector",
+    "max-task":"2",
+    "connect-topicname":"connect-topicname-jdbc-02",
+    "connection.url":"jdbc:mysql://*****:3306/{dbname}",
+    "connection.user":"******",
+    "connection.password":"******",
+    "pk.fields":"id",
+    "pk.mode":"record_value",
+    "insert.mode":"UPSERT",
+    "source-record-converter":"org.apache.rocketmq.connect.runtime.converter.JsonConverter"
+}
 ```
 
 >**注:** `rocketmq-jdbc-connect` 的启动依赖于`rocketmq-connect-runtime`项目的启动,需将打好的所有`jar`包放置到`runtime`项目中`pluginPaths`配置的路径后再执行上面的启动请求,该值配置在`runtime`项目下的`connect.conf`文件中
@@ -48,38 +56,41 @@ http://${runtime-ip}:${runtime-port}/connectors/${rocketmq-jdbc-connector-name}/
 ## rocketmq-connect-jdbc 参数说明
 * **jdbc-source-connector 参数说明**
 
-|         KEY            |  TYPE   | Must be filled | Description| Example
-|------------------------|---------|----------------|------------|---|
-|dbUrl                   | String  | YES            | source端 DB ip | 192.168.1.3|
-|dbPort                  | String  | YES            | source端 DB port | 3306 |
-|dbUsername              | String  | YES            | source端 DB 用户名 | root |
-|dbPassword              | String  | YES            | source端 DB 密码 | 123456 |
-|whiteDataBase           | String  | YES            | source端同步数据白名单,嵌套配置,为{DB名:{表名:{字段名:字段值}}},若无指定字段数据同步,字段名可设为NO-FILTER,值为任意 | {"DATABASE_TEST":{"TEST_DATA":{"name":"test"}}} |
-|mode                    | String  | YES            | source-connector 模式,目前仅支持bulk | bulk |
-|rocketmqTopic           | String  | NO             | source端同步数据的topic名字,必须和要同步的数据库表名一样 | TEST_DATA |
-|task-divide-strategy    | Integer | NO             | task 分配策略, 默认值为 0,表示按照topic分配任务,每一个table便是一个topic | 0 |
-|task-parallelism        | Integer | NO             | task parallelism,默认值为 1,表示将topic拆分为多少个任务进行执行 | 2 |
-|source-rocketmq         | String  | YES            | source 端获取路由信息连接到的RocketMQ nameserver 地址 | 192.168.1.3:9876 |
-|source-cluster          | String  | YES            | source 端获取路由信息连接到的RocketMQ broker cluster | DefaultCluster |
-|source-record-converter | String  | YES            | source data 解析 | org.apache.rocketmq.connect.runtime.converter.JsonConverter |
 
+|         KEY                 |  TYPE   | Must be filled | Description| Example
+|------------------------|----|---------|---------------|------------------|
+|connection.url               | String  | YES           | source端 jdbc连接 | jdbc:mysql://XXXXXXXXX:3306|
+|connection.user              | String  | YES           | source端 DB 用户名 | root |
+|connection.password          | String  | YES           | source端 DB 密码   | root |
+|connection.attempts          | String  | YES           | source端 DB连接重试次数 | 3 |
+|connection.backoff.ms        | Long    | YES           |  |
+|poll.interval.ms             | Long    | YES           |拉取间隔时间  | 3000ms |
+|batch.max.rows               | Integer | NO            |每次拉取数量 | 300 |
+|mode                         | Integer | NO            |拉取模式 | bulk、timestamp、incrementing、timestamp+incrementing |
+|incrementing.column.name     | Integer | NO            |增量字段,常用ID  | id |
+|timestamp.column.name        | String  | YES           |时间增量字段 | modified_time |
+|table.whitelist              | String  | YES           |需要扫描的表 | db.table,db.table01 |
+|max-task                     | Integer | YES           |任务数量,最大不能大于表的数量 | 2 |
+|source-record-converter      | Integer | YES           |data转换器  | org.apache.rocketmq.connect.runtime.converter.JsonConverter |
 ```  
-注:1. source/sink配置文件说明是以rocketmq-connect-jdcb为demo,不同source/sink connector配置有差异,请以具体sourc/sink connector为准
-    2. rocketmqTopic 在jdbc-source-connector中没有被用到,暂时保留的原因是为了配置显示一致性
+注:1.source拉取的数据写入到以表名自动创建的topic中,如果需要写入特定的topic中则需要指定"connect-topicname" 参数
+   2.topic.prefix参数可以为自动创建的topic增加前缀,用来进行逻辑的隔离
 ```  
 * **jdbc-sink-connector 参数说明**
 
-|         KEY            |  TYPE   | Must be filled | Description| Example
-|------------------------|---------|----------------|------------|---|
-|dbUrl                   | String  | YES            | sink端 DB ip | 192.168.1.2|
-|dbPort                  | String  | YES            | sink端 DB port | 3306 |
-|dbUsername              | String  | YES            | sink端 DB 用户名 | root |
-|dbPassword              | String  | YES            | sink端 DB 密码 | 123456 |
-|mode                    | String  | YES            | source-connector 模式,目前仅支持bulk | bulk |
-|topicNames              | String  | YES            | sink端同步数据的topic名字,必须和要同步的数据库表名一样 | TEST_DATA |
-|task-divide-strategy    | Integer | NO             | sink端 分配策略, 默认值为 0,表示按照topic分配任务,每一个table便是一个topic | 0 |
-|task-parallelism        | Integer | NO             | sink端 parallelism,默认值为 1,表示将topic拆分为多少个任务进行执行 | 2 |
-|source-rocketmq         | String  | YES            | sink端 端获取路由信息连接到的RocketMQ nameserver 地址 | 192.168.1.3:9876 |
-|source-cluster          | String  | YES            | sink端 端获取路由信息连接到的RocketMQ broker cluster | DefaultCluster |
-|source-record-converter | String  | YES            | sink端 data 解析 | org.apache.rocketmq.connect.runtime.converter.JsonConverter |
 
+|         KEY                 |  TYPE   | Must be filled | Description| Example
+|------------------------|----|---------|---------------|------------------|
+|connection.url               | String  | YES           | sink端 jdbc连接          | jdbc:mysql://XXXXXXXXX:3306|
+|connection.user              | String  | YES           | sink端 DB 用户名 | root |
+|connection.password          | String  | YES           | sink端 DB 密码   | root |
+|connection.attempts          | String  | NO           | sink端 DB连接重试次数 | 3 |
+|connection.backoff.ms        | Long    | NO           |  |
+|connect-topicname            | Long    | YES          |监听的topic  | topic-name |
+|pk.fields                     | String  | NO           |写入侧主键配置,用于更新使用 | id |
+|pk.mode                      | String  | NO           |获取主键的模式 | none、record_value |
+|insert.mode                  | Integer | YES           |写入模式 | UPDATE、UPSERT、INSERT |
+|max-task                     | Integer | NO           |任务数量 | 2 |
+|source-record-converter      | Integer | YES          |data转换器  | org.apache.rocketmq.connect.runtime.converter.JsonConverter |
+
+```  
diff --git a/connectors/rocketmq-connect-jdbc/pom.xml b/connectors/rocketmq-connect-jdbc/pom.xml
index da09af9..651f5a7 100644
--- a/connectors/rocketmq-connect-jdbc/pom.xml
+++ b/connectors/rocketmq-connect-jdbc/pom.xml
@@ -43,7 +43,15 @@
         <rocketmq.version>4.5.2</rocketmq.version>
     </properties>
 
+
+
     <build>
+        <resources>
+            <resource>
+                <directory>src/main/resources</directory>
+                <filtering>true</filtering>
+            </resource>
+        </resources>
         <plugins>
             <plugin>
                 <groupId>org.codehaus.mojo</groupId>
@@ -55,14 +63,14 @@
                 <artifactId>clirr-maven-plugin</artifactId>
                 <version>2.7</version>
             </plugin>
-			<plugin> 
-				<artifactId>maven-dependency-plugin</artifactId> 
-				<configuration> 
-				<outputDirectory>${project.build.directory}/lib</outputDirectory> 
-				<excludeTransitive>false</excludeTransitive> 
-				<stripVersion>true</stripVersion> 
-				</configuration> 
-			</plugin> 
+            <plugin>
+                <artifactId>maven-dependency-plugin</artifactId>
+                <configuration>
+                    <outputDirectory>${project.build.directory}/lib</outputDirectory>
+                    <excludeTransitive>false</excludeTransitive>
+                    <stripVersion>true</stripVersion>
+                </configuration>
+            </plugin>
             <plugin>
                 <artifactId>maven-compiler-plugin</artifactId>
                 <version>3.6.1</version>
@@ -136,38 +144,23 @@
                 <artifactId>findbugs-maven-plugin</artifactId>
                 <version>3.0.4</version>
             </plugin>
-			 <plugin>
-            <artifactId>maven-assembly-plugin</artifactId>
-            <version>3.0.0</version>
-            <configuration>
-                <archive>
-                    <manifest>
-                        <mainClass>org.apache.rocketmq.connect.jdbc.jdbcSourceConnector</mainClass>
-                    </manifest>
-                </archive>
-                <descriptorRefs>
-                    <descriptorRef>jar-with-dependencies</descriptorRef>
-                </descriptorRefs>
-            </configuration>
-            <executions>
-                <execution>
-                    <id>make-assembly</id>
-                    <phase>package</phase>
-                    <goals>
-                        <goal>single</goal>
-                    </goals>
-                </execution>
-            </executions>
-			</plugin>
             <plugin>
-                <groupId>org.apache.rat</groupId>
-                <artifactId>apache-rat-plugin</artifactId>
-                <version>0.12</version>
+                <artifactId>maven-assembly-plugin</artifactId>
+                <version>3.0.0</version>
                 <configuration>
-                    <excludes>
-                        <exclude>README.md</exclude>
-                    </excludes>
+                    <descriptorRefs>
+                        <descriptorRef>jar-with-dependencies</descriptorRef>
+                    </descriptorRefs>
                 </configuration>
+                <executions>
+                    <execution>
+                        <id>make-assembly</id>
+                        <phase>package</phase>
+                        <goals>
+                            <goal>single</goal>
+                        </goals>
+                    </execution>
+                </executions>
             </plugin>
         </plugins>
     </build>
@@ -199,7 +192,7 @@
         <dependency>
             <groupId>io.openmessaging</groupId>
             <artifactId>openmessaging-connector</artifactId>
-            <version>0.1.1</version>
+            <version>0.1.2</version>
             <scope>provided</scope>
         </dependency>
 		<dependency>
@@ -276,6 +269,15 @@
             <version>1.1.22</version>
         </dependency>
 
+        <dependency>
+            <groupId>com.4paradigm.openmldb</groupId>
+            <artifactId>openmldb-jdbc</artifactId>
+            <version>0.4.4-hotfix1</version>
+        </dependency>
+        <dependency>
+            <groupId>com.4paradigm.openmldb</groupId>
+            <artifactId>openmldb-native</artifactId>
+            <version>0.4.4-hotfix1</version>
+        </dependency>
     </dependencies>
-
 </project>
diff --git a/connectors/rocketmq-connect-jdbc/src/main/java/org/apache/rocketmq/connect/jdbc/common/CloneUtils.java b/connectors/rocketmq-connect-jdbc/src/main/java/org/apache/rocketmq/connect/jdbc/common/CloneUtils.java
deleted file mode 100644
index b9f2bc7..0000000
--- a/connectors/rocketmq-connect-jdbc/src/main/java/org/apache/rocketmq/connect/jdbc/common/CloneUtils.java
+++ /dev/null
@@ -1,45 +0,0 @@
-/*
- * 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.rocketmq.connect.jdbc.common;
-
-import java.io.ByteArrayInputStream;
-import java.io.ByteArrayOutputStream;
-import java.io.ObjectInputStream;
-import java.io.ObjectOutputStream;
-import java.io.Serializable;
-
-public class CloneUtils {
-    @SuppressWarnings("unchecked")
-    public static <T extends Serializable> T clone(T obj) {
-        T clonedObj = null;
-        try {
-            ByteArrayOutputStream baos = new ByteArrayOutputStream();
-            ObjectOutputStream oos = new ObjectOutputStream(baos);
-            oos.writeObject(obj);
-            oos.close();
-
-            ByteArrayInputStream bais = new ByteArrayInputStream(baos.toByteArray());
-            ObjectInputStream ois = new ObjectInputStream(bais);
-            clonedObj = (T) ois.readObject();
-            ois.close();
-        } catch (Exception e) {
-            e.printStackTrace();
-        }
-        return clonedObj;
-    }
-}
diff --git a/connectors/rocketmq-connect-jdbc/src/main/java/org/apache/rocketmq/connect/jdbc/common/DBUtils.java b/connectors/rocketmq-connect-jdbc/src/main/java/org/apache/rocketmq/connect/jdbc/common/DBUtils.java
deleted file mode 100644
index 963fbf6..0000000
--- a/connectors/rocketmq-connect-jdbc/src/main/java/org/apache/rocketmq/connect/jdbc/common/DBUtils.java
+++ /dev/null
@@ -1,212 +0,0 @@
-
-/**
- * Copyright 2015 Confluent Inc.
- * <p>
- * Licensed under the Apache License, Version 2.0 (the "License");
- * you may not use this file except in compliance with the License.
- * You may obtain a copy of the License at
- * <p>
- * http://www.apache.org/licenses/LICENSE-2.0
- * <p>
- * 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.rocketmq.connect.jdbc.common;
-
-import com.alibaba.druid.pool.DruidDataSource;
-import com.alibaba.druid.pool.DruidDataSourceFactory;
-import org.apache.rocketmq.connect.jdbc.config.Config;
-import org.apache.rocketmq.connect.jdbc.connector.JdbcSourceTask;
-import org.slf4j.Logger;
-import org.slf4j.LoggerFactory;
-
-import javax.sql.DataSource;
-import java.sql.*;
-import java.text.SimpleDateFormat;
-import java.util.*;
-import java.util.Date;
-
-/**
- * Utilties for interacting with a JDBC database.
- */
-public class DBUtils {
-
-    private static final Logger log = LoggerFactory.getLogger(JdbcSourceTask.class);
-
-    /**
-     * The default table types to include when listing tables if none are specified. Valid values
-     * are those specified by the @{java.sql.DatabaseMetaData#getTables} method's TABLE_TYPE column.
-     * The default only includes standard, user-defined tables.
-     */
-    public static final Set<String> DEFAULT_TABLE_TYPES = Collections.unmodifiableSet(
-            new HashSet<String>(Arrays.asList("TABLE"))
-    );
-
-    private static final int GET_TABLES_TYPE_COLUMN = 4;
-    private static final int GET_TABLES_NAME_COLUMN = 3;
-
-    private static final int GET_COLUMNS_COLUMN_NAME = 4;
-    private static final int GET_COLUMNS_IS_NULLABLE = 18;
-    private static final int GET_COLUMNS_IS_AUTOINCREMENT = 23;
-
-
-    private static ThreadLocal<SimpleDateFormat> DATE_FORMATTER = new ThreadLocal<SimpleDateFormat>() {
-        @Override
-        protected SimpleDateFormat initialValue() {
-            SimpleDateFormat sdf = new SimpleDateFormat("yyyy-MM-dd HH:mm:ss.SSS");
-            sdf.setTimeZone(TimeZone.getTimeZone("UTC"));
-            return sdf;
-        }
-    };
-
-    /**
-     * Get a list of tables in the database. This uses the default filters, which only include
-     * user-defined tables.
-     * @param conn database connection
-     * @return a list of tables
-     * @throws SQLException
-     */
-    public static List<String> getTables(Connection conn) throws SQLException {
-        return getTables(conn, DEFAULT_TABLE_TYPES);
-    }
-
-    /**
-     * Get a list of table names in the database.
-     * @param conn database connection
-     * @param types a set of table types that should be included in the results
-     * @throws SQLException
-     */
-    public static List<String> getTables(Connection conn, Set<String> types) throws SQLException {
-        DatabaseMetaData metadata = conn.getMetaData();
-        ResultSet rs = metadata.getTables(null, null, "%", null);
-        List<String> tableNames = new ArrayList<String>();
-        while (rs.next()) {
-            if (types.contains(rs.getString(GET_TABLES_TYPE_COLUMN))) {
-                String colName = rs.getString(GET_TABLES_NAME_COLUMN);
-                // SQLite JDBC driver does not correctly mark these as system tables
-                if (metadata.getDatabaseProductName().equals("SQLite") && colName.startsWith("sqlite_")) {
-                    continue;
-                }
-
-                tableNames.add(colName);
-            }
-        }
-        return tableNames;
-    }
-
-    /**
-     * Look up the autoincrement column for the specified table.
-     * @param conn database connection
-     * @param table the table to
-     * @return the name of the column that is an autoincrement column, or null if there is no
-     *         autoincrement column or more than one exists
-     * @throws SQLException
-     */
-    public static String getAutoincrementColumn(Connection conn, String table) throws SQLException {
-        String result = null;
-        int matches = 0;
-
-        ResultSet rs = conn.getMetaData().getColumns(null, null, table, "%");
-        // Some database drivers (SQLite) don't include all the columns
-        if (rs.getMetaData().getColumnCount() >= GET_COLUMNS_IS_AUTOINCREMENT) {
-            while (rs.next()) {
-                if (rs.getString(GET_COLUMNS_IS_AUTOINCREMENT).equals("YES")) {
-                    result = rs.getString(GET_COLUMNS_COLUMN_NAME);
-                    matches++;
-                }
-            }
-            return (matches == 1 ? result : null);
-        }
-
-        // Fallback approach is to query for a single row. This unfortunately does not work with any
-        // empty table
-        log.trace("Falling back to SELECT detection of auto-increment column for {}:{}", conn, table);
-        Statement stmt = conn.createStatement();
-        try {
-            String quoteString = getIdentifierQuoteString(conn);
-            rs = stmt.executeQuery("SELECT * FROM " + quoteString + table + quoteString + " LIMIT 1");
-            ResultSetMetaData rsmd = rs.getMetaData();
-            for (int i = 1; i < rsmd.getColumnCount(); i++) {
-                if (rsmd.isAutoIncrement(i)) {
-                    result = rsmd.getColumnName(i);
-                    matches++;
-                }
-            }
-        } finally {
-            rs.close();
-            stmt.close();
-        }
-        return (matches == 1 ? result : null);
-    }
-
-    public static boolean isColumnNullable(Connection conn, String table, String column)
-            throws SQLException {
-        ResultSet rs = conn.getMetaData().getColumns(null, null, table, column);
-        if (rs.getMetaData().getColumnCount() > GET_COLUMNS_IS_NULLABLE) {
-            // Should only be one match
-            if (!rs.next()) {
-                return false;
-            }
-            String val = rs.getString(GET_COLUMNS_IS_NULLABLE);
-            return rs.getString(GET_COLUMNS_IS_NULLABLE).equals("YES");
-        }
-
-        return false;
-    }
-
-    /**
-     * Format the given Date assuming UTC timezone in a format supported by SQL.
-     * @param date the date to convert to a String
-     * @return the formatted string
-     */
-    public static String formatUTC(Date date) {
-        return DATE_FORMATTER.get().format(date);
-    }
-
-    /**
-     * Get the string used for quoting identifiers in this database's SQL dialect.
-     * @param connection the database connection
-     * @return the quote string
-     * @throws SQLException
-     */
-    public static String getIdentifierQuoteString(Connection connection) throws SQLException {
-        String quoteString = connection.getMetaData().getIdentifierQuoteString();
-        quoteString = quoteString == null ? "" : quoteString;
-        return quoteString;
-    }
-
-    /**
-     * Quote the given string.
-     * @param orig the string to quote
-     * @param quote the quote character
-     * @return the quoted string
-     */
-    public static String quoteString(String orig, String quote) {
-        return quote + orig + quote;
-    }
-
-    public static DataSource initDataSource(Config config) throws Exception {
-        DruidDataSource dataSource = new DruidDataSource();
-        dataSource.setDriverClassName("com.mysql.cj.jdbc.Driver");
-        dataSource.setUrl("jdbc:mysql://" + config.getDbUrl() + ":" + config.getDbPort()  + "?useSSL=true&verifyServerCertificate=false&serverTimezone=GMT%2B8&characterEncoding=utf8");
-        dataSource.setUsername(config.getDbUsername());
-        dataSource.setPassword(config.getDbPassword());
-        dataSource.setInitialSize(1);
-        dataSource.setMaxActive(2);
-        dataSource.setMaxWait(60000);
-        dataSource.setTimeBetweenEvictionRunsMillis(60000);
-        dataSource.setConnectionErrorRetryAttempts(2);
-        dataSource.setBreakAfterAcquireFailure(true);
-        dataSource.setMinEvictableIdleTimeMillis(300000);
-        dataSource.setValidationQuery("SELECT 1 FROM DUAL");
-        dataSource.setTestWhileIdle(true);
-
-        log.info("init data source success");
-        return dataSource;
-    }
-}
-
diff --git a/connectors/rocketmq-connect-jdbc/src/main/java/org/apache/rocketmq/connect/jdbc/schema/column/TimeColumnParser.java b/connectors/rocketmq-connect-jdbc/src/main/java/org/apache/rocketmq/connect/jdbc/common/Dialect.java
similarity index 61%
copy from connectors/rocketmq-connect-jdbc/src/main/java/org/apache/rocketmq/connect/jdbc/schema/column/TimeColumnParser.java
copy to connectors/rocketmq-connect-jdbc/src/main/java/org/apache/rocketmq/connect/jdbc/common/Dialect.java
index 9926d81..c164462 100644
--- a/connectors/rocketmq-connect-jdbc/src/main/java/org/apache/rocketmq/connect/jdbc/schema/column/TimeColumnParser.java
+++ b/connectors/rocketmq-connect-jdbc/src/main/java/org/apache/rocketmq/connect/jdbc/common/Dialect.java
@@ -14,26 +14,29 @@
  * See the License for the specific language governing permissions and
  * limitations under the License.
  */
+package org.apache.rocketmq.connect.jdbc.common;
 
-package org.apache.rocketmq.connect.jdbc.schema.column;
+import java.lang.annotation.ElementType;
+import java.lang.annotation.Retention;
+import java.lang.annotation.RetentionPolicy;
+import java.lang.annotation.Target;
 
-import java.sql.Time;
-import java.sql.Timestamp;
-
-public class TimeColumnParser extends ColumnParser {
-
-    @Override
-    public Object getValue(Object value) {
-
-        if (value == null) {
-            return null;
-        }
-
-        if (value instanceof Timestamp) {
-
-            return new Time(((Timestamp) value).getTime());
-        }
+/**
+ * dialect annotation
+ */
+@Retention(RetentionPolicy.RUNTIME)
+@Target({ElementType.METHOD})
+public @interface Dialect {
+    /**
+     * dialect name
+     * @return
+     */
+    String name() default "";
+
+    /**
+     * support protocol
+     * @return
+     */
+    String protocol() default "";
 
-        return value;
-    }
 }
diff --git a/connectors/rocketmq-connect-jdbc/src/main/java/org/apache/rocketmq/connect/jdbc/strategy/DivideStrategyEnum.java b/connectors/rocketmq-connect-jdbc/src/main/java/org/apache/rocketmq/connect/jdbc/common/DialectType.java
similarity index 85%
rename from connectors/rocketmq-connect-jdbc/src/main/java/org/apache/rocketmq/connect/jdbc/strategy/DivideStrategyEnum.java
rename to connectors/rocketmq-connect-jdbc/src/main/java/org/apache/rocketmq/connect/jdbc/common/DialectType.java
index 0afa470..1e789f5 100644
--- a/connectors/rocketmq-connect-jdbc/src/main/java/org/apache/rocketmq/connect/jdbc/strategy/DivideStrategyEnum.java
+++ b/connectors/rocketmq-connect-jdbc/src/main/java/org/apache/rocketmq/connect/jdbc/common/DialectType.java
@@ -14,10 +14,12 @@
  * See the License for the specific language governing permissions and
  * limitations under the License.
  */
-package org.apache.rocketmq.connect.jdbc.strategy;
+package org.apache.rocketmq.connect.jdbc.common;
 
-public enum DivideStrategyEnum {
-
-    BY_TOPIC,
-    BY_QUEUE
+/**
+ * dialect type source or sink
+ */
+public enum DialectType {
+    SOURCE,
+    SINK,
 }
diff --git a/connectors/rocketmq-connect-jdbc/src/main/java/org/apache/rocketmq/connect/jdbc/schema/column/EnumColumnParser.java b/connectors/rocketmq-connect-jdbc/src/main/java/org/apache/rocketmq/connect/jdbc/common/JdbcSourceConfigConstants.java
similarity index 54%
rename from connectors/rocketmq-connect-jdbc/src/main/java/org/apache/rocketmq/connect/jdbc/schema/column/EnumColumnParser.java
rename to connectors/rocketmq-connect-jdbc/src/main/java/org/apache/rocketmq/connect/jdbc/common/JdbcSourceConfigConstants.java
index 0fd14ba..fbe00ec 100644
--- a/connectors/rocketmq-connect-jdbc/src/main/java/org/apache/rocketmq/connect/jdbc/schema/column/EnumColumnParser.java
+++ b/connectors/rocketmq-connect-jdbc/src/main/java/org/apache/rocketmq/connect/jdbc/common/JdbcSourceConfigConstants.java
@@ -14,33 +14,26 @@
  * See the License for the specific language governing permissions and
  * limitations under the License.
  */
+package org.apache.rocketmq.connect.jdbc.common;
 
-package org.apache.rocketmq.connect.jdbc.schema.column;
+import org.apache.commons.lang3.StringUtils;
 
-public class EnumColumnParser extends ColumnParser {
+public class JdbcSourceConfigConstants {
+  private static final String TABLE_NAME_KEY_PREFIX = "table";
+  private static final String QUERY_NAME_KEY_PREFIX = "query";
+  public static final String QUERY_NAME_VALUE = "query";
 
-    private String[] enumValues;
-
-    public EnumColumnParser(String colType) {
-        enumValues = extractEnumValues(colType);
+  public static String TABLE_NAME_KEY(String suffix){
+    if(StringUtils.isEmpty(suffix)){
+      return TABLE_NAME_KEY_PREFIX;
     }
+    return TABLE_NAME_KEY_PREFIX + "-" + suffix;
+  }
 
-    @Override
-    public Object getValue(Object value) {
-
-        if (value == null) {
-            return null;
-        }
-
-        if (value instanceof String) {
-            return value;
-        }
-
-        Integer i = (Integer) value;
-        if (i == 0) {
-            return null;
-        } else {
-            return enumValues[i - 1];
-        }
+  public static String QUERY_NAME_KEY(String suffix){
+    if(StringUtils.isEmpty(suffix)){
+      return QUERY_NAME_KEY_PREFIX;
     }
+    return QUERY_NAME_KEY_PREFIX + "-" + suffix;
+  }
 }
diff --git a/connectors/rocketmq-connect-jdbc/src/main/java/org/apache/rocketmq/connect/jdbc/common/ConstDefine.java b/connectors/rocketmq-connect-jdbc/src/main/java/org/apache/rocketmq/connect/jdbc/common/OperatorType.java
similarity index 84%
rename from connectors/rocketmq-connect-jdbc/src/main/java/org/apache/rocketmq/connect/jdbc/common/ConstDefine.java
rename to connectors/rocketmq-connect-jdbc/src/main/java/org/apache/rocketmq/connect/jdbc/common/OperatorType.java
index e6d2f7a..574861d 100644
--- a/connectors/rocketmq-connect-jdbc/src/main/java/org/apache/rocketmq/connect/jdbc/common/ConstDefine.java
+++ b/connectors/rocketmq-connect-jdbc/src/main/java/org/apache/rocketmq/connect/jdbc/common/OperatorType.java
@@ -16,8 +16,11 @@
  */
 package org.apache.rocketmq.connect.jdbc.common;
 
-public class ConstDefine {
-
-    public static String JDBC_CONNECTOR_ADMIN_PREFIX = "JDBC-CONNECTOR-ADMIN";
-    public static final String PREFIX = "jdbc";
+/**
+ * c=create , u=update, d=delete
+ */
+public enum OperatorType {
+    c,
+    u,
+    d;
 }
diff --git a/connectors/rocketmq-connect-jdbc/src/main/java/org/apache/rocketmq/connect/jdbc/common/Utils.java b/connectors/rocketmq-connect-jdbc/src/main/java/org/apache/rocketmq/connect/jdbc/common/Utils.java
deleted file mode 100644
index 5708e34..0000000
--- a/connectors/rocketmq-connect-jdbc/src/main/java/org/apache/rocketmq/connect/jdbc/common/Utils.java
+++ /dev/null
@@ -1,74 +0,0 @@
-/*
- * 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.rocketmq.connect.jdbc.common;
-
-import org.apache.commons.lang3.StringUtils;
-import org.apache.rocketmq.client.exception.MQClientException;
-import org.apache.rocketmq.common.protocol.route.BrokerData;
-import org.apache.rocketmq.common.protocol.route.TopicRouteData;
-import org.apache.rocketmq.remoting.exception.RemotingException;
-import org.apache.rocketmq.tools.admin.DefaultMQAdminExt;
-import org.slf4j.Logger;
-import org.slf4j.LoggerFactory;
-
-import java.util.ArrayList;
-import java.util.Collections;
-import java.util.List;
-
-public class Utils {
-    private static final Logger log = LoggerFactory.getLogger(Utils.class);
-
-    public static String createGroupName(String prefix) {
-        return new StringBuilder().append(prefix).append("-").append(System.currentTimeMillis()).toString();
-    }
-
-    public static String createGroupName(String prefix, String postfix) {
-        return new StringBuilder().append(prefix).append("-").append(postfix).toString();
-    }
-
-    public static String createTaskId(String prefix) {
-        return new StringBuilder().append(prefix).append("-").append(System.currentTimeMillis()).toString();
-    }
-
-    public static String createInstanceName(String namesrvAddr) {
-        String[] namesrvArray = namesrvAddr.split(";");
-        List<String> namesrvList = new ArrayList<>();
-        for (String ns : namesrvArray) {
-            if (!namesrvList.contains(ns)) {
-                namesrvList.add(ns);
-            }
-        }
-        Collections.sort(namesrvList);
-        return String.valueOf(namesrvList.toString().hashCode());
-    }
-
-    public static List<BrokerData> examineBrokerData(DefaultMQAdminExt defaultMQAdminExt, String topic,
-        String cluster) throws RemotingException, MQClientException, InterruptedException {
-        List<BrokerData> brokerList = new ArrayList<>();
-
-        TopicRouteData topicRouteData = defaultMQAdminExt.examineTopicRouteInfo(topic);
-        if (topicRouteData.getBrokerDatas() != null) {
-            for (BrokerData broker : topicRouteData.getBrokerDatas()) {
-                if (StringUtils.equals(broker.getCluster(), cluster)) {
-                    brokerList.add(broker);
-                }
-            }
-        }
-        return brokerList;
-    }
-
-}
diff --git a/connectors/rocketmq-connect-jdbc/src/main/java/org/apache/rocketmq/connect/jdbc/config/AbstractConfig.java b/connectors/rocketmq-connect-jdbc/src/main/java/org/apache/rocketmq/connect/jdbc/config/AbstractConfig.java
new file mode 100644
index 0000000..6daf89e
--- /dev/null
+++ b/connectors/rocketmq-connect-jdbc/src/main/java/org/apache/rocketmq/connect/jdbc/config/AbstractConfig.java
@@ -0,0 +1,133 @@
+/*
+ * 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.rocketmq.connect.jdbc.config;
+
+import com.beust.jcommander.internal.Lists;
+import io.openmessaging.KeyValue;
+import org.apache.rocketmq.connect.jdbc.util.QuoteMethod;
+
+import java.util.Arrays;
+import java.util.List;
+import java.util.Objects;
+import java.util.regex.Pattern;
+
+/**
+ * abstract config
+ */
+public abstract class AbstractConfig {
+
+    private static final Pattern COMMA_WITH_WHITESPACE = Pattern.compile("\\s*,\\s*");
+
+    // connection url
+    public static final String CONNECTION_PREFIX = "connection.";
+    public static final String CONNECTION_URL_CONFIG = CONNECTION_PREFIX + "url";
+    // connection user
+    public static final String CONNECTION_USER_CONFIG = CONNECTION_PREFIX + "user";
+    private static final String CONNECTION_USER_DOC = "JDBC connection user.";
+    // connection password
+    public static final String CONNECTION_PASSWORD_CONFIG = CONNECTION_PREFIX + "password";
+    private static final String CONNECTION_PASSWORD_DOC = "JDBC connection password.";
+    // connection attempts
+    public static final String CONNECTION_ATTEMPTS_CONFIG = CONNECTION_PREFIX + "attempts";
+    public static final String CONNECTION_ATTEMPTS_DOC = "Maximum number of attempts to retrieve a valid JDBC connection.Must be a positive integer.";
+    public static final int CONNECTION_ATTEMPTS_DEFAULT = 3;
+    // backoff ms
+    public static final String CONNECTION_BACKOFF_CONFIG = CONNECTION_PREFIX + "backoff.ms";
+    public static final String CONNECTION_BACKOFF_DOC = "Backoff time in milliseconds between connection attempts.";
+    public static final long CONNECTION_BACKOFF_DEFAULT = 10000L;
+    /**
+     * quote.sql.identifiers
+     */
+    public static final String QUOTE_SQL_IDENTIFIERS_CONFIG = "quote.sql.identifiers";
+    public static final String QUOTE_SQL_IDENTIFIERS_DEFAULT = QuoteMethod.ALWAYS.name().toString();
+    public static final String QUOTE_SQL_IDENTIFIERS_DOC =
+            "When to quote table names, column names, and other identifiers in SQL statements. "
+                    + "For backward compatibility, the default is ``always``.";
+
+
+    private String connectionDbUrl;
+    private String connectionDbUser;
+    private String connectionDbPassword;
+    private Integer attempts;
+    private Long backoffMs;
+    private String quoteSqlIdentifiers;
+
+    public AbstractConfig(KeyValue config){
+        connectionDbUrl=config.getString(CONNECTION_URL_CONFIG);
+        connectionDbUser=config.getString(CONNECTION_USER_CONFIG);
+        connectionDbPassword=config.getString(CONNECTION_PASSWORD_CONFIG);
+        attempts=config.getInt(CONNECTION_ATTEMPTS_CONFIG,CONNECTION_ATTEMPTS_DEFAULT);
+        backoffMs=config.getLong(CONNECTION_BACKOFF_CONFIG,CONNECTION_BACKOFF_DEFAULT);
+        quoteSqlIdentifiers=config.getString(QUOTE_SQL_IDENTIFIERS_CONFIG,QUOTE_SQL_IDENTIFIERS_DEFAULT);
+    }
+
+
+    public String getConnectionDbUrl() {
+        return connectionDbUrl;
+    }
+
+    public String getConnectionDbUser() {
+        return connectionDbUser;
+    }
+
+    public String getConnectionDbPassword() {
+        return connectionDbPassword;
+    }
+
+    public Integer getAttempts() {
+        return attempts;
+    }
+
+    public Long getBackoffMs() {
+        return backoffMs;
+    }
+
+    public String getQuoteSqlIdentifiers() {
+        return quoteSqlIdentifiers;
+    }
+
+    /**
+     * get list
+     * @param config
+     * @param key
+     * @return
+     */
+    protected List<String> getList(KeyValue config, String key){
+        if (!config.containsKey(key) || Objects.isNull(config.getString(key))){
+            return Lists.newArrayList();
+        }
+        return Arrays.asList(COMMA_WITH_WHITESPACE.split(config.getString(key), -1));
+    }
+
+    /**
+     * get list
+     * @param config
+     * @param key
+     * @return
+     */
+    protected List<String> getList(KeyValue config, String key,String defaultValue){
+        if (config.containsKey(key) ||Objects.isNull(config.getString(key))){
+            return Lists.newArrayList(defaultValue);
+        }
+        return Arrays.asList(COMMA_WITH_WHITESPACE.split(config.getString(key), -1));
+    }
+
+    protected Boolean getBoolean(KeyValue config, String key,Boolean defaultValue){
+        return config.containsKey(key) ? Boolean.getBoolean(config.getString(key)) : defaultValue;
+    }
+
+}
diff --git a/connectors/rocketmq-connect-jdbc/src/main/java/org/apache/rocketmq/connect/jdbc/config/Config.java b/connectors/rocketmq-connect-jdbc/src/main/java/org/apache/rocketmq/connect/jdbc/config/Config.java
deleted file mode 100644
index 1a9bbc9..0000000
--- a/connectors/rocketmq-connect-jdbc/src/main/java/org/apache/rocketmq/connect/jdbc/config/Config.java
+++ /dev/null
@@ -1,357 +0,0 @@
-/*
- * 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.rocketmq.connect.jdbc.config;
-
-import com.alibaba.fastjson.JSONObject;
-import org.slf4j.Logger;
-import org.slf4j.LoggerFactory;
-
-import java.util.HashSet;
-import java.util.List;
-import java.util.Set;
-
-public class Config {
-    private static final Logger LOG = LoggerFactory.getLogger(Config.class);
-
-    /* Database Connection Config */
-    private String dbUrl;
-    private String dbPort;
-    private String dbUsername;
-    private String dbPassword;
-    private String dataType;
-    private String rocketmqTopic;
-    private String jdbcBackoff;
-    private String jdbcAttempts;
-    private String catalogPattern;
-    private List tableWhitelist;
-    private List tableBlacklist;
-    private String schemaPattern;
-    private boolean numericPrecisionMapping = false;
-    private String bumericMapping;
-    private String dialectName = "";
-    private String whiteDataBase;
-    private String whiteTable;
-
-    public static final String CONN_TASK_PARALLELISM = "task-parallelism";
-    public static final String CONN_TASK_DIVIDE_STRATEGY = "task-divide-strategy";
-    public static final String CONN_WHITE_LIST = "whiteDataBase";
-    public static final String CONN_SOURCE_RECORD_CONVERTER = "source-record-converter";
-    public static final String CONN_DB_IP = "dbUrl";
-    public static final String CONN_DB_PORT = "dbPort";
-    public static final String CONN_DB_USERNAME = "dbUsername";
-    public static final String CONN_DB_PASSWORD = "dbPassword";
-    public static final String CONN_DATA_TYPE = "dataType";
-    public static final String CONN_TOPIC_NAMES = "topicNames";
-    public static final String CONN_DB_MODE = "mode";
-
-    public static final String CONN_SOURCE_RMQ = "source-rocketmq";
-    public static final String CONN_SOURCE_CLUSTER = "source-cluster";
-    public static final String REFRESH_INTERVAL = "refresh.interval";
-
-    /* Mode Config */
-    private String mode = "";
-    private String incrementingColumnName = "";
-    private String query = "";
-    private String timestampColmnName = "";
-    private boolean validateNonNull = true;
-
-    /*Connector config*/
-    private String tableTypes = "table";
-    private long pollInterval = 5000;
-    private int batchMaxRows = 100;
-    private long tablePollInterval = 60000;
-    private long timestampDelayInterval = 0;
-    private String dbTimezone = "GMT+8";
-    private String queueName;
-    private String jdbcUrl;
-    private String jdbcUsername;
-    private String jdbcPassword;
-
-    private Logger log = LoggerFactory.getLogger(Config.class);
-    public static final Set<String> REQUEST_CONFIG = new HashSet<String>() {
-        {
-            add(CONN_DB_IP);
-            add(CONN_DB_PORT);
-            add(CONN_DB_USERNAME);
-            add(CONN_DB_PASSWORD);
-            add(CONN_DB_MODE);
-            add(CONN_SOURCE_RMQ);
-            add(CONN_SOURCE_CLUSTER);
-        }
-    };
-
-    public String getQueueName() {
-        return queueName;
-    }
-
-    public void setQueueName(String queueName) {
-        this.queueName = queueName;
-    }
-
-    public String getDbUrl() {
-        return dbUrl;
-    }
-
-    public void setDbUrl(String dbUrl) {
-        this.dbUrl = dbUrl;
-    }
-
-    public String getDbPort() {
-        return dbPort;
-    }
-
-    public void setDbPort(String dbPort) {
-        this.dbPort = dbPort;
-    }
-
-    public String getDbUsername() {
-        return dbUsername;
-    }
-
-    public void setDbUsername(String dbUsername) {
-        this.dbUsername = dbUsername;
-    }
-
-    public String getDbPassword() {
-        return dbPassword;
-    }
-
-    public void setDbPassword(String dbPassword) {
-        this.dbPassword = dbPassword;
-    }
-
-    public String getDataType() {
-        return dataType;
-    }
-
-    public void setDataType(String dataType) {
-        this.dataType = dataType;
-    }
-
-    public String getRocketmqTopic() {
-        return rocketmqTopic;
-    }
-
-    public void setRocketmqTopic(String rocketmqTopic) {
-        this.rocketmqTopic = rocketmqTopic;
-    }
-
-    public String getJdbcBackoff() {
-        return jdbcBackoff;
-    }
-
-    public void setJdbcBackoff(String jdbcBackoff) {
-        this.jdbcBackoff = jdbcBackoff;
-    }
-
-    public String getJdbcAttempts() {
-        return jdbcAttempts;
-    }
-
-    public void setJdbcAttempts(String jdbcAttempts) {
-        this.jdbcAttempts = jdbcAttempts;
-    }
-
-    public String getCatalogPattern() {
-        return catalogPattern;
-    }
-
-    public void setCatalogPattern(String catalogPattern) {
-        this.catalogPattern = catalogPattern;
-    }
-
-    public List getTableWhitelist() {
-        return tableWhitelist;
-    }
-
-    public void setTableWhitelist(List tableWhitelist) {
-        this.tableWhitelist = tableWhitelist;
-    }
-
-    public List getTableBlacklist() {
-        return tableBlacklist;
-    }
-
-    public void setTableBlacklist(List tableBlacklist) {
-        this.tableBlacklist = tableBlacklist;
-    }
-
-    public String getSchemaPattern() {
-        return schemaPattern;
-    }
-
-    public void setSchemaPattern(String schemaPattern) {
-        this.schemaPattern = schemaPattern;
-    }
-
-    public boolean isNumericPrecisionMapping() {
-        return numericPrecisionMapping;
-    }
-
-    public void setNumericPrecisionMapping(boolean numericPrecisionMapping) {
-        this.numericPrecisionMapping = numericPrecisionMapping;
-    }
-
-    public String getBumericMapping() {
-        return bumericMapping;
-    }
-
-    public void setBumericMapping(String bumericMapping) {
-        this.bumericMapping = bumericMapping;
-    }
-
-    public String getDialectName() {
-        return dialectName;
-    }
-
-    public void setDialectName(String dialectName) {
-        this.dialectName = dialectName;
-    }
-
-    public String getMode() {
-        return mode;
-    }
-
-    public void setMode(String mode) {
-        this.mode = mode;
-    }
-
-    public String getIncrementingColumnName() {
-        return incrementingColumnName;
-    }
-
-    public void setIncrementingColumnName(String incrementingColumnName) {
-        this.incrementingColumnName = incrementingColumnName;
-    }
-
-    public String getQuery() {
-        return query;
-    }
-
-    public void setQuery(String query) {
-        this.query = query;
-    }
-
-    public String getTimestampColmnName() {
-        return timestampColmnName;
-    }
-
-    public void setTimestampColmnName(String timestampColmnName) {
-        this.timestampColmnName = timestampColmnName;
-    }
-
-    public boolean isValidateNonNull() {
-        return validateNonNull;
-    }
-
-    public void setValidateNonNull(boolean validateNonNull) {
-        this.validateNonNull = validateNonNull;
-    }
-
-    public String getTableTypes() {
-        return tableTypes;
-    }
-
-    public void setTableTypes(String tableTypes) {
-        this.tableTypes = tableTypes;
-    }
-
-    public long getPollInterval() {
-        return pollInterval;
-    }
-
-    public void setPollInterval(int pollInterval) {
-        this.pollInterval = pollInterval;
-    }
-
-    public int getBatchMaxRows() {
-        return batchMaxRows;
-    }
-
-    public void setBatchMaxRows(int batchMaxRows) {
-        this.batchMaxRows = batchMaxRows;
-    }
-
-    public long getTablePollInterval() {
-        return tablePollInterval;
-    }
-
-    public void setTablePollInterval(long tablePollInterval) {
-        this.tablePollInterval = tablePollInterval;
-    }
-
-    public long getTimestampDelayInterval() {
-        return timestampDelayInterval;
-    }
-
-    public void setTimestampDelayInterval(long timestampDelayInterval) {
-        this.timestampDelayInterval = timestampDelayInterval;
-    }
-
-    public String getDbTimezone() {
-        return dbTimezone;
-    }
-
-    public void setDbTimezone(String dbTimezone) {
-        this.dbTimezone = dbTimezone;
-    }
-
-    public String getWhiteDataBase() {
-        return whiteDataBase;
-    }
-
-    public void setWhiteDataBase(String whiteDataBase) {
-        this.whiteDataBase = whiteDataBase;
-    }
-
-    public String getWhiteTable() {
-        return whiteTable;
-    }
-
-    public void setWhiteTable(String whiteTable) {
-        this.whiteTable = whiteTable;
-    }
-
-    public void setPollInterval(long pollInterval) {
-        this.pollInterval = pollInterval;
-    }
-
-    public String getJdbcUrl() {
-        return jdbcUrl;
-    }
-
-    public void setJdbcUrl(String jdbcUrl) {
-        this.jdbcUrl = jdbcUrl;
-    }
-
-    public String getJdbcUsername() {
-        return jdbcUsername;
-    }
-
-    public void setJdbcUsername(String jdbcUsername) {
-        this.jdbcUsername = jdbcUsername;
-    }
-
-    public String getJdbcPassword() {
-        return jdbcPassword;
-    }
-
-    public void setJdbcPassword(String jdbcPassword) {
-        this.jdbcPassword = jdbcPassword;
-    }
-}
\ No newline at end of file
diff --git a/connectors/rocketmq-connect-jdbc/src/main/java/org/apache/rocketmq/connect/jdbc/config/ConfigDef.java b/connectors/rocketmq-connect-jdbc/src/main/java/org/apache/rocketmq/connect/jdbc/config/ConfigDef.java
new file mode 100644
index 0000000..c54ce11
--- /dev/null
+++ b/connectors/rocketmq-connect-jdbc/src/main/java/org/apache/rocketmq/connect/jdbc/config/ConfigDef.java
@@ -0,0 +1,573 @@
+/*
+ * 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.rocketmq.connect.jdbc.config;
+
+import org.apache.commons.lang3.StringUtils;
+import org.apache.rocketmq.connect.jdbc.exception.ConfigException;
+
+import java.util.ArrayList;
+import java.util.Arrays;
+import java.util.Collections;
+import java.util.HashMap;
+import java.util.LinkedHashMap;
+import java.util.List;
+import java.util.Locale;
+import java.util.Map;
+import java.util.Set;
+import java.util.function.BiConsumer;
+import java.util.function.Supplier;
+import java.util.regex.Pattern;
+import java.util.stream.Collectors;
+
+/**
+ * config def
+ */
+public class ConfigDef {
+
+    private static final Pattern COMMA_WITH_WHITESPACE = Pattern.compile("\\s*,\\s*");
+    /**
+     * A unique Java object which represents the lack of a default value.
+     */
+    public static final Object NO_DEFAULT_VALUE = new Object();
+
+    private final Map<String, ConfigKey> configKeys;
+    public ConfigDef() {
+        configKeys = new LinkedHashMap<>();
+    }
+
+
+    public ConfigDef define(ConfigKey key) {
+        if (configKeys.containsKey(key.name)) {
+            throw new ConfigException("Configuration " + key.name + " is defined twice.");
+        }
+        configKeys.put(key.name, key);
+        return this;
+    }
+
+    /**
+     * 不验证
+     * @param name
+     * @param type
+     * @param defaultValue
+     * @param validator
+     * @param documentation
+     * @return
+     */
+    public ConfigDef define(String name, Type type, Object defaultValue, Validator validator,  String documentation) {
+        return define(new ConfigKey(name, type, defaultValue, validator, documentation));
+    }
+
+    /**
+     * 加验证
+     * @param name
+     * @param type
+     * @param defaultValue
+     * @param documentation
+     * @return
+     */
+    public ConfigDef define(String name, Type type, Object defaultValue, String documentation) {
+        return define(new ConfigKey(name, type, defaultValue, null, documentation));
+    }
+
+
+    /**
+     * Get the configuration keys
+     * @return a map containing all configuration keys
+     */
+    public Map<String, ConfigKey> configKeys() {
+        return configKeys;
+    }
+
+    /**
+     * parse props
+     * @param props
+     * @return
+     */
+    public Map<String, Object> parse(Map<?, ?> props) {
+        // parse all known keys
+        Map<String, Object> values = new HashMap<>();
+        for (ConfigKey key : configKeys.values()) {
+            values.put(key.name, parseValue(key, props.get(key.name), props.containsKey(key.name)));
+        }
+        return values;
+    }
+
+    /**
+     * convert + validate
+     * @param key
+     * @param value
+     * @param isSet
+     * @return
+     */
+    Object parseValue(ConfigKey key, Object value, boolean isSet) {
+        Object parsedValue;
+        if (isSet) {
+            parsedValue = parseType(key.name, value, key.type);
+            // props map doesn't contain setting, the key is required because no default value specified - its an error
+        } else if (NO_DEFAULT_VALUE.equals(key.defaultValue)) {
+            throw new ConfigException("Missing required configuration \"" + key.name + "\" which has no default value.");
+        } else {
+            parsedValue = key.defaultValue;
+        }
+        if (key.validator != null) {
+            key.validator.ensureValid(key.name, parsedValue);
+        }
+        return parsedValue;
+    }
+
+
+
+    /**
+     * Parse a value according to its expected type.
+     * @param name  The config name
+     * @param value The config value
+     * @param type  The expected type
+     * @return The parsed object
+     */
+    public static Object parseType(String name, Object value, Type type) {
+        try {
+            if (value == null) {
+                return null;
+            }
+            String trimmed = null;
+            if (value instanceof String) {
+                trimmed = ((String) value).trim();
+            }
+            switch (type) {
+                case BOOLEAN:
+                    if (value instanceof String) {
+                        if (trimmed.equalsIgnoreCase("true")) {
+                            return true;
+                        } else if (trimmed.equalsIgnoreCase("false")) {
+                            return false;
+                        } else {
+                            throw new ConfigException(name, value, "Expected value to be either true or false");
+                        }
+                    } else if (value instanceof Boolean) {
+                        return value;
+                    } else {
+                        throw new ConfigException(name, value, "Expected value to be either true or false");
+                    }
+                case STRING:
+                    if (value instanceof String) {
+                        return trimmed;
+                    } else {
+                        throw new ConfigException(name, value, "Expected value to be a string, but it was a " + value.getClass().getName());
+                    }
+                case INT:
+                    if (value instanceof Integer) {
+                        return value;
+                    } else if (value instanceof String) {
+                        return Integer.parseInt(trimmed);
+                    } else {
+                        throw new ConfigException(name, value, "Expected value to be a 32-bit integer, but it was a " + value.getClass().getName());
+                    }
+                case SHORT:
+                    if (value instanceof Short) {
+                        return value;
+                    } else if (value instanceof String) {
+                        return Short.parseShort(trimmed);
+                    } else {
+                        throw new ConfigException(name, value, "Expected value to be a 16-bit integer (short), but it was a " + value.getClass().getName());
+                    }
+                case LONG:
+                    if (value instanceof Integer) {
+                        return ((Integer) value).longValue();
+                    }
+                    if (value instanceof Long) {
+                        return value;
+                    } else if (value instanceof String) {
+                        return Long.parseLong(trimmed);
+                    } else {
+                        throw new ConfigException(name, value, "Expected value to be a 64-bit integer (long), but it was a " + value.getClass().getName());
+                    }
+                case DOUBLE:
+                    if (value instanceof Number) {
+                        return ((Number) value).doubleValue();
+                    } else if (value instanceof String) {
+                        return Double.parseDouble(trimmed);
+                    } else {
+                        throw new ConfigException(name, value, "Expected value to be a double, but it was a " + value.getClass().getName());
+                    }
+                case LIST:
+                    if (value instanceof List) {
+                        return value;
+                    } else if (value instanceof String) {
+                        if (trimmed.isEmpty()) {
+                            return Collections.emptyList();
+                        } else {
+                            return Arrays.asList(COMMA_WITH_WHITESPACE.split(trimmed, -1));
+                        }
+                    } else {
+                        throw new ConfigException(name, value, "Expected a comma separated list.");
+                    }
+                default:
+                    throw new IllegalStateException("Unknown type.");
+            }
+        } catch (NumberFormatException e) {
+            throw new ConfigException(name, value, "Not a number of type " + type);
+        }
+    }
+
+    public static String convertToString(Object parsedValue, Type type) {
+        if (parsedValue == null) {
+            return null;
+        }
+
+        if (type == null) {
+            return parsedValue.toString();
+        }
+
+        switch (type) {
+            case BOOLEAN:
+            case SHORT:
+            case INT:
+            case LONG:
+            case DOUBLE:
+            case STRING:
+            case PASSWORD:
+                return parsedValue.toString();
+            case LIST:
+                List<?> valueList = (List<?>) parsedValue;
+                return StringUtils.join(valueList, ",");
+            case CLASS:
+                Class<?> clazz = (Class<?>) parsedValue;
+                return clazz.getName();
+            default:
+                throw new IllegalStateException("Unknown type.");
+        }
+    }
+
+    /**
+     * The config types
+     */
+    public enum Type {
+        BOOLEAN, STRING, INT, SHORT, LONG, DOUBLE, LIST, CLASS, PASSWORD
+    }
+
+
+
+
+
+    /**
+     * Validation logic the user may provide to perform single configuration validation.
+     */
+    public interface Validator {
+        /**
+         * Perform single configuration validation.
+         * @param name The name of the configuration
+         * @param value The value of the configuration
+         * @throws ConfigException if the value is invalid.
+         */
+        void ensureValid(String name, Object value);
+    }
+
+    /**
+     * Validation logic for numeric ranges
+     */
+    public static class Range implements Validator {
+        private final Number min;
+        private final Number max;
+        /**
+         *  A numeric range with inclusive upper bound and inclusive lower bound
+         * @param min  the lower bound
+         * @param max  the upper bound
+         */
+        private Range(Number min, Number max) {
+            this.min = min;
+            this.max = max;
+        }
+        /**
+         * A numeric range that checks only the lower bound
+         * @param min The minimum acceptable value
+         */
+        public static Range atLeast(Number min) {
+            return new Range(min, null);
+        }
+
+        /**
+         * A numeric range that checks both the upper (inclusive) and lower bound
+         */
+        public static Range between(Number min, Number max) {
+            return new Range(min, max);
+        }
+
+        @Override
+        public void ensureValid(String name, Object o) {
+            if (o == null) {
+                throw new ConfigException(name, null, "Value must be non-null");
+            }
+            Number n = (Number) o;
+            if (min != null && n.doubleValue() < min.doubleValue()) {
+                throw new ConfigException(name, o, "Value must be at least " + min);
+            }
+            if (max != null && n.doubleValue() > max.doubleValue()) {
+                throw new ConfigException(name, o, "Value must be no more than " + max);
+            }
+        }
+
+        @Override
+        public String toString() {
+            if (min == null && max == null) {
+                return "[...]";
+            } else if (min == null) {
+                return "[...," + max + "]";
+            } else if (max == null) {
+                return "[" + min + ",...]";
+            } else {
+                return "[" + min + ",...," + max + "]";
+            }
+        }
+    }
+
+    public static class ValidList implements Validator {
+
+        final ValidString validString;
+
+        private ValidList(List<String> validStrings) {
+            this.validString = new ValidString(validStrings);
+        }
+
+        public static ValidList in(String... validStrings) {
+            return new ValidList(Arrays.asList(validStrings));
+        }
+
+        @Override
+        public void ensureValid(final String name, final Object value) {
+            @SuppressWarnings("unchecked")
+            List<String> values = (List<String>) value;
+            for (String string : values) {
+                validString.ensureValid(name, string);
+            }
+        }
+
+        @Override
+        public String toString() {
+            return validString.toString();
+        }
+    }
+
+    public static class ValidString implements Validator {
+        final List<String> validStrings;
+
+        private ValidString(List<String> validStrings) {
+            this.validStrings = validStrings;
+        }
+
+        public static ValidString in(String... validStrings) {
+            return new ValidString(Arrays.asList(validStrings));
+        }
+
+        @Override
+        public void ensureValid(String name, Object o) {
+            String s = (String) o;
+            if (!validStrings.contains(s)) {
+                throw new ConfigException(name, o, "String must be one of: " + StringUtils.join(validStrings, ", "));
+            }
+
+        }
+
+        @Override
+        public String toString() {
+            return "[" + StringUtils.join(validStrings, ", ") + "]";
+        }
+    }
+
+    public static class CaseInsensitiveValidString implements Validator {
+
+        final Set<String> validStrings;
+
+        private CaseInsensitiveValidString(List<String> validStrings) {
+            this.validStrings = validStrings.stream()
+                    .map(s -> s.toUpperCase(Locale.ROOT))
+                    .collect(Collectors.toSet());
+        }
+
+        public static CaseInsensitiveValidString in(String... validStrings) {
+            return new CaseInsensitiveValidString(Arrays.asList(validStrings));
+        }
+
+        @Override
+        public void ensureValid(String name, Object o) {
+            String s = (String) o;
+            if (s == null || !validStrings.contains(s.toUpperCase(Locale.ROOT))) {
+                throw new ConfigException(name, o, "String must be one of (case insensitive): " + StringUtils.join(validStrings, ", "));
+            }
+        }
+
+        @Override
+        public String toString() {
+            return "(case insensitive) [" + StringUtils.join(validStrings, ", ") + "]";
+        }
+    }
+
+    public static class NonNullValidator implements Validator {
+        @Override
+        public void ensureValid(String name, Object value) {
+            if (value == null) {
+                // Pass in the string null to avoid the spotbugs warning
+                throw new ConfigException(name, "null", "entry must be non null");
+            }
+        }
+
+        @Override
+        public String toString() {
+            return "non-null string";
+        }
+    }
+
+    public static class LambdaValidator implements Validator {
+        BiConsumer<String, Object> ensureValid;
+        Supplier<String> toStringFunction;
+
+        private LambdaValidator(BiConsumer<String, Object> ensureValid,
+                                Supplier<String> toStringFunction) {
+            this.ensureValid = ensureValid;
+            this.toStringFunction = toStringFunction;
+        }
+
+        public static LambdaValidator with(BiConsumer<String, Object> ensureValid,
+                                           Supplier<String> toStringFunction) {
+            return new LambdaValidator(ensureValid, toStringFunction);
+        }
+
+        @Override
+        public void ensureValid(String name, Object value) {
+            ensureValid.accept(name, value);
+        }
+
+        @Override
+        public String toString() {
+            return toStringFunction.get();
+        }
+    }
+
+    public static class CompositeValidator implements Validator {
+        private final List<Validator> validators;
+
+        private CompositeValidator(List<Validator> validators) {
+            this.validators = Collections.unmodifiableList(validators);
+        }
+
+        public static CompositeValidator of(Validator... validators) {
+            return new CompositeValidator(Arrays.asList(validators));
+        }
+
+        @Override
+        public void ensureValid(String name, Object value) {
+            for (Validator validator: validators) {
+                validator.ensureValid(name, value);
+            }
+        }
+
+        @Override
+        public String toString() {
+            if (validators == null) {
+                return "";
+            }
+            StringBuilder desc = new StringBuilder();
+            for (Validator v: validators) {
+                if (desc.length() > 0) {
+                    desc.append(',').append(' ');
+                }
+                desc.append(String.valueOf(v));
+            }
+            return desc.toString();
+        }
+    }
+
+    public static class NonEmptyString implements Validator {
+
+        @Override
+        public void ensureValid(String name, Object o) {
+            String s = (String) o;
+            if (s != null && s.isEmpty()) {
+                throw new ConfigException(name, o, "String must be non-empty");
+            }
+        }
+
+        @Override
+        public String toString() {
+            return "non-empty string";
+        }
+    }
+
+    public static class NonEmptyStringWithoutControlChars implements Validator {
+
+        public static NonEmptyStringWithoutControlChars nonEmptyStringWithoutControlChars() {
+            return new NonEmptyStringWithoutControlChars();
+        }
+        @Override
+        public void ensureValid(String name, Object value) {
+            String s = (String) value;
+
+            if (s == null) {
+                // This can happen during creation of the config object due to no default value being defined for the
+                // name configuration - a missing name parameter is caught when checking for mandatory parameters,
+                // thus we can ok a null value here
+                return;
+            } else if (s.isEmpty()) {
+                throw new ConfigException(name, value, "String may not be empty");
+            }
+
+            // Check name string for illegal characters
+            ArrayList<Integer> foundIllegalCharacters = new ArrayList<>();
+
+            for (int i = 0; i < s.length(); i++) {
+                if (Character.isISOControl(s.codePointAt(i))) {
+                    foundIllegalCharacters.add(s.codePointAt(i));
+                }
+            }
+            if (!foundIllegalCharacters.isEmpty()) {
+                throw new ConfigException(name, value, "String may not contain control sequences but had the following ASCII chars: " + StringUtils.join(foundIllegalCharacters, ","));
+            }
+        }
+
+        @Override
+        public String toString() {
+            return "non-empty string without ISO control characters";
+        }
+    }
+
+    public static class ConfigKey {
+        public final String name;
+        public final Type type;
+        public final String documentation;
+        public final Object defaultValue;
+        public final Validator validator;
+
+        public ConfigKey(String name, Type type, Object defaultValue, Validator validator, String documentation) {
+            this.name = name;
+            this.type = type;
+            this.defaultValue = NO_DEFAULT_VALUE.equals(defaultValue) ? NO_DEFAULT_VALUE : parseType(name, defaultValue, type);
+            this.validator = validator;
+            if (this.validator != null && hasDefault()) {
+                this.validator.ensureValid(name, this.defaultValue);
+            }
+            this.documentation = documentation;
+        }
+
+        public boolean hasDefault() {
+            return !NO_DEFAULT_VALUE.equals(this.defaultValue);
+        }
+
+        public Type type() {
+            return type;
+        }
+    }
+
+}
diff --git a/connectors/rocketmq-connect-jdbc/src/main/java/org/apache/rocketmq/connect/jdbc/config/ConfigUtil.java b/connectors/rocketmq-connect-jdbc/src/main/java/org/apache/rocketmq/connect/jdbc/config/ConfigUtil.java
deleted file mode 100644
index 70b9e6b..0000000
--- a/connectors/rocketmq-connect-jdbc/src/main/java/org/apache/rocketmq/connect/jdbc/config/ConfigUtil.java
+++ /dev/null
@@ -1,69 +0,0 @@
-/*
- * 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.rocketmq.connect.jdbc.config;
-
-import io.openmessaging.KeyValue;
-import java.lang.reflect.Method;
-
-public class ConfigUtil {
-    public static <T> void load(KeyValue props, Object object) {
-
-        properties2Object(props, object);
-    }
-
-    private static <T> void properties2Object(final KeyValue p, final Object object) {
-
-        Method[] methods = object.getClass().getMethods();
-        for (Method method : methods) {
-            String mn = method.getName();
-            if (mn.startsWith("set")) {
-                try {
-                    String tmp = mn.substring(4);
-                    String first = mn.substring(3, 4);
-
-                    String key = first.toLowerCase() + tmp;
-                    String property = p.getString(key);
-                    if (property != null) {
-                        Class<?>[] pt = method.getParameterTypes();
-                        if (pt != null && pt.length > 0) {
-                            String cn = pt[0].getSimpleName();
-                            Object arg;
-                            if (cn.equals("int") || cn.equals("Integer")) {
-                                arg = Integer.parseInt(property);
-                            } else if (cn.equals("long") || cn.equals("Long")) {
-                                arg = Long.parseLong(property);
-                            } else if (cn.equals("double") || cn.equals("Double")) {
-                                arg = Double.parseDouble(property);
-                            } else if (cn.equals("boolean") || cn.equals("Boolean")) {
-                                arg = Boolean.parseBoolean(property);
-                            } else if (cn.equals("float") || cn.equals("Float")) {
-                                arg = Float.parseFloat(property);
-                            } else if (cn.equals("String")) {
-                                arg = property;
-                            } else {
-                                continue;
-                            }
-                            method.invoke(object, arg);
-                        }
-                    }
-                } catch (Throwable ignored) {
-                }
-            }
-        }
-    }
-}
diff --git a/connectors/rocketmq-connect-jdbc/src/main/java/org/apache/rocketmq/connect/jdbc/config/DbConnectorConfig.java b/connectors/rocketmq-connect-jdbc/src/main/java/org/apache/rocketmq/connect/jdbc/config/DbConnectorConfig.java
deleted file mode 100644
index 91ffd3c..0000000
--- a/connectors/rocketmq-connect-jdbc/src/main/java/org/apache/rocketmq/connect/jdbc/config/DbConnectorConfig.java
+++ /dev/null
@@ -1,101 +0,0 @@
-/*
- * 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.rocketmq.connect.jdbc.config;
-
-import io.openmessaging.KeyValue;
-import org.apache.rocketmq.connect.jdbc.strategy.TaskDivideStrategy;
-
-public abstract class DbConnectorConfig {
-
-    public TaskDivideStrategy taskDivideStrategy;
-    public String dbUrl;
-    public String dbPort;
-    public String dbUserName;
-    public String dbPassword;
-    public String converter;
-    public int taskParallelism;
-    public String mode;
-
-    public abstract void validate(KeyValue config);
-
-    public abstract <T> T getWhiteTopics();
-
-    public TaskDivideStrategy getTaskDivideStrategy() {
-        return taskDivideStrategy;
-    }
-
-    public void setTaskDivideStrategy(TaskDivideStrategy taskDivideStrategy) {
-        this.taskDivideStrategy = taskDivideStrategy;
-    }
-
-    public String getDbUrl() {
-        return dbUrl;
-    }
-
-    public void setDbUrl(String dbUrl) {
-        this.dbUrl = dbUrl;
-    }
-
-    public String getDbPort() {
-        return dbPort;
-    }
-
-    public void setDbPort(String dbPort) {
-        this.dbPort = dbPort;
-    }
-
-    public String getDbUserName() {
-        return dbUserName;
-    }
-
-    public void setDbUserName(String dbUserName) {
-        this.dbUserName = dbUserName;
-    }
-
-    public String getDbPassword() {
-        return dbPassword;
-    }
-
-    public void setDbPassword(String dbPassword) {
-        this.dbPassword = dbPassword;
-    }
-
-    public String getConverter() {
-        return converter;
-    }
-
-    public void setConverter(String converter) {
-        this.converter = converter;
-    }
-
-    public int getTaskParallelism() {
-        return taskParallelism;
-    }
-
-    public void setTaskParallelism(int taskParallelism) {
-        this.taskParallelism = taskParallelism;
-    }
-
-    public String getMode() {
-        return mode;
-    }
-
-    public void setMode(String mode) {
-        this.mode = mode;
-    }
-}
diff --git a/connectors/rocketmq-connect-jdbc/src/main/java/org/apache/rocketmq/connect/jdbc/config/SinkDbConnectorConfig.java b/connectors/rocketmq-connect-jdbc/src/main/java/org/apache/rocketmq/connect/jdbc/config/SinkDbConnectorConfig.java
deleted file mode 100644
index 075e8ce..0000000
--- a/connectors/rocketmq-connect-jdbc/src/main/java/org/apache/rocketmq/connect/jdbc/config/SinkDbConnectorConfig.java
+++ /dev/null
@@ -1,115 +0,0 @@
-/*
- * 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.rocketmq.connect.jdbc.config;
-
-import io.openmessaging.KeyValue;
-import org.apache.rocketmq.connect.jdbc.strategy.DivideStrategyEnum;
-import org.apache.rocketmq.connect.jdbc.strategy.DivideTaskByQueue;
-import org.apache.rocketmq.connect.jdbc.strategy.DivideTaskByTopic;
-
-import java.util.HashSet;
-import java.util.List;
-import java.util.Map;
-import java.util.Set;
-
-public class SinkDbConnectorConfig extends DbConnectorConfig{
-
-    private Set<String> whiteList;
-    private String srcNamesrvs;
-    private String srcCluster;
-    private long refreshInterval;
-    private Map<String, Set<TaskTopicInfo>> topicRouteMap;
-
-    public SinkDbConnectorConfig(){
-    }
-
-    @Override
-    public void validate(KeyValue config) {
-        this.taskParallelism = config.getInt(Config.CONN_TASK_PARALLELISM, 1);
-
-        int strategy = config.getInt(Config.CONN_TASK_DIVIDE_STRATEGY, DivideStrategyEnum.BY_TOPIC.ordinal());
-        if (strategy == DivideStrategyEnum.BY_QUEUE.ordinal()) {
-            this.taskDivideStrategy = new DivideTaskByQueue();
-        } else {
-            this.taskDivideStrategy = new DivideTaskByTopic();
-        }
-
-        buildWhiteList(config);
-
-        this.converter = config.getString(Config.CONN_SOURCE_RECORD_CONVERTER);
-        this.dbUrl = config.getString(Config.CONN_DB_IP);
-        this.dbPort = config.getString(Config.CONN_DB_PORT);
-        this.dbUserName = config.getString(Config.CONN_DB_USERNAME);
-        this.dbPassword = config.getString(Config.CONN_DB_PASSWORD);
-
-        this.srcNamesrvs = config.getString(Config.CONN_SOURCE_RMQ);
-        this.srcCluster = config.getString(Config.CONN_SOURCE_CLUSTER);
-        this.refreshInterval = config.getLong(Config.REFRESH_INTERVAL, 3);
-        this.mode = config.getString(Config.CONN_DB_MODE, "bulk");
-
-    }
-
-    private void buildWhiteList(KeyValue config) {
-        this.whiteList = new HashSet<>();
-        String whiteListStr = config.getString(Config.CONN_TOPIC_NAMES, "");
-        String[] wl = whiteListStr.trim().split(",");
-        if (wl.length <= 0)
-            throw new IllegalArgumentException("White list must be not empty.");
-        else {
-            this.whiteList.clear();
-            for (String t : wl) {
-                this.whiteList.add(t.trim());
-            }
-        }
-    }
-
-
-    public Set<String> getWhiteList() {
-        return whiteList;
-    }
-
-    public void setWhiteList(Set<String> whiteList) {
-        this.whiteList = whiteList;
-    }
-
-    public String getSrcNamesrvs() {
-        return this.srcNamesrvs;
-    }
-
-    public String getSrcCluster() {
-        return this.srcCluster;
-    }
-
-    public long getRefreshInterval() {
-        return this.refreshInterval;
-    }
-
-    public Map<String, Set<TaskTopicInfo>> getTopicRouteMap() {
-        return topicRouteMap;
-    }
-
-    public void setTopicRouteMap(Map<String, Set<TaskTopicInfo>> topicRouteMap) {
-        this.topicRouteMap = topicRouteMap;
-    }
-
-    @Override
-    public Set<String> getWhiteTopics() {
-        return getWhiteList();
-    }
-
-}
diff --git a/connectors/rocketmq-connect-jdbc/src/main/java/org/apache/rocketmq/connect/jdbc/config/SourceDbConnectorConfig.java b/connectors/rocketmq-connect-jdbc/src/main/java/org/apache/rocketmq/connect/jdbc/config/SourceDbConnectorConfig.java
deleted file mode 100644
index 9ab3f79..0000000
--- a/connectors/rocketmq-connect-jdbc/src/main/java/org/apache/rocketmq/connect/jdbc/config/SourceDbConnectorConfig.java
+++ /dev/null
@@ -1,90 +0,0 @@
-/*
- * 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.rocketmq.connect.jdbc.config;
-
-import com.alibaba.fastjson.JSONObject;
-import io.openmessaging.KeyValue;
-import org.apache.rocketmq.connect.jdbc.strategy.DivideStrategyEnum;
-import org.apache.rocketmq.connect.jdbc.strategy.DivideTaskByQueue;
-import org.apache.rocketmq.connect.jdbc.strategy.DivideTaskByTopic;
-
-import java.util.HashMap;
-import java.util.Map;
-
-public class SourceDbConnectorConfig extends DbConnectorConfig{
-
-    private Map<String, String> whiteMap;
-
-    public SourceDbConnectorConfig(){
-    }
-
-    @Override
-    public void validate(KeyValue config) {
-        this.taskParallelism = config.getInt(Config.CONN_TASK_PARALLELISM, 1);
-
-        int strategy = config.getInt(Config.CONN_TASK_DIVIDE_STRATEGY, DivideStrategyEnum.BY_TOPIC.ordinal());
-        if (strategy == DivideStrategyEnum.BY_QUEUE.ordinal()) {
-            this.taskDivideStrategy = new DivideTaskByQueue();
-        } else {
-            this.taskDivideStrategy = new DivideTaskByTopic();
-        }
-
-        buildWhiteMap(config);
-
-        this.converter = config.getString(Config.CONN_SOURCE_RECORD_CONVERTER);
-        this.dbUrl = config.getString(Config.CONN_DB_IP);
-        this.dbPort = config.getString(Config.CONN_DB_PORT);
-        this.dbUserName = config.getString(Config.CONN_DB_USERNAME);
-        this.dbPassword = config.getString(Config.CONN_DB_PASSWORD);
-        this.mode = config.getString(Config.CONN_DB_MODE, "bulk");
-
-    }
-
-    private void buildWhiteMap(KeyValue config) {
-        this.whiteMap = new HashMap<>(16);
-        String whiteListStr = config.getString(Config.CONN_WHITE_LIST, "");
-        JSONObject whiteDataBaseObject = JSONObject.parseObject(whiteListStr);
-        if(whiteDataBaseObject.keySet().size() <= 0){
-            throw new IllegalArgumentException("white data base must be not empty.");
-        }else {
-            this.whiteMap.clear();
-            for (String dbName : whiteDataBaseObject.keySet()){
-                JSONObject whiteTableObject = (JSONObject) whiteDataBaseObject.get(dbName);
-                for (String tableName : whiteTableObject.keySet()){
-                    String dbTableKey = dbName + "-" + tableName;
-                    this.whiteMap.put(dbTableKey, whiteTableObject.getString(tableName));
-                }
-            }
-        }
-    }
-
-
-    public Map<String, String> getWhiteMap() {
-        return whiteMap;
-    }
-
-    public void setWhiteMap(Map<String, String> whiteMap) {
-        this.whiteMap = whiteMap;
-    }
-
-    @Override
-    public Map<String, String> getWhiteTopics() {
-        return getWhiteMap();
-    }
-
-}
diff --git a/connectors/rocketmq-connect-jdbc/src/main/java/org/apache/rocketmq/connect/jdbc/config/TaskDivideConfig.java b/connectors/rocketmq-connect-jdbc/src/main/java/org/apache/rocketmq/connect/jdbc/config/TaskDivideConfig.java
deleted file mode 100644
index 8b15a2f..0000000
--- a/connectors/rocketmq-connect-jdbc/src/main/java/org/apache/rocketmq/connect/jdbc/config/TaskDivideConfig.java
+++ /dev/null
@@ -1,112 +0,0 @@
-/*
- * 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.rocketmq.connect.jdbc.config;
-
-public class TaskDivideConfig {
-
-    private String dbUrl;
-
-    private String dbPort;
-
-    private String dbUserName;
-
-    private String dbPassword;
-
-    private String srcRecordConverter;
-
-    private int dataType;
-
-    private int taskParallelism;
-
-    private String mode;
-
-    public TaskDivideConfig(String dbUrl, String dbPort, String dbUserName, String dbPassword, String srcRecordConverter,
-                            int dataType, int taskParallelism, String mode) {
-        this.dbUrl = dbUrl;
-        this.dbPort = dbPort;
-        this.dbUserName = dbUserName;
-        this.dbPassword = dbPassword;
-        this.srcRecordConverter = srcRecordConverter;
-        this.dataType = dataType;
-        this.taskParallelism = taskParallelism;
-        this.mode = mode;
-    }
-
-    public String getDbUrl() {
-        return dbUrl;
-    }
-
-    public void setDbUrl(String dbUrl) {
-        this.dbUrl = dbUrl;
-    }
-
-    public String getDbPort() {
-        return dbPort;
-    }
-
-    public void setDbPort(String dbPort) {
-        this.dbPort = dbPort;
-    }
-
-    public String getDbUserName() {
-        return dbUserName;
-    }
-
-    public void setDbUserName(String dbUserName) {
-        this.dbUserName = dbUserName;
-    }
-
-    public String getDbPassword() {
-        return dbPassword;
-    }
-
-    public void setDbPassword(String dbPassword) {
-        this.dbPassword = dbPassword;
-    }
-
-    public String getSrcRecordConverter() {
-        return srcRecordConverter;
-    }
-
-    public void setSrcRecordConverter(String srcRecordConverter) {
-        this.srcRecordConverter = srcRecordConverter;
-    }
-
-    public int getDataType() {
-        return dataType;
-    }
-
-    public void setDataType(int dataType) {
-        this.dataType = dataType;
-    }
-
-    public int getTaskParallelism() {
-        return taskParallelism;
-    }
-
-    public void setTaskParallelism(int taskParallelism) {
-        this.taskParallelism = taskParallelism;
-    }
-
-    public String getMode() {
-        return mode;
-    }
-
-    public void setMode(String mode) {
-        this.mode = mode;
-    }
-}
diff --git a/connectors/rocketmq-connect-jdbc/src/main/java/org/apache/rocketmq/connect/jdbc/connector/JdbcSinkConfig.java b/connectors/rocketmq-connect-jdbc/src/main/java/org/apache/rocketmq/connect/jdbc/connector/JdbcSinkConfig.java
new file mode 100644
index 0000000..d211a23
--- /dev/null
+++ b/connectors/rocketmq-connect-jdbc/src/main/java/org/apache/rocketmq/connect/jdbc/connector/JdbcSinkConfig.java
@@ -0,0 +1,302 @@
+/*
+ * 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.rocketmq.connect.jdbc.connector;
+
+import io.openmessaging.KeyValue;
+import org.apache.rocketmq.connect.jdbc.config.AbstractConfig;
+import org.apache.rocketmq.connect.jdbc.dialect.DatabaseDialect;
+import org.apache.rocketmq.connect.jdbc.schema.table.TableId;
+import org.apache.rocketmq.connect.jdbc.util.TableType;
+
+import java.time.ZoneId;
+import java.util.EnumSet;
+import java.util.HashSet;
+import java.util.List;
+import java.util.Objects;
+import java.util.Set;
+import java.util.TimeZone;
+import java.util.stream.Collectors;
+
+/**
+ * jdbc sink config
+ */
+public class JdbcSinkConfig extends AbstractConfig {
+
+    public enum InsertMode {
+        INSERT,
+        UPSERT,
+        UPDATE;
+
+    }
+
+    public enum PrimaryKeyMode {
+        NONE,
+        RECORD_KEY,
+        RECORD_VALUE;
+    }
+
+    public static final String TABLE_NAME_FORMAT = "table.name.format";
+    public static final String TABLE_NAME_FORMAT_DEFAULT = "${topic}";
+    private static final String TABLE_NAME_FORMAT_DISPLAY = "Table Name Format";
+
+    /**
+     * table name from header
+     */
+    public static final String TABLE_NAME_FROM_HEADER = "table.name.from.header";
+    private static final boolean TABLE_NAME_FROM_HEADER_DEFAULT = Boolean.FALSE;
+    private static final String TABLE_NAME_FROM_HEADER_DISPLAY = "Table from header";
+
+    /**
+     * max retries
+     */
+    public static final String MAX_RETRIES = "max.retries";
+    private static final int MAX_RETRIES_DEFAULT = 10;
+    private static final String MAX_RETRIES_DOC =
+            "The maximum number of times to retry on errors before failing the task.";
+
+    public static final String RETRY_BACKOFF_MS = "retry.backoff.ms";
+    private static final int RETRY_BACKOFF_MS_DEFAULT = 3000;
+
+    public static final String BATCH_SIZE = "batch.size";
+    private static final int BATCH_SIZE_DEFAULT = 3000;
+
+
+    public static final String DELETE_ENABLED = "delete.enabled";
+    private static final boolean DELETE_ENABLED_DEFAULT = false;
+
+
+    public static final String AUTO_CREATE = "auto.create";
+    private static final boolean AUTO_CREATE_DEFAULT = false;
+
+    public static final String AUTO_EVOLVE = "auto.evolve";
+    private static final boolean AUTO_EVOLVE_DEFAULT = false;
+    private static final String AUTO_EVOLVE_DOC =
+            "Whether to automatically add columns in the table schema when found to be missing relative "
+                    + "to the record schema by issuing ``ALTER``.";
+
+    public static final String INSERT_MODE = "insert.mode";
+    private static final String INSERT_MODE_DEFAULT = "insert";
+    private static final String INSERT_MODE_DOC =
+            "The insertion mode to use. Supported modes are:\n"
+                    + "``insert``\n"
+                    + "    Use standard SQL ``INSERT`` statements.\n"
+                    + "``upsert``\n"
+                    + "    Use the appropriate upsert semantics for the target database if it is supported by "
+                    + "the connector, e.g. ``INSERT OR IGNORE``.\n"
+                    + "``update``\n"
+                    + "    Use the appropriate update semantics for the target database if it is supported by "
+                    + "the connector, e.g. ``UPDATE``.";
+    private static final String INSERT_MODE_DISPLAY = "Insert Mode";
+
+
+    public static final String PK_FIELDS = "pk.fields";
+    private static final String PK_FIELDS_DEFAULT = "";
+    private static final String PK_FIELDS_DOC =
+            "List of comma-separated primary key field names. The runtime interpretation of this config"
+                    + " depends on the ``pk.mode``:\n"
+                    + "``none``\n"
+                    + "    Ignored as no fields are used as primary key in this mode.\n"
+                    + "``record_key``\n"
+                    + "    If empty, all fields from the key struct will be used, otherwise used to extract the"
+                    + " desired fields - for primitive key only a single field name must be configured.\n"
+                    + "``record_value``\n"
+                    + "    If empty, all fields from the value struct will be used, otherwise used to extract "
+                    + "the desired fields.";
+    private static final String PK_FIELDS_DISPLAY = "Primary Key Fields";
+
+    public static final String PK_MODE = "pk.mode";
+    private static final String PK_MODE_DEFAULT = "none";
+    private static final String PK_MODE_DOC =
+            "The primary key mode, also refer to ``" + PK_FIELDS + "`` documentation for interplay. "
+                    + "Supported modes are:\n"
+                    + "``none``\n"
+                    + "    No keys utilized.\n"
+                    + "``record_value``\n"
+                    + "    Field(s) from the record value are used, which must be a struct.";
+    private static final String PK_MODE_DISPLAY = "Primary Key Mode";
+
+    public static final String FIELDS_WHITELIST = "fields.whitelist";
+    private static final String FIELDS_WHITELIST_DEFAULT = "";
+    private static final String FIELDS_WHITELIST_DOC =
+            "List of comma-separated record value field names. If empty, all fields from the record "
+                    + "value are utilized, otherwise used to filter to the desired fields.\n"
+                    + "Note that ``" + PK_FIELDS + "`` is applied independently in the context of which field"
+                    + "(s) form the primary key columns in the destination database,"
+                    + " while this configuration is applicable for the other columns.";
+    private static final String FIELDS_WHITELIST_DISPLAY = "Fields Whitelist";
+
+
+    public static final String DIALECT_NAME_CONFIG = "dialect.name";
+    public static final String DIALECT_NAME_DEFAULT = "";
+
+
+    public static final String DB_TIMEZONE_CONFIG = "db.timezone";
+    public static final String DB_TIMEZONE_DEFAULT = "UTC";
+
+    // table types
+    public static final String TABLE_TYPES_CONFIG = "table.types";
+    public static final String TABLE_TYPES_DEFAULT = TableType.TABLE.toString();
+    private static final String TABLE_TYPES_DOC =
+            "The comma-separated types of database tables to which the sink connector can write. "
+                    + "By default this is ``" + TableType.TABLE + "``, but any combination of ``"
+                    + TableType.TABLE + "`` and ``" + TableType.VIEW + "`` is allowed. Not all databases "
+                    + "support writing to views, and when they do the the sink connector will fail if the "
+                    + "view definition does not match the records' schemas (regardless of ``"
+                    + AUTO_EVOLVE + "``).";
+
+    // white list tables
+    public static final String TABLE_WHITE_LIST_CONFIG = "tables.whitelist";
+    public static final String TABLE_WHITE_LIST_DEFAULT = "";
+    private static final String TABLE_WHITE_LIST_DOC =
+            "Table white list.<br>db1.table01,db1.table02</br>";
+
+
+    private String tableNameFormat;
+    private boolean tableFromHeader;
+    private int maxRetries;
+    private int retryBackoffMs;
+    private int batchSize;
+    private boolean deleteEnabled;
+    private boolean autoCreate;
+    private boolean autoEvolve;
+    private InsertMode insertMode;
+    public final PrimaryKeyMode pkMode;
+    private List<String> pkFields;
+    private Set<String> fieldsWhitelist;
+    private Set<String> tableWhitelist;
+    private String dialectName;
+    private TimeZone timeZone;
+    private EnumSet<TableType> tableTypes;
+
+    public JdbcSinkConfig(KeyValue config) {
+        super(config);
+        tableNameFormat = config.getString(TABLE_NAME_FORMAT,TABLE_NAME_FORMAT_DEFAULT).trim();
+        tableFromHeader=getBoolean(config,TABLE_NAME_FROM_HEADER,false);
+        batchSize = config.getInt(BATCH_SIZE);
+        deleteEnabled = getBoolean(config,DELETE_ENABLED,DELETE_ENABLED_DEFAULT);
+        maxRetries = config.getInt(MAX_RETRIES,MAX_RETRIES_DEFAULT);
+        retryBackoffMs =config.getInt(RETRY_BACKOFF_MS,RETRY_BACKOFF_MS_DEFAULT);
+        autoCreate = getBoolean(config,AUTO_CREATE,AUTO_CREATE_DEFAULT);
+        autoEvolve = getBoolean(config,AUTO_EVOLVE,AUTO_EVOLVE_DEFAULT);
+        if (Objects.nonNull(config.getString(INSERT_MODE))){
+            insertMode = InsertMode.valueOf(config.getString(INSERT_MODE,INSERT_MODE_DEFAULT).toUpperCase());
+        }
+
+        pkMode = PrimaryKeyMode.valueOf(config.getString(PK_MODE,PK_MODE_DEFAULT).toUpperCase());
+        pkFields = getList(config,PK_FIELDS);
+        dialectName = config.getString(DIALECT_NAME_CONFIG);
+        fieldsWhitelist = new HashSet<>(getList(config,FIELDS_WHITELIST));
+        // table white list
+        tableWhitelist =new HashSet<>(getList(config,TABLE_WHITE_LIST_CONFIG));
+        String dbTimeZone = config.getString(DB_TIMEZONE_CONFIG,DB_TIMEZONE_DEFAULT);
+        timeZone = TimeZone.getTimeZone(ZoneId.of(dbTimeZone));
+        tableTypes = TableType.parse(getList(config,TABLE_TYPES_CONFIG,TABLE_TYPES_DEFAULT));
+
+    }
+
+    public String getTableNameFormat() {
+        return tableNameFormat;
+    }
+
+    public boolean isTableFromHeader() {
+        return tableFromHeader;
+    }
+
+    public int getMaxRetries() {
+        return maxRetries;
+    }
+
+    public int getRetryBackoffMs() {
+        return retryBackoffMs;
+    }
+
+    public int getBatchSize() {
+        return batchSize;
+    }
+
+    public boolean isDeleteEnabled() {
+        return deleteEnabled;
+    }
+
+    public boolean isAutoCreate() {
+        return autoCreate;
+    }
+
+    public boolean isAutoEvolve() {
+        return autoEvolve;
+    }
+
+    public InsertMode getInsertMode() {
+        return insertMode;
+    }
+
+    public PrimaryKeyMode getPkMode() {
+        return pkMode;
+    }
+
+    public List<String> getPkFields() {
+        return pkFields;
+    }
+
+    public Set<String> getFieldsWhitelist() {
+        return fieldsWhitelist;
+    }
+
+    public Set<String> getTableWhitelist() {
+        return tableWhitelist;
+    }
+
+    public String getDialectName() {
+        return dialectName;
+    }
+
+    public TimeZone getTimeZone() {
+        return timeZone;
+    }
+
+    public EnumSet<TableType> getTableTypes() {
+        return tableTypes;
+    }
+
+    /**
+     * filter white table
+     * @param dbDialect
+     * @param tableId
+     * @return
+     */
+    public boolean filterWhiteTable(DatabaseDialect dbDialect, TableId tableId) {
+        // not filter table
+        if (tableWhitelist.isEmpty()){
+            return true;
+        }
+        for (String tableName: tableWhitelist){
+            TableId table=dbDialect.parseToTableId(tableName);
+            if(table.catalogName() != null && table.catalogName().equals(tableId.catalogName())){
+                return true;
+            }
+            if (table.tableName().equals(tableId.tableName())){
+                return true;
+            }
+        }
+        return false;
+    }
+
+    public Set<String> tableTypeNames() {
+        return tableTypes.stream().map(TableType::toString).collect(Collectors.toSet());
+    }
+
+}
diff --git a/connectors/rocketmq-connect-jdbc/src/main/java/org/apache/rocketmq/connect/jdbc/connector/JdbcSinkConnector.java b/connectors/rocketmq-connect-jdbc/src/main/java/org/apache/rocketmq/connect/jdbc/connector/JdbcSinkConnector.java
index 936811e..0fbbf8d 100644
--- a/connectors/rocketmq-connect-jdbc/src/main/java/org/apache/rocketmq/connect/jdbc/connector/JdbcSinkConnector.java
+++ b/connectors/rocketmq-connect-jdbc/src/main/java/org/apache/rocketmq/connect/jdbc/connector/JdbcSinkConnector.java
@@ -18,190 +18,73 @@
 package org.apache.rocketmq.connect.jdbc.connector;
 
 import io.openmessaging.KeyValue;
-import io.openmessaging.connector.api.Task;
-import io.openmessaging.connector.api.sink.SinkConnector;
-import java.util.ArrayList;
-import java.util.HashMap;
-import java.util.HashSet;
-import java.util.List;
-import java.util.Map;
-import java.util.Set;
-import java.util.concurrent.Executors;
-import java.util.concurrent.ScheduledExecutorService;
-import java.util.concurrent.ScheduledFuture;
-import java.util.concurrent.TimeUnit;
-import org.apache.commons.lang3.concurrent.BasicThreadFactory;
-import org.apache.rocketmq.client.exception.MQClientException;
-import org.apache.rocketmq.common.protocol.route.BrokerData;
-import org.apache.rocketmq.common.protocol.route.QueueData;
-import org.apache.rocketmq.common.protocol.route.TopicRouteData;
-import org.apache.rocketmq.connect.jdbc.common.CloneUtils;
-import org.apache.rocketmq.connect.jdbc.common.ConstDefine;
-import org.apache.rocketmq.connect.jdbc.common.Utils;
-import org.apache.rocketmq.connect.jdbc.config.Config;
-import org.apache.rocketmq.connect.jdbc.config.DataType;
-import org.apache.rocketmq.connect.jdbc.config.DbConnectorConfig;
-import org.apache.rocketmq.connect.jdbc.config.SinkDbConnectorConfig;
-import org.apache.rocketmq.connect.jdbc.config.TaskDivideConfig;
-import org.apache.rocketmq.connect.jdbc.config.TaskTopicInfo;
-import org.apache.rocketmq.remoting.RPCHook;
-import org.apache.rocketmq.tools.admin.DefaultMQAdminExt;
+import io.openmessaging.connector.api.component.connector.ConnectorContext;
+import io.openmessaging.connector.api.component.task.Task;
+import io.openmessaging.connector.api.component.task.sink.SinkConnector;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 
+import java.util.ArrayList;
+import java.util.List;
+
 
+/**
+ * jdbc sink connector
+ */
 public class JdbcSinkConnector extends SinkConnector {
     private static final Logger log = LoggerFactory.getLogger(JdbcSinkConnector.class);
-    private DbConnectorConfig dbConnectorConfig;
-    private volatile boolean configValid = false;
-    private ScheduledExecutorService executor;
-    private HashMap<String, Set<TaskTopicInfo>> topicRouteMap;
-
-    private DefaultMQAdminExt srcMQAdminExt;
-
-    private volatile boolean adminStarted;
-
-    private ScheduledFuture<?> listenerHandle;
-    public JdbcSinkConnector() {
-        topicRouteMap = new HashMap<>();
-        dbConnectorConfig = new SinkDbConnectorConfig();
-        executor = Executors.newSingleThreadScheduledExecutor(new BasicThreadFactory.Builder().namingPattern("JdbcSinkConnector-SinkWatcher-%d").daemon(true).build());
-    }
-
-    private synchronized void startMQAdminTools() {
-        if (!configValid || adminStarted) {
-            return;
-        }
-        RPCHook rpcHook = null;
-        this.srcMQAdminExt = new DefaultMQAdminExt(rpcHook);
-        this.srcMQAdminExt.setNamesrvAddr(((SinkDbConnectorConfig) this.dbConnectorConfig).getSrcNamesrvs());
-        this.srcMQAdminExt.setAdminExtGroup(Utils.createGroupName(ConstDefine.JDBC_CONNECTOR_ADMIN_PREFIX));
-        this.srcMQAdminExt.setInstanceName(Utils.createInstanceName(((SinkDbConnectorConfig) this.dbConnectorConfig).getSrcNamesrvs()));
-
-        try {
-            this.srcMQAdminExt.start();
-            log.info("RocketMQ srcMQAdminExt started");
-
-        } catch (MQClientException e) {
-            log.error("Replicator start failed for `srcMQAdminExt` exception.", e);
-        }
-
-        adminStarted = true;
-    }
+    private KeyValue connectConfig;
+    private ConnectorContext context;
 
     @Override
-    public String verifyAndSetConfig(KeyValue config) {
-        for (String requestKey : Config.REQUEST_CONFIG) {
-            if (!config.containsKey(requestKey)) {
-                return "Request config key: " + requestKey;
-            }
-        }
-        try {
-            this.dbConnectorConfig.validate(config);
-        } catch (IllegalArgumentException e) {
-            return e.getMessage();
-        }
-        this.configValid = true;
-
-        return "";
+    public void start(ConnectorContext context) {
+        this.context=context;
     }
 
+    /**
+     * Should invoke before start the connector.
+     * @param config
+     * @return error message
+     */
     @Override
-    public void start() {
-        startMQAdminTools();
-        startListener();
-    }
-
-    public void startListener() {
-        listenerHandle = executor.scheduleAtFixedRate(new Runnable() {
-            boolean first = true;
-            HashMap<String, Set<TaskTopicInfo>> origin = null;
-
-            @Override
-            public void run() {
-                buildRoute();
-                if (first) {
-                    origin = CloneUtils.clone(topicRouteMap);
-                    first = false;
-                }
-                if (!compare(origin, topicRouteMap)) {
-                    context.requestTaskReconfiguration();
-                    origin = CloneUtils.clone(topicRouteMap);
-                }
-            }
-        }, ((SinkDbConnectorConfig) dbConnectorConfig).getRefreshInterval(), ((SinkDbConnectorConfig) dbConnectorConfig).getRefreshInterval(), TimeUnit.SECONDS);
-    }
-
-    public boolean compare(Map<String, Set<TaskTopicInfo>> origin, Map<String, Set<TaskTopicInfo>> updated) {
-        if (origin.size() != updated.size()) {
-            return false;
-        }
-        for (Map.Entry<String, Set<TaskTopicInfo>> entry : origin.entrySet()) {
-            if (!updated.containsKey(entry.getKey())) {
-                return false;
-            }
-            Set<TaskTopicInfo> originTasks = entry.getValue();
-            Set<TaskTopicInfo> updateTasks = updated.get(entry.getKey());
-            if (originTasks.size() != updateTasks.size()) {
-                return false;
-            }
-
-            if (!originTasks.containsAll(updateTasks)) {
-                return false;
-            }
-        }
-
-        return true;
-    }
-
-    public void buildRoute() {
-        String srcCluster = ((SinkDbConnectorConfig) this.dbConnectorConfig).getSrcCluster();
-        try {
-            for (String topic : ((SinkDbConnectorConfig) this.dbConnectorConfig).getWhiteList()) {
-
-                // different from BrokerData with cluster field, which can ensure the brokerData is from expected cluster.
-                // QueueData use brokerName as unique info on cluster of rocketmq. so when we want to get QueueData of
-                // expected cluster, we should get brokerNames of expected cluster, and then filter queueDatas.
-                List<BrokerData> brokerList = Utils.examineBrokerData(this.srcMQAdminExt, topic, srcCluster);
-                Set<String> brokerNameSet = new HashSet<String>();
-                for (BrokerData b : brokerList) {
-                    brokerNameSet.add(b.getBrokerName());
-                }
-
-                TopicRouteData topicRouteData = srcMQAdminExt.examineTopicRouteInfo(topic);
-                if (!topicRouteMap.containsKey(topic)) {
-                    topicRouteMap.put(topic, new HashSet<>(16));
-                }
-                for (QueueData qd : topicRouteData.getQueueDatas()) {
-                    if (brokerNameSet.contains(qd.getBrokerName())) {
-                        for (int i = 0; i < qd.getReadQueueNums(); i++) {
-                            TaskTopicInfo taskTopicInfo = new TaskTopicInfo(topic, qd.getBrokerName(), i, null);
-                            topicRouteMap.get(topic).add(taskTopicInfo);
-                        }
-                    }
-                }
-            }
-        } catch (Exception e) {
-            log.error("Fetch topic list error.", e);
-        } finally {
-            srcMQAdminExt.shutdown();
-        }
+    public void validate(KeyValue config) {
+        // do validate config
     }
 
-
+    /**
+     * Init the component
+     *
+     * @param config
+     */
     @Override
-    public void stop() {
-        listenerHandle.cancel(true);
+    public void init(KeyValue config) {
+        this.connectConfig=config;
     }
 
     @Override
-    public void pause() {
-
-    }
+    public void stop() {}
 
     @Override
-    public void resume() {
+    public void pause() {}
 
+    @Override
+    public void resume() {}
+
+    /**
+     * Returns a set of configurations for Tasks based on the current configuration,
+     * producing at most count configurations.
+     *
+     * @param maxTasks maximum number of configurations to generate
+     * @return configurations for Tasks
+     */
+    @Override
+    public List<KeyValue> taskConfigs(int maxTasks) {
+        log.info("Starting task config !!! ");
+        List<KeyValue> configs = new ArrayList<>();
+        for (int i =0 ; i< maxTasks; i++){
+            configs.add(this.connectConfig);
+        }
+        return configs;
     }
 
     @Override
@@ -209,30 +92,4 @@ public class JdbcSinkConnector extends SinkConnector {
         return JdbcSinkTask.class;
     }
 
-    @Override
-    public List<KeyValue> taskConfigs() {
-        log.info("List.start");
-        if (!configValid) {
-            return new ArrayList<KeyValue>();
-        }
-
-        startMQAdminTools();
-
-        buildRoute();
-
-        TaskDivideConfig tdc = new TaskDivideConfig(
-                this.dbConnectorConfig.getDbUrl(),
-                this.dbConnectorConfig.getDbPort(),
-                this.dbConnectorConfig.getDbUserName(),
-                this.dbConnectorConfig.getDbPassword(),
-                this.dbConnectorConfig.getConverter(),
-                DataType.COMMON_MESSAGE.ordinal(),
-                this.dbConnectorConfig.getTaskParallelism(),
-                this.dbConnectorConfig.getMode()
-        );
-
-        ((SinkDbConnectorConfig) this.dbConnectorConfig).setTopicRouteMap(topicRouteMap);
-
-        return this.dbConnectorConfig.getTaskDivideStrategy().divide(this.dbConnectorConfig, tdc);
-    }
 }
diff --git a/connectors/rocketmq-connect-jdbc/src/main/java/org/apache/rocketmq/connect/jdbc/connector/JdbcSinkTask.java b/connectors/rocketmq-connect-jdbc/src/main/java/org/apache/rocketmq/connect/jdbc/connector/JdbcSinkTask.java
index 96b4cbb..4fd844a 100644
--- a/connectors/rocketmq-connect-jdbc/src/main/java/org/apache/rocketmq/connect/jdbc/connector/JdbcSinkTask.java
+++ b/connectors/rocketmq-connect-jdbc/src/main/java/org/apache/rocketmq/connect/jdbc/connector/JdbcSinkTask.java
@@ -18,134 +18,131 @@
 package org.apache.rocketmq.connect.jdbc.connector;
 
 
-import com.alibaba.fastjson.JSONObject;
 import io.openmessaging.KeyValue;
-import io.openmessaging.connector.api.common.QueueMetaData;
-import io.openmessaging.connector.api.data.EntryType;
-import io.openmessaging.connector.api.data.Field;
-import io.openmessaging.connector.api.data.Schema;
-import io.openmessaging.connector.api.data.SinkDataEntry;
-import io.openmessaging.connector.api.sink.SinkTask;
-import org.apache.rocketmq.connect.jdbc.config.Config;
-import org.apache.rocketmq.connect.jdbc.common.DBUtils;
-import org.apache.rocketmq.connect.jdbc.config.ConfigUtil;
+import io.openmessaging.connector.api.component.task.sink.SinkTask;
+import io.openmessaging.connector.api.component.task.sink.SinkTaskContext;
+import io.openmessaging.connector.api.data.ConnectRecord;
+import io.openmessaging.connector.api.errors.ConnectException;
+import io.openmessaging.connector.api.errors.RetriableException;
+import org.apache.rocketmq.connect.jdbc.dialect.DatabaseDialect;
+import org.apache.rocketmq.connect.jdbc.dialect.DatabaseDialectFactory;
+import org.apache.rocketmq.connect.jdbc.schema.db.DbStructure;
 import org.apache.rocketmq.connect.jdbc.sink.Updater;
+import org.apache.rocketmq.connect.jdbc.exception.TableAlterOrCreateException;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 
-import javax.sql.DataSource;
-import java.sql.Connection;
-import java.util.Collection;
-import java.util.HashMap;
+import java.sql.SQLException;
 import java.util.List;
-import java.util.Map;
-import java.util.concurrent.BlockingQueue;
-import java.util.concurrent.LinkedBlockingQueue;
-import java.util.concurrent.TimeUnit;
 
+/**
+ * jdbc sink task
+ */
 public class JdbcSinkTask extends SinkTask {
 
     private static final Logger log = LoggerFactory.getLogger(JdbcSinkTask.class);
-
-    private Config config;
-    private DataSource dataSource;
-    private Connection connection;
+    private SinkTaskContext context;
+    private KeyValue originalConfig;
+    private JdbcSinkConfig config;
+    private DatabaseDialect dialect;
+    int remainingRetries;
     private Updater updater;
-    private BlockingQueue<Updater> tableQueue = new LinkedBlockingQueue<Updater>();
 
-    public JdbcSinkTask() {
-        this.config = new Config();
-    }
 
+    /**
+     * Put the records to the sink
+     * @param records
+     */
     @Override
-    public void put(Collection<SinkDataEntry> sinkDataEntries) {
+    public void put(List<ConnectRecord> records) throws ConnectException {
+        if (records.isEmpty()) {
+            return;
+        }
+        final int recordsCount = records.size();
+        log.debug("Received {} records.", recordsCount);
         try {
-            if (tableQueue.size() > 1) {
-                updater = tableQueue.poll(1000, TimeUnit.MILLISECONDS);
-            } else {
-                updater = tableQueue.peek();
+            updater.write(records);
+        } catch (TableAlterOrCreateException tace) {
+            throw tace;
+        } catch (SQLException sqle) {
+            SQLException sqlAllMessagesException = getAllMessagesException(sqle);
+            if (remainingRetries > 0) {
+                updater.closeQuietly();
+                init(originalConfig);
+                remainingRetries--;
+                throw new RetriableException(sqlAllMessagesException);
             }
+        }
+        remainingRetries = config.getMaxRetries();
+    }
 
-            for (SinkDataEntry record : sinkDataEntries) {
-                Map<Field, Object[]> fieldMap = new HashMap<>();
-                Object[] payloads = record.getPayload();
-                Schema schema = record.getSchema();
-                EntryType entryType = record.getEntryType();
-                String tableName = schema.getName();
-                String dbName = schema.getDataSource();
-                List<Field> fields = schema.getFields();
-                Boolean parseError = false;
-                if (!fields.isEmpty()) {
-                    for (Field field : fields) {
-                        Object fieldValue = payloads[field.getIndex()];
-                        Object[] value = JSONObject.parseArray((String)fieldValue).toArray();
-                        if (value.length == 2) {
-                            fieldMap.put(field, value);
-                        } else {
-                            log.error("parseArray error, fieldValue:{}", fieldValue);
-                            parseError = true;
-                        }
-                    }
-                }
-                if (!parseError) {
-                    Boolean isSuccess = updater.push(dbName, tableName, fieldMap, entryType);
-                    if (!isSuccess) {
-                        log.error("push data error, dbName:{}, tableName:{}, entryType:{}, fieldMap:{}", dbName, tableName, fieldMap, entryType);
-                    }
-                }
-            }
-        } catch (Exception e) {
-            log.error("put sinkDataEntries error, {}", e);
+    private SQLException getAllMessagesException(SQLException sqle) {
+        String sqleAllMessages = "Exception chain:" + System.lineSeparator();
+        for (Throwable e : sqle) {
+            sqleAllMessages += e + System.lineSeparator();
         }
+        SQLException sqlAllMessagesException = new SQLException(sqleAllMessages);
+        sqlAllMessagesException.setNextException(sqle);
+        return sqlAllMessagesException;
     }
 
+
     @Override
-    public void commit(Map<QueueMetaData, Long> map) {
+    public void start(SinkTaskContext context) {
+        this.context=context;
+    }
 
+    /**
+     * Should invoke before start the connector.
+     *
+     * @param config
+     * @return error message
+     */
+    @Override
+    public void validate(KeyValue config) {
+        // to do nothing
     }
 
+    /**
+     * Init the component
+     * @param keyValue
+     */
     @Override
-    public void start(KeyValue props) {
-        try {
-            ConfigUtil.load(props, this.config);
-            dataSource = DBUtils.initDataSource(config);
-            connection = dataSource.getConnection();
-            log.info("init data source success");
-        } catch (Exception e) {
-            log.error("Cannot start Jdbc Sink Task because of configuration error{}", e);
-        }
-        String mode = config.getMode();
-        if (mode.equals("bulk")) {
-            Updater updater = new Updater(config, connection);
-            try {
-                updater.start();
-                tableQueue.add(updater);
-            } catch (Exception e) {
-                log.error("fail to start updater{}", e);
-            }
+    public void init(KeyValue keyValue) {
+        originalConfig = keyValue;
+        config = new JdbcSinkConfig(keyValue);
+        remainingRetries = config.getMaxRetries();
+        if (config.getDialectName() != null && !config.getDialectName().trim().isEmpty()) {
+            dialect = DatabaseDialectFactory.create(config.getDialectName(), config);
+        } else {
+            dialect = DatabaseDialectFactory.findDialectFor(config.getConnectionDbUrl(), config);
         }
+        final DbStructure dbStructure = new DbStructure(dialect);
+        log.info("Initializing writer using SQL dialect: {}", dialect.getClass().getSimpleName());
+        this.updater = new Updater(config, dialect, dbStructure);
     }
 
     @Override
     public void stop() {
+        log.info("Stopping task");
         try {
-            if (connection != null){
-                connection.close();
-                log.info("jdbc sink task connection is closed.");
+            updater.closeQuietly();
+        } finally {
+            try {
+                if (dialect != null) {
+                    dialect.close();
+                }
+            } catch (Throwable t) {
+                log.warn("Error while closing the {} dialect: ", dialect.name(), t);
+            } finally {
+                dialect = null;
             }
-        } catch (Throwable e) {
-            log.warn("sink task stop error while closing connection to {}", "jdbc", e);
         }
     }
 
     @Override
-    public void pause() {
-
-    }
-
+    public void pause() {}
     @Override
-    public void resume() {
-
-    }
+    public void resume() {}
 
 }
diff --git a/connectors/rocketmq-connect-jdbc/src/main/java/org/apache/rocketmq/connect/jdbc/connector/JdbcSourceConfig.java b/connectors/rocketmq-connect-jdbc/src/main/java/org/apache/rocketmq/connect/jdbc/connector/JdbcSourceConfig.java
new file mode 100644
index 0000000..bc3028f
--- /dev/null
+++ b/connectors/rocketmq-connect-jdbc/src/main/java/org/apache/rocketmq/connect/jdbc/connector/JdbcSourceConfig.java
@@ -0,0 +1,335 @@
+/*
+ * 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.rocketmq.connect.jdbc.connector;
+
+import io.openmessaging.KeyValue;
+import org.apache.rocketmq.connect.jdbc.config.AbstractConfig;
+import org.apache.rocketmq.connect.jdbc.util.NumericMapping;
+import org.apache.rocketmq.connect.jdbc.util.TableType;
+
+import java.time.ZoneId;
+import java.util.EnumSet;
+import java.util.HashSet;
+import java.util.List;
+import java.util.Set;
+import java.util.TimeZone;
+
+/**
+ * jdbc source config
+ */
+public class JdbcSourceConfig extends AbstractConfig {
+    /**
+     * table load mode
+     */
+    public enum TableLoadMode {
+        MODE_BULK("bulk"),
+        MODE_TIMESTAMP("timestamp"),
+        MODE_INCREMENTING( "incrementing"),
+        MODE_TIMESTAMP_INCREMENTING("timestamp+incrementing");
+        private String name;
+        TableLoadMode(String name) {
+            this.name=name;
+        }
+
+        public String getName() {
+            return name;
+        }
+
+        public void setName(String name) {
+            this.name = name;
+        }
+        public static TableLoadMode findTableLoadModeByName(String name){
+            for (TableLoadMode mode : TableLoadMode.values()){
+                if (mode.getName().equals(name)){
+                    return mode;
+                }
+            }
+            throw new IllegalArgumentException("Unsupports mode "+ name);
+        }
+    }
+
+    //source poll interval ms
+    public static final String POLL_INTERVAL_MS_CONFIG = "poll.interval.ms";
+    private static final String POLL_INTERVAL_MS_DOC = "Frequency in ms to poll for new data in each table.";
+    public static final int POLL_INTERVAL_MS_DEFAULT = 5000;
+    // batch max rows
+
+    public static final String BATCH_MAX_ROWS_CONFIG = "batch.max.rows";
+    public static final int BATCH_MAX_ROWS_DEFAULT = 100;
+
+    // numeric precision mapping
+    public static final String NUMERIC_PRECISION_MAPPING_CONFIG = "numeric.precision.mapping";
+    public static final boolean NUMERIC_PRECISION_MAPPING_DEFAULT = false;
+
+    // numeric mapping
+    public static final String NUMERIC_MAPPING_CONFIG = "numeric.mapping";
+    public static final String NUMERIC_MAPPING_DEFAULT = null;
+
+    // dialect name
+
+    public static final String DIALECT_NAME_CONFIG = "dialect.name";
+    public static final String DIALECT_NAME_DEFAULT = "";
+
+    // table load mode
+    public static final String MODE_CONFIG = "mode";
+
+    // incrementing column name
+    public static final String INCREMENTING_COLUMN_NAME_CONFIG = "incrementing.column.name";
+    public static final String INCREMENTING_COLUMN_NAME_DEFAULT = "";
+
+    // timestamp column name
+    public static final String TIMESTAMP_COLUMN_NAME_CONFIG = "timestamp.column.name";
+    public static final String TIMESTAMP_COLUMN_NAME_DEFAULT = "";
+
+
+    // timestamp initial
+    public static final String TIMESTAMP_INITIAL_CONFIG = "timestamp.initial";
+    public static final Long TIMESTAMP_INITIAL_DEFAULT = null;
+    public static final long TIMESTAMP_INITIAL_CURRENT = Long.valueOf(-1);
+
+    // Metadata Change Monitoring Interval (ms)
+    public static final String TABLE_POLL_INTERVAL_MS_CONFIG = "table.poll.interval.ms";
+    public static final long TABLE_POLL_INTERVAL_MS_DEFAULT = 60 * 1000;
+
+    // table white list
+    public static final String TABLE_WHITELIST_CONFIG = "table.whitelist";
+    public static final String TABLE_WHITELIST_DEFAULT = "";
+
+    // table black list
+    public static final String TABLE_BLACKLIST_CONFIG = "table.blacklist";
+    public static final String TABLE_BLACKLIST_DEFAULT = "";
+
+    public static final String SCHEMA_PATTERN_CONFIG = "schema.pattern";
+    private static final String SCHEMA_PATTERN_DOC =
+            "Schema pattern to fetch table metadata from the database.\n"
+                    + "  * ``\"\"`` retrieves those without a schema.\n"
+                    + "  * null (default) indicates that the schema name is not used to narrow the search and "
+                    + "that all table metadata is fetched, regardless of the schema.";
+    private static final String SCHEMA_PATTERN_DISPLAY = "Schema pattern";
+    public static final String SCHEMA_PATTERN_DEFAULT = null;
+
+    public static final String CATALOG_PATTERN_CONFIG = "catalog.pattern";
+    private static final String CATALOG_PATTERN_DOC =
+            "Catalog pattern to fetch table metadata from the database.\n"
+                    + "  * ``\"\"`` retrieves those without a catalog \n"
+                    + "  * null (default) indicates that the schema name is not used to narrow the search and "
+                    + "that all table metadata is fetched, regardless of the catalog.";
+    private static final String CATALOG_PATTERN_DISPLAY = "Schema pattern";
+    public static final String CATALOG_PATTERN_DEFAULT = null;
+
+    public static final String QUERY_CONFIG = "query";
+    public static final String QUERY_DEFAULT = "";
+
+    public static final String TOPIC_PREFIX_CONFIG = "topic.prefix";
+    private static final String TOPIC_PREFIX_DOC =
+            "Prefix to prepend to table names to generate the name of the Kafka topic to publish data "
+                    + "to, or in the case of a custom query, the full name of the topic to publish to.";
+    private static final String TOPIC_PREFIX_DISPLAY = "Topic Prefix";
+
+    /**
+     * validate non null
+     */
+    public static final String VALIDATE_NON_NULL_CONFIG = "validate.non.null";
+    private static final String VALIDATE_NON_NULL_DOC =
+            "By default, the JDBC connector will validate that all incrementing and timestamp tables "
+                    + "have NOT NULL set for the columns being used as their ID/timestamp. If the tables don't,"
+                    + " JDBC connector will fail to start. Setting this to false will disable these checks.";
+    public static final boolean VALIDATE_NON_NULL_DEFAULT = true;
+    private static final String VALIDATE_NON_NULL_DISPLAY = "Validate Non Null";
+
+    public static final String TIMESTAMP_DELAY_INTERVAL_MS_CONFIG = "timestamp.delay.interval.ms";
+    private static final String TIMESTAMP_DELAY_INTERVAL_MS_DOC =
+            "How long to wait after a row with certain timestamp appears before we include it in the "
+                    + "result. You may choose to add some delay to allow transactions with earlier timestamp to"
+                    + " complete. The first execution will fetch all available records (i.e. starting at "
+                    + "timestamp 0) until current time minus the delay. Every following execution will get data"
+                    + " from the last time we fetched until current time minus the delay.";
+    public static final long TIMESTAMP_DELAY_INTERVAL_MS_DEFAULT = 0;
+    private static final String TIMESTAMP_DELAY_INTERVAL_MS_DISPLAY = "Delay Interval (ms)";
+
+
+    public static final String DB_TIMEZONE_CONFIG = "db.timezone";
+    public static final String DB_TIMEZONE_DEFAULT = "UTC";
+    private static final String DB_TIMEZONE_CONFIG_DOC =
+            "Name of the JDBC timezone used in the connector when "
+                    + "querying with time-based criteria. Defaults to UTC.";
+    private static final String DB_TIMEZONE_CONFIG_DISPLAY = "DB time zone";
+
+    public static final String TABLE_TYPE_DEFAULT = "TABLE";
+    public static final String TABLE_TYPE_CONFIG = "table.types";
+    private static final String TABLE_TYPE_DOC =
+            "By default, the JDBC connector will only detect tables with type TABLE from the source "
+                    + "Database. This config allows a command separated list of table types to extract. Options"
+                    + " include:\n"
+                    + "  * TABLE\n"
+                    + "  * VIEW\n"
+                    + "  * SYSTEM TABLE\n"
+                    + "  * GLOBAL TEMPORARY\n"
+                    + "  * LOCAL TEMPORARY\n"
+                    + "  * ALIAS\n"
+                    + "  * SYNONYM\n"
+                    + "  In most cases it only makes sense to have either TABLE or VIEW.";
+
+
+    // The suffix to add at offset partition's key
+    public static final String OFFSET_SUFFIX_CONFIG = "offset.suffix";
+    public static final String OFFSET_SUFFIX_DEFAULT = "";
+    public static final String OFFSET_SUFFIX_DOC = "Add this suffix to offset partition's key. " +
+            "So every time when create connector can use new offset";
+
+    // query suffix
+    public static final String QUERY_SUFFIX_CONFIG = "query.suffix";
+    public static final String QUERY_SUFFIX_DEFAULT = "";
+    public static final String QUERY_SUFFIX_DOC = "Suffix to append at the end of the generated query.";
+
+    private int pollIntervalMs;
+    private int batchMaxRows;
+    private Boolean numericPrecisionMapping;
+    private String numericMapping;
+    private String dialectName;
+    private String mode;
+    private String incrementingColumnName;
+    private List<String> timestampColumnNames;
+    private long timestampDelayIntervalMs;
+    private Long timestampInitial;
+    private Set<String> tableWhitelist;
+    private Set<String> tableBlacklist;
+    private String schemaPattern;
+    private String catalogPattern;
+    private String query;
+    private String topicPrefix;
+    private boolean validateNonNull;
+    private EnumSet<TableType> tableTypes;
+    private TimeZone timeZone;
+    private String offsetSuffix;
+    private String querySuffix;
+    public JdbcSourceConfig(KeyValue config) {
+        super(config);
+        this.pollIntervalMs = config.getInt(POLL_INTERVAL_MS_CONFIG,POLL_INTERVAL_MS_DEFAULT);
+        this.batchMaxRows = config.getInt(BATCH_MAX_ROWS_CONFIG,BATCH_MAX_ROWS_DEFAULT);
+        this.numericPrecisionMapping = getBoolean(config,NUMERIC_PRECISION_MAPPING_CONFIG,NUMERIC_PRECISION_MAPPING_DEFAULT);
+        this.numericMapping = config.getString(NUMERIC_MAPPING_CONFIG,NUMERIC_MAPPING_DEFAULT);
+        this.dialectName = config.getString(DIALECT_NAME_CONFIG, DIALECT_NAME_DEFAULT );
+        this.mode = config.getString(MODE_CONFIG);
+        this.incrementingColumnName = config.getString(INCREMENTING_COLUMN_NAME_CONFIG);
+        this.timestampColumnNames = getList(config,TIMESTAMP_COLUMN_NAME_CONFIG);
+        timestampDelayIntervalMs=config.getLong(TIMESTAMP_DELAY_INTERVAL_MS_CONFIG);
+        this.timestampInitial = config.containsKey(TIMESTAMP_INITIAL_CONFIG) ? config.getLong(TIMESTAMP_INITIAL_CONFIG) : TIMESTAMP_INITIAL_DEFAULT;
+        this.tableWhitelist = new HashSet<>(getList(config,TABLE_WHITELIST_CONFIG));
+        this.tableBlacklist = new HashSet<>(getList(config,TABLE_BLACKLIST_CONFIG));
+        this.schemaPattern = config.getString(SCHEMA_PATTERN_CONFIG);
+        this.catalogPattern = config.getString(CATALOG_PATTERN_CONFIG);
+        this.query = config.getString(QUERY_CONFIG);
+        this.topicPrefix = config.getString(TOPIC_PREFIX_CONFIG);
+        this.validateNonNull = getBoolean(config,VALIDATE_NON_NULL_CONFIG,VALIDATE_NON_NULL_DEFAULT);
+        tableTypes = TableType.parse(getList(config,TABLE_TYPE_CONFIG,TABLE_TYPE_DEFAULT)) ;
+        String dbTimeZone =config.getString(DB_TIMEZONE_CONFIG,DB_TIMEZONE_DEFAULT);
+        this.timeZone=TimeZone.getTimeZone(ZoneId.of(dbTimeZone));
+        this.querySuffix=config.getString(QUERY_SUFFIX_CONFIG,QUERY_SUFFIX_DEFAULT);
+        this.offsetSuffix=config.getString(OFFSET_SUFFIX_CONFIG,OFFSET_SUFFIX_DEFAULT);
+    }
+
+
+    public NumericMapping numericMapping() {
+        return NumericMapping.get(this);
+    }
+
+    public int getPollIntervalMs() {
+        return pollIntervalMs;
+    }
+
+    public int getBatchMaxRows() {
+        return batchMaxRows;
+    }
+
+    public Boolean getNumericPrecisionMapping() {
+        return numericPrecisionMapping;
+    }
+
+    public String getNumericMapping() {
+        return numericMapping;
+    }
+
+    public String getDialectName() {
+        return dialectName;
+    }
+
+    public String getMode() {
+        return mode;
+    }
+
+    public String getIncrementingColumnName() {
+        return incrementingColumnName;
+    }
+
+    public List<String> getTimestampColumnNames() {
+        return timestampColumnNames;
+    }
+
+    public Long getTimestampInitial() {
+        return timestampInitial;
+    }
+
+    public Set<String> getTableWhitelist() {
+        return tableWhitelist;
+    }
+
+    public Set<String> getTableBlacklist() {
+        return tableBlacklist;
+    }
+
+    public String getSchemaPattern() {
+        return schemaPattern;
+    }
+
+    public String getCatalogPattern() {
+        return catalogPattern;
+    }
+
+    public String getQuery() {
+        return query;
+    }
+
+    public String getTopicPrefix() {
+        return topicPrefix;
+    }
+
+    public boolean isValidateNonNull() {
+        return validateNonNull;
+    }
+
+    public EnumSet<TableType> getTableTypes() {
+        return tableTypes;
+    }
+
+    public TimeZone getTimeZone() {
+        return timeZone;
+    }
+
+    public long getTimestampDelayIntervalMs() {
+        return timestampDelayIntervalMs;
+    }
+
+
+    public String getOffsetSuffix() {
+        return offsetSuffix;
+    }
+
+    public String getQuerySuffix() {
+        return querySuffix;
+    }
+}
diff --git a/connectors/rocketmq-connect-jdbc/src/main/java/org/apache/rocketmq/connect/jdbc/connector/JdbcSourceConnector.java b/connectors/rocketmq-connect-jdbc/src/main/java/org/apache/rocketmq/connect/jdbc/connector/JdbcSourceConnector.java
index ee62133..f7f55eb 100644
--- a/connectors/rocketmq-connect-jdbc/src/main/java/org/apache/rocketmq/connect/jdbc/connector/JdbcSourceConnector.java
+++ b/connectors/rocketmq-connect-jdbc/src/main/java/org/apache/rocketmq/connect/jdbc/connector/JdbcSourceConnector.java
@@ -17,89 +17,97 @@
 
 package org.apache.rocketmq.connect.jdbc.connector;
 
-import java.util.ArrayList;
 import java.util.List;
 
-import org.apache.rocketmq.connect.jdbc.config.*;
+import com.beust.jcommander.internal.Lists;
+import io.openmessaging.connector.api.component.task.Task;
+import io.openmessaging.connector.api.component.task.source.SourceConnector;
+import io.openmessaging.internal.DefaultKeyValue;
+import org.apache.commons.lang3.StringUtils;
+import org.apache.rocketmq.connect.jdbc.util.ConnectorGroupUtils;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 
 import io.openmessaging.KeyValue;
-import io.openmessaging.connector.api.Task;
-import io.openmessaging.connector.api.source.SourceConnector;
 
 public class JdbcSourceConnector extends SourceConnector {
     private static final Logger log = LoggerFactory.getLogger(JdbcSourceConnector.class);
-    private DbConnectorConfig dbConnectorConfig;
-    private volatile boolean configValid = false;
+    private JdbcSourceConfig jdbcSourceConfig;
+    private KeyValue originalConfig;
 
-    public JdbcSourceConnector() {
-        dbConnectorConfig = new SourceDbConnectorConfig();
-    }
 
+    /**
+     * Should invoke before start the connector.
+     *
+     * @param config
+     * @return error message
+     */
     @Override
-    public String verifyAndSetConfig(KeyValue config) {
-
-        log.info("JdbcSourceConnector verifyAndSetConfig enter");
-        for (String requestKey : Config.REQUEST_CONFIG) {
-
-            if (!config.containsKey(requestKey)) {
-                return "Request config key: " + requestKey;
-            }
-        }
-        try {
-            this.dbConnectorConfig.validate(config);
-        } catch (IllegalArgumentException e) {
-            return e.getMessage();
-        }
-        this.configValid = true;
-
-        return "";
+    public void validate(KeyValue config) {
+        jdbcSourceConfig = new JdbcSourceConfig(config);
+        // validate config
     }
 
+    /**
+     * Init the component
+     *
+     * @param config
+     */
     @Override
-    public void start() {
-        log.info("JdbcSourceConnector start");
+    public void init(KeyValue config) {
+        if (config.containsKey("connect-topicname")){
+            config.put("connect-topicname","");
+        }
+        originalConfig = config;
     }
 
     @Override
     public void stop() {
-
     }
 
     @Override
     public void pause() {
-
     }
 
     @Override
     public void resume() {
-
     }
 
+    /**
+     * Returns a set of configurations for Tasks based on the current configuration,
+     * producing at most count configurations.
+     *
+     * @param maxTasks maximum number of configurations to generate
+     * @return configurations for Tasks
+     */
     @Override
-    public Class<? extends Task> taskClass() {
-        return JdbcSourceTask.class;
+    public List<KeyValue> taskConfigs(int maxTasks) {
+
+        log.info("Connector task config divide["+maxTasks+"]" );
+        List<KeyValue> keyValues=Lists.newArrayList();
+        List<String> tables = Lists.newArrayList();
+        log.info("Connector table white list["+jdbcSourceConfig.getTableWhitelist()+"]" );
+        jdbcSourceConfig.getTableWhitelist().forEach(table->{
+            tables.add(table);
+        });
+        maxTasks = tables.size() > maxTasks ? maxTasks: tables.size();
+
+        List<List<String>> tablesGrouped =
+                ConnectorGroupUtils.groupPartitions(tables, maxTasks);
+        for (List<String> tableGroup:tablesGrouped) {
+            KeyValue keyValue = new DefaultKeyValue();
+            for (String key: originalConfig.keySet()){
+                keyValue.put(key,originalConfig.getString(key));
+            }
+            keyValue.put(JdbcSourceTaskConfig.TABLES_CONFIG,StringUtils.join(tableGroup,","));
+            keyValues.add(keyValue);
+        }
+        return keyValues;
     }
 
     @Override
-    public List<KeyValue> taskConfigs() {
-        log.info("List.start");
-        if (!configValid) {
-            return new ArrayList<KeyValue>();
-        }
-
-        TaskDivideConfig tdc = new TaskDivideConfig(
-                this.dbConnectorConfig.getDbUrl(),
-                this.dbConnectorConfig.getDbPort(),
-                this.dbConnectorConfig.getDbUserName(),
-                this.dbConnectorConfig.getDbPassword(),
-                this.dbConnectorConfig.getConverter(),
-                DataType.COMMON_MESSAGE.ordinal(),
-                this.dbConnectorConfig.getTaskParallelism(),
-                this.dbConnectorConfig.getMode()
-        );
-        return this.dbConnectorConfig.getTaskDivideStrategy().divide(this.dbConnectorConfig, tdc);
+    public Class<? extends Task> taskClass() {
+        return JdbcSourceTask.class;
     }
 
 }
diff --git a/connectors/rocketmq-connect-jdbc/src/main/java/org/apache/rocketmq/connect/jdbc/connector/JdbcSourceTask.java b/connectors/rocketmq-connect-jdbc/src/main/java/org/apache/rocketmq/connect/jdbc/connector/JdbcSourceTask.java
index f36623f..6ea61b1 100644
--- a/connectors/rocketmq-connect-jdbc/src/main/java/org/apache/rocketmq/connect/jdbc/connector/JdbcSourceTask.java
+++ b/connectors/rocketmq-connect-jdbc/src/main/java/org/apache/rocketmq/connect/jdbc/connector/JdbcSourceTask.java
@@ -18,166 +18,314 @@
 
 package org.apache.rocketmq.connect.jdbc.connector;
 
-import io.openmessaging.connector.api.source.SourceTask;
-
-import java.nio.ByteBuffer;
-import java.nio.charset.StandardCharsets;
-import java.sql.Connection;
-import java.util.*;
-import java.util.concurrent.BlockingQueue;
-import java.util.concurrent.LinkedBlockingQueue;
-import java.util.concurrent.TimeUnit;
-import java.util.concurrent.atomic.AtomicInteger;
-
-import org.apache.rocketmq.connect.jdbc.common.ConstDefine;
-import org.apache.rocketmq.connect.jdbc.config.Config;
-import org.apache.rocketmq.connect.jdbc.common.DBUtils;
-import org.apache.rocketmq.connect.jdbc.config.ConfigUtil;
-import org.apache.rocketmq.connect.jdbc.schema.Table;
-import org.apache.rocketmq.connect.jdbc.source.Querier;
-import org.apache.rocketmq.connect.jdbc.source.TimestampIncrementingQuerier;
-import org.apache.rocketmq.connect.jdbc.schema.column.*;
+import io.openmessaging.KeyValue;
+import io.openmessaging.connector.api.component.task.source.SourceTask;
+import io.openmessaging.connector.api.component.task.source.SourceTaskContext;
+import io.openmessaging.connector.api.data.*;
+import org.apache.commons.lang3.StringUtils;
+import org.apache.rocketmq.connect.jdbc.dialect.DatabaseDialect;
+import org.apache.rocketmq.connect.jdbc.dialect.DatabaseDialectFactory;
+import org.apache.rocketmq.connect.jdbc.dialect.provider.CachedConnectionProvider;
+import org.apache.rocketmq.connect.jdbc.source.offset.SourceOffsetCompute;
+import org.apache.rocketmq.connect.jdbc.source.querier.BulkQuerier;
+import org.apache.rocketmq.connect.jdbc.source.querier.Querier;
+import org.apache.rocketmq.connect.jdbc.source.querier.TimestampIncrementingQuerier;
 
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 
-import com.alibaba.fastjson.JSON;
-
-import io.openmessaging.KeyValue;
-import io.openmessaging.connector.api.data.EntryType;
-import io.openmessaging.connector.api.data.Schema;
-import io.openmessaging.connector.api.data.SourceDataEntry;
-
-import com.alibaba.fastjson.JSONObject;
-import io.openmessaging.connector.api.data.DataEntryBuilder;
-import io.openmessaging.connector.api.data.Field;
-
-import javax.sql.DataSource;
+import java.sql.Connection;
+import java.sql.SQLException;
+import java.util.ArrayList;
+import java.util.Collections;
+import java.util.List;
+import java.util.Map;
+import java.util.TimeZone;
+import java.util.concurrent.BlockingQueue;
+import java.util.concurrent.LinkedBlockingQueue;
+import java.util.concurrent.atomic.AtomicBoolean;
+import java.util.function.Function;
+import java.util.stream.Collectors;
 
+/**
+ * jdbc source task
+ */
 public class JdbcSourceTask extends SourceTask {
 
     private static final Logger log = LoggerFactory.getLogger(JdbcSourceTask.class);
+    private static final int CONSECUTIVE_EMPTY_RESULTS_BEFORE_RETURN = 3;
 
-    private Config config;
-
-    private DataSource dataSource;
-
-    private Connection connection;
+    private JdbcSourceTaskConfig config;
+    private DatabaseDialect dialect;
+    private CachedConnectionProvider cachedConnectionProvider;
 
     BlockingQueue<Querier> tableQueue = new LinkedBlockingQueue<Querier>();
-    static final String INCREMENTING_FIELD = "incrementing";
-    static final String TIMESTAMP_FIELD = "timestamp";
-    private Querier querier;
-
-    public JdbcSourceTask() {
-        this.config = new Config();
-    }
+    private final AtomicBoolean running = new AtomicBoolean(false);
 
     @Override
-    public Collection<SourceDataEntry> poll() {
-        List<SourceDataEntry> res = new ArrayList<>();
-        try {
-            if (tableQueue.size() > 1)
-                querier = tableQueue.poll(1000, TimeUnit.MILLISECONDS);
-            else
-                querier = tableQueue.peek();
-            Timer timer = new Timer();
-            try {
-                Thread.currentThread();
-                Thread.sleep(1000);//毫秒
-            } catch (Exception e) {
-                throw e;
+    public List<ConnectRecord> poll() {
+        log.trace(" Polling for new data");
+        // 连续空的次数
+        Map<Querier, Integer> consecutiveEmptyResults = tableQueue.stream().collect(Collectors.toMap(Function.identity(), (q) -> 0));
+        while (running.get()) {
+            final Querier querier = tableQueue.peek();
+            if (!querier.querying()) {
+                // If not in the middle of an update, wait for next update time
+                final long nextUpdate = querier.getLastUpdate() + config.getPollIntervalMs();
+                final long now = System.currentTimeMillis();
+                final long sleepMs = Math.min(nextUpdate - now, 100);
+                if (sleepMs > 0) {
+                    log.trace("Waiting {} ms to poll {} next", nextUpdate - now, querier.toString());
+                    try {
+                        Thread.sleep(sleepMs);
+                    } catch (InterruptedException e) {
+                        Thread.currentThread().interrupt();
+                    }
+                    continue;
+                }
             }
-            querier.poll();
-            for (Table dataRow : querier.getList()) {
-                JSONObject jsonObject = new JSONObject();
-                jsonObject.put("nextQuery", "database");
-                jsonObject.put("nextPosition", "table");
-                Schema schema = new Schema();
-                schema.setDataSource(dataRow.getDatabase());
-                schema.setName(dataRow.getName());
-                schema.setFields(new ArrayList<>());
-                for (int i = 0; i < dataRow.getColList().size(); i++) {
-                    String columnName = dataRow.getColList().get(i);
-                    String rawDataType = dataRow.getRawDataTypeList().get(i);
-                    Field field = new Field(i, columnName, ColumnParser.mapConnectorFieldType(rawDataType));
-                    schema.getFields().add(field);
+
+            // poll data
+            final List<ConnectRecord> results = new ArrayList<>();
+            try {
+                log.debug("Checking for next block of results from {}", querier);
+                querier.maybeStartQuery(cachedConnectionProvider);
+                int batchMaxRows = config.getBatchMaxRows();
+                boolean hasNext = true;
+                while (results.size() < batchMaxRows && (hasNext = querier.hasNext())) {
+                    results.add(querier.extractRecord());
                 }
-                DataEntryBuilder dataEntryBuilder = new DataEntryBuilder(schema);
-                dataEntryBuilder.timestamp(System.currentTimeMillis()).queue(dataRow.getName())
-                        .entryType(EntryType.UPDATE);
-                for (int i = 0; i < dataRow.getColList().size(); i++) {
-                    Object[] value = new Object[2];
-                    value[0] = value[1] = dataRow.getParserList().get(i).getValue(dataRow.getDataList().get(i));
-                    dataEntryBuilder.putFiled(dataRow.getColList().get(i), JSONObject.toJSONString(value));
+                if (!hasNext) {
+                    // the querier to the tail of the queue
+                    resetAndRequeueHead(querier);
+                }
+
+                if (results.isEmpty()) {
+                    consecutiveEmptyResults.compute(querier, (k, v) -> v + 1);
+                    log.trace("No updates for {}", querier);
+                    if (Collections.min(consecutiveEmptyResults.values()) >= CONSECUTIVE_EMPTY_RESULTS_BEFORE_RETURN) {
+                        log.warn("More than " + CONSECUTIVE_EMPTY_RESULTS_BEFORE_RETURN + " consecutive empty results for all queriers, returning");
+                        return null;
+                    } else {
+                        continue;
+                    }
+                } else {
+                    consecutiveEmptyResults.put(querier, 0);
                 }
 
-                SourceDataEntry sourceDataEntry = dataEntryBuilder.buildSourceDataEntry(
-                        ByteBuffer.wrap((ConstDefine.PREFIX + config.getDbUrl() + config.getDbPort()).getBytes(StandardCharsets.UTF_8)),
-                        ByteBuffer.wrap(jsonObject.toJSONString().getBytes(StandardCharsets.UTF_8)));
-                res.add(sourceDataEntry);
-                log.debug("sourceDataEntry : {}", JSONObject.toJSONString(sourceDataEntry));
+                log.debug("Returning {} records for {}", results.size(), querier.toString());
+                return results;
+            } catch (SQLException sqle) {
+                log.error("Failed to run query for table {}: {}", querier.toString(), sqle);
+                resetAndRequeueHead(querier);
+                throw new RuntimeException(sqle);
+            } catch (Throwable t) {
+                resetAndRequeueHead(querier);
+                // This task has failed, so close any resources (may be reopened if needed) before throwing
+                closeResources();
+                throw t;
             }
-        } catch (Exception e) {
-            log.error("JDBC task poll error, current config:" + JSON.toJSONString(config), e);
         }
-        log.debug("dataEntry poll successfully,{}", JSONObject.toJSONString(res));
-        return res;
+        // Only in case of shutdown
+        final Querier querier = tableQueue.peek();
+        if (querier != null) {
+            resetAndRequeueHead(querier);
+        }
+        closeResources();
+        return null;
     }
 
+    private void resetAndRequeueHead(Querier querier) {
+        log.debug("Resetting querier {}", querier.toString());
+        tableQueue.poll();
+        if(running.get()){
+            querier.reset(System.currentTimeMillis());
+        }else{  //不跑了,可能是暂停,也可能是停止
+            querier.reset(0);
+        }
+        tableQueue.add(querier);
+    }
+
+
+    /**
+     * Should invoke before start the connector.
+     * @param config
+     * @return error message
+     */
     @Override
-    public void start(KeyValue props) {
-        try {
-            ConfigUtil.load(props, this.config);
-            dataSource = DBUtils.initDataSource(config);
-            connection = dataSource.getConnection();
-            log.info("init data source success");
-        } catch (Exception e) {
-            log.error("Cannot start Jdbc Source Task because of configuration error{}", e);
+    public void validate(KeyValue config) { }
+
+    /**
+     * start jdbc task
+     * @param context
+     */
+    @Override
+    public void start(SourceTaskContext context) {
+        // compute table offset
+        Map<String,Map<String, Object>> offsetValues= SourceOffsetCompute.initOffset(config, context, dialect, cachedConnectionProvider);
+        for (String tableOrQuery: offsetValues.keySet()) {
+            this.buildAndAddQuerier(
+                    JdbcSourceConfig.TableLoadMode.findTableLoadModeByName(this.config.getMode()),
+                    this.config.getQuerySuffix(),
+                    this.config.getIncrementingColumnName(),
+                    this.config.getTimestampColumnNames(),
+                    this.config.getTimestampDelayIntervalMs(),
+                    this.config.getTimeZone(), tableOrQuery,
+                    offsetValues.get(tableOrQuery)
+            );
         }
-        Map<Map<String, String>, Map<String, Object>> offsets = null;
-        String mode = config.getMode();
-        if (mode.equals("bulk")) {
-            Querier querier = new Querier(config, connection);
-            try {
-                querier.start();
+        running.set(true);
+        log.info("Started JDBC source task");
+    }
+
+    /**
+     * build and add querier
+     * @param loadMode
+     * @param querySuffix
+     * @param incrementingColumn
+     * @param timestampColumns
+     * @param timestampDelayInterval
+     * @param timeZone
+     * @param tableOrQuery
+     * @param offset
+     */
+    private void buildAndAddQuerier(JdbcSourceConfig.TableLoadMode loadMode, String querySuffix, String incrementingColumn, List<String> timestampColumns, Long timestampDelayInterval, TimeZone timeZone, String tableOrQuery, Map<String, Object> offset) {
+        String topicPrefix = config.getTopicPrefix();
+        Querier.QueryMode queryMode = !StringUtils.isEmpty(config.getQuery()) ? Querier.QueryMode.QUERY : Querier.QueryMode.TABLE;
+        Querier querier = null;
+        switch (loadMode){
+            case MODE_BULK:
+                querier = new BulkQuerier(
+                        dialect,
+                        queryMode,
+                        tableOrQuery,
+                        topicPrefix,
+                        querySuffix,
+                        this.config.getOffsetSuffix()
+                );
                 tableQueue.add(querier);
-            } catch (Exception e) {
-                log.error("start querier failed in bulk mode{}", e);
-            }
-        } else {
-            TimestampIncrementingQuerier querier = new TimestampIncrementingQuerier();
-            try {
-                querier.setConfig(config);
-                querier.start();
+                break;
+            case MODE_INCREMENTING:
+                querier = new TimestampIncrementingQuerier(
+                        dialect,
+                        queryMode,
+                        tableOrQuery,
+                        topicPrefix,
+                        null,
+                        incrementingColumn,
+                        offset,
+                        timestampDelayInterval,
+                        timeZone,
+                        querySuffix,
+                        this.config.getOffsetSuffix()
+                );
                 tableQueue.add(querier);
-            } catch (Exception e) {
-                log.error("fail to start querier{}", e);
-            }
+                break;
+            case MODE_TIMESTAMP:
+                 querier = new TimestampIncrementingQuerier(
+                        dialect,
+                        queryMode,
+                        tableOrQuery,
+                        topicPrefix,
+                        timestampColumns,
+                        null,
+                        offset,
+                        timestampDelayInterval,
+                        timeZone,
+                        querySuffix,
+                        this.config.getOffsetSuffix()
+                );
+                tableQueue.add(querier);
+                break;
+            case MODE_TIMESTAMP_INCREMENTING:
+                 querier = new TimestampIncrementingQuerier(
+                        dialect,
+                        queryMode,
+                        tableOrQuery,
+                        topicPrefix,
+                        timestampColumns,
+                        incrementingColumn,
+                        offset,
+                        timestampDelayInterval,
+                        timeZone,
+                        querySuffix,
+                         this.config.getOffsetSuffix()
+                );
+                tableQueue.add(querier);
+                break;
+        }
+    }
 
+    /**
+         * Init the component
+         * @param props
+         */
+    @Override
+    public void init(KeyValue props) {
+        try {
+            config=new JdbcSourceTaskConfig(props);
+            final String dialectName = config.getDialectName();
+            final String url = config.getConnectionDbUrl();
+            if (dialectName != null && !dialectName.trim().isEmpty()) {
+                dialect = DatabaseDialectFactory.create(dialectName, config);
+            } else {
+                dialect = DatabaseDialectFactory.findDialectFor(url, config);
+            }
+            final int maxConnAttempts = config.getAttempts();
+            final long retryBackoff = config.getBackoffMs();
+            cachedConnectionProvider = connectionProvider(maxConnAttempts, retryBackoff);
+            log.info("Using JDBC dialect {}", dialect.name());
+        } catch (Exception e) {
+            log.error("Cannot start Jdbc Source Task because of configuration error{}", e);
         }
+    }
 
+    protected CachedConnectionProvider connectionProvider(int maxConnAttempts, long retryBackoff) {
+        return new CachedConnectionProvider(dialect, maxConnAttempts, retryBackoff) {
+            @Override
+            protected void onConnect(final Connection connection) throws SQLException {
+                super.onConnect(connection);
+                connection.setAutoCommit(false);
+            }
+        };
     }
 
+
     @Override
     public void stop() {
+        running.set(true);
+    }
+
+
+    protected void closeResources() {
+        log.info("Closing resources for JDBC source task");
         try {
-            if (connection != null) {
-                connection.close();
-                log.info("jdbc source task connection is closed.");
+            if (cachedConnectionProvider != null) {
+                cachedConnectionProvider.close();
+            }
+        } catch (Throwable t) {
+            log.warn("Error while closing the connections", t);
+        } finally {
+            cachedConnectionProvider = null;
+            try {
+                if (dialect != null) {
+                    dialect.close();
+                }
+            } catch (Throwable t) {
+                log.warn("Error while closing the {} dialect: ", dialect.name(), t);
+            } finally {
+                dialect = null;
             }
-        } catch (Throwable e) {
-            log.warn("source task stop error while closing connection to {}", "jdbc", e);
         }
     }
 
     @Override
     public void pause() {
-
+        //TODO do nothing
     }
 
     @Override
     public void resume() {
-
+        //TODO do nothing
     }
 }
diff --git a/connectors/rocketmq-connect-jdbc/src/main/java/org/apache/rocketmq/connect/jdbc/strategy/TaskDivideStrategy.java b/connectors/rocketmq-connect-jdbc/src/main/java/org/apache/rocketmq/connect/jdbc/connector/JdbcSourceTaskConfig.java
similarity index 63%
rename from connectors/rocketmq-connect-jdbc/src/main/java/org/apache/rocketmq/connect/jdbc/strategy/TaskDivideStrategy.java
rename to connectors/rocketmq-connect-jdbc/src/main/java/org/apache/rocketmq/connect/jdbc/connector/JdbcSourceTaskConfig.java
index 736fcac..d0a2d8e 100644
--- a/connectors/rocketmq-connect-jdbc/src/main/java/org/apache/rocketmq/connect/jdbc/strategy/TaskDivideStrategy.java
+++ b/connectors/rocketmq-connect-jdbc/src/main/java/org/apache/rocketmq/connect/jdbc/connector/JdbcSourceTaskConfig.java
@@ -14,19 +14,26 @@
  * See the License for the specific language governing permissions and
  * limitations under the License.
  */
-package org.apache.rocketmq.connect.jdbc.strategy;
+package org.apache.rocketmq.connect.jdbc.connector;
 
 import io.openmessaging.KeyValue;
-import io.openmessaging.connector.api.Task;
-import org.apache.rocketmq.connect.jdbc.config.DbConnectorConfig;
-import org.apache.rocketmq.connect.jdbc.config.TaskDivideConfig;
-import org.apache.rocketmq.connect.jdbc.config.TaskTopicInfo;
-
 import java.util.List;
-import java.util.Map;
 
-public abstract class TaskDivideStrategy {
 
-    public abstract List<KeyValue> divide(DbConnectorConfig dbConnectorConfig, TaskDivideConfig tdc);
+/**
+ * task config
+ */
+public class JdbcSourceTaskConfig extends JdbcSourceConfig {
+  public static final String TABLES_CONFIG = "tables";
+  private static final String TABLES_DOC = "List of tables for this task to watch for changes.";
+
+  private List<String> tables;
+  public JdbcSourceTaskConfig(KeyValue config) {
+    super(config);
+    this.tables=getList(config,TABLES_CONFIG);
+  }
 
+  public List<String> getTables() {
+    return tables;
+  }
 }
diff --git a/connectors/rocketmq-connect-jdbc/src/main/java/org/apache/rocketmq/connect/jdbc/dialect/DatabaseDialect.java b/connectors/rocketmq-connect-jdbc/src/main/java/org/apache/rocketmq/connect/jdbc/dialect/DatabaseDialect.java
new file mode 100644
index 0000000..fc15fa8
--- /dev/null
+++ b/connectors/rocketmq-connect-jdbc/src/main/java/org/apache/rocketmq/connect/jdbc/dialect/DatabaseDialect.java
@@ -0,0 +1,319 @@
+/*
+ * 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.rocketmq.connect.jdbc.dialect;
+
+import io.openmessaging.connector.api.data.ConnectRecord;
+import io.openmessaging.connector.api.data.Schema;
+import org.apache.rocketmq.connect.jdbc.connector.JdbcSinkConfig;
+import org.apache.rocketmq.connect.jdbc.schema.column.ColumnDefinition;
+import org.apache.rocketmq.connect.jdbc.schema.column.ColumnId;
+import org.apache.rocketmq.connect.jdbc.schema.table.TableDefinition;
+import org.apache.rocketmq.connect.jdbc.schema.table.TableId;
+import org.apache.rocketmq.connect.jdbc.source.metadata.ColumnMapping;
+import org.apache.rocketmq.connect.jdbc.sink.metadata.FieldsMetadata;
+import org.apache.rocketmq.connect.jdbc.sink.metadata.SchemaPair;
+import org.apache.rocketmq.connect.jdbc.sink.metadata.SinkRecordField;
+import org.apache.rocketmq.connect.jdbc.dialect.provider.ConnectionProvider;
+import org.apache.rocketmq.connect.jdbc.source.TimestampIncrementingCriteria;
+import org.apache.rocketmq.connect.jdbc.util.ExpressionBuilder;
+import org.apache.rocketmq.connect.jdbc.util.IdentifierRules;
+
+import java.io.IOException;
+import java.sql.Connection;
+import java.sql.PreparedStatement;
+import java.sql.ResultSet;
+import java.sql.ResultSetMetaData;
+import java.sql.SQLException;
+import java.sql.Timestamp;
+import java.util.Calendar;
+import java.util.Collection;
+import java.util.List;
+import java.util.Map;
+
+/**
+ * database dialect
+ */
+public interface DatabaseDialect extends ConnectionProvider {
+
+  /**
+   * dialect name
+   * @return
+   */
+  String name();
+
+  /**
+   * create jdbc prepared statement
+   * @param connection
+   * @param query
+   * @return
+   * @throws SQLException
+   */
+  PreparedStatement createPreparedStatement(Connection connection, String query) throws SQLException;
+
+  /**
+   * parse to Table Id
+   * @param fqn
+   * @return
+   */
+  TableId parseToTableId(String fqn);
+
+  /**
+   * Get the identifier rules for this database.
+   * @return the identifier rules
+   */
+  IdentifierRules identifierRules();
+
+  /**
+   *  Get a new expression builder that can be used to build expressions with quoted identifiers.
+   * @return
+   */
+  ExpressionBuilder expressionBuilder();
+  Timestamp currentTimeOnDB(Connection connection, Calendar cal) throws SQLException;
+
+  /**
+   *  Get a list of identifiers of the non-system tables in the database.
+   * @param connection
+   * @return
+   * @throws SQLException
+   */
+  List<TableId> tableIds(Connection connection) throws SQLException;
+
+  /**
+   * table exists
+   * @param connection
+   * @param tableId
+   * @return
+   * @throws SQLException
+   */
+  boolean tableExists(Connection connection, TableId tableId) throws SQLException;
+
+
+  /**
+   * Create the definition for the columns described by the database metadata.
+   */
+  Map<ColumnId, ColumnDefinition> describeColumns(Connection connection, String tablePattern, String columnPattern) throws SQLException;
+
+  /**
+   * Create the definition for the columns described by the database metadata.
+   */
+  Map<ColumnId, ColumnDefinition> describeColumns(Connection connection, String catalogPattern, String schemaPattern, String tablePattern, String columnPattern) throws SQLException;
+
+  /**
+   * Create the definition for the columns in the result set.
+   */
+  Map<ColumnId, ColumnDefinition> describeColumns(Connection conn, TableId tableId, ResultSetMetaData rsMetadata) throws SQLException;
+
+  /**
+   * describe table info
+   * @param connection
+   * @param tableId
+   * @return
+   * @throws SQLException
+   */
+  TableDefinition describeTable(Connection connection, TableId tableId) throws SQLException;
+
+  /**
+   * describe columns by query sql
+   * @param connection
+   * @param tableId
+   * @return
+   * @throws SQLException
+   */
+  Map<ColumnId, ColumnDefinition> describeColumnsByQuerying(Connection connection, TableId tableId) throws SQLException;
+
+  /**
+   * 添加字段到schema中
+   * @param column
+   * @param schema
+   * @param index
+   * @return
+   */
+  String addFieldToSchema(ColumnDefinition column, Schema schema,int index);
+
+  /**
+   * Apply the supplied DDL statements using the given connection. This gives the dialect the
+   * opportunity to execute the statements with a different autocommit setting.
+   */
+  void applyDdlStatements(Connection connection, List<String> statements) throws SQLException;
+
+  /**
+   * build dml statement
+   * @param table
+   * @param keyColumns
+   * @param nonKeyColumns
+   * @return
+   */
+  String buildInsertStatement(TableId table, Collection<ColumnId> keyColumns, Collection<ColumnId> nonKeyColumns);
+
+  /**
+   * update statement
+   * @param table
+   * @param keyColumns
+   * @param nonKeyColumns
+   * @return
+   */
+  String buildUpdateStatement(TableId table, Collection<ColumnId> keyColumns, Collection<ColumnId> nonKeyColumns);
+
+  /**
+   * upsert statment
+   * @param table
+   * @param keyColumns
+   * @param nonKeyColumns
+   * @return
+   */
+  String buildUpsertQueryStatement(TableId table, Collection<ColumnId> keyColumns, Collection<ColumnId> nonKeyColumns);
+
+  /**
+   * delete statement
+   * @param table
+   * @param keyColumns
+   * @return
+   */
+  default String buildDeleteStatement(TableId table, Collection<ColumnId> keyColumns) {
+    throw new UnsupportedOperationException();
+  }
+
+
+  /**
+   * build select table
+   */
+  String buildSelectTableMode();
+  void buildSelectTable(ExpressionBuilder builder, TableId tableId);
+
+
+  /** drop table
+   * @param table
+   * @param options
+   * @return
+   */
+  String buildDropTableStatement(TableId table, DropOptions options);
+
+  /**
+   * create table
+   * @param table
+   * @param fields
+   * @return
+   */
+  String buildCreateTableStatement(TableId table, Collection<SinkRecordField> fields);
+  /**
+   * build alter table
+   * @param table
+   * @param fields
+   * @return
+   */
+  List<String> buildAlterTable(TableId table, Collection<SinkRecordField> fields);
+
+  /**
+   * Create a component that can bind record values into the supplied prepared statement.
+   *
+   * @param statement      the prepared statement
+   * @param pkMode         the primary key mode; may not be null
+   * @param schemaPair     the key and value schemas; may not be null
+   * @param fieldsMetadata the field metadata; may not be null
+   * @param tableDefinition the table definition; may be null
+   * @param insertMode     the insert mode; may not be null
+   * @return the statement binder; may not be null
+   */
+  StatementBinder statementBinder(
+      PreparedStatement statement,
+      JdbcSinkConfig.PrimaryKeyMode pkMode,
+      SchemaPair schemaPair,
+      FieldsMetadata fieldsMetadata,
+      TableDefinition tableDefinition,
+      JdbcSinkConfig.InsertMode insertMode
+  );
+
+  /**
+   * value column types
+   * @param rsMetadata
+   * @param columns
+   * @throws io.openmessaging.connector.api.errors.ConnectException
+   */
+  void validateColumnTypes(
+          ResultSetMetaData rsMetadata,
+          List<ColumnId> columns
+  ) throws io.openmessaging.connector.api.errors.ConnectException;
+
+
+  /**
+   * bind field
+   * @param statement
+   * @param index
+   * @param schema
+   * @param value
+   * @param colDef
+   * @throws SQLException
+   */
+  void bindField(PreparedStatement statement, int index, Schema schema, Object value, ColumnDefinition colDef) throws SQLException ;
+
+  /**
+   * criteria for
+   * @param incrementingColumn
+   * @param timestampColumns
+   * @return
+   */
+  TimestampIncrementingCriteria criteriaFor(
+          ColumnId incrementingColumn,
+          List<ColumnId> timestampColumns
+  );
+
+  /**
+   * get min timestamp value
+   * @param con
+   * @param tableOrQuery
+   * @param timestampColumns
+   * @return
+   * @throws SQLException
+   */
+  Long getMinTimestampValue(Connection con, String tableOrQuery, List<String> timestampColumns) throws SQLException;
+
+  /**
+   * A function to bind the values from a sink record into a prepared statement.
+   */
+  @FunctionalInterface
+  interface StatementBinder {
+    /**
+     * bind record
+     * @param record
+     * @throws SQLException
+     */
+    void bindRecord(ConnectRecord record) throws SQLException;
+  }
+
+
+  /**
+   * Create a function that converts column values for the column defined by the specified mapping.
+   * @param mapping
+   * @return
+   */
+  ColumnConverter createColumnConverter(ColumnMapping mapping);
+
+  /**
+   * A function that obtains a column value from the current row of the specified result set.
+   */
+  @FunctionalInterface
+  interface ColumnConverter {
+    /**
+     * convert
+     * @param resultSet
+     * @return
+     * @throws SQLException
+     * @throws IOException
+     */
+    Object convert(ResultSet resultSet) throws SQLException, IOException;
+  }
+}
\ No newline at end of file
diff --git a/connectors/rocketmq-connect-jdbc/src/main/java/org/apache/rocketmq/connect/jdbc/dialect/DatabaseDialectFactory.java b/connectors/rocketmq-connect-jdbc/src/main/java/org/apache/rocketmq/connect/jdbc/dialect/DatabaseDialectFactory.java
new file mode 100644
index 0000000..61d37b5
--- /dev/null
+++ b/connectors/rocketmq-connect-jdbc/src/main/java/org/apache/rocketmq/connect/jdbc/dialect/DatabaseDialectFactory.java
@@ -0,0 +1,192 @@
+/*
+ * 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.rocketmq.connect.jdbc.dialect;
+
+import io.openmessaging.connector.api.errors.ConnectException;
+import org.apache.rocketmq.connect.jdbc.config.AbstractConfig;
+import org.apache.rocketmq.connect.jdbc.dialect.provider.DatabaseDialectProvider;
+import org.apache.rocketmq.connect.jdbc.dialect.provider.JdbcUrlInfo;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import java.security.AccessController;
+import java.security.PrivilegedAction;
+import java.util.HashSet;
+import java.util.Iterator;
+import java.util.ServiceLoader;
+import java.util.Set;
+import java.util.concurrent.ConcurrentMap;
+import java.util.concurrent.ConcurrentSkipListMap;
+import java.util.concurrent.atomic.AtomicInteger;
+import java.util.regex.Matcher;
+import java.util.regex.Pattern;
+
+/**
+ * load and find database dialect
+ */
+public class DatabaseDialectFactory {
+
+  /**
+   * regex jdbc protocol
+   */
+  private static final Pattern PROTOCOL_PATTERN = Pattern.compile("jdbc:([^:]+):(.*)");
+  private static final Logger LOG = LoggerFactory.getLogger(DatabaseDialectFactory.class);
+  private static final ConcurrentMap<String, DatabaseDialectProvider> REGISTRY = new ConcurrentSkipListMap<>();
+
+  static {
+    loadAllDialects();
+  }
+
+  /**
+   * load all database dialect
+   */
+  private static void loadAllDialects() {
+    LOG.debug("Searching for and loading all JDBC source dialects on the classpath");
+    final AtomicInteger count = new AtomicInteger();
+    AccessController.doPrivileged(new PrivilegedAction<Void>() {
+      @Override
+      public Void run() {
+        ServiceLoader<DatabaseDialectProvider> loadedDialects =
+                ServiceLoader.load(
+                        DatabaseDialectProvider.class,
+                        this.getClass().getClassLoader()
+                );
+        Iterator<DatabaseDialectProvider> dialectIterator = loadedDialects.iterator();
+        try {
+          while (dialectIterator.hasNext()) {
+            try {
+              DatabaseDialectProvider provider = dialectIterator.next();
+              REGISTRY.put(provider.getClass().getName(), provider);
+              count.incrementAndGet();
+              LOG.debug("Found '{}' provider {}", provider, provider.getClass());
+            } catch (Throwable t) {
+              LOG.debug("Skipping dialect provider after error while loading", t);
+            }
+          }
+        } catch (Throwable t) {
+          LOG.debug("Error loading dialect providers", t);
+        }
+        return null;
+      }
+    });
+    LOG.debug("Registered {} source dialects", count.get());
+  }
+
+
+  /**
+   * find dialect for
+   * @param jdbcUrl
+   * @param config
+   * @return
+   * @throws ConnectException
+   */
+  public static DatabaseDialect findDialectFor(
+          String jdbcUrl,
+          AbstractConfig config
+  ) throws ConnectException {
+    final JdbcUrlInfo info = extractJdbcUrlInfo(jdbcUrl);
+    LOG.debug("Finding best dialect for {}", info);
+    DatabaseDialectProvider bestMatch = null;
+    for (DatabaseDialectProvider provider : REGISTRY.values()) {
+      if (provider.protocolName().equals(info.subProtocol())){
+        bestMatch=provider;
+        break;
+      }
+    }
+    LOG.debug("Using dialect {}  against {}", bestMatch, info);
+    return bestMatch.create(config);
+  }
+
+  /**
+   * create database dialect
+   * @param dialectName
+   * @param config
+   * @return
+   * @throws ConnectException
+   */
+  public static DatabaseDialect create(
+          String dialectName,
+          AbstractConfig config
+  ) throws ConnectException {
+    LOG.debug("Looking for named dialect '{}'", dialectName);
+    Set<String> dialectNames = new HashSet<>();
+    for (DatabaseDialectProvider provider : REGISTRY.values()) {
+      dialectNames.add(provider.dialectName());
+      if (provider.dialectName().equals(dialectName)) {
+        return provider.create(config);
+      }
+    }
+    for (DatabaseDialectProvider provider : REGISTRY.values()) {
+      if (provider.dialectName().equalsIgnoreCase(dialectName)) {
+        return provider.create(config);
+      }
+    }
+    throw new ConnectException(
+        "Unable to find dialect with name '" + dialectName + "' in the available dialects: "
+        + dialectNames
+    );
+  }
+
+  public static JdbcUrlInfo extractJdbcUrlInfo(final String url) {
+    Matcher matcher = PROTOCOL_PATTERN.matcher(url);
+    if (matcher.matches()) {
+      return new JdbcUrlDetails(matcher.group(1), matcher.group(2), url);
+    }
+    throw new ConnectException("Not a valid JDBC URL: " + url);
+  }
+
+
+
+  static class JdbcUrlDetails implements JdbcUrlInfo {
+    final String subprotocol;
+    final String subname;
+    final String url;
+
+    public JdbcUrlDetails(
+        String subprotocol,
+        String subname,
+        String url
+    ) {
+      this.subprotocol = subprotocol;
+      this.subname = subname;
+      this.url = url;
+    }
+
+    @Override
+    public String subProtocol() {
+      return subprotocol;
+    }
+
+    @Override
+    public String subName() {
+      return subname;
+    }
+
+    @Override
+    public String url() {
+      return url;
+    }
+
+    @Override
+    public String toString() {
+      return "JDBC subprotocol '" + subprotocol + "' and source '" + url + "'";
+    }
+  }
+
+  private DatabaseDialectFactory() {
+  }
+}
diff --git a/connectors/rocketmq-connect-jdbc/src/main/java/org/apache/rocketmq/connect/jdbc/dialect/DropOptions.java b/connectors/rocketmq-connect-jdbc/src/main/java/org/apache/rocketmq/connect/jdbc/dialect/DropOptions.java
new file mode 100644
index 0000000..4e8122f
--- /dev/null
+++ b/connectors/rocketmq-connect-jdbc/src/main/java/org/apache/rocketmq/connect/jdbc/dialect/DropOptions.java
@@ -0,0 +1,100 @@
+/*
+ * 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.rocketmq.connect.jdbc.dialect;
+
+import java.util.Objects;
+
+public class DropOptions {
+
+  private final boolean ifExists;
+  private final boolean cascade;
+
+  /**
+   * Create a new instance with the default settings.
+   */
+  public DropOptions() {
+    this(false, false);
+  }
+
+  protected DropOptions(
+      boolean ifExists,
+      boolean cascade
+  ) {
+    this.ifExists = ifExists;
+    this.cascade = cascade;
+  }
+
+  /**
+   * Get whether the 'IF EXISTS' clause should be used with the 'DROP' statement.
+   *
+   * @return true if the object should be dropped only if it already exists, or false otherwise
+   */
+  public boolean ifExists() {
+    return ifExists;
+  }
+
+  /**
+   * Get whether the 'DROP' statement should cascade to dependent objects.
+   *
+   * @return true if dependent objects should also be dropped, or false otherwise
+   */
+  public boolean cascade() {
+    return cascade;
+  }
+
+  /**
+   * Set whether the 'IF EXISTS' clause should be used with the 'DROP' statement.
+   *
+   * @param ifExists true if the object should be dropped only if it already exists
+   * @return a new options object with the current state plus the new if-exists state; never null
+   */
+  public DropOptions setIfExists(boolean ifExists) {
+    return new DropOptions(ifExists, cascade);
+  }
+
+  /**
+   * Set whether the 'DROP' statement should cascade to dependent objects.
+   *
+   * @param cascade true if dependent objects should also be dropped, or false otherwise
+   * @return a new options object with the current state plus the new cascade state; never null
+   */
+  public DropOptions setCascade(boolean cascade) {
+    return new DropOptions(ifExists, cascade);
+  }
+
+  @Override
+  public String toString() {
+    return "DropOptions{ifExists=" + ifExists + ", cascade=" + cascade + "}";
+  }
+
+  @Override
+  public int hashCode() {
+    return Objects.hash(ifExists, cascade);
+  }
+
+  @Override
+  public boolean equals(Object obj) {
+    if (obj == this) {
+      return true;
+    }
+    if (obj instanceof DropOptions) {
+      DropOptions that = (DropOptions) obj;
+      return this.ifExists() == that.ifExists() && this.cascade() == that.cascade();
+    }
+    return false;
+  }
+}
diff --git a/connectors/rocketmq-connect-jdbc/src/main/java/org/apache/rocketmq/connect/jdbc/dialect/PreparedStatementBinder.java b/connectors/rocketmq-connect-jdbc/src/main/java/org/apache/rocketmq/connect/jdbc/dialect/PreparedStatementBinder.java
new file mode 100644
index 0000000..d74ec52
--- /dev/null
+++ b/connectors/rocketmq-connect-jdbc/src/main/java/org/apache/rocketmq/connect/jdbc/dialect/PreparedStatementBinder.java
@@ -0,0 +1,130 @@
+/*
+ * 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.rocketmq.connect.jdbc.dialect;
+
+import com.alibaba.fastjson.JSON;
+import com.alibaba.fastjson.JSONArray;
+import io.openmessaging.connector.api.data.ConnectRecord;
+import io.openmessaging.connector.api.data.Field;
+import io.openmessaging.connector.api.data.Schema;
+import io.openmessaging.connector.api.errors.ConnectException;
+import org.apache.rocketmq.connect.jdbc.connector.JdbcSinkConfig;
+import org.apache.rocketmq.connect.jdbc.sink.metadata.FieldsMetadata;
+import org.apache.rocketmq.connect.jdbc.sink.metadata.SchemaPair;
+import org.apache.rocketmq.connect.jdbc.schema.column.ColumnDefinition;
+import org.apache.rocketmq.connect.jdbc.schema.table.TableDefinition;
+
+import java.sql.PreparedStatement;
+import java.sql.SQLException;
+import java.util.Objects;
+
+/**
+ * prepared statement binder
+ */
+public class PreparedStatementBinder implements DatabaseDialect.StatementBinder {
+
+  private final JdbcSinkConfig.PrimaryKeyMode pkMode;
+  private final PreparedStatement statement;
+  private final SchemaPair schemaPair;
+  private final FieldsMetadata fieldsMetadata;
+  private final JdbcSinkConfig.InsertMode insertMode;
+  private final DatabaseDialect dialect;
+  private final TableDefinition tabDef;
+
+  public PreparedStatementBinder(
+      DatabaseDialect dialect,
+      PreparedStatement statement,
+      JdbcSinkConfig.PrimaryKeyMode pkMode,
+      SchemaPair schemaPair,
+      FieldsMetadata fieldsMetadata,
+      TableDefinition tabDef,
+      JdbcSinkConfig.InsertMode insertMode
+  ) {
+    this.dialect = dialect;
+    this.pkMode = pkMode;
+    this.statement = statement;
+    this.schemaPair = schemaPair;
+    this.fieldsMetadata = fieldsMetadata;
+    this.insertMode = insertMode;
+    this.tabDef = tabDef;
+  }
+
+  @Override
+  public void bindRecord(ConnectRecord record) throws SQLException {
+    final boolean isDelete = Objects.isNull(record.getData());
+
+    int index = 1;
+    if (isDelete) {
+      bindKeyFields(record, index);
+    } else {
+      switch (insertMode) {
+        case INSERT:
+        case UPSERT:
+          index = bindKeyFields(record, index);
+          bindNonKeyFields(record,index);
+          break;
+
+        case UPDATE:
+          index = bindNonKeyFields(record, index);
+          bindKeyFields(record, index);
+          break;
+        default:
+          throw new AssertionError();
+
+      }
+    }
+    statement.addBatch();
+  }
+
+  protected int bindKeyFields(ConnectRecord record, int index) throws SQLException {
+    switch (pkMode) {
+      case NONE:
+        if (!fieldsMetadata.keyFieldNames.isEmpty()) {
+          throw new AssertionError();
+        }
+        break;
+      case RECORD_VALUE: {
+        Object[] data=JSONArray.parseArray(JSON.toJSONString(record.getData())).stream().toArray();
+        for (String fieldName : fieldsMetadata.keyFieldNames) {
+          final Field field = schemaPair.schema.getField(fieldName);
+          bindField(index++, field.getSchema(), data[field.getIndex()] , fieldName);
+        }
+      }
+      break;
+      default:
+        throw new ConnectException("Unknown primary key mode: " + pkMode);
+    }
+    return index;
+  }
+
+  protected int bindNonKeyFields(
+          ConnectRecord record,
+          int index
+  ) throws SQLException {
+    for (final String fieldName : fieldsMetadata.nonKeyFieldNames) {
+      final Field field = record.getSchema().getField(fieldName);
+      bindField(index++, field.getSchema(), ((Object[])record.getData())[field.getIndex()] , fieldName);
+    }
+    return index;
+  }
+
+  protected void bindField(int index, Schema schema, Object value, String fieldName)
+      throws SQLException {
+    ColumnDefinition colDef = tabDef == null ? null : tabDef.definitionForColumn(fieldName);
+    dialect.bindField(statement, index, schema, value, colDef);
+  }
+}
diff --git a/connectors/rocketmq-connect-jdbc/src/main/java/org/apache/rocketmq/connect/jdbc/dialect/impl/GenericDatabaseDialect.java b/connectors/rocketmq-connect-jdbc/src/main/java/org/apache/rocketmq/connect/jdbc/dialect/impl/GenericDatabaseDialect.java
new file mode 100644
index 0000000..f177148
--- /dev/null
+++ b/connectors/rocketmq-connect-jdbc/src/main/java/org/apache/rocketmq/connect/jdbc/dialect/impl/GenericDatabaseDialect.java
@@ -0,0 +1,1888 @@
+/*
+ * 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.rocketmq.connect.jdbc.dialect.impl;
+
+import io.openmessaging.connector.api.data.Field;
+import io.openmessaging.connector.api.data.FieldType;
+import io.openmessaging.connector.api.data.Schema;
+import io.openmessaging.connector.api.data.SchemaBuilder;
+import lombok.SneakyThrows;
+import org.apache.rocketmq.connect.jdbc.config.AbstractConfig;
+import org.apache.rocketmq.connect.jdbc.connector.JdbcSinkConfig;
+import org.apache.rocketmq.connect.jdbc.connector.JdbcSourceConfig;
+import org.apache.rocketmq.connect.jdbc.dialect.DatabaseDialect;
+import org.apache.rocketmq.connect.jdbc.dialect.DatabaseDialectFactory;
+import org.apache.rocketmq.connect.jdbc.dialect.DropOptions;
+import org.apache.rocketmq.connect.jdbc.dialect.PreparedStatementBinder;
+import org.apache.rocketmq.connect.jdbc.schema.column.ColumnDefAdjuster;
+import org.apache.rocketmq.connect.jdbc.schema.column.ColumnDefinition;
+import org.apache.rocketmq.connect.jdbc.schema.column.ColumnId;
+import org.apache.rocketmq.connect.jdbc.schema.table.TableDefinition;
+import org.apache.rocketmq.connect.jdbc.schema.table.TableId;
+import org.apache.rocketmq.connect.jdbc.source.metadata.ColumnMapping;
+import org.apache.rocketmq.connect.jdbc.sink.metadata.FieldsMetadata;
+import org.apache.rocketmq.connect.jdbc.sink.metadata.SchemaPair;
+import org.apache.rocketmq.connect.jdbc.sink.metadata.SinkRecordField;
+import org.apache.rocketmq.connect.jdbc.dialect.provider.DatabaseDialectProvider;
+import org.apache.rocketmq.connect.jdbc.dialect.provider.JdbcUrlInfo;
+import org.apache.rocketmq.connect.jdbc.schema.column.parser.DateColumnParser;
+import org.apache.rocketmq.connect.jdbc.schema.column.parser.TimeColumnParser;
+import org.apache.rocketmq.connect.jdbc.schema.column.parser.TimestampColumnParser;
+import org.apache.rocketmq.connect.jdbc.source.TimestampIncrementingCriteria;
+import org.apache.rocketmq.connect.jdbc.util.*;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import java.io.IOException;
+import java.net.URL;
+import java.nio.ByteBuffer;
+import java.sql.Blob;
+import java.sql.Clob;
+import java.sql.Connection;
+import java.sql.DatabaseMetaData;
+import java.sql.DriverManager;
+import java.sql.PreparedStatement;
+import java.sql.ResultSet;
+import java.sql.ResultSetMetaData;
+import java.sql.SQLException;
+import java.sql.SQLXML;
+import java.sql.Statement;
+import java.sql.Timestamp;
+import java.sql.Types;
+import java.time.ZoneOffset;
+import java.util.ArrayList;
+import java.util.Arrays;
+import java.util.Calendar;
+import java.util.Collection;
+import java.util.Collections;
+import java.util.HashMap;
+import java.util.HashSet;
+import java.util.LinkedHashMap;
+import java.util.List;
+import java.util.Locale;
+import java.util.Map;
+import java.util.Properties;
+import java.util.Queue;
+import java.util.Set;
+import java.util.TimeZone;
+import java.util.concurrent.ConcurrentLinkedQueue;
+import java.util.concurrent.atomic.AtomicReference;
+import java.util.stream.Collectors;
+
+/**
+ * generic database dialect
+ */
+public class GenericDatabaseDialect implements DatabaseDialect {
+
+  protected static final int NUMERIC_TYPE_SCALE_LOW = -84;
+  protected static final int NUMERIC_TYPE_SCALE_HIGH = 127;
+  protected static final int NUMERIC_TYPE_SCALE_UNSET = -127;
+
+  private static final int MAX_INTEGER_TYPE_PRECISION = 18;
+
+  /**
+   * The provider for GenericDatabaseDialect
+   */
+  public static class Provider extends DatabaseDialectProvider {
+    public Provider() {
+      super(GenericDatabaseDialect.class.getSimpleName(),"");
+    }
+    @Override
+    public DatabaseDialect create(AbstractConfig config) {
+      return new GenericDatabaseDialect(config);
+    }
+  }
+
+  private static final Logger glog = LoggerFactory.getLogger(GenericDatabaseDialect.class);
+
+  @Deprecated
+  protected final Logger log = LoggerFactory.getLogger(GenericDatabaseDialect.class);
+  protected AbstractConfig config;
+  /**
+   * Whether to map {@code NUMERIC} JDBC types by precision.
+   */
+  protected NumericMapping mapNumerics=NumericMapping.NONE;
+  protected String catalogPattern;
+  protected  String schemaPattern;
+  protected Set<String> tableTypes;
+  protected String jdbcUrl;
+  protected JdbcUrlInfo jdbcUrlInfo;
+  private QuoteMethod quoteSqlIdentifiers=QuoteMethod.ALWAYS;
+  private IdentifierRules defaultIdentifierRules;
+  private AtomicReference<IdentifierRules> identifierRules = new AtomicReference<>();
+  private Queue<Connection> connections = new ConcurrentLinkedQueue<>();
+  private volatile JdbcDriverInfo jdbcDriverInfo;
+  private int batchMaxRows;
+  private TimeZone timeZone;
+
+  public GenericDatabaseDialect(AbstractConfig config) {
+    this(config, IdentifierRules.DEFAULT);
+  }
+
+  protected GenericDatabaseDialect(
+          AbstractConfig config, IdentifierRules defaultIdentifierRules
+  ) {
+    this.config = config;
+    this.defaultIdentifierRules = defaultIdentifierRules;
+    this.jdbcUrl = config.getConnectionDbUrl();
+    this.jdbcUrlInfo = DatabaseDialectFactory.extractJdbcUrlInfo(jdbcUrl);
+    if (config instanceof JdbcSinkConfig) {
+      JdbcSinkConfig sinkConfig = (JdbcSinkConfig) config;
+      catalogPattern = JdbcSourceConfig.CATALOG_PATTERN_DEFAULT;
+      schemaPattern = JdbcSourceConfig.SCHEMA_PATTERN_DEFAULT;
+      tableTypes = sinkConfig.tableTypeNames();
+      quoteSqlIdentifiers = QuoteMethod.get(config.getQuoteSqlIdentifiers());
+      mapNumerics = NumericMapping.NONE;
+      batchMaxRows = 0;
+      timeZone = sinkConfig.getTimeZone();
+    } else {
+      JdbcSourceConfig sourceConfig = (JdbcSourceConfig) config;
+      catalogPattern = sourceConfig.getCatalogPattern();
+      schemaPattern = sourceConfig.getSchemaPattern();
+      tableTypes = sourceConfig.getTableTypes().stream().map(TableType::toString).collect(Collectors.toSet());
+      quoteSqlIdentifiers = QuoteMethod.get(config.getQuoteSqlIdentifiers());
+      mapNumerics = sourceConfig.numericMapping();
+      batchMaxRows = sourceConfig.getBatchMaxRows();
+      timeZone = sourceConfig.getTimeZone();
+    }
+
+  }
+
+  @Override
+  public String name() {
+    return getClass().getSimpleName().replace("DatabaseDialect", "");
+  }
+
+  /**
+   * init jdbc connection
+   * @return
+   * @throws SQLException
+   */
+  @SneakyThrows
+  @Override
+  public Connection getConnection() throws SQLException {
+    // These config names are the same for both source and sink configs ...
+    String username = config.getConnectionDbUser();
+    String dbPassword = config.getConnectionDbPassword();
+    Properties properties = new Properties();
+    if (username != null) {
+      properties.setProperty("user", username);
+    }
+    if (dbPassword != null) {
+      properties.setProperty("password", dbPassword);
+    }
+    properties = addConnectionProperties(properties);
+    DriverManager.setLoginTimeout(40);
+    Connection connection = DriverManager.getConnection(jdbcUrl, properties);
+    if (jdbcDriverInfo == null) {
+      jdbcDriverInfo = createJdbcDriverInfo(connection);
+    }
+    connections.add(connection);
+    return connection;
+  }
+
+  @Override
+  public void close() {
+    Connection conn;
+    while ((conn = connections.poll()) != null) {
+      try {
+        conn.close();
+      } catch (Throwable e) {
+        glog.warn("Error while closing connection to {}", jdbcDriverInfo, e);
+      }
+    }
+  }
+
+  @Override
+  public boolean isConnectionValid(
+          Connection connection,
+          int timeout
+  ) throws SQLException {
+    if (jdbcDriverInfo().jdbcMajorVersion() >= 4) {
+      return connection.isValid(timeout);
+    }
+    // issue a test query ...
+    String query = checkConnectionQuery();
+    if (query != null) {
+      try (Statement statement = connection.createStatement()) {
+        if (statement.execute(query)) {
+          ResultSet rs = null;
+          try {
+            // do nothing with the result set
+            rs = statement.getResultSet();
+          } finally {
+            if (rs != null) {
+              rs.close();
+            }
+          }
+        }
+      }
+    }
+    return true;
+  }
+
+  /**
+   * check connection query
+   * @return the check connection query; may be null if the connection should not be queried
+   */
+  protected String checkConnectionQuery() {
+    return "SELECT 1";
+  }
+
+  protected JdbcDriverInfo jdbcDriverInfo() {
+    if (jdbcDriverInfo == null) {
+      try (Connection connection = getConnection()) {
+        jdbcDriverInfo = createJdbcDriverInfo(connection);
+      } catch (SQLException e) {
+        throw new io.openmessaging.connector.api.errors.ConnectException("Unable to get JDBC driver information", e);
+      }
+    }
+    return jdbcDriverInfo;
+  }
+
+  protected JdbcDriverInfo createJdbcDriverInfo(Connection connection) throws SQLException {
+    DatabaseMetaData metadata = connection.getMetaData();
+    return new JdbcDriverInfo(
+            metadata.getJDBCMajorVersion(),
+            metadata.getJDBCMinorVersion(),
+            metadata.getDriverName(),
+            metadata.getDatabaseProductName(),
+            metadata.getDatabaseProductVersion()
+    );
+  }
+
+  /**
+   * add connect properties
+   * @param properties
+   * @return
+   */
+  protected Properties addConnectionProperties(Properties properties) {
+    // todo 设置 config中的属性
+    return properties;
+  }
+
+  @Override
+  public PreparedStatement createPreparedStatement(
+          Connection db,
+          String query
+  ) throws SQLException {
+    glog.trace("Creating a PreparedStatement '{}'", query);
+    PreparedStatement stmt = db.prepareStatement(query);
+    initializePreparedStatement(stmt);
+    return stmt;
+  }
+
+  /**
+   * init PreparedStatement
+   * @param stmt
+   * @throws SQLException
+   */
+  protected void initializePreparedStatement(PreparedStatement stmt) throws SQLException {
+    if (batchMaxRows > 0) {
+      stmt.setFetchSize(batchMaxRows);
+    }
+  }
+
+  @Override
+  public TableId parseToTableId(String fqn) {
+    List<String> parts = identifierRules().parseQualifiedIdentifier(fqn);
+    if (parts.isEmpty()) {
+      throw new IllegalArgumentException("Invalid fully qualified name: '" + fqn + "'");
+    }
+    if (parts.size() == 1) {
+      return new TableId(null, null, parts.get(0));
+    }
+    if (parts.size() == 3) {
+      return new TableId(parts.get(0), parts.get(1), parts.get(2));
+    }
+    assert parts.size() >= 2;
+    if (useCatalog()) {
+      return new TableId(parts.get(0), null, parts.get(1));
+    }
+    return new TableId(null, parts.get(0), parts.get(1));
+  }
+
+  /**
+   * Return whether the database uses JDBC catalogs.
+   * @return true if catalogs are used, or false otherwise
+   */
+  protected boolean useCatalog() {
+    return false;
+  }
+
+  @Override
+  public List<TableId> tableIds(Connection conn) throws SQLException {
+    DatabaseMetaData metadata = conn.getMetaData();
+    String[] tableTypes = tableTypes(metadata, this.tableTypes);
+    String tableTypeDisplay = displayableTableTypes(tableTypes, ", ");
+    glog.debug("Using {} dialect to get {}", this, tableTypeDisplay);
+    try (ResultSet rs = metadata.getTables(catalogPattern(), schemaPattern(), "%", tableTypes)) {
+      List<TableId> tableIds = new ArrayList<>();
+      while (rs.next()) {
+        String catalogName = rs.getString(1);
+        String schemaName = rs.getString(2);
+        String tableName = rs.getString(3);
+        TableId tableId = new TableId(catalogName, schemaName, tableName);
+        if (includeTable(tableId)) {
+          tableIds.add(tableId);
+        }
+      }
+      glog.debug("Used {} dialect to find {} {}", this, tableIds.size(), tableTypeDisplay);
+      return tableIds;
+    }
+  }
+
+  protected String catalogPattern() {
+    return catalogPattern;
+  }
+
+  protected String schemaPattern() {
+    return schemaPattern;
+  }
+
+  /**
+   * Determine whether the table with the specific name is to be included in the tables.
+   *
+   * <p>This method can be overridden to exclude certain database tables.
+   *
+   * @param table the identifier of the table; may be null
+   * @return true if the table should be included; false otherwise
+   */
+  protected boolean includeTable(TableId table) {
+    return true;
+  }
+
+  /**
+   * Find the available table types that are returned by the JDBC driver that case insensitively
+   * match the specified types.
+   *
+   * @param metadata the database metadata; may not be null but may be empty if no table types
+   * @param types    the case-independent table types that are desired
+   * @return the array of table types take directly from the list of available types returned by the
+   *     JDBC driver; never null
+   * @throws SQLException if there is an error with the database connection
+   */
+  protected String[] tableTypes(
+          DatabaseMetaData metadata,
+          Set<String> types
+  ) throws SQLException {
+    glog.debug("Using {} dialect to check support for {}", this, types);
+    // Compute the uppercase form of the desired types ...
+    Set<String> uppercaseTypes = new HashSet<>();
+    for (String type : types) {
+      if (type != null) {
+        uppercaseTypes.add(type.toUpperCase(Locale.ROOT));
+      }
+    }
+    // Now find out the available table types ...
+    Set<String> matchingTableTypes = new HashSet<>();
+    try (ResultSet rs = metadata.getTableTypes()) {
+      while (rs.next()) {
+        String tableType = rs.getString(1);
+        if (tableType != null && uppercaseTypes.contains(tableType.toUpperCase(Locale.ROOT))) {
+          matchingTableTypes.add(tableType);
+        }
+      }
+    }
+    String[] result = matchingTableTypes.toArray(new String[matchingTableTypes.size()]);
+    glog.debug("Used {} dialect to find table types: {}", this, result);
+    return result;
+  }
+
+  @Override
+  public IdentifierRules identifierRules() {
+    if (identifierRules.get() == null) {
+      // Otherwise try to get the actual quote string and separator from the database, since
+      // many databases allow them to be changed
+      try (Connection connection = getConnection()) {
+        DatabaseMetaData metaData = connection.getMetaData();
+        String leadingQuoteStr = metaData.getIdentifierQuoteString();
+        String trailingQuoteStr = leadingQuoteStr; // JDBC does not distinguish
+        String separator = metaData.getCatalogSeparator();
+        if (leadingQuoteStr == null || leadingQuoteStr.isEmpty()) {
+          leadingQuoteStr = defaultIdentifierRules.leadingQuoteString();
+          trailingQuoteStr = defaultIdentifierRules.trailingQuoteString();
+        }
+        if (separator == null || separator.isEmpty()) {
+          separator = defaultIdentifierRules.identifierDelimiter();
+        }
+        identifierRules.set(new IdentifierRules(separator, leadingQuoteStr, trailingQuoteStr));
+      } catch (SQLException e) {
+        if (defaultIdentifierRules != null) {
+          identifierRules.set(defaultIdentifierRules);
+          glog.warn("Unable to get identifier metadata; using default rules", e);
+        } else {
+          throw new io.openmessaging.connector.api.errors.ConnectException("Unable to get identifier metadata", e);
+        }
+      }
+    }
+    return identifierRules.get();
+  }
+
+  @Override
+  public ExpressionBuilder expressionBuilder() {
+    return identifierRules().expressionBuilder()
+            .setQuoteIdentifiers(quoteSqlIdentifiers);
+  }
+
+  /**
+   * Return current time at the database
+   * @param conn database connection
+   * @param cal  calendar
+   * @return the current time at the database
+   */
+  @Override
+  public Timestamp currentTimeOnDB(
+          Connection conn,
+          Calendar cal
+  ) throws SQLException {
+    String query = currentTimestampDatabaseQuery();
+    assert query != null;
+    assert !query.isEmpty();
+    try (Statement stmt = conn.createStatement()) {
+      glog.debug("executing query " + query + " to get current time from database");
+      try (ResultSet rs = stmt.executeQuery(query)) {
+        if (rs.next()) {
+          return rs.getTimestamp(1, cal);
+        } else {
+          throw new io.openmessaging.connector.api.errors.ConnectException(
+                  "Unable to get current time from DB using " + this + " and query '" + query + "'"
+          );
+        }
+      }
+    } catch (SQLException e) {
+      glog.error("Failed to get current time from DB using {} and query '{}'", this, query, e);
+      throw e;
+    }
+  }
+
+  /**
+   * Get the query string to determine the current timestamp in the database.
+   * @return the query string; never null or empty
+   */
+  protected String currentTimestampDatabaseQuery() {
+    return "SELECT CURRENT_TIMESTAMP";
+  }
+
+  @Override
+  public boolean tableExists(
+          Connection connection,
+          TableId tableId
+  ) throws SQLException {
+    DatabaseMetaData metadata = connection.getMetaData();
+    String[] tableTypes = tableTypes(metadata, this.tableTypes);
+    String tableTypeDisplay = displayableTableTypes(tableTypes, "/");
+    glog.info("Checking {} dialect for existence of {} {}", this, tableTypeDisplay, tableId);
+    try (ResultSet rs = connection.getMetaData().getTables(
+            tableId.catalogName(),
+            tableId.schemaName(),
+            tableId.tableName(),
+            tableTypes
+    )) {
+      final boolean exists = rs.next();
+      glog.info(
+              "Using {} dialect {} {} {}",
+              this,
+              tableTypeDisplay,
+              tableId,
+              exists ? "present" : "absent"
+      );
+      return exists;
+    }
+  }
+
+  protected String displayableTableTypes(String[] types, String delim) {
+    return Arrays.stream(types).sorted().collect(Collectors.joining(delim));
+  }
+
+  @Override
+  public Map<ColumnId, ColumnDefinition> describeColumns(
+          Connection connection,
+          String tablePattern,
+          String columnPattern
+  ) throws SQLException {
+    //if the table pattern is fqn, then just use the actual table name
+    TableId tableId = parseToTableId(tablePattern);
+    String catalog = tableId.catalogName() != null ? tableId.catalogName() : catalogPattern;
+    String schema = tableId.schemaName() != null ? tableId.schemaName() : schemaPattern;
+    return describeColumns(connection, catalog , schema, tableId.tableName(), columnPattern);
+  }
+
+  @Override
+  public Map<ColumnId, ColumnDefinition> describeColumns(
+          Connection connection,
+          String catalogPattern,
+          String schemaPattern,
+          String tablePattern,
+          String columnPattern
+  ) throws SQLException {
+    glog.debug(
+            "Querying {} dialect column metadata for catalog:{} schema:{} table:{}",
+            this,
+            catalogPattern,
+            schemaPattern,
+            tablePattern);
+
+    // Get the primary keys of the table(s) ...
+    final Set<ColumnId> pkColumns = primaryKeyColumns(
+            connection,
+            catalogPattern,
+            schemaPattern,
+            tablePattern
+    );
+    Map<ColumnId, ColumnDefinition> results = new HashMap<>();
+    try (ResultSet rs = connection.getMetaData().getColumns(
+            catalogPattern,
+            schemaPattern,
+            tablePattern,
+            columnPattern
+    )) {
+      final int rsColumnCount = rs.getMetaData().getColumnCount();
+      while (rs.next()) {
+        final String catalogName = rs.getString(1);
+        final String schemaName = rs.getString(2);
+        final String tableName = rs.getString(3);
+        final TableId tableId = new TableId(catalogName, schemaName, tableName);
+        final String columnName = rs.getString(4);
+        final ColumnId columnId = new ColumnId(tableId, columnName, null);
+        final int jdbcType = rs.getInt(5);
+        final String typeName = rs.getString(6);
+        final int precision = rs.getInt(7);
+        final int scale = rs.getInt(9);
+        final String typeClassName = null;
+        ColumnDefinition.Nullability nullability;
+        final int nullableValue = rs.getInt(11);
+        switch (nullableValue) {
+          case DatabaseMetaData.columnNoNulls:
+            nullability = ColumnDefinition.Nullability.NOT_NULL;
+            break;
+          case DatabaseMetaData.columnNullable:
+            nullability = ColumnDefinition.Nullability.NULL;
+            break;
+          case DatabaseMetaData.columnNullableUnknown:
+          default:
+            nullability = ColumnDefinition.Nullability.UNKNOWN;
+            break;
+        }
+        Boolean autoIncremented = null;
+        if (rsColumnCount >= 23) {
+          // Not all drivers include all columns ...
+          String isAutoIncremented = rs.getString(23);
+          if ("yes".equalsIgnoreCase(isAutoIncremented)) {
+            autoIncremented = Boolean.TRUE;
+          } else if ("no".equalsIgnoreCase(isAutoIncremented)) {
+            autoIncremented = Boolean.FALSE;
+          }
+        }
+        Boolean signed = null;
+        Boolean caseSensitive = null;
+        Boolean searchable = null;
+        Boolean currency = null;
+        Integer displaySize = null;
+        boolean isPrimaryKey = pkColumns.contains(columnId);
+        if (isPrimaryKey) {
+          // Some DBMSes report pks as null
+          nullability = ColumnDefinition.Nullability.NOT_NULL;
+        }
+        ColumnDefinition defn = columnDefinition(
+                rs,
+                columnId,
+                jdbcType,
+                typeName,
+                typeClassName,
+                nullability,
+                ColumnDefinition.Mutability.UNKNOWN,
+                precision,
+                scale,
+                signed,
+                displaySize,
+                autoIncremented,
+                caseSensitive,
+                searchable,
+                currency,
+                isPrimaryKey
+        );
+        results.put(columnId, defn);
+      }
+      return results;
+    }
+  }
+
+  @Override
+  public Map<ColumnId, ColumnDefinition> describeColumns(Connection conn,
+        TableId tableId,ResultSetMetaData rsMetadata) throws SQLException {
+    ColumnDefAdjuster adjuster;
+    //querySql模式tableId会是空的
+    if(tableId == null){
+      adjuster = new ColumnDefAdjuster();
+    }else{
+      String catalog = tableId.catalogName() != null ? tableId.catalogName() : catalogPattern;
+      String schema = tableId.schemaName() != null ? tableId.schemaName() : schemaPattern;
+      adjuster = ColumnDefAdjuster.create(
+              conn, catalog, schema, tableId.tableName(), null);
+    }
+    Map<ColumnId, ColumnDefinition> result = new LinkedHashMap<>();
+    for (int i = 1; i <= rsMetadata.getColumnCount(); ++i) {
+      ColumnDefinition defn = describeColumn(rsMetadata, adjuster, i);
+      result.put(defn.id(), defn);
+    }
+    return result;
+  }
+
+  /**
+   * Create a definition for the specified column in the result set.
+   * @param rsMetadata the result set metadata; may not be null
+   * @param column     the column number, starting at 1 for the first column
+   * @return the column definition; never null
+   * @throws SQLException if there is an error accessing the result set metadata
+   */
+  protected ColumnDefinition describeColumn(
+          ResultSetMetaData rsMetadata,
+          ColumnDefAdjuster adjuster,
+          int column
+  ) throws SQLException {
+    String catalog = rsMetadata.getCatalogName(column);
+    String schema = rsMetadata.getSchemaName(column);
+    String tableName = rsMetadata.getTableName(column);
+    TableId tableId = new TableId(catalog, schema, tableName);
+    String name = rsMetadata.getColumnName(column);
+    String alias = rsMetadata.getColumnLabel(column);
+    ColumnId id = new ColumnId(tableId, name, alias);
+
+    ColumnDefinition.Nullability nullability = adjuster.nullable(name);
+    if(nullability == null){
+      nullability = columnIsNullAble(rsMetadata, column);
+    }
+
+    ColumnDefinition.Mutability mutability = ColumnDefinition.Mutability.MAYBE_WRITABLE;
+    if (rsMetadata.isReadOnly(column)) {
+      mutability = ColumnDefinition.Mutability.READ_ONLY;
+    } else if (rsMetadata.isWritable(column)) {
+      mutability = ColumnDefinition.Mutability.MAYBE_WRITABLE;
+    } else if (rsMetadata.isDefinitelyWritable(column)) {
+      mutability = ColumnDefinition.Mutability.WRITABLE;
+    }
+    return new ColumnDefinition(
+            id,
+            rsMetadata.getColumnType(column),
+            rsMetadata.getColumnTypeName(column),
+            rsMetadata.getColumnClassName(column),
+            nullability,
+            mutability,
+            rsMetadata.getPrecision(column),
+            rsMetadata.getScale(column),
+            rsMetadata.isSigned(column),
+            rsMetadata.getColumnDisplaySize(column),
+            rsMetadata.isAutoIncrement(column),
+            false,
+            rsMetadata.isSearchable(column),
+            rsMetadata.isCurrency(column),
+            false
+    );
+  }
+
+  protected ColumnDefinition.Nullability columnIsNullAble(ResultSetMetaData rsMetadata, int column) throws SQLException {
+    switch (rsMetadata.isNullable(column)) {
+      case ResultSetMetaData.columnNullable:
+        return ColumnDefinition.Nullability.NULL;
+      case ResultSetMetaData.columnNoNulls:
+        return ColumnDefinition.Nullability.NOT_NULL;
+      case ResultSetMetaData.columnNullableUnknown:
+      default:
+        return ColumnDefinition.Nullability.UNKNOWN;
+    }
+  }
+
+  protected Set<ColumnId> primaryKeyColumns(
+          Connection connection,
+          String catalogPattern,
+          String schemaPattern,
+          String tablePattern
+  ) throws SQLException {
+
+    // Get the primary keys of the table(s) ...
+    final Set<ColumnId> pkColumns = new HashSet<>();
+    try (ResultSet rs = connection.getMetaData().getPrimaryKeys(
+            catalogPattern, schemaPattern, tablePattern)) {
+      while (rs.next()) {
+        String catalogName = rs.getString(1);
+        String schemaName = rs.getString(2);
+        String tableName = rs.getString(3);
+        TableId tableId = new TableId(catalogName, schemaName, tableName);
+        final String colName = rs.getString(4);
+        ColumnId columnId = new ColumnId(tableId, colName);
+        pkColumns.add(columnId);
+      }
+    }
+    return pkColumns;
+  }
+
+  @Override
+  public Map<ColumnId, ColumnDefinition> describeColumnsByQuerying(
+          Connection db,
+          TableId tableId
+  ) throws SQLException {
+    String queryStr = "SELECT * FROM {} LIMIT 1";
+    String quotedName = expressionBuilder().append(tableId).toString();
+    try (PreparedStatement stmt = db.prepareStatement(queryStr)) {
+      stmt.setString(1, quotedName);
+      try (ResultSet rs = stmt.executeQuery()) {
+        ResultSetMetaData rsmd = rs.getMetaData();
+        return describeColumns(db, tableId, rsmd);
+      }
+    }
+  }
+
+
+  @Override
+  public TableDefinition describeTable(
+          Connection connection,
+          TableId tableId
+  ) throws SQLException {
+    Map<ColumnId, ColumnDefinition> columnDefns = describeColumns(connection, tableId.catalogName(),
+            tableId.schemaName(),
+            tableId.tableName(), null
+    );
+    if (columnDefns.isEmpty()) {
+      return null;
+    }
+    TableType tableType = tableTypeFor(connection, tableId);
+    return new TableDefinition(tableId, columnDefns.values(), tableType);
+  }
+
+  protected TableType tableTypeFor(
+          Connection connection,
+          TableId tableId
+  ) throws SQLException {
+    DatabaseMetaData metadata = connection.getMetaData();
+    String[] tableTypes = tableTypes(metadata, this.tableTypes);
+    String tableTypeDisplay = displayableTableTypes(tableTypes, "/");
+    glog.info("Checking {} dialect for type of {} {}", this, tableTypeDisplay, tableId);
+    try (ResultSet rs = connection.getMetaData().getTables(
+            tableId.catalogName(),
+            tableId.schemaName(),
+            tableId.tableName(),
+            tableTypes
+    )) {
+      if (rs.next()) {
+        //final String catalogName = rs.getString(1);
+        //final String schemaName = rs.getString(2);
+        //final String tableName = rs.getString(3);
+        final String tableType = rs.getString(4);
+        try {
+          return TableType.get(tableType);
+        } catch (IllegalArgumentException e) {
+          glog.warn(
+                  "{} dialect found unknown type '{}' for {} {}; using TABLE",
+                  this,
+                  tableType,
+                  tableTypeDisplay,
+                  tableId
+          );
+          return TableType.TABLE;
+        }
+      }
+    }
+    glog.warn(
+            "{} dialect did not find type for {} {}; using TABLE",
+            this,
+            tableTypeDisplay,
+            tableId
+    );
+    return TableType.TABLE;
+  }
+
+  /**
+   * Create a ColumnDefinition with supplied values and the result set from the {@link
+   * DatabaseMetaData#getColumns(String, String, String, String)} call. By default that method does
+   * not describe whether the column is signed, case sensitive, searchable, currency, or the
+   * preferred display size.
+   *
+   * <p>Subclasses can override this method to extract additional non-standard characteristics from
+   * the result set, and override the characteristics determined using the standard JDBC metadata
+   * columns and supplied as parameters.
+   *
+   * @param resultSet        the result set
+   * @param id               the column identifier
+   * @param jdbcType         the JDBC type of the column
+   * @param typeName         the name of the column's type
+   * @param classNameForType the name of the class used as instances of the value when {@link
+   *                         ResultSet#getObject(int)} is called
+   * @param nullability      the nullability of the column
+   * @param mutability       the mutability of the column
+   * @param precision        the precision of the column for numeric values, or the length for
+   *                         non-numeric values
+   * @param scale            the scale of the column for numeric values; ignored for other values
+   * @param signedNumbers    true if the column holds signed numeric values; null if not known
+   * @param displaySize      the preferred display size for the column values; null if not known
+   * @param autoIncremented  true if the column is auto-incremented; null if not known
+   * @param caseSensitive    true if the column values are case-sensitive; null if not known
+   * @param searchable       true if the column is searchable; null if no; null if not known known
+   * @param currency         true if the column is a currency value
+   * @param isPrimaryKey     true if the column is part of the primary key; null if not known known
+   * @return the column definition; never null
+   */
+  protected ColumnDefinition columnDefinition(
+          ResultSet resultSet,
+          ColumnId id,
+          int jdbcType,
+          String typeName,
+          String classNameForType,
+          ColumnDefinition.Nullability nullability,
+          ColumnDefinition.Mutability mutability,
+          int precision,
+          int scale,
+          Boolean signedNumbers,
+          Integer displaySize,
+          Boolean autoIncremented,
+          Boolean caseSensitive,
+          Boolean searchable,
+          Boolean currency,
+          Boolean isPrimaryKey
+  ) {
+    return new ColumnDefinition(
+            id,
+            jdbcType,
+            typeName,
+            classNameForType,
+            nullability,
+            mutability,
+            precision,
+            scale,
+            signedNumbers != null ? signedNumbers.booleanValue() : false,
+            displaySize != null ? displaySize.intValue() : 0,
+            autoIncremented != null ? autoIncremented.booleanValue() : false,
+            caseSensitive != null ? caseSensitive.booleanValue() : false,
+            searchable != null ? searchable.booleanValue() : false,
+            currency != null ? currency.booleanValue() : false,
+            isPrimaryKey != null ? isPrimaryKey.booleanValue() : false
+    );
+  }
+
+//  @Override
+//  public TimestampIncrementingCriteria criteriaFor(
+//          ColumnId incrementingColumn,
+//          Long incrementingColumnStep,
+//          List<ColumnId> timestampColumns,
+//          Long timestampColumnStep
+//  ) {
+//    return new TimestampIncrementingCriteria(
+//            incrementingColumn, incrementingColumnStep, timestampColumns,timestampColumnStep, timeZone);
+//  }
+
+  /**
+   * Determine the name of the field. By default this is the column alias or name.
+   * @param columnDefinition the column definition; never null
+   * @return the field name; never null
+   */
+  protected String fieldNameFor(ColumnDefinition columnDefinition) {
+    return columnDefinition.id().aliasOrName();
+  }
+
+
+  @Override
+  public String addFieldToSchema(
+          ColumnDefinition columnDefn,
+          Schema schema,
+          int index
+  ) {
+    return addFieldToSchema(columnDefn, schema, fieldNameFor(columnDefn), index,columnDefn.type(),
+            columnDefn.isOptional()
+    );
+  }
+
+  /**
+   * add field to schema
+   * @param columnDefn
+   * @param builder
+   * @param fieldName
+   * @param index
+   * @param sqlType
+   * @param optional
+   * @return
+   */
+  @SuppressWarnings("fallthrough")
+  protected String addFieldToSchema(
+          final ColumnDefinition columnDefn,
+          final Schema builder,
+          final String fieldName,
+          final int index,
+          final int sqlType,
+          final boolean optional
+  ) {
+    int precision = columnDefn.precision();
+    int scale = columnDefn.scale();
+    switch (sqlType) {
+      case Types.NULL: {
+        glog.debug("JDBC type 'NULL' not currently supported for column '{}'", fieldName);
+        return null;
+      }
+      case Types.BOOLEAN: {
+        builder.addField(new Field(index, fieldName, SchemaBuilder.bool().build()));
+        break;
+      }
+
+      // ints <= 8 bits
+      case Types.BIT: {
+        builder.addField(new Field(index, fieldName, SchemaBuilder.int8().build()));
+        break;
+      }
+
+      case Types.TINYINT: {
+        if (columnDefn.isSignedNumber()) {
+          builder.addField(new Field(index, fieldName, SchemaBuilder.int8().build()));
+        } else {
+          builder.addField(new Field(index, fieldName, SchemaBuilder.int32().build()));
+        }
+        break;
+      }
+
+      // 16 bit ints
+      case Types.SMALLINT: {
+        builder.addField(new Field(index, fieldName, SchemaBuilder.int32().build()));
+        break;
+      }
+
+      // 32 bit ints
+      case Types.INTEGER: {
+        if (columnDefn.isSignedNumber()) {
+          builder.addField(new Field(index, fieldName, SchemaBuilder.int32().build()));
+        } else {
+          builder.addField(new Field(index, fieldName, SchemaBuilder.int64().build()));
+        }
+        break;
+      }
+
+      // 64 bit ints
+      case Types.BIGINT: {
+        builder.addField(new Field(index, fieldName, SchemaBuilder.int64().build()));
+        break;
+      }
+
+      // REAL is a single precision floating point value, i.e. a Java float
+      case Types.REAL: {
+        builder.addField(new Field(index, fieldName, SchemaBuilder.float32().build()));
+        break;
+      }
+
+      // FLOAT is, confusingly, double precision and effectively the same as DOUBLE. See REAL
+      // for single precision
+      case Types.FLOAT:
+      case Types.DOUBLE:
+      case Types.DECIMAL:
+        builder.addField(new Field(index, fieldName, SchemaBuilder.float64().build()));
+        break;
+
+      /**
+       * numeric
+       */
+      case Types.NUMERIC:
+        if (mapNumerics == NumericMapping.PRECISION_ONLY) {
+          glog.debug("NUMERIC with precision: '{}' and scale: '{}'", precision, scale);
+          if (scale == 0 && precision <= MAX_INTEGER_TYPE_PRECISION) { // integer
+            builder.addField(new Field(index, fieldName, integerSchema(optional, precision)));
+            break;
+          }
+        } else if (mapNumerics == NumericMapping.BEST_FIT) {
+          glog.debug("NUMERIC with precision: '{}' and scale: '{}'", precision, scale);
+          if (precision <= MAX_INTEGER_TYPE_PRECISION) { // fits in primitive data types.
+            if (scale < 1 && scale >= NUMERIC_TYPE_SCALE_LOW) { // integer
+              builder.addField(new Field(index,fieldName, integerSchema(optional, precision)));
+              break;
+            } else if (scale > 0) { // floating point - use double in all cases
+              builder.addField(new Field(index, fieldName, SchemaBuilder.float64().build()));
+              break;
+            }
+          }
+        } else if (mapNumerics == NumericMapping.BEST_FIT_EAGER_DOUBLE) {
+          glog.debug("NUMERIC with precision: '{}' and scale: '{}'", precision, scale);
+          if (scale < 1 && scale >= NUMERIC_TYPE_SCALE_LOW) { // integer
+            if (precision <= MAX_INTEGER_TYPE_PRECISION) { // fits in primitive data types.
+              builder.addField(new Field(index, fieldName, integerSchema(optional, precision)));
+              break;
+            }
+          } else if (scale > 0) { // floating point - use double in all cases
+            builder.addField(new Field(index, fieldName, SchemaBuilder.float64().build()));
+            break;
+          }
+        }
+
+      case Types.CHAR:
+      case Types.VARCHAR:
+      case Types.LONGVARCHAR:
+      case Types.NCHAR:
+      case Types.NVARCHAR:
+      case Types.LONGNVARCHAR:
+      case Types.CLOB:
+      case Types.NCLOB:
+      case Types.DATALINK:
+      case Types.SQLXML: {
+        // Some of these types will have fixed size, but we drop this from the schema conversion
+        // since only fixed byte arrays can have a fixed size
+        builder.addField(new Field(index, fieldName, SchemaBuilder.string().build()));
+        break;
+      }
+
+      // Binary == fixed bytes
+      // BLOB, VARBINARY, LONGVARBINARY == bytes
+      case Types.BINARY:
+      case Types.BLOB:
+      case Types.VARBINARY:
+      case Types.LONGVARBINARY: {
+        builder.addField(new Field(index, fieldName, SchemaBuilder.bytes().build()));
+        break;
+      }
+
+      // Date is day + moth + year
+      case Types.DATE: {
+        SchemaBuilder dateSchemaBuilder = DateColumnParser.builder();
+        builder.addField(new Field(index, fieldName,dateSchemaBuilder.build()));
+        break;
+      }
+
+      // Time is a time of day -- hour, minute, seconds, nanoseconds
+      case Types.TIME: {
+        SchemaBuilder timeSchemaBuilder = TimestampColumnParser.builder();
+        builder.addField(new Field(index, fieldName, timeSchemaBuilder.build()));
+        break;
+      }
+
+      // Timestamp is a date + time
+      case Types.TIMESTAMP: {
+        SchemaBuilder tsSchemaBuilder = TimestampColumnParser.builder();
+        builder.addField(new Field(index, fieldName,tsSchemaBuilder.build()));
+        break;
+      }
+
+      case Types.ARRAY:
+      case Types.JAVA_OBJECT:
+      case Types.OTHER:
+      case Types.DISTINCT:
+      case Types.STRUCT:
+      case Types.REF:
+      case Types.ROWID:
+      default: {
+        glog.warn("JDBC type {} ({}) not currently supported", sqlType, columnDefn.typeName());
+        return null;
+      }
+    }
+    return fieldName;
+  }
+
+  private Schema integerSchema(boolean optional, int precision) {
+    Schema schema;
+    if (precision > 9) {
+      schema =SchemaBuilder.int64().build();
+    } else if (precision > 2) {
+      schema =SchemaBuilder.int32().build();
+    } else {
+      schema =SchemaBuilder.int8().build();
+    }
+    return schema;
+  }
+
+  /**
+   * execute ddl
+   * @param connection the connection to use
+   * @param statements the list of DDL statements to execute
+   * @throws SQLException
+   */
+  @Override
+  public void applyDdlStatements(
+          Connection connection,
+          List<String> statements
+  ) throws SQLException {
+    try (Statement statement = connection.createStatement()) {
+      for (String ddlStatement : statements) {
+        statement.executeUpdate(ddlStatement);
+      }
+    }
+  }
+
+  @Override
+  public ColumnConverter createColumnConverter(
+          ColumnMapping mapping
+  ) {
+    return columnConverterFor(
+            mapping,
+            mapping.columnDefn(),
+            mapping.columnNumber(),
+            jdbcDriverInfo().jdbcVersionAtLeast(4, 0)
+    );
+  }
+
+  @SuppressWarnings({"deprecation", "fallthrough"})
+  protected ColumnConverter columnConverterFor(
+          final ColumnMapping mapping,
+          final ColumnDefinition defn,
+          final int col,
+          final boolean isJdbc4
+  ) {
+    switch (mapping.columnDefn().type()) {
+
+      case Types.BOOLEAN: {
+        return rs -> rs.getBoolean(col);
+      }
+
+      case Types.BIT: {
+        /**
+         * BIT should be either 0 or 1.
+         * TODO: Postgres handles this differently, returning a string "t" or "f". See the
+         * elasticsearch-jdbc plugin for an example of how this is handled
+         */
+        return rs -> rs.getByte(col);
+      }
+
+      // 8 bits int
+      case Types.TINYINT: {
+        if (defn.isSignedNumber()) {
+          return rs -> rs.getByte(col);
+        } else {
+          return rs -> rs.getShort(col);
+        }
+      }
+
+      // 16 bits int
+      case Types.SMALLINT: {
+        if (defn.isSignedNumber()) {
+          return rs -> rs.getShort(col);
+        } else {
+          return rs -> rs.getInt(col);
+        }
+      }
+
+      // 32 bits int
+      case Types.INTEGER: {
+        if (defn.isSignedNumber()) {
+          return rs -> rs.getInt(col);
+        } else {
+          return rs -> rs.getLong(col);
+        }
+      }
+
+      // 64 bits int
+      case Types.BIGINT: {
+        return rs -> rs.getLong(col);
+      }
+
+      // REAL is a single precision floating point value, i.e. a Java float
+      case Types.REAL: {
+        return rs -> rs.getFloat(col);
+      }
+
+      // FLOAT is, confusingly, double precision and effectively the same as DOUBLE. See REAL
+      // for single precision
+      case Types.FLOAT:
+      case Types.DOUBLE: {
+        return rs -> rs.getDouble(col);
+      }
+
+      case Types.NUMERIC:
+        if (mapNumerics == NumericMapping.PRECISION_ONLY) {
+          int precision = defn.precision();
+          int scale = defn.scale();
+          glog.trace("NUMERIC with precision: '{}' and scale: '{}'", precision, scale);
+          if (scale == 0 && precision <= MAX_INTEGER_TYPE_PRECISION) { // integer
+            if (precision > 9) {
+              return rs -> rs.getLong(col);
+            } else if (precision > 4) {
+              return rs -> rs.getInt(col);
+            } else if (precision > 2) {
+              return rs -> rs.getShort(col);
+            } else {
+              return rs -> rs.getByte(col);
+            }
+          }
+        } else if (mapNumerics == NumericMapping.BEST_FIT) {
+          int precision = defn.precision();
+          int scale = defn.scale();
+          glog.trace("NUMERIC with precision: '{}' and scale: '{}'", precision, scale);
+          if (precision <= MAX_INTEGER_TYPE_PRECISION) { // fits in primitive data types.
+            if (scale < 1 && scale >= NUMERIC_TYPE_SCALE_LOW) { // integer
+              if (precision > 9) {
+                return rs -> rs.getLong(col);
+              } else if (precision > 4) {
+                return rs -> rs.getInt(col);
+              } else if (precision > 2) {
+                return rs -> rs.getShort(col);
+              } else {
+                return rs -> rs.getByte(col);
+              }
+            } else if (scale > 0) { // floating point - use double in all cases
+              return rs -> rs.getDouble(col);
+            }
+          }
+        } else if (mapNumerics == NumericMapping.BEST_FIT_EAGER_DOUBLE) {
+          int precision = defn.precision();
+          int scale = defn.scale();
+          glog.trace("NUMERIC with precision: '{}' and scale: '{}'", precision, scale);
+          if (scale < 1 && scale >= NUMERIC_TYPE_SCALE_LOW) { // integer
+            if (precision <= MAX_INTEGER_TYPE_PRECISION) { // fits in primitive data types.
+              if (precision > 9) {
+                return rs -> rs.getLong(col);
+              } else if (precision > 4) {
+                return rs -> rs.getInt(col);
+              } else if (precision > 2) {
+                return rs -> rs.getShort(col);
+              } else {
+                return rs -> rs.getByte(col);
+              }
+            }
+          } else if (scale > 0) { // floating point - use double in all cases
+            return rs -> rs.getDouble(col);
+          }
+        }
+        // fallthrough
+
+      case Types.DECIMAL: {
+        final int precision = defn.precision();
+        glog.debug("DECIMAL with precision: '{}' and scale: '{}'", precision, defn.scale());
+        final int scale = decimalScale(defn);
+        return rs -> rs.getBigDecimal(col, scale);
+      }
+
+      case Types.CHAR:
+      case Types.VARCHAR:
+      case Types.LONGVARCHAR: {
+        return rs -> rs.getString(col);
+      }
+
+      case Types.NCHAR:
+      case Types.NVARCHAR:
+      case Types.LONGNVARCHAR: {
+        return rs -> rs.getNString(col);
+      }
+
+      // Binary == fixed, VARBINARY and LONGVARBINARY == bytes
+      case Types.BINARY:
+      case Types.VARBINARY:
+      case Types.LONGVARBINARY: {
+        return rs -> rs.getBytes(col);
+      }
+
+      // Date is day + month + year
+      case Types.DATE: {
+        return rs -> rs.getDate(col,
+                DateTimeUtils.getTimeZoneCalendar(TimeZone.getTimeZone(ZoneOffset.UTC)));
+      }
+
+      // Time is a time of day -- hour, minute, seconds, nanoseconds
+      case Types.TIME: {
+        return rs -> rs.getTime(col, DateTimeUtils.getTimeZoneCalendar(timeZone));
+      }
+
+      // Timestamp is a date + time
+      case Types.TIMESTAMP: {
+        return rs -> rs.getTimestamp(col, DateTimeUtils.getTimeZoneCalendar(timeZone));
+      }
+
+      // Datalink is basically a URL -> string
+      case Types.DATALINK: {
+        return rs -> {
+          URL url = rs.getURL(col);
+          return (url != null ? url.toString() : null);
+        };
+      }
+
+      // BLOB == fixed
+      case Types.BLOB: {
+        return rs -> {
+          Blob blob = rs.getBlob(col);
+          if (blob == null) {
+            return null;
+          } else {
+            try {
+              if (blob.length() > Integer.MAX_VALUE) {
+                throw new IOException("Can't process BLOBs longer than " + Integer.MAX_VALUE);
+              }
+              return blob.getBytes(1, (int) blob.length());
+            } finally {
+              if (isJdbc4) {
+                free(blob);
+              }
+            }
+          }
+        };
+      }
+      case Types.CLOB:
+        return rs -> {
+          Clob clob = rs.getClob(col);
+          if (clob == null) {
+            return null;
+          } else {
+            try {
+              if (clob.length() > Integer.MAX_VALUE) {
+                throw new IOException("Can't process CLOBs longer than " + Integer.MAX_VALUE);
+              }
+              return clob.getSubString(1, (int) clob.length());
+            } finally {
+              if (isJdbc4) {
+                free(clob);
+              }
+            }
+          }
+        };
+      case Types.NCLOB: {
+        return rs -> {
+          Clob clob = rs.getNClob(col);
+          if (clob == null) {
+            return null;
+          } else {
+            try {
+              if (clob.length() > Integer.MAX_VALUE) {
+                throw new IOException("Can't process NCLOBs longer than " + Integer.MAX_VALUE);
+              }
+              return clob.getSubString(1, (int) clob.length());
+            } finally {
+              if (isJdbc4) {
+                free(clob);
+              }
+            }
+          }
+        };
+      }
+
+      // XML -> string
+      case Types.SQLXML: {
+        return rs -> {
+          SQLXML xml = rs.getSQLXML(col);
+          return xml != null ? xml.getString() : null;
+        };
+      }
+
+      case Types.NULL:
+      case Types.ARRAY:
+      case Types.JAVA_OBJECT:
+      case Types.OTHER:
+      case Types.DISTINCT:
+      case Types.STRUCT:
+      case Types.REF:
+      case Types.ROWID:
+      default: {
+        // These are not currently supported, but we don't want to log something for every single
+        // record we translate. There will already be errors logged for the schema translation
+        break;
+      }
+    }
+    return null;
+  }
+
+  protected int decimalScale(ColumnDefinition defn) {
+    return defn.scale() == NUMERIC_TYPE_SCALE_UNSET ? NUMERIC_TYPE_SCALE_HIGH : defn.scale();
+  }
+
+  /**
+   * Called when the object has been fully read and {@link Blob#free()} should be called.
+   *
+   * @param blob the Blob; never null
+   * @throws SQLException if there is a problem calling free()
+   */
+  protected void free(Blob blob) throws SQLException {
+    blob.free();
+  }
+
+  /**
+   * Called when the object has been fully read and {@link Clob#free()} should be called.
+   *
+   * @param clob the Clob; never null
+   * @throws SQLException if there is a problem calling free()
+   */
+  protected void free(Clob clob) throws SQLException {
+    clob.free();
+  }
+
+  @Override
+  @SuppressWarnings("deprecation")
+  public String buildInsertStatement(
+          TableId table,
+          Collection<ColumnId> keyColumns,
+          Collection<ColumnId> nonKeyColumns
+  ) {
+    ExpressionBuilder builder = expressionBuilder();
+    builder.append("INSERT INTO ");
+    builder.append(table);
+    builder.append("(");
+    builder.appendList()
+            .delimitedBy(",")
+            .transformedBy(ExpressionBuilder.columnNames())
+            .of(keyColumns, nonKeyColumns);
+    builder.append(") VALUES(");
+    builder.appendMultiple(",", "?", keyColumns.size() + nonKeyColumns.size());
+    builder.append(")");
+    return builder.toString();
+  }
+
+  @Override
+  public String buildUpdateStatement(
+          TableId table,
+          Collection<ColumnId> keyColumns,
+          Collection<ColumnId> nonKeyColumns
+  ) {
+    ExpressionBuilder builder = expressionBuilder();
+    builder.append("UPDATE ");
+    builder.append(table);
+    builder.append(" SET ");
+    builder.appendList()
+            .delimitedBy(", ")
+            .transformedBy(ExpressionBuilder.columnNamesWith(" = ?"))
+            .of(nonKeyColumns);
+    if (!keyColumns.isEmpty()) {
+      builder.append(" WHERE ");
+      builder.appendList()
+              .delimitedBy(" AND ")
+              .transformedBy(ExpressionBuilder.columnNamesWith(" = ?"))
+              .of(keyColumns);
+    }
+    return builder.toString();
+  }
+
+  @Override
+  @SuppressWarnings("deprecation")
+  public String buildUpsertQueryStatement(
+          TableId table,
+          Collection<ColumnId> keyColumns,
+          Collection<ColumnId> nonKeyColumns
+  ) {
+    throw new UnsupportedOperationException();
+  }
+
+  @Override
+  public final String buildDeleteStatement(
+          TableId table,
+          Collection<ColumnId> keyColumns
+  ) {
+    ExpressionBuilder builder = expressionBuilder();
+    builder.append("DELETE FROM ");
+    builder.append(table);
+    if (!keyColumns.isEmpty()) {
+      builder.append(" WHERE ");
+      builder.appendList()
+              .delimitedBy(" AND ")
+              .transformedBy(ExpressionBuilder.columnNamesWith(" = ?"))
+              .of(keyColumns);
+    }
+    return builder.toString();
+  }
+
+  /**
+   * table moode
+   * @return
+   */
+  @Override
+  public String buildSelectTableMode() {
+    return "SELECT * FROM ";
+  }
+
+  @Override
+  public void buildSelectTable(ExpressionBuilder builder, TableId tableId) {
+    String mode = buildSelectTableMode();
+    builder.append(mode).append(tableId);
+  }
+
+  @Override
+  public StatementBinder statementBinder(
+          PreparedStatement statement,
+          JdbcSinkConfig.PrimaryKeyMode pkMode,
+          SchemaPair schemaPair,
+          FieldsMetadata fieldsMetadata,
+          TableDefinition tableDefinition,
+          JdbcSinkConfig.InsertMode insertMode) {
+    return new PreparedStatementBinder(
+            this,
+            statement,
+            pkMode,
+            schemaPair,
+            fieldsMetadata,
+            tableDefinition,
+            insertMode
+    );
+  }
+
+  @Override
+  public void bindField(
+          PreparedStatement statement,
+          int index, Schema schema,
+          Object value,
+          ColumnDefinition colDef) throws SQLException {
+    if (value == null) {
+      Integer type = getSqlTypeForSchema(schema);
+      if (type != null) {
+        statement.setNull(index, type);
+      } else {
+        statement.setObject(index, null);
+      }
+    } else {
+      boolean bound = maybeBindLogical(statement, index, schema, value, null);
+      if (!bound) {
+        bound = maybeBindPrimitive(statement, index, schema, value);
+      }
+      if (!bound) {
+        throw new io.openmessaging.connector.api.errors.ConnectException("Unsupported source data type: " + schema.getFieldType());
+      }
+    }
+  }
+
+  protected boolean maybeBindLogical(
+          PreparedStatement statement,
+          int index,
+          Schema schema,
+          Object value,
+          ColumnDefinition colDef
+  ) throws SQLException {
+    if (schema.getName() != null) {
+      switch (schema.getName()) {
+        case DateColumnParser.LOGICAL_NAME:
+          java.sql.Date date;
+          if(value instanceof java.util.Date){
+            date = new java.sql.Date(((java.util.Date) value).getTime());
+          }else{
+            date = new java.sql.Date((int)value);
+          }
+          statement.setDate(
+                  index, date,
+                  DateTimeUtils.getTimeZoneCalendar(timeZone)
+          );
+          return true;
+        case TimeColumnParser.LOGICAL_NAME:
+          java.sql.Time time;
+          if(value instanceof java.util.Date){
+            time = new java.sql.Time(((java.util.Date) value).getTime());
+          }else{
+            time = new java.sql.Time((int)value);
+          }
+          statement.setTime(
+                  index, time,
+                  DateTimeUtils.getTimeZoneCalendar(timeZone)
+          );
+          return true;
+        case TimestampColumnParser.LOGICAL_NAME:
+          java.sql.Timestamp timestamp;
+          if(value instanceof java.util.Date){
+            timestamp = new java.sql.Timestamp(((java.util.Date) value).getTime());
+          }else{
+            timestamp = new java.sql.Timestamp((long)value);
+          }
+          statement.setTimestamp(
+                  index, timestamp,
+                  DateTimeUtils.getTimeZoneCalendar(timeZone)
+          );
+          return true;
+        default:
+          return false;
+      }
+    }
+    return false;
+  }
+
+
+  @Override
+  public TimestampIncrementingCriteria criteriaFor(ColumnId incrementingColumn, List<ColumnId> timestampColumns) {
+    return new TimestampIncrementingCriteria(
+            incrementingColumn, timestampColumns, timeZone);
+  }
+
+  @Override
+  public Long getMinTimestampValue(Connection con, String tableOrQuery, List<String> timestampColumns) throws SQLException {
+    if(timestampColumns==null || timestampColumns.isEmpty()){
+      return null;
+    }
+    StringBuilder builder = new StringBuilder();
+    builder.append("SELECT ");
+    boolean appendComma = false;
+    for(String column : timestampColumns){
+      builder.append("MIN(");
+      builder.append(column);
+      builder.append(")");
+      if(appendComma){
+        builder.append(",");
+      }else{
+        appendComma = true;
+      }
+    }
+    builder.append(" FROM ");
+    builder.append(tableOrQuery);
+    String querySql = builder.toString();
+    PreparedStatement st = con.prepareStatement(querySql);
+    ResultSet resultSet = st.executeQuery();
+    ResultSetMetaData metaData = resultSet.getMetaData();
+    long minTimestampValue = Long.MAX_VALUE;
+    for(int i = 1; i <= metaData.getColumnCount(); ++i){
+      long t = resultSet.getLong(i);
+      minTimestampValue = Math.min(minTimestampValue, t);
+    }
+    st.close();
+    return minTimestampValue;
+  }
+
+  /**
+   * Dialects not supporting `setObject(index, null)` can override this method
+   * to provide a specific sqlType, as per the JDBC documentation
+   * https://docs.oracle.com/javase/7/docs/api/java/sql/PreparedStatement.html
+   *
+   * @param schema the schema
+   * @return the SQL type
+   */
+  protected Integer getSqlTypeForSchema(Schema schema) {
+    return null;
+  }
+
+  protected boolean maybeBindPrimitive(
+          PreparedStatement statement,
+          int index,
+          Schema schema,
+          Object value
+  ) throws SQLException {
+    switch (schema.getFieldType()) {
+      case INT8:
+        statement.setByte(index, (Byte) value);
+        break;
+      case INT32:
+        statement.setInt(index, (Integer) value);
+        break;
+      case INT64:
+        statement.setLong(index, (Long) value);
+        break;
+      case FLOAT32:
+        statement.setFloat(index, (Float) value);
+        break;
+      case FLOAT64:
+        statement.setDouble(index, (Double) value);
+        break;
+      case BOOLEAN:
+        statement.setBoolean(index, (Boolean) value);
+        break;
+      case STRING:
+        statement.setString(index, (String) value);
+        break;
+      case BYTES:
+        final byte[] bytes;
+        if (value instanceof ByteBuffer) {
+          final ByteBuffer buffer = ((ByteBuffer) value).slice();
+          bytes = new byte[buffer.remaining()];
+          buffer.get(bytes);
+        } else {
+          bytes = (byte[]) value;
+        }
+        statement.setBytes(index, bytes);
+        break;
+      case DATETIME:
+        java.sql.Date date;
+        if(value instanceof java.util.Date){
+          date = new java.sql.Date(((java.util.Date) value).getTime());
+        }else{
+          date = new java.sql.Date((int)value);
+        }
+        statement.setDate(
+                index, date,
+                DateTimeUtils.getTimeZoneCalendar(timeZone)
+        );
+
+
+
+        break;
+      default:
+        return false;
+    }
+    return true;
+  }
+
+  @Override
+  public String buildCreateTableStatement(
+          TableId table,
+          Collection<SinkRecordField> fields) {
+    ExpressionBuilder builder = expressionBuilder();
+    final List<String> pkFieldNames = extractPrimaryKeyFieldNames(fields);
+    builder.append("CREATE TABLE ");
+    builder.append(table);
+    builder.append(" (");
+    writeColumnsSpec(builder, fields);
+    if (!pkFieldNames.isEmpty()) {
+      builder.append(",");
+      builder.append(System.lineSeparator());
+      builder.append("PRIMARY KEY(");
+      builder.appendList()
+              .delimitedBy(",")
+              .transformedBy(ExpressionBuilder.quote())
+              .of(pkFieldNames);
+      builder.append(")");
+    }
+    builder.append(")");
+    return builder.toString();
+  }
+
+  @Override
+  public String buildDropTableStatement(
+          TableId table,
+          DropOptions options
+  ) {
+    ExpressionBuilder builder = expressionBuilder();
+
+    builder.append("DROP TABLE ");
+    builder.append(table);
+    if (options.ifExists()) {
+      builder.append(" IF EXISTS");
+    }
+    if (options.cascade()) {
+      builder.append(" CASCADE");
+    }
+    return builder.toString();
+  }
+
+  @Override
+  public List<String> buildAlterTable(
+          TableId table,
+          Collection<SinkRecordField> fields
+  ) {
+    final boolean newlines = fields.size() > 1;
+
+    final ExpressionBuilder.Transform<SinkRecordField> transform = (builder, field) -> {
+      if (newlines) {
+        builder.appendNewLine();
+      }
+      builder.append("ADD ");
+      writeColumnSpec(builder, field);
+    };
+
+    ExpressionBuilder builder = expressionBuilder();
+    builder.append("ALTER TABLE ");
+    builder.append(table);
+    builder.append(" ");
+    builder.appendList()
+            .delimitedBy(",")
+            .transformedBy(transform)
+            .of(fields);
+    return Collections.singletonList(builder.toString());
+  }
+
+  @Override
+  public void validateColumnTypes(
+          ResultSetMetaData rsMetadata,
+          List<ColumnId> columns
+  ) throws io.openmessaging.connector.api.errors.ConnectException { }
+
+  protected List<String> extractPrimaryKeyFieldNames(Collection<SinkRecordField> fields) {
+    final List<String> pks = new ArrayList<>();
+    for (SinkRecordField f : fields) {
+      // todo add check column info
+      if (f.isPrimaryKey()) {
+        pks.add(f.name());
+      }
+    }
+    return pks;
+  }
+
+  protected void writeColumnsSpec(
+          ExpressionBuilder builder,
+          Collection<SinkRecordField> fields
+  ) {
+    ExpressionBuilder.Transform<SinkRecordField> transform = (b, field) -> {
+      b.append(System.lineSeparator());
+      writeColumnSpec(b, field);
+    };
+    builder.appendList().delimitedBy(",").transformedBy(transform).of(fields);
+  }
+
+  protected void writeColumnSpec(
+          ExpressionBuilder builder,
+          SinkRecordField f
+  ) {
+    builder.appendColumnName(f.name());
+    builder.append(" ");
+    String sqlType = getSqlType(f);
+    builder.append(sqlType);
+    if (f.defaultValue() != null) {
+      builder.append(" DEFAULT ");
+      formatColumnValue(
+              builder,
+              f.schemaType(),
+              f.defaultValue()
+      );
+    } else if (isColumnOptional(f)) {
+      builder.append(" NULL");
+    } else {
+      builder.append(" NOT NULL");
+    }
+  }
+
+  protected boolean isColumnOptional(SinkRecordField field) {
+    return field.isOptional();
+  }
+
+  protected void formatColumnValue(
+          ExpressionBuilder builder,
+          FieldType type,
+          Object value) {
+    switch (type) {
+      case INT8:
+      case INT32:
+      case INT64:
+      case FLOAT32:
+      case FLOAT64:
+        // no escaping required
+        builder.append(value);
+        break;
+      case BOOLEAN:
+        // 1 & 0 for boolean is more portable rather than TRUE/FALSE
+        builder.append((Boolean) value ? '1' : '0');
+        break;
+      case STRING:
+        builder.appendStringQuoted(value);
+        break;
+      case BYTES:
+        final byte[] bytes;
+        if (value instanceof ByteBuffer) {
+          final ByteBuffer buffer = ((ByteBuffer) value).slice();
+          bytes = new byte[buffer.remaining()];
+          buffer.get(bytes);
+        } else {
+          bytes = (byte[]) value;
+        }
+        builder.appendBinaryLiteral(bytes);
+        break;
+      default:
+        throw new io.openmessaging.connector.api.errors.ConnectException("Unsupported type for column value: " + type);
+    }
+  }
+
+  protected String getSqlType(SinkRecordField f) {
+    throw new io.openmessaging.connector.api.errors.ConnectException(String.format(
+            "%s (%s) type doesn't have a mapping to the SQL database column type", f.schemaName(),
+            f.schemaType()
+    ));
+  }
+
+
+  /**
+   * 隐藏密码
+   * @param url
+   * @return
+   */
+  protected String sanitizedUrl(String url) {
+    // Only replace standard URL-type properties ...
+    return url.replaceAll("(?i)([?&]([^=&]*)password([^=&]*)=)[^&]*", "$1****");
+  }
+
+  @Override
+  public String identifier() {
+    return name() + " database " + sanitizedUrl(jdbcUrl);
+  }
+
+  @Override
+  public String toString() {
+    return name();
+  }
+}
diff --git a/connectors/rocketmq-connect-jdbc/src/main/java/org/apache/rocketmq/connect/jdbc/dialect/impl/MySqlDatabaseDialect.java b/connectors/rocketmq-connect-jdbc/src/main/java/org/apache/rocketmq/connect/jdbc/dialect/impl/MySqlDatabaseDialect.java
new file mode 100644
index 0000000..b6bd1dc
--- /dev/null
+++ b/connectors/rocketmq-connect-jdbc/src/main/java/org/apache/rocketmq/connect/jdbc/dialect/impl/MySqlDatabaseDialect.java
@@ -0,0 +1,149 @@
+/*
+ * 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.rocketmq.connect.jdbc.dialect.impl;
+
+import lombok.SneakyThrows;
+import org.apache.rocketmq.connect.jdbc.config.AbstractConfig;
+import org.apache.rocketmq.connect.jdbc.dialect.DatabaseDialect;
+import org.apache.rocketmq.connect.jdbc.sink.metadata.SinkRecordField;
+import org.apache.rocketmq.connect.jdbc.dialect.provider.DatabaseDialectProvider;
+import org.apache.rocketmq.connect.jdbc.schema.column.ColumnId;
+import org.apache.rocketmq.connect.jdbc.util.ExpressionBuilder;
+import org.apache.rocketmq.connect.jdbc.util.IdentifierRules;
+import org.apache.rocketmq.connect.jdbc.schema.table.TableId;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import java.sql.PreparedStatement;
+import java.sql.ResultSet;
+import java.sql.SQLException;
+import java.util.Collection;
+
+/**
+ * mysql database dialect
+ */
+public class MySqlDatabaseDialect extends GenericDatabaseDialect {
+
+  private final Logger log = LoggerFactory.getLogger(MySqlDatabaseDialect.class);
+
+  /**
+   * The provider for {@link MySqlDatabaseDialect}.
+   */
+  public static class Provider extends DatabaseDialectProvider {
+    @SneakyThrows
+    public Provider() {
+      super(MySqlDatabaseDialect.class.getSimpleName(), "mysql");
+      Class.forName("com.mysql.cj.jdbc.Driver");
+    }
+
+    @Override
+    public DatabaseDialect create(AbstractConfig config) {
+      return new MySqlDatabaseDialect(config);
+    }
+  }
+
+  /**
+   * Create a new dialect instance with the given connector configuration.
+   *
+   * @param config the connector configuration; may not be null
+   */
+  public MySqlDatabaseDialect(AbstractConfig config) {
+    super(config, new IdentifierRules(".", "`", "`"));
+  }
+
+  /**
+   * initialize prepared statement
+   * @param stmt
+   * @throws SQLException
+   */
+  @Override
+  protected void initializePreparedStatement(PreparedStatement stmt) throws SQLException {
+    stmt.setFetchSize(Integer.MIN_VALUE);
+    log.trace("Initializing PreparedStatement fetch direction to FETCH_FORWARD for '{}'", stmt);
+    stmt.setFetchDirection(ResultSet.FETCH_FORWARD);
+  }
+
+  /**
+   * get sql type
+   * @param field
+   * @return
+   */
+  @Override
+  protected String getSqlType(SinkRecordField field) {
+    switch (field.schemaType()) {
+      case INT8:
+        return "TINYINT";
+      case INT32:
+        return "INT";
+      case INT64:
+        return "BIGINT";
+      case FLOAT32:
+        return "FLOAT";
+      case FLOAT64:
+        return "DOUBLE";
+      case BOOLEAN:
+        return "TINYINT";
+      case STRING:
+        return "TEXT";
+      case BYTES:
+        return "VARBINARY(1024)";
+      default:
+        return super.getSqlType(field);
+    }
+  }
+
+  @Override
+  public String buildUpsertQueryStatement(
+      TableId table,
+      Collection<ColumnId> keyColumns,
+      Collection<ColumnId> nonKeyColumns
+  ) {
+    //MySql doesn't support SQL 2003:merge so here how the upsert is handled
+    final ExpressionBuilder.Transform<ColumnId> transform = (builder, col) -> {
+      builder.appendColumnName(col.name());
+      builder.append("=values(");
+      builder.appendColumnName(col.name());
+      builder.append(")");
+    };
+
+    ExpressionBuilder builder = expressionBuilder();
+    builder.append("insert into ");
+    builder.append(table);
+    builder.append("(");
+    builder.appendList()
+           .delimitedBy(",")
+           .transformedBy(ExpressionBuilder.columnNames())
+           .of(keyColumns, nonKeyColumns);
+    builder.append(") values(");
+    builder.appendMultiple(",", "?", keyColumns.size() + nonKeyColumns.size());
+    builder.append(") on duplicate key update ");
+    builder.appendList()
+           .delimitedBy(",")
+           .transformedBy(transform)
+           .of(nonKeyColumns.isEmpty() ? keyColumns : nonKeyColumns);
+    return builder.toString();
+  }
+
+  @Override
+  protected String sanitizedUrl(String url) {
+    // MySQL can also have "username:password@" at the beginning of the host list and
+    // in parenthetical properties
+    return super.sanitizedUrl(url)
+                .replaceAll("(?i)([(,]password=)[^,)]*", "$1****")
+                .replaceAll("(://[^:]*:)([^@]*)@", "$1****@");
+  }
+}
diff --git a/connectors/rocketmq-connect-jdbc/src/main/java/org/apache/rocketmq/connect/jdbc/dialect/impl/OpenMLDBDatabaseDialect.java b/connectors/rocketmq-connect-jdbc/src/main/java/org/apache/rocketmq/connect/jdbc/dialect/impl/OpenMLDBDatabaseDialect.java
new file mode 100644
index 0000000..daf4ace
--- /dev/null
+++ b/connectors/rocketmq-connect-jdbc/src/main/java/org/apache/rocketmq/connect/jdbc/dialect/impl/OpenMLDBDatabaseDialect.java
@@ -0,0 +1,68 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.rocketmq.connect.jdbc.dialect.impl;
+
+import lombok.SneakyThrows;
+import org.apache.rocketmq.connect.jdbc.config.AbstractConfig;
+import org.apache.rocketmq.connect.jdbc.dialect.DatabaseDialect;
+import org.apache.rocketmq.connect.jdbc.dialect.provider.DatabaseDialectProvider;
+import org.apache.rocketmq.connect.jdbc.util.IdentifierRules;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+
+/**
+ * openmldb database dialect
+ */
+public class OpenMLDBDatabaseDialect extends GenericDatabaseDialect {
+
+  private final Logger log = LoggerFactory.getLogger(OpenMLDBDatabaseDialect.class);
+
+  /**
+   * The provider for {@link OpenMLDBDatabaseDialect}.
+   */
+  public static class Provider extends DatabaseDialectProvider {
+    @SneakyThrows
+    public Provider() {
+      super(OpenMLDBDatabaseDialect.class.getSimpleName(), "openmldb");
+      Class.forName("com._4paradigm.openmldb.jdbc.SQLDriver");
+    }
+
+    @Override
+    public DatabaseDialect create(AbstractConfig config) {
+      return new OpenMLDBDatabaseDialect(config);
+    }
+  }
+
+  /**
+   * create openMLDB database dialect
+   * @param config
+   */
+  public OpenMLDBDatabaseDialect(AbstractConfig config) {
+    super(config, new IdentifierRules(".", "`", "`"));
+  }
+
+
+  @Override
+  protected String sanitizedUrl(String url) {
+    // MySQL can also have "username:password@" at the beginning of the host list and
+    // in parenthetical properties
+    return super.sanitizedUrl(url)
+                .replaceAll("(?i)([(,]password=)[^,)]*", "$1****")
+                .replaceAll("(://[^:]*:)([^@]*)@", "$1****@");
+  }
+}
diff --git a/connectors/rocketmq-connect-jdbc/src/main/java/org/apache/rocketmq/connect/jdbc/dialect/provider/CachedConnectionProvider.java b/connectors/rocketmq-connect-jdbc/src/main/java/org/apache/rocketmq/connect/jdbc/dialect/provider/CachedConnectionProvider.java
new file mode 100644
index 0000000..4286a04
--- /dev/null
+++ b/connectors/rocketmq-connect-jdbc/src/main/java/org/apache/rocketmq/connect/jdbc/dialect/provider/CachedConnectionProvider.java
@@ -0,0 +1,128 @@
+/*
+ * 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.rocketmq.connect.jdbc.dialect.provider;
+
+import io.openmessaging.connector.api.errors.ConnectException;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import java.sql.Connection;
+import java.sql.SQLException;
+
+/**
+ * cached connection provider
+ */
+public class CachedConnectionProvider implements ConnectionProvider {
+
+  private static final Logger log = LoggerFactory.getLogger(CachedConnectionProvider.class);
+
+  private static final int VALIDITY_CHECK_TIMEOUT_S = 5;
+
+  private final ConnectionProvider provider;
+  private final int maxConnectionAttempts;
+  private final long connectionRetryBackoff;
+
+  private int count = 0;
+  private Connection connection;
+
+  public CachedConnectionProvider(
+      ConnectionProvider provider,
+      int maxConnectionAttempts,
+      long connectionRetryBackoff
+  ) {
+    this.provider = provider;
+    this.maxConnectionAttempts = maxConnectionAttempts;
+    this.connectionRetryBackoff = connectionRetryBackoff;
+  }
+
+  @Override
+  public synchronized Connection getConnection() {
+    try {
+      if (connection == null) {
+        newConnection();
+      } else if (!isConnectionValid(connection, VALIDITY_CHECK_TIMEOUT_S)) {
+        log.info("The database connection is invalid. Reconnecting...");
+        close();
+        newConnection();
+      }
+    } catch (SQLException sqle) {
+      throw new ConnectException(sqle);
+    }
+    return connection;
+  }
+
+  @Override
+  public boolean isConnectionValid(Connection connection, int timeout) {
+    try {
+      return provider.isConnectionValid(connection, timeout);
+    } catch (SQLException sqle) {
+      log.debug("Unable to check if the underlying connection is valid", sqle);
+      return false;
+    }
+  }
+
+  private void newConnection() throws SQLException {
+    int attempts = 0;
+    while (attempts < maxConnectionAttempts) {
+      try {
+        ++count;
+        log.info("Attempting to open connection #{} to {}", count, provider);
+        connection = provider.getConnection();
+        onConnect(connection);
+        return;
+      } catch (SQLException sqle) {
+        attempts++;
+        if (attempts < maxConnectionAttempts) {
+          log.info("Unable to connect to database on attempt {}/{}. Will retry in {} ms.", attempts,
+                   maxConnectionAttempts, connectionRetryBackoff, sqle
+          );
+          try {
+            Thread.sleep(connectionRetryBackoff);
+          } catch (InterruptedException e) {
+            // this is ok because just woke up early
+          }
+        } else {
+          throw sqle;
+        }
+      }
+    }
+  }
+
+  @Override
+  public synchronized void close() {
+    if (connection != null) {
+      try {
+        log.info("Closing connection #{} to {}", count, provider);
+        connection.close();
+      } catch (SQLException sqle) {
+        log.warn("Ignoring error closing connection", sqle);
+      } finally {
+        connection = null;
+        provider.close();
+      }
+    }
+  }
+
+  @Override
+  public String identifier() {
+    return provider.identifier();
+  }
+
+  protected void onConnect(Connection connection) throws SQLException {
+  }
+
+}
diff --git a/connectors/rocketmq-connect-jdbc/src/main/java/org/apache/rocketmq/connect/jdbc/dialect/provider/ConnectionProvider.java b/connectors/rocketmq-connect-jdbc/src/main/java/org/apache/rocketmq/connect/jdbc/dialect/provider/ConnectionProvider.java
new file mode 100644
index 0000000..3c5f1d3
--- /dev/null
+++ b/connectors/rocketmq-connect-jdbc/src/main/java/org/apache/rocketmq/connect/jdbc/dialect/provider/ConnectionProvider.java
@@ -0,0 +1,65 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.rocketmq.connect.jdbc.dialect.provider;
+
+import java.sql.Connection;
+import java.sql.SQLException;
+
+/**
+ * A provider of JDBC {@link Connection} instances.
+ */
+public interface ConnectionProvider extends AutoCloseable {
+
+  /**
+   * Create a connection.
+   * @return the connection; never null
+   * @throws SQLException if there is a problem getting the connection
+   */
+  Connection getConnection() throws SQLException;
+
+  /**
+   * Determine if the specified connection is valid.
+   *
+   * @param connection the database connection; may not be null
+   * @param timeout    The time in seconds to wait for the database operation used to validate
+   *                   the connection to complete.  If the timeout period expires before the
+   *                   operation completes, this method returns false.  A value of 0 indicates a
+   *                   timeout is not applied to the database operation.
+   * @return true if it is valid, or false otherwise
+   * @throws SQLException if there is an error with the database connection
+   */
+  boolean isConnectionValid(
+      Connection connection,
+      int timeout
+  ) throws SQLException;
+
+  /**
+   * Close this connection provider.
+   */
+  @Override
+  void close();
+
+  /**
+   * Get the publicly viewable identifier for this connection provider and / or the database.
+   * The resulting value should not contain any secrets or passwords.
+   *
+   * @return the identifier; never null
+   */
+  default String identifier() {
+    return toString();
+  }
+}
diff --git a/connectors/rocketmq-connect-jdbc/src/main/java/org/apache/rocketmq/connect/jdbc/schema/column/BigIntColumnParser.java b/connectors/rocketmq-connect-jdbc/src/main/java/org/apache/rocketmq/connect/jdbc/dialect/provider/DatabaseDialectProvider.java
similarity index 50%
rename from connectors/rocketmq-connect-jdbc/src/main/java/org/apache/rocketmq/connect/jdbc/schema/column/BigIntColumnParser.java
rename to connectors/rocketmq-connect-jdbc/src/main/java/org/apache/rocketmq/connect/jdbc/dialect/provider/DatabaseDialectProvider.java
index 610f07d..d72da5d 100644
--- a/connectors/rocketmq-connect-jdbc/src/main/java/org/apache/rocketmq/connect/jdbc/schema/column/BigIntColumnParser.java
+++ b/connectors/rocketmq-connect-jdbc/src/main/java/org/apache/rocketmq/connect/jdbc/dialect/provider/DatabaseDialectProvider.java
@@ -14,37 +14,45 @@
  * See the License for the specific language governing permissions and
  * limitations under the License.
  */
+package org.apache.rocketmq.connect.jdbc.dialect.provider;
 
-package org.apache.rocketmq.connect.jdbc.schema.column;
+import org.apache.rocketmq.connect.jdbc.config.AbstractConfig;
+import org.apache.rocketmq.connect.jdbc.dialect.DatabaseDialect;
 
-import java.math.BigInteger;
 
-public class BigIntColumnParser extends ColumnParser {
+/**
+ * database dialect provider
+ */
+public abstract class DatabaseDialectProvider {
+
 
-    private static BigInteger max = BigInteger.ONE.shiftLeft(64);
+  private final String name;
+  private final String protocol;
 
-    private boolean signed;
+  protected DatabaseDialectProvider(String name,String protocol) {
+    assert name != null;
+    this.name = name;
+    this.protocol = protocol;
+  }
+  /**
+   * create database dialect
+   * @param config
+   * @return
+   */
+  public abstract DatabaseDialect create(AbstractConfig config);
 
-    public BigIntColumnParser(String colType) {
-        this.signed = !colType.matches(".* unsigned$");
-    }
+  public String protocolName() {
+    return protocol;
+  }
 
-    @Override
-    public Object getValue(Object value) {
+  public String dialectName() {
+    return name;
+  }
 
-        if (value == null) {
-            return null;
-        }
+  @Override
+  public String toString() {
+    return dialectName();
+  }
 
-        if (value instanceof BigInteger) {
-            return value;
-        }
 
-        Long l = (Long) value;
-        if (!signed && l < 0) {
-            return max.add(BigInteger.valueOf(l));
-        } else {
-            return l;
-        }
-    }
 }
diff --git a/connectors/rocketmq-connect-jdbc/src/main/java/org/apache/rocketmq/connect/jdbc/schema/column/TimeColumnParser.java b/connectors/rocketmq-connect-jdbc/src/main/java/org/apache/rocketmq/connect/jdbc/dialect/provider/JdbcUrlInfo.java
similarity index 61%
rename from connectors/rocketmq-connect-jdbc/src/main/java/org/apache/rocketmq/connect/jdbc/schema/column/TimeColumnParser.java
rename to connectors/rocketmq-connect-jdbc/src/main/java/org/apache/rocketmq/connect/jdbc/dialect/provider/JdbcUrlInfo.java
index 9926d81..82d4136 100644
--- a/connectors/rocketmq-connect-jdbc/src/main/java/org/apache/rocketmq/connect/jdbc/schema/column/TimeColumnParser.java
+++ b/connectors/rocketmq-connect-jdbc/src/main/java/org/apache/rocketmq/connect/jdbc/dialect/provider/JdbcUrlInfo.java
@@ -14,26 +14,29 @@
  * See the License for the specific language governing permissions and
  * limitations under the License.
  */
-
-package org.apache.rocketmq.connect.jdbc.schema.column;
-
-import java.sql.Time;
-import java.sql.Timestamp;
-
-public class TimeColumnParser extends ColumnParser {
-
-    @Override
-    public Object getValue(Object value) {
-
-        if (value == null) {
-            return null;
-        }
-
-        if (value instanceof Timestamp) {
-
-            return new Time(((Timestamp) value).getTime());
-        }
-
-        return value;
-    }
-}
+package org.apache.rocketmq.connect.jdbc.dialect.provider;
+
+public interface JdbcUrlInfo {
+
+    /**
+     * Get the subprotocol in the JDBC URL.
+     *
+     * @return the subprotocol
+     */
+    String subProtocol();
+
+    /**
+     * Get the subname in the JDBC URL, which is everything after the ':' character following the
+     * subprotocol.
+     *
+     * @return the subname
+     */
+    String subName();
+
+    /**
+     * Get the full JDBC URL.
+     *
+     * @return the URL.
+     */
+    String url();
+  }
diff --git a/connectors/rocketmq-connect-jdbc/src/main/java/org/apache/rocketmq/connect/jdbc/config/TaskTopicInfo.java b/connectors/rocketmq-connect-jdbc/src/main/java/org/apache/rocketmq/connect/jdbc/exception/ConfigException.java
similarity index 60%
rename from connectors/rocketmq-connect-jdbc/src/main/java/org/apache/rocketmq/connect/jdbc/config/TaskTopicInfo.java
rename to connectors/rocketmq-connect-jdbc/src/main/java/org/apache/rocketmq/connect/jdbc/exception/ConfigException.java
index 5c2a21e..1c181a4 100644
--- a/connectors/rocketmq-connect-jdbc/src/main/java/org/apache/rocketmq/connect/jdbc/config/TaskTopicInfo.java
+++ b/connectors/rocketmq-connect-jdbc/src/main/java/org/apache/rocketmq/connect/jdbc/exception/ConfigException.java
@@ -14,24 +14,24 @@
  * See the License for the specific language governing permissions and
  * limitations under the License.
  */
-package org.apache.rocketmq.connect.jdbc.config;
+package org.apache.rocketmq.connect.jdbc.exception;
 
-import org.apache.rocketmq.common.message.MessageQueue;
 
-public class TaskTopicInfo extends MessageQueue {
-
-    private String targetTopic;
+/**
+ * config exception
+ */
+public class ConfigException extends RuntimeException {
+    private static final long serialVersionUID = 1L;
 
-    public TaskTopicInfo(String sourceTopic, String brokerName, int queueId, String targetTopic) {
-        super(sourceTopic, brokerName, queueId);
-        this.targetTopic = targetTopic;
+    public ConfigException(String message) {
+        super(message);
     }
 
-    public String getTargetTopic() {
-        return this.targetTopic;
+    public ConfigException(String name, Object value) {
+        this(name, value, (String)null);
     }
 
-    public void setTargetTopic(String targetTopic) {
-        this.targetTopic = targetTopic;
+    public ConfigException(String name, Object value, String message) {
+        super("Invalid value " + value + " for configuration " + name + (message == null ? "" : ": " + message));
     }
-}
+}
\ No newline at end of file
diff --git a/connectors/rocketmq-connect-jdbc/src/main/java/org/apache/rocketmq/connect/jdbc/schema/column/DefaultColumnParser.java b/connectors/rocketmq-connect-jdbc/src/main/java/org/apache/rocketmq/connect/jdbc/exception/TableAlterOrCreateException.java
similarity index 65%
rename from connectors/rocketmq-connect-jdbc/src/main/java/org/apache/rocketmq/connect/jdbc/schema/column/DefaultColumnParser.java
rename to connectors/rocketmq-connect-jdbc/src/main/java/org/apache/rocketmq/connect/jdbc/exception/TableAlterOrCreateException.java
index ee3075a..e32f26d 100644
--- a/connectors/rocketmq-connect-jdbc/src/main/java/org/apache/rocketmq/connect/jdbc/schema/column/DefaultColumnParser.java
+++ b/connectors/rocketmq-connect-jdbc/src/main/java/org/apache/rocketmq/connect/jdbc/exception/TableAlterOrCreateException.java
@@ -14,24 +14,21 @@
  * See the License for the specific language governing permissions and
  * limitations under the License.
  */
+package org.apache.rocketmq.connect.jdbc.exception;
 
-package org.apache.rocketmq.connect.jdbc.schema.column;
+import io.openmessaging.connector.api.errors.ConnectException;
 
-import org.apache.commons.codec.binary.Base64;
-
-public class DefaultColumnParser extends ColumnParser {
-
-    @Override
-    public Object getValue(Object value) {
-
-        if (value == null) {
-            return null;
-        }
+/**
+ * table alter or create exception
+ */
+public class TableAlterOrCreateException extends ConnectException {
 
-        if (value instanceof byte[]) {
-            return Base64.encodeBase64String((byte[]) value);
-        }
+  public TableAlterOrCreateException(String reason) {
+    super(reason);
+  }
 
-        return value;
-    }
+  public TableAlterOrCreateException(String reason, Throwable throwable) {
+    super(reason, throwable);
+  }
 }
+
diff --git a/connectors/rocketmq-connect-jdbc/src/main/java/org/apache/rocketmq/connect/jdbc/schema/Database.java b/connectors/rocketmq-connect-jdbc/src/main/java/org/apache/rocketmq/connect/jdbc/schema/Database.java
deleted file mode 100644
index 33a9a22..0000000
--- a/connectors/rocketmq-connect-jdbc/src/main/java/org/apache/rocketmq/connect/jdbc/schema/Database.java
+++ /dev/null
@@ -1,109 +0,0 @@
-/*
- * 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.rocketmq.connect.jdbc.schema;
-
-import org.apache.rocketmq.connect.jdbc.schema.column.ColumnParser;
-import java.sql.Connection;
-import java.sql.PreparedStatement;
-import java.sql.ResultSet;
-import java.sql.SQLException;
-import java.util.*;
-import org.slf4j.Logger;
-import org.slf4j.LoggerFactory;
-
-public class Database {
-    private static final Logger LOGGER = LoggerFactory.getLogger(Database.class);
-
-    private static final String SQL = "select table_name,column_name,data_type,column_type,character_set_name " +
-        "from information_schema.columns " +
-        "where table_schema = ? order by ORDINAL_POSITION";
-    private String name;
-
-    private Connection connection;
-
-    private Map<String, Table> tableMap = new HashMap<String, Table>();
-
-    public Set<String> tableWhiteList;
-
-    public Map<String, Map<String, String>> tableFilterMap;
-
-    public Database(String name, Connection connection, Set<String> tableWhiteList, Map<String, Map<String, String>> tableFilterMap) {
-        this.name = name;
-        this.connection = connection;
-        this.tableFilterMap = tableFilterMap;
-        this.tableWhiteList = tableWhiteList;
-    }
-
-    public void init() throws SQLException {
-        PreparedStatement ps = null;
-        ResultSet rs = null;
-
-        try {
-            ps = connection.prepareStatement(SQL);
-            ps.setString(1, name);
-            rs = ps.executeQuery();
-
-            while (rs.next()) {
-                String tableName = rs.getString(1);
-                String colName = rs.getString(2);
-                String dataType = rs.getString(3);
-                String colType = rs.getString(4);
-                String charset = rs.getString(5);
-
-                ColumnParser columnParser = ColumnParser.getColumnParser(dataType, colType, charset);
-                if (!tableWhiteList.contains(tableName)){
-                    continue;
-                }
-                if (!tableMap.containsKey(tableName)) {
-                    addTable(tableName);
-                }
-                Table table = tableMap.get(tableName);
-                table.addCol(colName);
-                table.addParser(columnParser);
-                table.addRawDataType(dataType);
-                table.setFilterMap(tableFilterMap.get(tableName));
-            }
-
-        } finally {
-            if (rs != null) {
-                rs.close();
-            }
-            if (ps != null) {
-                ps.close();
-            }
-        }
-
-    }
-
-    private void addTable(String tableName) {
-
-        LOGGER.info("Schema load -- DATABASE:{},\tTABLE:{}", name, tableName);
-
-        Table table = new Table(name, tableName);
-        tableMap.put(tableName, table);
-    }
-
-    public Table getTable(String tableName) {
-
-        return tableMap.get(tableName);
-    }
-
-    public Map<String, Table> getTableMap() {
-        return tableMap;
-    }
-}
diff --git a/connectors/rocketmq-connect-jdbc/src/main/java/org/apache/rocketmq/connect/jdbc/schema/Schema.java b/connectors/rocketmq-connect-jdbc/src/main/java/org/apache/rocketmq/connect/jdbc/schema/Schema.java
deleted file mode 100644
index 1cfaf2c..0000000
--- a/connectors/rocketmq-connect-jdbc/src/main/java/org/apache/rocketmq/connect/jdbc/schema/Schema.java
+++ /dev/null
@@ -1,123 +0,0 @@
-/*
- * 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.rocketmq.connect.jdbc.schema;
-
-import java.sql.Connection;
-import java.sql.PreparedStatement;
-import java.sql.ResultSet;
-import java.sql.SQLException;
-import java.util.*;
-import org.slf4j.Logger;
-import org.slf4j.LoggerFactory;
-
-public class Schema {
-    private static final Logger LOGGER = LoggerFactory.getLogger(Schema.class);
-
-    private static final String SQL = "select schema_name from information_schema.schemata";
-
-    private static final List<String> IGNORED_DATABASES = new ArrayList<>(
-        Arrays.asList(new String[] {"information_schema", "mysql", "performance_schema", "sys"})
-    );
-
-    private Connection connection;
-
-    private Map<String, Database> dbMap;
-
-    public Map<String, Set<String>> dbTableMap;
-
-    public Map<String, Map<String, String>> tableFilterMap;
-
-    public Schema(Connection connection) {
-        this.connection = connection;
-        this.dbTableMap = new HashMap<>();
-        this.tableFilterMap = new HashMap<>();
-    }
-
-    public void load() throws SQLException {
-
-        dbMap = new HashMap<>();
-
-        PreparedStatement ps = null;
-        ResultSet rs = null;
-
-        try {
-            ps = connection.prepareStatement(SQL);
-            rs = ps.executeQuery();
-
-            while (rs.next()) {
-                String dbName = rs.getString(1);
-                if (!IGNORED_DATABASES.contains(dbName) && dbTableMap.keySet().contains(dbName)) {
-                    Database database = new Database(dbName, connection, dbTableMap.get(dbName), tableFilterMap);
-                    dbMap.put(dbName, database);
-                }
-            }
-
-        } finally {
-            if (rs != null) {
-                rs.close();
-            }
-            if (ps != null) {
-                ps.close();
-            }
-        }
-
-        for (Database db : dbMap.values()) {
-            db.init();
-        }
-
-    }
-
-    public Table getTable(String dbName, String tableName) {
-
-        if (dbMap == null) {
-            reload();
-        }
-
-        Database database = dbMap.get(dbName);
-        if (database == null) {
-            return null;
-        }
-
-        Table table = database.getTable(tableName);
-        if (table == null) {
-            return null;
-        }
-
-        return table;
-    }
-
-    private void reload() {
-
-        while (true) {
-            try {
-                load();
-                break;
-            } catch (Exception e) {
-                LOGGER.error("Reload schema error.", e);
-            }
-        }
-    }
-
-    public void reset() {
-        dbMap = null;
-    }
-
-    public Map<String, Database> getDbMap() {
-        return dbMap;
-    }
-}
diff --git a/connectors/rocketmq-connect-jdbc/src/main/java/org/apache/rocketmq/connect/jdbc/schema/Table.java b/connectors/rocketmq-connect-jdbc/src/main/java/org/apache/rocketmq/connect/jdbc/schema/Table.java
deleted file mode 100644
index 891fb9a..0000000
--- a/connectors/rocketmq-connect-jdbc/src/main/java/org/apache/rocketmq/connect/jdbc/schema/Table.java
+++ /dev/null
@@ -1,101 +0,0 @@
-/*
- * 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.rocketmq.connect.jdbc.schema;
-
-import org.apache.rocketmq.connect.jdbc.schema.column.ColumnParser;
-
-import java.util.HashMap;
-import java.util.LinkedList;
-import java.util.List;
-import java.util.Map;
-
-public class Table {
-
-    private String database;
-    private String name;
-    private List<String> colList = new LinkedList<>();
-    private List<ColumnParser> parserList = new LinkedList<>();
-    private List<String> rawDataTypeList = new LinkedList<>();
-    private List<Object> dataList = new LinkedList<>();
-    private Map<String, String> filterMap = new HashMap<>();
-
-    public Table(String database, String table) {
-        this.database = database;
-        this.name = table;
-    }
-
-    public void addCol(String column) {
-        colList.add(column);
-    }
-
-    public void setParserList(List<ColumnParser> parserList) {
-        this.parserList = parserList;
-    }
-
-    public void setRawDataTypeList(List<String> rawDataTypeList) {
-        this.rawDataTypeList = rawDataTypeList;
-    }
-
-    public void addParser(ColumnParser columnParser) {
-        parserList.add(columnParser);
-    }
-
-    public void addRawDataType(String rawDataType) {
-        this.rawDataTypeList.add(rawDataType);
-    }
-
-    public List<String> getColList() {
-        return colList;
-    }
-
-    public List<String> getRawDataTypeList() {
-        return rawDataTypeList;
-    }
-
-    public String getDatabase() {
-        return database;
-    }
-
-    public String getName() {
-        return name;
-    }
-
-    public List<ColumnParser> getParserList() {
-        return parserList;
-    }
-
-    public List<Object> getDataList() {
-        return dataList;
-    }
-
-    public void setDataList(List<Object> dataList) {
-        this.dataList = dataList;
-    }
-
-    public void setColList(List<String> colList) {
-        this.colList = colList;
-    }
-
-    public Map<String, String> getFilterMap() {
-        return filterMap;
-    }
-
-    public void setFilterMap(Map<String, String> filterMap) {
-        this.filterMap = filterMap;
-    }
-}
\ No newline at end of file
diff --git a/connectors/rocketmq-connect-jdbc/src/main/java/org/apache/rocketmq/connect/jdbc/schema/column/ColumnDefAdjuster.java b/connectors/rocketmq-connect-jdbc/src/main/java/org/apache/rocketmq/connect/jdbc/schema/column/ColumnDefAdjuster.java
new file mode 100644
index 0000000..4f589c0
--- /dev/null
+++ b/connectors/rocketmq-connect-jdbc/src/main/java/org/apache/rocketmq/connect/jdbc/schema/column/ColumnDefAdjuster.java
@@ -0,0 +1,72 @@
+/*
+ * 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.rocketmq.connect.jdbc.schema.column;
+
+import java.sql.Connection;
+import java.sql.DatabaseMetaData;
+import java.sql.ResultSet;
+import java.sql.SQLException;
+import java.util.HashMap;
+import java.util.Map;
+
+public class ColumnDefAdjuster {
+    Map<String, ColumnDefinition.Nullability> nullable = new HashMap<>();
+
+    public ColumnDefAdjuster() {
+    }
+
+    public static ColumnDefAdjuster create(Connection conn,
+                                           String catalogPattern,
+                                           String schemaPattern,
+                                           String tablePattern,
+                                           String columnPattern){
+        ColumnDefAdjuster adjuster = new ColumnDefAdjuster();
+        try(ResultSet rs = conn.getMetaData().getColumns(
+                catalogPattern, schemaPattern, tablePattern, columnPattern)){
+            final int rsColumnCount = rs.getMetaData().getColumnCount();
+            while (rs.next()) {
+                final String columnName = rs.getString(4);
+                ColumnDefinition.Nullability nullability;
+                final int nullableValue = rs.getInt(11);
+                switch (nullableValue) {
+                    case DatabaseMetaData.columnNoNulls:
+                        nullability = ColumnDefinition.Nullability.NOT_NULL;
+                        break;
+                    case DatabaseMetaData.columnNullable:
+                        nullability = ColumnDefinition.Nullability.NULL;
+                        break;
+                    case DatabaseMetaData.columnNullableUnknown:
+                    default:
+                        nullability = ColumnDefinition.Nullability.UNKNOWN;
+                        break;
+                }
+                adjuster.nullable.put(columnName, nullability);
+            }
+        }catch (SQLException e){
+            //pass
+        }
+
+        return adjuster;
+    }
+
+    public ColumnDefinition.Nullability nullable(String columnName){
+        if(nullable == null || !nullable.containsKey(columnName)){
+            return null;
+        }
+        return nullable.get(columnName);
+    }
+}
diff --git a/connectors/rocketmq-connect-jdbc/src/main/java/org/apache/rocketmq/connect/jdbc/schema/column/ColumnDefinition.java b/connectors/rocketmq-connect-jdbc/src/main/java/org/apache/rocketmq/connect/jdbc/schema/column/ColumnDefinition.java
new file mode 100644
index 0000000..25ff655
--- /dev/null
+++ b/connectors/rocketmq-connect-jdbc/src/main/java/org/apache/rocketmq/connect/jdbc/schema/column/ColumnDefinition.java
@@ -0,0 +1,339 @@
+/*
+ * 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.rocketmq.connect.jdbc.schema.column;
+
+import org.apache.rocketmq.connect.jdbc.schema.table.TableId;
+
+import java.sql.Types;
+import java.util.Objects;
+
+/**
+ * column definition
+ */
+public class ColumnDefinition {
+
+  /**
+   * The nullability of a column.
+   */
+  public enum Nullability {
+    NULL, NOT_NULL, UNKNOWN
+  }
+
+  /**
+   * The mutability of a column.
+   */
+  public enum Mutability {
+    READ_ONLY, MAYBE_WRITABLE, WRITABLE, UNKNOWN
+  }
+
+  private final ColumnId id;
+  private final String typeName;
+  private final int jdbcType;
+  private final int displaySize;
+  private final int precision;
+  private final int scale;
+  private final boolean autoIncremented;
+  private final boolean caseSensitive;
+  private final boolean searchable;
+  private final boolean currency;
+  private final boolean signedNumbers;
+  private final boolean isPrimaryKey;
+  private final Nullability nullability;
+  private final Mutability mutability;
+  private final String classNameForType;
+
+  public ColumnDefinition(
+      ColumnId id,
+      int jdbcType,
+      String typeName,
+      String classNameForType,
+      Nullability nullability,
+      Mutability mutability,
+      int precision,
+      int scale,
+      boolean signedNumbers,
+      int displaySize,
+      boolean autoIncremented,
+      boolean caseSensitive,
+      boolean searchable,
+      boolean currency,
+      boolean isPrimaryKey
+  ) {
+    this.id = id;
+    this.typeName = typeName;
+    this.jdbcType = jdbcType;
+    this.displaySize = displaySize;
+    this.precision = precision;
+    this.scale = scale;
+    this.autoIncremented = autoIncremented;
+    this.caseSensitive = caseSensitive;
+    this.searchable = searchable;
+    this.currency = currency;
+    this.signedNumbers = signedNumbers;
+    this.nullability = nullability != null ? nullability : Nullability.UNKNOWN;
+    this.mutability = mutability != null ? mutability : Mutability.MAYBE_WRITABLE;
+    this.classNameForType = classNameForType;
+    this.isPrimaryKey = isPrimaryKey;
+  }
+
+
+  /**
+   * Indicates whether the column is automatically numbered.
+   *
+   * @return <code>true</code> if so; <code>false</code> otherwise
+   */
+  public boolean isAutoIncrement() {
+    return autoIncremented;
+  }
+
+  /**
+   * Indicates whether the column's case matters.
+   *
+   * @return <code>true</code> if so; <code>false</code> otherwise
+   */
+  public boolean isCaseSensitive() {
+    return caseSensitive;
+  }
+
+  /**
+   * Indicates whether the column can be used in a where clause.
+   *
+   * @return <code>true</code> if so; <code>false</code> otherwise
+   */
+  public boolean isSearchable() {
+    return searchable;
+  }
+
+  /**
+   * Indicates whether the column is a cash value.
+   *
+   * @return <code>true</code> if so; <code>false</code> otherwise
+   */
+  public boolean isCurrency() {
+    return currency;
+  }
+
+  /**
+   * Indicates whether the column is part of the table's primary key.
+   *
+   * @return <code>true</code> if so; <code>false</code> otherwise
+   */
+  public boolean isPrimaryKey() {
+    return isPrimaryKey;
+  }
+
+  /**
+   * Indicates the nullability of values in the column.
+   *
+   * @return the nullability status of the given column; never null
+   */
+  public Nullability nullability() {
+    return nullability;
+  }
+
+  /**
+   * Indicates whether values in the column are optional. This is equivalent to calling:
+   * <pre>
+   *   nullability() == Nullability.NULL || nullability() == Nullability.UNKNOWN
+   * </pre>
+   *
+   * @return <code>true</code> if so; <code>false</code> otherwise
+   */
+  public boolean isOptional() {
+    return nullability == Nullability.NULL || nullability == Nullability.UNKNOWN;
+  }
+
+  /**
+   * Indicates whether values in the column are signed numbers.
+   *
+   * @return <code>true</code> if so; <code>false</code> otherwise
+   */
+  public boolean isSignedNumber() {
+    return signedNumbers;
+  }
+
+  /**
+   * Indicates the column's normal maximum width in characters.
+   *
+   * @return the normal maximum number of characters allowed as the width of the designated column
+   */
+  public int displaySize() {
+    return displaySize;
+  }
+
+  /**
+   * Get the column's identifier.
+   *
+   * @return column identifier; never null
+   */
+  public ColumnId id() {
+    return id;
+  }
+
+  /**
+   * Get the column's table identifier.
+   *
+   * @return the table identifier; never null
+   */
+  public TableId tableId() {
+    return id.tableId();
+  }
+
+  /**
+   * Get the column's specified column size. For numeric data, this is the maximum precision.  For
+   * character data, this is the length in characters. For datetime datatypes, this is the length in
+   * characters of the String representation (assuming the maximum allowed precision of the
+   * fractional seconds component). For binary data, this is the length in bytes. For the ROWID
+   * datatype, this is the length in bytes. 0 is returned for data types where the column size is
+   * not applicable.
+   *
+   * @return precision
+   */
+  public int precision() {
+    return precision;
+  }
+
+  /**
+   * Gets the column's number of digits to right of the decimal point. 0 is returned for data types
+   * where the scale is not applicable.
+   *
+   * @return scale
+   */
+  public int scale() {
+    return scale;
+  }
+
+  /**
+   * Retrieves the column's JDBC type.
+   *
+   * @return SQL type from java.sql.Types
+   * @see Types
+   */
+  public int type() {
+    return jdbcType;
+  }
+
+  /**
+   * Retrieves the designated column's database-specific type name.
+   *
+   * @return type name used by the database. If the column type is a user-defined type, then a
+   *     fully-qualified type name is returned.
+   */
+  public String typeName() {
+    return typeName;
+  }
+
+  /**
+   * Indicates whether the designated column is mutable.
+   *
+   * @return the mutability; never null
+   */
+  public Mutability mutability() {
+    return mutability;
+  }
+
+  /**
+   * Returns the fully-qualified name of the Java class whose instances are manufactured if the
+   * method {@link java.sql.ResultSet#getObject(int)} is called to retrieve a value from the column.
+   * {@link java.sql.ResultSet#getObject(int)} may return a subclass of the class returned by this
+   * method.
+   *
+   * @return the fully-qualified name of the class in the Java programming language that would be
+   *     used by the method <code>ResultSet.getObject</code> to retrieve the value in the specified
+   *     column. This is the class name used for custom mapping.
+   */
+  public String classNameForType() {
+    return classNameForType;
+  }
+
+
+  @Override
+  public int hashCode() {
+    return id.hashCode();
+  }
+
+  @Override
+  public boolean equals(Object obj) {
+    if (obj == this) {
+      return true;
+    }
+    if (obj instanceof ColumnDefinition) {
+      ColumnDefinition that = (ColumnDefinition) obj;
+      return Objects.equals(this.id, that.id)
+             && this.jdbcType == that.jdbcType
+             && this.displaySize == that.displaySize
+             && this.precision == that.precision
+             && this.scale == that.scale
+             && this.autoIncremented == that.autoIncremented
+             && this.caseSensitive == that.caseSensitive
+             && this.searchable == that.searchable
+             && this.currency == that.currency
+             && this.signedNumbers == that.signedNumbers
+             && this.nullability == that.nullability
+             && Objects.equals(this.typeName, that.typeName)
+             && Objects.equals(this.classNameForType, that.classNameForType)
+             && Objects.equals(this.isPrimaryKey, that.isPrimaryKey);
+    }
+    return false;
+  }
+
+  @Override
+  public String toString() {
+    return "Column{'" + id.name() + '\'' + ", isPrimaryKey=" + isPrimaryKey + ", allowsNull="
+           + isOptional() + ", sqlType=" + typeName + '}';
+  }
+
+  /**
+   * Obtain a {@link ColumnDefinition} that has all the same characteristics as this column except
+   * that belongs to the specified table.
+   *
+   * @param tableId the new table ID; may not be null
+   * @return this definition if the specified table ID matches this definition's {@link #tableId()},
+   *     or a new definition that is a copy of this definition except with an {@link #id() ID} that
+   *     uses the specified table; never null
+   */
+  public ColumnDefinition forTable(TableId tableId) {
+    if (tableId().equals(tableId)) {
+      return this;
+    }
+    ColumnId newId = new ColumnId(tableId, this.id().name());
+    return new ColumnDefinition(newId, jdbcType, typeName, classNameForType, nullability,
+                                mutability, precision, scale, signedNumbers, displaySize,
+                                autoIncremented, caseSensitive, searchable, currency, isPrimaryKey
+    );
+  }
+
+  /**
+   * Obtain a {@link ColumnDefinition} that has all the same characteristics as this column except
+   * that it does or does not belong to the table's primary key
+   *
+   * @param isPrimaryKey true if the resulting column definition should be part of the table's
+   *                     primary key, or false otherwise
+   * @return a definition that is the same as this definition except it is or is not part of the
+   *     tables primary key, or may be this object if {@link #isPrimaryKey()} already matches the
+   *     supplied parameter; never null
+   */
+  public ColumnDefinition asPartOfPrimaryKey(boolean isPrimaryKey) {
+    if (isPrimaryKey == isPrimaryKey()) {
+      return this;
+    }
+    return new ColumnDefinition(id, jdbcType, typeName, classNameForType, nullability, mutability,
+                                precision, scale, signedNumbers, displaySize, autoIncremented,
+                                caseSensitive, searchable, currency, isPrimaryKey
+    );
+  }
+}
diff --git a/connectors/rocketmq-connect-jdbc/src/main/java/org/apache/rocketmq/connect/jdbc/schema/column/ColumnId.java b/connectors/rocketmq-connect-jdbc/src/main/java/org/apache/rocketmq/connect/jdbc/schema/column/ColumnId.java
new file mode 100644
index 0000000..a237d70
--- /dev/null
+++ b/connectors/rocketmq-connect-jdbc/src/main/java/org/apache/rocketmq/connect/jdbc/schema/column/ColumnId.java
@@ -0,0 +1,105 @@
+/*
+ * 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.rocketmq.connect.jdbc.schema.column;
+
+import org.apache.rocketmq.connect.jdbc.util.ExpressionBuilder;
+import org.apache.rocketmq.connect.jdbc.util.QuoteMethod;
+import org.apache.rocketmq.connect.jdbc.schema.table.TableId;
+
+import java.util.Objects;
+
+/**
+ * column id
+ */
+public class ColumnId implements ExpressionBuilder.Expressable {
+
+  private final TableId tableId;
+  private final String name;
+  private final String alias;
+  private final int hash;
+
+  public ColumnId(
+      TableId tableId,
+      String columnName
+  ) {
+    this(tableId, columnName, null);
+  }
+
+  public ColumnId(
+      TableId tableId,
+      String columnName,
+      String alias
+  ) {
+    assert columnName != null;
+    this.tableId = tableId;
+    this.name = columnName;
+    this.alias = alias != null && !alias.trim().isEmpty() ? alias : name;
+    this.hash = Objects.hash(this.tableId, this.name);
+  }
+
+  public TableId tableId() {
+    return tableId;
+  }
+
+  public String name() {
+    return name;
+  }
+
+  public String aliasOrName() {
+    return alias;
+  }
+
+  @Override
+  public void appendTo(ExpressionBuilder builder, boolean useQuotes) {
+    appendTo(builder, useQuotes ? QuoteMethod.ALWAYS : QuoteMethod.NEVER);
+  }
+
+  @Override
+  public void appendTo(
+      ExpressionBuilder builder,
+      QuoteMethod useQuotes
+  ) {
+    if (tableId != null) {
+      builder.append(tableId);
+      builder.appendIdentifierDelimiter();
+    }
+    builder.appendColumnName(this.name, useQuotes);
+  }
+
+  @Override
+  public int hashCode() {
+    return hash;
+  }
+
+  @Override
+  public boolean equals(Object obj) {
+    if (obj == this) {
+      return true;
+    }
+    if (obj instanceof ColumnId) {
+      ColumnId that = (ColumnId) obj;
+      return Objects.equals(this.name, that.name) && Objects.equals(this.alias, that.alias)
+             && Objects.equals(this.tableId, that.tableId);
+    }
+    return false;
+  }
+
+  @Override
+  public String toString() {
+    return ExpressionBuilder.create().append(this).toString();
+  }
+}
diff --git a/connectors/rocketmq-connect-jdbc/src/main/java/org/apache/rocketmq/connect/jdbc/schema/column/ColumnParser.java b/connectors/rocketmq-connect-jdbc/src/main/java/org/apache/rocketmq/connect/jdbc/schema/column/ColumnParser.java
deleted file mode 100644
index 341064e..0000000
--- a/connectors/rocketmq-connect-jdbc/src/main/java/org/apache/rocketmq/connect/jdbc/schema/column/ColumnParser.java
+++ /dev/null
@@ -1,104 +0,0 @@
-/*
- * 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.rocketmq.connect.jdbc.schema.column;
-
-import io.openmessaging.connector.api.data.FieldType;
-import java.util.regex.Matcher;
-import java.util.regex.Pattern;
-
-public abstract class ColumnParser {
-	
-    public static ColumnParser getColumnParser(String dataType, String colType, String charset) {
-
-        switch (dataType) {
-            case "tinyint":
-            case "smallint":
-            case "mediumint":
-            case "int":
-                return new IntColumnParser(dataType, colType);
-            case "bigint":
-                return new BigIntColumnParser(colType);
-            case "tinytext":
-            case "text":
-            case "mediumtext":
-            case "longtext":
-            case "varchar":
-            case "char":
-                return new StringColumnParser(charset);
-            case "date":
-            case "datetime":
-            case "timestamp":
-                return new DateTimeColumnParser();
-            case "time":
-                return new TimeColumnParser();
-            case "year":
-                return new YearColumnParser();
-            case "enum":
-                return new EnumColumnParser(colType);
-            case "set":
-                return new SetColumnParser(colType);
-            default:
-                return new DefaultColumnParser();
-        }
-    }
-
-    public static FieldType mapConnectorFieldType(String dataType) {
-
-        switch (dataType) {
-            case "tinyint":
-            case "smallint":
-            case "mediumint":
-            case "int":
-                return FieldType.INT32;
-            case "bigint":
-                return FieldType.BIG_INTEGER;
-            case "tinytext":
-            case "text":
-            case "mediumtext":
-            case "longtext":
-            case "varchar":
-            case "char":
-                return FieldType.STRING;
-            case "date":
-            case "datetime":
-            case "timestamp":
-            case "time":
-            case "year":
-                return FieldType.DATETIME;
-            case "enum":
-                return null;
-            case "set":
-                return null;
-            default:
-                return FieldType.BYTES;
-        }
-    }
-
-    public static String[] extractEnumValues(String colType) {
-        String[] enumValues = {};
-        Matcher matcher = Pattern.compile("(enum|set)\\((.*)\\)").matcher(colType);
-        if (matcher.matches()) {
-            enumValues = matcher.group(2).replace("'", "").split(",");
-        }
-
-        return enumValues;
-    }
-
-    public abstract Object getValue(Object value);
-
-}
diff --git a/connectors/rocketmq-connect-jdbc/src/main/java/org/apache/rocketmq/connect/jdbc/schema/column/DateTimeColumnParser.java b/connectors/rocketmq-connect-jdbc/src/main/java/org/apache/rocketmq/connect/jdbc/schema/column/DateTimeColumnParser.java
deleted file mode 100644
index c9b39e3..0000000
--- a/connectors/rocketmq-connect-jdbc/src/main/java/org/apache/rocketmq/connect/jdbc/schema/column/DateTimeColumnParser.java
+++ /dev/null
@@ -1,53 +0,0 @@
-/*
- * 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.rocketmq.connect.jdbc.schema.column;
-
-import java.sql.Timestamp;
-import java.text.SimpleDateFormat;
-import java.util.Date;
-import java.util.TimeZone;
-
-public class DateTimeColumnParser extends ColumnParser {
-
-    private static SimpleDateFormat dateTimeFormat;
-    private static SimpleDateFormat dateTimeUtcFormat;
-
-    static {
-        dateTimeFormat = new SimpleDateFormat("yyyy-MM-dd HH:mm:ss");
-        dateTimeUtcFormat = new SimpleDateFormat("yyyy-MM-dd HH:mm:ss");
-        dateTimeUtcFormat.setTimeZone(TimeZone.getTimeZone("GMT+8"));
-    }
-
-    @Override
-    public Object getValue(Object value) {
-
-        if (value == null) {
-            return null;
-        }
-
-        if (value instanceof Timestamp) {
-            return dateTimeFormat.format(value);
-        }
-
-        if (value instanceof Long) {
-            return dateTimeUtcFormat.format(new Date((Long) value));
-        }
-
-        return value;
-    }
-}
diff --git a/connectors/rocketmq-connect-jdbc/src/main/java/org/apache/rocketmq/connect/jdbc/schema/column/IntColumnParser.java b/connectors/rocketmq-connect-jdbc/src/main/java/org/apache/rocketmq/connect/jdbc/schema/column/IntColumnParser.java
deleted file mode 100644
index 36c6078..0000000
--- a/connectors/rocketmq-connect-jdbc/src/main/java/org/apache/rocketmq/connect/jdbc/schema/column/IntColumnParser.java
+++ /dev/null
@@ -1,66 +0,0 @@
-/*
- * 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.rocketmq.connect.jdbc.schema.column;
-
-public class IntColumnParser extends ColumnParser {
-
-    private int bits;
-    private boolean signed;
-
-    public IntColumnParser(String dataType, String colType) {
-
-        switch (dataType) {
-            case "tinyint":
-                bits = 8;
-                break;
-            case "smallint":
-                bits = 16;
-                break;
-            case "mediumint":
-                bits = 24;
-                break;
-            case "int":
-                bits = 32;
-        }
-
-        this.signed = !colType.matches(".* unsigned$");
-    }
-
-    @Override
-    public Object getValue(Object value) {
-
-        if (value == null) {
-            return null;
-        }
-
-        if (value instanceof Long) {
-            return value;
-        }
-
-        if (value instanceof Integer) {
-            Integer i = (Integer) value;
-            if (signed || i > 0) {
-                return i;
-            } else {
-                return (1L << bits) + i;
-            }
-        }
-
-        return value;
-    }
-}
diff --git a/connectors/rocketmq-connect-jdbc/src/main/java/org/apache/rocketmq/connect/jdbc/schema/column/SetColumnParser.java b/connectors/rocketmq-connect-jdbc/src/main/java/org/apache/rocketmq/connect/jdbc/schema/column/SetColumnParser.java
deleted file mode 100644
index d1e6bbc..0000000
--- a/connectors/rocketmq-connect-jdbc/src/main/java/org/apache/rocketmq/connect/jdbc/schema/column/SetColumnParser.java
+++ /dev/null
@@ -1,54 +0,0 @@
-/*
- * 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.rocketmq.connect.jdbc.schema.column;
-
-public class SetColumnParser extends ColumnParser {
-
-    private String[] enumValues;
-
-    public SetColumnParser(String colType) {
-        enumValues = extractEnumValues(colType);
-    }
-
-    @Override
-    public Object getValue(Object value) {
-        if (value == null) {
-            return null;
-        }
-
-        if (value instanceof String) {
-            return value;
-        }
-
-        StringBuilder builder = new StringBuilder();
-        long l = (Long) value;
-
-        boolean needSplit = false;
-        for (int i = 0; i < enumValues.length; i++) {
-            if (((l >> i) & 1) == 1) {
-                if (needSplit)
-                    builder.append(",");
-
-                builder.append(enumValues[i]);
-                needSplit = true;
-            }
-        }
-
-        return builder.toString();
-    }
-}
diff --git a/connectors/rocketmq-connect-jdbc/src/main/java/org/apache/rocketmq/connect/jdbc/schema/column/StringColumnParser.java b/connectors/rocketmq-connect-jdbc/src/main/java/org/apache/rocketmq/connect/jdbc/schema/column/StringColumnParser.java
deleted file mode 100644
index cd4f04f..0000000
--- a/connectors/rocketmq-connect-jdbc/src/main/java/org/apache/rocketmq/connect/jdbc/schema/column/StringColumnParser.java
+++ /dev/null
@@ -1,57 +0,0 @@
-/*
- * 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.rocketmq.connect.jdbc.schema.column;
-
-import org.apache.commons.codec.Charsets;
-
-public class StringColumnParser extends ColumnParser {
-
-    private String charset;
-
-    public StringColumnParser(String charset) {
-        this.charset = charset.toLowerCase();
-    }
-
-    @Override
-    public Object getValue(Object value) {
-
-        if (value == null) {
-            return null;
-        }
-
-        if (value instanceof String) {
-            return value;
-        }
-
-        byte[] bytes = (byte[]) value;
-
-        switch (charset) {
-            case "utf8":
-            case "utf8mb4":
-                return new String(bytes, Charsets.UTF_8);
-            case "latin1":
-            case "ascii":
-                return new String(bytes, Charsets.ISO_8859_1);
-            case "ucs2":
-                return new String(bytes, Charsets.UTF_16);
-            default:
-                return new String(bytes, Charsets.toCharset(charset));
-
-        }
-    }
-}
diff --git a/connectors/rocketmq-connect-jdbc/src/main/java/org/apache/rocketmq/connect/jdbc/schema/column/parser/DateColumnParser.java b/connectors/rocketmq-connect-jdbc/src/main/java/org/apache/rocketmq/connect/jdbc/schema/column/parser/DateColumnParser.java
new file mode 100644
index 0000000..386fe42
--- /dev/null
+++ b/connectors/rocketmq-connect-jdbc/src/main/java/org/apache/rocketmq/connect/jdbc/schema/column/parser/DateColumnParser.java
@@ -0,0 +1,61 @@
+/*
+ * 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.rocketmq.connect.jdbc.schema.column.parser;
+
+import io.openmessaging.connector.api.data.Schema;
+import io.openmessaging.connector.api.data.SchemaBuilder;
+import io.openmessaging.connector.api.errors.ConnectException;
+
+import java.util.Calendar;
+import java.util.TimeZone;
+
+public class DateColumnParser {
+    public static final String LOGICAL_NAME = "org.apache.rocketmq.connect.jdbc.schema.column.parser.DateColumnParser";
+    private static final long MILLIS_PER_DAY = 86400000L;
+    private static final TimeZone UTC = TimeZone.getTimeZone("UTC");
+    public static final Schema SCHEMA = builder().build();
+
+    public DateColumnParser() {
+    }
+
+    public static SchemaBuilder builder() {
+        return SchemaBuilder.int32().name("org.apache.kafka.connect.data.Date");
+    }
+
+    public static int fromLogical(Schema schema, java.util.Date value) {
+        if (!LOGICAL_NAME.equals(schema.getName())) {
+            throw new ConnectException("Requested conversion of Date object but the schema does not match.");
+        } else {
+            Calendar calendar = Calendar.getInstance(UTC);
+            calendar.setTime(value);
+            if (calendar.get(11) == 0 && calendar.get(12) == 0 && calendar.get(13) == 0 && calendar.get(14) == 0) {
+                long unixMillis = calendar.getTimeInMillis();
+                return (int)(unixMillis / 86400000L);
+            } else {
+                throw new ConnectException("Kafka Connect Date type should not have any time fields set to non-zero values.");
+            }
+        }
+    }
+
+    public static java.util.Date toLogical(Schema schema, int value) {
+        if (!LOGICAL_NAME.equals(schema.getName())) {
+            throw new ConnectException("Requested conversion of Date object but the schema does not match.");
+        } else {
+            return new java.util.Date((long)value * 86400000L);
+        }
+    }
+}
\ No newline at end of file
diff --git a/connectors/rocketmq-connect-jdbc/src/main/java/org/apache/rocketmq/connect/jdbc/schema/column/parser/TimeColumnParser.java b/connectors/rocketmq-connect-jdbc/src/main/java/org/apache/rocketmq/connect/jdbc/schema/column/parser/TimeColumnParser.java
new file mode 100644
index 0000000..e5a4198
--- /dev/null
+++ b/connectors/rocketmq-connect-jdbc/src/main/java/org/apache/rocketmq/connect/jdbc/schema/column/parser/TimeColumnParser.java
@@ -0,0 +1,63 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.rocketmq.connect.jdbc.schema.column.parser;
+
+import io.openmessaging.connector.api.data.Schema;
+import io.openmessaging.connector.api.data.SchemaBuilder;
+import io.openmessaging.connector.api.errors.ConnectException;
+
+import java.util.Calendar;
+import java.util.Date;
+import java.util.TimeZone;
+
+public class TimeColumnParser{
+    public static final String LOGICAL_NAME = "org.apache.rocketmq.connect.jdbc.schema.column.parser.TimeColumnParser";
+    private static final long MILLIS_PER_DAY = 86400000L;
+    private static final TimeZone UTC = TimeZone.getTimeZone("UTC");
+    public static final Schema SCHEMA = builder().build();
+
+    public static SchemaBuilder builder() {
+        return SchemaBuilder.int32().name("org.apache.rocketmq.connect.jdbc.schema.column.parser.TimeColumnParser");
+    }
+
+    public static int fromLogical(Schema schema, Date value) {
+        if (!LOGICAL_NAME.equals(schema.getName())) {
+            throw new ConnectException("Requested conversion of Time object but the schema does not match.");
+        } else {
+            Calendar calendar = Calendar.getInstance(UTC);
+            calendar.setTime(value);
+            long unixMillis = calendar.getTimeInMillis();
+            if (unixMillis >= 0L && unixMillis <= 86400000L) {
+                return (int)unixMillis;
+            } else {
+                throw new ConnectException("Kafka Connect Time type should not have any date fields set to non-zero values.");
+            }
+        }
+    }
+
+    public static Date toLogical(Schema schema, int value) {
+        if (!LOGICAL_NAME.equals(schema.getName())) {
+            throw new ConnectException("Requested conversion of Date object but the schema does not match.");
+        } else if (value >= 0 && (long)value <= 86400000L) {
+            return new Date((long)value);
+        } else {
+            throw new ConnectException("Time values must use number of milliseconds greater than 0 and less than 86400000");
+        }
+    }
+
+}
diff --git a/connectors/rocketmq-connect-jdbc/src/main/java/org/apache/rocketmq/connect/jdbc/schema/column/parser/TimestampColumnParser.java b/connectors/rocketmq-connect-jdbc/src/main/java/org/apache/rocketmq/connect/jdbc/schema/column/parser/TimestampColumnParser.java
new file mode 100644
index 0000000..b3595ad
--- /dev/null
+++ b/connectors/rocketmq-connect-jdbc/src/main/java/org/apache/rocketmq/connect/jdbc/schema/column/parser/TimestampColumnParser.java
@@ -0,0 +1,54 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.rocketmq.connect.jdbc.schema.column.parser;
+
+import io.openmessaging.connector.api.data.Schema;
+import io.openmessaging.connector.api.data.SchemaBuilder;
+import io.openmessaging.connector.api.errors.ConnectException;
+
+import java.util.Date;
+
+/**
+ * time stamp column parser
+ */
+public class TimestampColumnParser {
+    public static final String LOGICAL_NAME = "org.apache.rocketmq.connect.jdbc.schema.column.parser.TimestampColumnParser";
+    public static final Schema SCHEMA = builder().build();
+
+    public TimestampColumnParser() {
+    }
+
+    public static SchemaBuilder builder() {
+        return SchemaBuilder.int64().name("org.apache.kafka.connect.data.Timestamp");
+    }
+
+    public static long fromLogical(Schema schema, Date value) {
+        if (!LOGICAL_NAME.equals(schema.getName())) {
+            throw new ConnectException("Requested conversion of Timestamp object but the schema does not match.");
+        } else {
+            return value.getTime();
+        }
+    }
+
+    public static Date toLogical(Schema schema, long value) {
+        if (!LOGICAL_NAME.equals(schema.getName())) {
+            throw new ConnectException("Requested conversion of Timestamp object but the schema does not match.");
+        } else {
+            return new Date(value);
+        }
+    }
+}
diff --git a/connectors/rocketmq-connect-jdbc/src/main/java/org/apache/rocketmq/connect/jdbc/schema/db/DbStructure.java b/connectors/rocketmq-connect-jdbc/src/main/java/org/apache/rocketmq/connect/jdbc/schema/db/DbStructure.java
new file mode 100644
index 0000000..e2cc7e7
--- /dev/null
+++ b/connectors/rocketmq-connect-jdbc/src/main/java/org/apache/rocketmq/connect/jdbc/schema/db/DbStructure.java
@@ -0,0 +1,310 @@
+/*
+ * 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.rocketmq.connect.jdbc.schema.db;
+
+import org.apache.rocketmq.connect.jdbc.connector.JdbcSinkConfig;
+import org.apache.rocketmq.connect.jdbc.dialect.DatabaseDialect;
+import org.apache.rocketmq.connect.jdbc.schema.column.ColumnDefinition;
+import org.apache.rocketmq.connect.jdbc.schema.table.TableDefinition;
+import org.apache.rocketmq.connect.jdbc.schema.table.TableDefinitions;
+import org.apache.rocketmq.connect.jdbc.schema.table.TableId;
+import org.apache.rocketmq.connect.jdbc.sink.metadata.FieldsMetadata;
+import org.apache.rocketmq.connect.jdbc.sink.metadata.SinkRecordField;
+import org.apache.rocketmq.connect.jdbc.exception.TableAlterOrCreateException;
+import org.apache.rocketmq.connect.jdbc.util.*;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import java.sql.Connection;
+import java.sql.SQLException;
+import java.util.Collection;
+import java.util.Collections;
+import java.util.HashSet;
+import java.util.List;
+import java.util.Map;
+import java.util.Set;
+import java.util.concurrent.ConcurrentHashMap;
+
+/**
+ *
+ */
+public class DbStructure {
+  private static final Logger log = LoggerFactory.getLogger(DbStructure.class);
+
+  private final DatabaseDialect dbDialect;
+  private final TableDefinitions tableDefns;
+
+  public DbStructure(DatabaseDialect dbDialect) {
+    this.dbDialect = dbDialect;
+    this.tableDefns = new TableDefinitions(dbDialect);
+  }
+
+  /**
+   * create 或者 alter 表
+   * Create or amend table.
+   *
+   * @param config the connector configuration
+   * @param connection the database connection handle
+   * @param tableId the table ID
+   * @param fieldsMetadata the fields metadata
+   * @return whether a DDL operation was performed
+   * @throws SQLException if a DDL operation was deemed necessary but failed
+   */
+  public boolean createOrAmendIfNecessary(
+      final JdbcSinkConfig config,
+      final Connection connection,
+      final TableId tableId,
+      final FieldsMetadata fieldsMetadata
+  ) throws SQLException {
+    if (tableDefns.get(connection, tableId) == null) {
+      // Table does not yet exist, so attempt to create it ...
+      try {
+        create(config, connection, tableId, fieldsMetadata);
+      } catch (SQLException sqle) {
+        log.warn("Create failed, will attempt amend if table already exists", sqle);
+        try {
+          TableDefinition newDefn = tableDefns.refresh(connection, tableId);
+          if (newDefn == null) {
+            throw sqle;
+          }
+        } catch (SQLException e) {
+          throw sqle;
+        }
+      }
+    }
+    return amendIfNecessary(config, connection, tableId, fieldsMetadata, config.getMaxRetries());
+  }
+
+  public void applyDdl(String ddl) throws SQLException {
+    Connection connection = dbDialect.getConnection();
+    dbDialect.applyDdlStatements(connection, Collections.singletonList(ddl));
+  }
+  /**
+   * Get the definition for the table with the given ID. This returns a cached definition if
+   * there is one; otherwise, it reads the definition from the database
+   *
+   * @param connection the connection that may be used to fetch the table definition if not
+   *                   already known; may not be null
+   * @param tableId    the ID of the table; may not be null
+   * @return the table definition; or null if the table does not exist
+   * @throws SQLException if there is an error getting the definition from the database
+   */
+  public TableDefinition tableDefinition(
+      Connection connection,
+      TableId tableId
+  ) throws SQLException {
+    TableDefinition defn = tableDefns.get(connection, tableId);
+    if (defn != null) {
+      return defn;
+    }
+    return tableDefns.refresh(connection, tableId);
+  }
+
+  /**
+   * @throws SQLException if CREATE failed
+   */
+  void create(
+      final JdbcSinkConfig config,
+      final Connection connection,
+      final TableId tableId,
+      final FieldsMetadata fieldsMetadata
+  ) throws SQLException{
+    if (!config.isAutoCreate()) {
+      throw new TableAlterOrCreateException(
+          String.format("Table %s is missing and auto-creation is disabled", tableId)
+      );
+    }
+    String sql = dbDialect.buildCreateTableStatement(tableId, fieldsMetadata.allFields.values());
+    log.info("Creating table with sql: {}", sql);
+    dbDialect.applyDdlStatements(connection, Collections.singletonList(sql));
+  }
+
+  /**
+   * @return whether an ALTER was successfully performed
+   * @throws SQLException if ALTER was deemed necessary but failed
+   */
+  boolean amendIfNecessary(
+      final JdbcSinkConfig config,
+      final Connection connection,
+      final TableId tableId,
+      final FieldsMetadata fieldsMetadata,
+      final int maxRetries
+  ) throws SQLException, TableAlterOrCreateException {
+    final TableDefinition tableDefn = tableDefns.get(connection, tableId);
+    final Set<SinkRecordField> missingFields = missingFields(
+        fieldsMetadata.allFields.values(),
+        tableDefn.columnNames()
+    );
+
+    if (missingFields.isEmpty()) {
+      return false;
+    }
+    // At this point there are missing fields
+    TableType type = tableDefn.type();
+    switch (type) {
+      case TABLE:
+        // Rather than embed the logic and change lots of lines, just break out
+        break;
+      case VIEW:
+      default:
+        throw new TableAlterOrCreateException(
+            String.format(
+                "%s %s is missing fields (%s) and ALTER %s is unsupported",
+                type.capitalized(),
+                tableId,
+                missingFields,
+                type.jdbcName()
+            )
+        );
+    }
+
+    final Set<SinkRecordField> replacedMissingFields = new HashSet<>();
+    for (SinkRecordField missingField: missingFields) {
+      if (!missingField.isOptional() && missingField.defaultValue() == null) {
+        throw new TableAlterOrCreateException(String.format(
+            "Cannot ALTER %s %s to add missing field %s, as the field is not optional and does "
+            + "not have a default value",
+            type.jdbcName(),
+            tableId,
+            missingField
+        ));
+      }
+    }
+
+    if (!config.isAutoCreate()) {
+      throw new TableAlterOrCreateException(String.format(
+          "%s %s is missing fields (%s) and auto-evolution is disabled",
+          type.capitalized(),
+          tableId,
+          replacedMissingFields
+      ));
+    }
+
+    final List<String> amendTableQueries = dbDialect.buildAlterTable(tableId, replacedMissingFields);
+    log.info(
+        "Amending {} to add missing fields:{} maxRetries:{} with SQL: {}",
+        type,
+        replacedMissingFields,
+        maxRetries,
+        amendTableQueries
+    );
+    try {
+      dbDialect.applyDdlStatements(connection, amendTableQueries);
+    } catch (SQLException sqle) {
+      if (maxRetries <= 0) {
+        throw new TableAlterOrCreateException(
+            String.format(
+                "Failed to amend %s '%s' to add missing fields: %s",
+                type,
+                tableId,
+                replacedMissingFields
+            ),
+            sqle
+        );
+      }
+      log.warn("Amend failed, re-attempting", sqle);
+      tableDefns.refresh(connection, tableId);
+      // Perhaps there was a race with other tasks to add the columns
+      return amendIfNecessary(
+          config,
+          connection,
+          tableId,
+          fieldsMetadata,
+          maxRetries - 1
+      );
+    }
+
+    tableDefns.refresh(connection, tableId);
+    return true;
+  }
+
+  Set<SinkRecordField> missingFields(
+      Collection<SinkRecordField> fields,
+      Set<String> dbColumnNames
+  ) {
+    final Set<SinkRecordField> missingFields = new HashSet<>();
+    for (SinkRecordField field : fields) {
+      if (!dbColumnNames.contains(field.name())) {
+        log.debug("Found missing field: {}", field);
+        missingFields.add(field);
+      }
+    }
+
+    if (missingFields.isEmpty()) {
+      return missingFields;
+    }
+
+    // check if the missing fields can be located by ignoring case
+    Set<String> columnNamesLowerCase = new HashSet<>();
+    for (String columnName: dbColumnNames) {
+      columnNamesLowerCase.add(columnName.toLowerCase());
+    }
+
+    if (columnNamesLowerCase.size() != dbColumnNames.size()) {
+      log.warn(
+          "Table has column names that differ only by case. Original columns={}",
+          dbColumnNames
+      );
+    }
+
+    final Set<SinkRecordField> missingFieldsIgnoreCase = new HashSet<>();
+    for (SinkRecordField missing: missingFields) {
+      if (!columnNamesLowerCase.contains(missing.name().toLowerCase())) {
+        missingFieldsIgnoreCase.add(missing);
+      }
+    }
+
+    if (missingFieldsIgnoreCase.size() > 0) {
+      log.info(
+          "Unable to find fields {} among column names {}",
+          missingFieldsIgnoreCase,
+          dbColumnNames
+      );
+    }
+
+    return missingFieldsIgnoreCase;
+  }
+
+
+  /**
+   * delete field
+   * @param fields
+   * @param dbColumnNames
+   * @return
+   */
+  Set<ColumnDefinition> alterDeleteFields(
+          Collection<SinkRecordField> fields,
+          Map<String, ColumnDefinition> dbColumnNames
+  ) {
+
+    final Map<String,SinkRecordField> recordFields = new ConcurrentHashMap<>();
+    for (SinkRecordField field : fields) {
+      recordFields.put(field.name(),field);
+    }
+
+    Set<ColumnDefinition> deleteColumns=new HashSet<>();
+    // filter delete column name
+    for (String columnName : dbColumnNames.keySet()) {
+      if (!recordFields.containsKey(columnName)) {
+        log.debug("Found delete field: {}", columnName);
+        ColumnDefinition columnDefinition=dbColumnNames.get(columnName);
+        deleteColumns.add(dbColumnNames.get(columnName));
+      }
+    }
+    return deleteColumns;
+  }
+}
diff --git a/connectors/rocketmq-connect-jdbc/src/main/java/org/apache/rocketmq/connect/jdbc/schema/table/TableDefinition.java b/connectors/rocketmq-connect-jdbc/src/main/java/org/apache/rocketmq/connect/jdbc/schema/table/TableDefinition.java
new file mode 100644
index 0000000..7027e9b
--- /dev/null
+++ b/connectors/rocketmq-connect-jdbc/src/main/java/org/apache/rocketmq/connect/jdbc/schema/table/TableDefinition.java
@@ -0,0 +1,123 @@
+/*
+ * 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.rocketmq.connect.jdbc.schema.table;
+
+import org.apache.rocketmq.connect.jdbc.schema.column.ColumnDefinition;
+import org.apache.rocketmq.connect.jdbc.util.TableType;
+
+import java.util.Collection;
+import java.util.LinkedHashMap;
+import java.util.Map;
+import java.util.Objects;
+import java.util.Set;
+
+
+/**
+ * A description of a table.
+ */
+public class TableDefinition {
+  private final TableId id;
+  private final Map<String, ColumnDefinition> columnsByName = new LinkedHashMap<>();
+  private final Map<String, String> pkColumnNames = new LinkedHashMap<>();
+  private final TableType type;
+
+  public TableDefinition(
+      TableId id,
+      Iterable<ColumnDefinition> columns
+  ) {
+    this(id, columns, TableType.TABLE);
+  }
+
+  public TableDefinition(
+      TableId id,
+      Iterable<ColumnDefinition> columns,
+      TableType type
+  ) {
+    this.id = id;
+    this.type = Objects.requireNonNull(type);
+    for (ColumnDefinition defn : columns) {
+      String columnName = defn.id().name();
+      columnsByName.put(
+          columnName,
+          defn.forTable(this.id)
+      );
+      if (defn.isPrimaryKey()) {
+        this.pkColumnNames.put(
+            columnName,
+            columnName
+        );
+      }
+    }
+  }
+
+  public TableId id() {
+    return id;
+  }
+
+  public TableType type() {
+    return type;
+  }
+
+  public int columnCount() {
+    return columnsByName.size();
+  }
+
+  public ColumnDefinition definitionForColumn(String name) {
+    return columnsByName.get(name);
+  }
+
+  public Collection<ColumnDefinition> definitionsForColumns() {
+    return columnsByName.values();
+  }
+
+  public Collection<String> primaryKeyColumnNames() {
+    return pkColumnNames.values();
+  }
+
+  public Set<String> columnNames() {
+    return columnsByName.keySet();
+  }
+
+  @Override
+  public int hashCode() {
+    return id.hashCode();
+  }
+
+  @Override
+  public boolean equals(Object obj) {
+    if (obj == null) {
+      return true;
+    }
+    if (obj instanceof TableDefinition) {
+      TableDefinition that = (TableDefinition) obj;
+      return Objects.equals(this.id(), that.id())
+             && Objects.equals(this.type(), that.type())
+             && Objects.equals(this.definitionsForColumns(), that.definitionsForColumns());
+    }
+    return false;
+  }
+
+  @Override
+  public String toString() {
+    return String.format(
+        "Table{name='%s', type=%s columns=%s}",
+        id,
+        type,
+        definitionsForColumns()
+    );
+  }
+}
diff --git a/connectors/rocketmq-connect-jdbc/src/main/java/org/apache/rocketmq/connect/jdbc/schema/table/TableDefinitions.java b/connectors/rocketmq-connect-jdbc/src/main/java/org/apache/rocketmq/connect/jdbc/schema/table/TableDefinitions.java
new file mode 100644
index 0000000..5169490
--- /dev/null
+++ b/connectors/rocketmq-connect-jdbc/src/main/java/org/apache/rocketmq/connect/jdbc/schema/table/TableDefinitions.java
@@ -0,0 +1,93 @@
+/*
+ * 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.rocketmq.connect.jdbc.schema.table;
+
+import org.apache.rocketmq.connect.jdbc.dialect.DatabaseDialect;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import java.sql.Connection;
+import java.sql.SQLException;
+import java.util.HashMap;
+import java.util.Map;
+
+/**
+ * A simple cache of {@link TableDefinition} keyed.
+ */
+public class TableDefinitions {
+
+  private static final Logger log = LoggerFactory.getLogger(TableDefinitions.class);
+
+  private final Map<TableId, TableDefinition> cache = new HashMap<>();
+  private final DatabaseDialect dialect;
+
+  /**
+   * Create an instance that uses the specified database dialect.
+   *
+   * @param dialect the database dialect; may not be null
+   */
+  public TableDefinitions(DatabaseDialect dialect) {
+    this.dialect = dialect;
+  }
+
+  /**
+   * Get the {@link TableDefinition} for the given table.
+   *
+   * @param connection the JDBC connection to use; may not be null
+   * @param tableId    the table identifier; may not be null
+   * @return the cached {@link TableDefinition}, or null if there is no such table
+   * @throws SQLException if there is any problem using the connection
+   */
+  public TableDefinition get(
+      Connection connection,
+      final TableId tableId
+  ) throws SQLException {
+    TableDefinition dbTable = cache.get(tableId);
+    if (dbTable == null) {
+      if (dialect.tableExists(connection, tableId)) {
+        dbTable = dialect.describeTable(connection, tableId);
+        if (dbTable != null) {
+          log.info("Setting metadata for table {} to {}", tableId, dbTable);
+          cache.put(tableId, dbTable);
+        }
+      }
+    }
+    return dbTable;
+  }
+
+  /**
+   * Refresh the cached {@link TableDefinition} for the given table.
+   *
+   * @param connection the JDBC connection to use; may not be null
+   * @param tableId    the table identifier; may not be null
+   * @return the refreshed {@link TableDefinition}, or null if there is no such table
+   * @throws SQLException if there is any problem using the connection
+   */
+  public TableDefinition refresh(
+      Connection connection,
+      TableId tableId
+  ) throws SQLException {
+    TableDefinition dbTable = dialect.describeTable(connection, tableId);
+    if (dbTable != null) {
+      log.info("Refreshing metadata for table {} to {}", tableId, dbTable);
+      cache.put(dbTable.id(), dbTable);
+    } else {
+      log.warn("Failed to refresh metadata for table {}", tableId);
+    }
+    return dbTable;
+  }
+}
diff --git a/connectors/rocketmq-connect-jdbc/src/main/java/org/apache/rocketmq/connect/jdbc/schema/table/TableId.java b/connectors/rocketmq-connect-jdbc/src/main/java/org/apache/rocketmq/connect/jdbc/schema/table/TableId.java
new file mode 100644
index 0000000..e004fd2
--- /dev/null
+++ b/connectors/rocketmq-connect-jdbc/src/main/java/org/apache/rocketmq/connect/jdbc/schema/table/TableId.java
@@ -0,0 +1,144 @@
+/*
+ * 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.rocketmq.connect.jdbc.schema.table;
+
+import org.apache.commons.lang3.StringUtils;
+import org.apache.rocketmq.connect.jdbc.util.ExpressionBuilder;
+import org.apache.rocketmq.connect.jdbc.util.QuoteMethod;
+
+import java.util.Objects;
+
+public class TableId implements Comparable<TableId>, ExpressionBuilder.Expressable {
+
+  private final String catalogName;
+  private final String schemaName;
+  private final String tableName;
+  private final int hash;
+
+  public TableId(
+      String catalogName,
+      String schemaName,
+      String tableName
+  ) {
+    this.catalogName = catalogName == null || catalogName.isEmpty() ? null : catalogName;
+    this.schemaName = schemaName == null || schemaName.isEmpty() ? null : schemaName;
+    this.tableName = tableName;
+    this.hash = Objects.hash(catalogName, schemaName, tableName);
+  }
+
+  public String catalogName() {
+    return catalogName;
+  }
+
+  public String schemaName() {
+    return schemaName;
+  }
+
+  public String dbName(){
+    if(StringUtils.isNotBlank(catalogName)){
+      return catalogName;
+    }
+    return schemaName;
+  }
+
+  public String tableName() {
+    return tableName;
+  }
+
+  @Override
+  public void appendTo(ExpressionBuilder builder, boolean useQuotes) {
+    appendTo(builder, useQuotes ? QuoteMethod.ALWAYS : QuoteMethod.NEVER);
+  }
+
+  @Override
+  public void appendTo(
+      ExpressionBuilder builder,
+      QuoteMethod useQuotes
+  ) {
+    if (catalogName != null) {
+      builder.appendIdentifier(catalogName, useQuotes);
+      builder.appendIdentifierDelimiter();
+    }
+    if (schemaName != null) {
+      builder.appendIdentifier(schemaName, useQuotes);
+      builder.appendIdentifierDelimiter();
+    }
+    builder.appendTableName(tableName, useQuotes);
+  }
+
+  @Override
+  public int hashCode() {
+    return hash;
+  }
+
+  @Override
+  public boolean equals(Object obj) {
+    if (obj == this) {
+      return true;
+    }
+    if (obj instanceof TableId) {
+      TableId that = (TableId) obj;
+      return Objects.equals(this.catalogName, that.catalogName)
+             && Objects.equals(this.schemaName, that.schemaName)
+             && Objects.equals(this.tableName, that.tableName);
+    }
+    return false;
+  }
+
+  @Override
+  public int compareTo(TableId that) {
+    if (that == this) {
+      return 0;
+    }
+    int diff = this.tableName.compareTo(that.tableName);
+    if (diff != 0) {
+      return diff;
+    }
+    if (this.schemaName == null) {
+      if (that.schemaName != null) {
+        return -1;
+      }
+    } else {
+      if (that.schemaName == null) {
+        return 1;
+      }
+      diff = this.schemaName.compareTo(that.schemaName);
+      if (diff != 0) {
+        return diff;
+      }
+    }
+    if (this.catalogName == null) {
+      if (that.catalogName != null) {
+        return -1;
+      }
+    } else {
+      if (that.catalogName == null) {
+        return 1;
+      }
+      diff = this.catalogName.compareTo(that.catalogName);
+      if (diff != 0) {
+        return diff;
+      }
+    }
+    return 0;
+  }
+
+  @Override
+  public String toString() {
+    return ExpressionBuilder.create().append(this).toString();
+  }
+}
diff --git a/connectors/rocketmq-connect-jdbc/src/main/java/org/apache/rocketmq/connect/jdbc/sink/BufferedRecords.java b/connectors/rocketmq-connect-jdbc/src/main/java/org/apache/rocketmq/connect/jdbc/sink/BufferedRecords.java
new file mode 100644
index 0000000..f20927c
--- /dev/null
+++ b/connectors/rocketmq-connect-jdbc/src/main/java/org/apache/rocketmq/connect/jdbc/sink/BufferedRecords.java
@@ -0,0 +1,330 @@
+/*
+ * 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.rocketmq.connect.jdbc.sink;
+
+import io.openmessaging.connector.api.data.ConnectRecord;
+import io.openmessaging.connector.api.data.Schema;
+import io.openmessaging.connector.api.errors.ConnectException;
+import org.apache.rocketmq.connect.jdbc.connector.JdbcSinkConfig;
+import org.apache.rocketmq.connect.jdbc.dialect.DatabaseDialect;
+import org.apache.rocketmq.connect.jdbc.schema.db.DbStructure;
+import org.apache.rocketmq.connect.jdbc.sink.metadata.FieldsMetadata;
+import org.apache.rocketmq.connect.jdbc.sink.metadata.SchemaPair;
+import org.apache.rocketmq.connect.jdbc.schema.column.ColumnId;
+import org.apache.rocketmq.connect.jdbc.schema.table.TableId;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import java.sql.Connection;
+import java.sql.PreparedStatement;
+import java.sql.SQLException;
+import java.sql.Statement;
+import java.util.ArrayList;
+import java.util.Collection;
+import java.util.List;
+import java.util.Objects;
+import java.util.Optional;
+import java.util.stream.Collectors;
+import static java.util.Objects.isNull;
+import static java.util.Objects.nonNull;
+
+/**
+ * buffered records
+ */
+public class BufferedRecords {
+  private static final Logger log = LoggerFactory.getLogger(BufferedRecords.class);
+
+
+  private final TableId tableId;
+  private final JdbcSinkConfig config;
+  private final DatabaseDialect dbDialect;
+  private final DbStructure dbStructure;
+  private final Connection connection;
+
+  private List<ConnectRecord> records = new ArrayList<>();
+  private Schema schema;
+  private FieldsMetadata fieldsMetadata;
+  private PreparedStatement updatePreparedStatement;
+  private PreparedStatement deletePreparedStatement;
+  private DatabaseDialect.StatementBinder updateStatementBinder;
+  private DatabaseDialect.StatementBinder deleteStatementBinder;
+
+  public BufferedRecords(
+      JdbcSinkConfig config,
+      TableId tableId,
+      DatabaseDialect dbDialect,
+      DbStructure dbStructure,
+      Connection connection
+  ) {
+    this.tableId = tableId;
+    this.config = config;
+    this.dbDialect = dbDialect;
+    this.dbStructure = dbStructure;
+    this.connection = connection;
+  }
+
+  /**
+   * add record
+   * @param record
+   * @return
+   * @throws SQLException
+   */
+  public List<ConnectRecord> add(ConnectRecord record) throws SQLException {
+    final List<ConnectRecord> flushed = new ArrayList<>();
+    boolean schemaChanged = false;
+    // check and update schema
+    if (!Objects.equals(schema, record.getSchema())) {
+      schema = record.getSchema();
+      schemaChanged = true;
+    }
+
+    if (schemaChanged || updateStatementBinder == null) {
+      // Each batch needs to have the same schemas, so get the buffered records out
+      flushed.addAll(flush());
+      // re-initialize everything that depends on the record schema
+      final SchemaPair schemaPair = new SchemaPair(
+          record.getSchema(),
+          record.getExtensions()
+      );
+      // 抽取字段
+      fieldsMetadata = FieldsMetadata.extract(
+          tableId.tableName(),
+          config.pkMode,
+          config.getPkFields(),
+          config.getFieldsWhitelist(),
+          schemaPair
+      );
+      // create or alter table
+      dbStructure.createOrAmendIfNecessary(
+          config,
+          connection,
+          tableId,
+          fieldsMetadata
+      );
+      final String insertSql = getInsertSql();
+      final String deleteSql = getDeleteSql();
+      log.debug(
+          "{} sql: {} deleteSql: {} meta: {}",
+          config.getInsertMode(),
+          insertSql,
+          deleteSql,
+          fieldsMetadata
+      );
+      close();
+      updatePreparedStatement = dbDialect.createPreparedStatement(connection, insertSql);
+      updateStatementBinder = dbDialect.statementBinder(
+          updatePreparedStatement,
+          config.pkMode,
+          schemaPair,
+          fieldsMetadata,
+          dbStructure.tableDefinition(connection, tableId),
+          config.getInsertMode()
+      );
+      if (config.isDeleteEnabled() && nonNull(deleteSql)) {
+        deletePreparedStatement = dbDialect.createPreparedStatement(connection, deleteSql);
+        deleteStatementBinder = dbDialect.statementBinder(
+            deletePreparedStatement,
+            config.pkMode,
+            schemaPair,
+            fieldsMetadata,
+            dbStructure.tableDefinition(connection, tableId),
+            config.getInsertMode()
+        );
+      }
+    }
+
+    records.add(record);
+    if (records.size() >= config.getBatchSize()) {
+      flushed.addAll(flush());
+    }
+    return flushed;
+  }
+
+  public List<ConnectRecord> flush() throws SQLException {
+    if (records.isEmpty()) {
+      log.debug("Records is empty");
+      return new ArrayList<>();
+    }
+    log.debug("Flushing {} buffered records", records.size());
+    for (ConnectRecord record : records) {
+      if (isNull(record.getData()) && nonNull(deleteStatementBinder)) {
+        deleteStatementBinder.bindRecord(record);
+      } else {
+        updateStatementBinder.bindRecord(record);
+      }
+    }
+    Optional<Long> totalUpdateCount = executeUpdates();
+    long totalDeleteCount = executeDeletes();
+
+    final long expectedCount = updateRecordCount();
+    log.trace("{} records:{} resulting in totalUpdateCount:{} totalDeleteCount:{}",
+        config.getInsertMode(), records.size(), totalUpdateCount, totalDeleteCount
+    );
+    if (totalUpdateCount.filter(total -> total != expectedCount).isPresent()
+        && config.getInsertMode() == JdbcSinkConfig.InsertMode.INSERT) {
+      throw new ConnectException(String.format(
+          "Update count (%d) did not sum up to total number of records inserted (%d)",
+          totalUpdateCount.get(),
+          expectedCount
+      ));
+    }
+    if (!totalUpdateCount.isPresent()) {
+      log.info(
+          "{} records:{} , but no count of the number of rows it affected is available",
+          config.getInsertMode(),
+          records.size()
+      );
+    }
+
+    final List<ConnectRecord> flushedRecords = records;
+    records = new ArrayList<>();
+    return flushedRecords;
+  }
+
+  /**
+   * @return an optional count of all updated rows or an empty optional if no info is available
+   */
+  private Optional<Long> executeUpdates() throws SQLException {
+    Optional<Long> count = Optional.empty();
+    if (nonNull(updatePreparedStatement)){
+      try{
+        for (int updateCount : updatePreparedStatement.executeBatch()) {
+          if (updateCount != Statement.SUCCESS_NO_INFO) {
+            count = count.isPresent()
+                    ? count.map(total -> total + updateCount)
+                    : Optional.of((long) updateCount);
+          }
+        }
+      }catch(SQLException e){
+        log.error("updatePreparedStatement.executeBatch failed, errCode={}, sqlState={}, error msg={}, cause={}, sql={}",
+                e.getErrorCode(), e.getSQLState(), e.getMessage(), e.getCause(), updatePreparedStatement);
+        throw e;
+      }
+    }
+    return count;
+  }
+
+  private long executeDeletes() throws SQLException {
+    long totalDeleteCount = 0;
+    if (nonNull(deletePreparedStatement)) {
+      for (int updateCount : deletePreparedStatement.executeBatch()) {
+        if (updateCount != Statement.SUCCESS_NO_INFO) {
+          totalDeleteCount += updateCount;
+        }
+      }
+    }
+    return totalDeleteCount;
+  }
+
+  private long updateRecordCount() {
+    return records
+        .stream()
+        // ignore deletes
+        .filter(record -> nonNull(record.getData()) || !config.isDeleteEnabled())
+        .count();
+  }
+
+  public void close() throws SQLException {
+    log.debug(
+        "Closing BufferedRecords with updatePreparedStatement: {} deletePreparedStatement: {}",
+        updatePreparedStatement,
+        deletePreparedStatement
+    );
+    if (nonNull(updatePreparedStatement)) {
+      updatePreparedStatement.close();
+      updatePreparedStatement = null;
+    }
+    if (nonNull(deletePreparedStatement)) {
+      deletePreparedStatement.close();
+      deletePreparedStatement = null;
+    }
+  }
+
+  private String getInsertSql() throws SQLException {
+    switch (config.getInsertMode()) {
+      case INSERT:
+        return dbDialect.buildInsertStatement(
... 4701 lines suppressed ...