Implement new flink-connector-rocketmq base on FLIP-27/FLIP-191 (#94)

diff --git a/.gitignore b/.gitignore
index f70c86a..7c74295 100644
--- a/.gitignore
+++ b/.gitignore
@@ -34,3 +34,4 @@
 
 # Ignore it while not disable for some reason.
 dependency-reduced-pom.xml
+/logs/output.log.*
\ No newline at end of file
diff --git a/pom.xml b/pom.xml
index 1b29948..67f322d 100644
--- a/pom.xml
+++ b/pom.xml
@@ -18,11 +18,13 @@
 <project xmlns="http://maven.apache.org/POM/4.0.0"
          xmlns:xsi="http://www.w3.org/2001/XMLSchema-instance"
          xsi:schemaLocation="http://maven.apache.org/POM/4.0.0 http://maven.apache.org/xsd/maven-4.0.0.xsd">
+
     <modelVersion>4.0.0</modelVersion>
 
-    <groupId>org.apache.rocketmq</groupId>
-    <artifactId>rocketmq-flink</artifactId>
-    <version>1.0.0-SNAPSHOT</version>
+    <groupId>org.apache.flink</groupId>
+    <artifactId>flink-connector-rocketmq</artifactId>
+    <name>Flink : Connectors : RocketMQ</name>
+    <version>1.15.0</version>
     <packaging>jar</packaging>
 
     <properties>
@@ -33,72 +35,98 @@
         <!-- compiler settings properties -->
         <maven.compiler.source>1.8</maven.compiler.source>
         <maven.compiler.target>1.8</maven.compiler.target>
-        <rocketmq.version>4.9.2</rocketmq.version>
-        <flink.version>1.15.0</flink.version>
-        <commons-lang.version>2.6</commons-lang.version>
-        <spotless.version>2.4.2</spotless.version>
-        <jaxb-api.version>2.3.1</jaxb-api.version>
+        <!-- rocketmq version -->
+        <rocketmq.version>5.1.1</rocketmq.version>
         <rocketmq.schema.registry.version>0.1.0</rocketmq.schema.registry.version>
+        <!-- dependency version -->
+        <slf4j-api.version>2.0.5</slf4j-api.version>
+        <logback.version>1.3.5</logback.version>
+        <commons-lang.version>2.6</commons-lang.version>
+        <spotless.version>2.13.0</spotless.version>
+        <junit.version>4.13.2</junit.version>
+        <junit-jupiter.version>5.9.2</junit-jupiter.version>
+        <powermock.version>1.7.4</powermock.version>
     </properties>
 
     <dependencies>
-        <dependency>
-            <groupId>org.apache.flink</groupId>
-            <artifactId>flink-java</artifactId>
-            <version>${flink.version}</version>
-            <scope>provided</scope>
-        </dependency>
+        <!-- Core -->
         <dependency>
             <groupId>org.apache.flink</groupId>
             <artifactId>flink-streaming-java</artifactId>
-            <version>${flink.version}</version>
+            <version>${project.version}</version>
             <scope>provided</scope>
         </dependency>
-        <dependency>
-            <groupId>org.apache.flink</groupId>
-            <artifactId>flink-clients</artifactId>
-            <version>${flink.version}</version>
-        </dependency>
+
+        <!-- Connectors -->
         <dependency>
             <groupId>org.apache.flink</groupId>
             <artifactId>flink-connector-base</artifactId>
-            <version>${flink.version}</version>
+            <version>${project.version}</version>
+            <scope>provided</scope>
         </dependency>
+
         <dependency>
             <groupId>org.apache.flink</groupId>
-            <artifactId>flink-table-common</artifactId>
-            <version>${flink.version}</version>
+            <artifactId>flink-clients</artifactId>
+            <version>${project.version}</version>
+            <scope>test</scope>
         </dependency>
+
         <dependency>
             <groupId>org.apache.flink</groupId>
-            <artifactId>flink-table-api-java-bridge</artifactId>
-            <version>${flink.version}</version>
+            <artifactId>flink-runtime-web</artifactId>
+            <version>${project.version}</version>
+            <scope>test</scope>
         </dependency>
+
+        <!-- Table ecosystem -->
         <dependency>
             <groupId>org.apache.flink</groupId>
             <artifactId>flink-table-runtime</artifactId>
-            <version>${flink.version}</version>
+            <version>${project.version}</version>
+            <scope>provided</scope>
         </dependency>
-        <dependency>
-            <groupId>org.apache.flink</groupId>
-            <artifactId>flink-queryable-state-runtime</artifactId>
-            <version>${flink.version}</version>
-        </dependency>
+
         <dependency>
             <groupId>org.apache.flink</groupId>
             <artifactId>flink-avro</artifactId>
-            <version>${flink.version}</version>
+            <version>${project.version}</version>
+            <scope>provided</scope>
         </dependency>
+
+        <!-- RocketMQ -->
         <dependency>
             <groupId>org.apache.rocketmq</groupId>
             <artifactId>rocketmq-client</artifactId>
             <version>${rocketmq.version}</version>
+            <exclusions>
+                <exclusion>
+                    <groupId>io.github.aliyunmq</groupId>
+                    <artifactId>rocketmq-logback-classic</artifactId>
+                </exclusion>
+                <exclusion>
+                    <groupId>io.github.aliyunmq</groupId>
+                    <artifactId>rocketmq-shaded-slf4j-api-bridge</artifactId>
+                </exclusion>
+            </exclusions>
         </dependency>
+
         <dependency>
             <groupId>org.apache.rocketmq</groupId>
             <artifactId>rocketmq-acl</artifactId>
             <version>${rocketmq.version}</version>
+            <exclusions>
+                <exclusion>
+                    <groupId>io.github.aliyunmq</groupId>
+                    <artifactId>rocketmq-logback-classic</artifactId>
+                </exclusion>
+                <exclusion>
+                    <groupId>io.github.aliyunmq</groupId>
+                    <artifactId>rocketmq-shaded-slf4j-api-bridge</artifactId>
+                </exclusion>
+            </exclusions>
         </dependency>
+
         <dependency>
             <groupId>org.apache.rocketmq</groupId>
             <artifactId>rocketmq-common</artifactId>
@@ -110,21 +138,23 @@
                 </exclusion>
             </exclusions>
         </dependency>
+
         <dependency>
             <groupId>org.apache.rocketmq</groupId>
-            <artifactId>rocketmq-namesrv</artifactId>
+            <artifactId>rocketmq-tools</artifactId>
             <version>${rocketmq.version}</version>
+            <exclusions>
+                <exclusion>
+                    <groupId>io.github.aliyunmq</groupId>
+                    <artifactId>rocketmq-logback-classic</artifactId>
+                </exclusion>
+                <exclusion>
+                    <groupId>io.github.aliyunmq</groupId>
+                    <artifactId>rocketmq-shaded-slf4j-api-bridge</artifactId>
+                </exclusion>
+            </exclusions>
         </dependency>
-        <dependency>
-            <groupId>org.apache.rocketmq</groupId>
-            <artifactId>rocketmq-broker</artifactId>
-            <version>${rocketmq.version}</version>
-        </dependency>
-        <dependency>
-            <groupId>org.apache.rocketmq</groupId>
-            <artifactId>rocketmq-test</artifactId>
-            <version>${rocketmq.version}</version>
-        </dependency>
+
         <dependency>
             <groupId>org.apache.rocketmq</groupId>
             <artifactId>schema-registry-client</artifactId>
@@ -137,26 +167,51 @@
             <version>${commons-lang.version}</version>
         </dependency>
 
+        <dependency>
+            <groupId>org.slf4j</groupId>
+            <artifactId>slf4j-api</artifactId>
+            <version>${slf4j-api.version}</version>
+        </dependency>
+
+        <dependency>
+            <groupId>ch.qos.logback</groupId>
+            <artifactId>logback-classic</artifactId>
+            <version>${logback.version}</version>
+        </dependency>
+
+        <dependency>
+            <groupId>ch.qos.logback</groupId>
+            <artifactId>logback-core</artifactId>
+            <version>${logback.version}</version>
+        </dependency>
+
         <!--test -->
         <dependency>
             <groupId>junit</groupId>
             <artifactId>junit</artifactId>
-            <scope>test</scope>
-            <version>4.12</version>
-        </dependency>
-        <dependency>
-            <groupId>org.powermock</groupId>
-            <artifactId>powermock-module-junit4</artifactId>
-            <version>1.5.5</version>
-            <scope>test</scope>
-        </dependency>
-        <dependency>
-            <groupId>org.powermock</groupId>
-            <artifactId>powermock-api-mockito</artifactId>
-            <version>1.5.5</version>
+            <version>${junit.version}</version>
             <scope>test</scope>
         </dependency>
 
+        <dependency>
+            <groupId>org.powermock</groupId>
+            <artifactId>powermock-api-mockito</artifactId>
+            <version>${powermock.version}</version>
+            <scope>test</scope>
+        </dependency>
+
+        <dependency>
+            <groupId>org.powermock</groupId>
+            <artifactId>powermock-module-junit4</artifactId>
+            <version>${powermock.version}</version>
+            <scope>test</scope>
+        </dependency>
+        <dependency>
+            <groupId>org.junit.jupiter</groupId>
+            <artifactId>junit-jupiter</artifactId>
+            <version>${junit-jupiter.version}</version>
+            <scope>test</scope>
+        </dependency>
     </dependencies>
 
     <build>
@@ -164,7 +219,7 @@
             <plugin>
                 <groupId>org.apache.maven.plugins</groupId>
                 <artifactId>maven-shade-plugin</artifactId>
-                <version>2.4.3</version>
+                <version>3.2.4</version>
                 <executions>
                     <execution>
                         <phase>package</phase>
@@ -187,7 +242,7 @@
                                     <resource>reference.conf</resource>
                                 </transformer>
                                 <transformer implementation="org.apache.maven.plugins.shade.resource.ManifestResourceTransformer">
-                                    <mainClass>org.apache.rocketmq.flink.legacy.example.RocketMQFlinkExample</mainClass>
+                                    <mainClass>org.apache.flink.connector.rocketmq.legacy.example.RocketMQFlinkExample</mainClass>
                                 </transformer>
                             </transformers>
                         </configuration>
@@ -196,8 +251,9 @@
             </plugin>
 
             <plugin>
+                <groupId>org.apache.maven.plugins</groupId>
                 <artifactId>maven-compiler-plugin</artifactId>
-                <version>3.5.1</version>
+                <version>3.10.0</version>
                 <configuration>
                     <source>${maven.compiler.source}</source>
                     <target>${maven.compiler.target}</target>
@@ -206,6 +262,7 @@
                     <showWarnings>true</showWarnings>
                 </configuration>
             </plugin>
+
             <plugin>
                 <groupId>org.apache.maven.plugins</groupId>
                 <artifactId>maven-surefire-plugin</artifactId>
@@ -214,6 +271,7 @@
                     <skipTests>${maven.test.skip}</skipTests>
                 </configuration>
             </plugin>
+
             <plugin>
                 <groupId>org.apache.rat</groupId>
                 <artifactId>apache-rat-plugin</artifactId>
@@ -227,7 +285,8 @@
                     </excludes>
                 </configuration>
             </plugin>
-	    <plugin>
+
+	        <plugin>
                 <groupId>org.jacoco</groupId>
                 <artifactId>jacoco-maven-plugin</artifactId>
                 <version>0.8.7</version>
@@ -240,6 +299,7 @@
                     </execution>
                 </executions>
             </plugin>
+
             <plugin>
                 <groupId>org.eluder.coveralls</groupId>
                 <artifactId>coveralls-maven-plugin</artifactId>
@@ -252,27 +312,8 @@
                     </dependency>
                 </dependencies>
             </plugin>
-            <plugin>
-                <artifactId>maven-checkstyle-plugin</artifactId>
-                <version>2.17</version>
-                <executions>
-                    <execution>
-                        <id>verify</id>
-                        <phase>verify</phase>
-                        <configuration>
-                            <configLocation>style/rmq_checkstyle.xml</configLocation>
-                            <encoding>UTF-8</encoding>
-                            <consoleOutput>true</consoleOutput>
-                            <failsOnError>true</failsOnError>
-                            <includeTestSourceDirectory>false</includeTestSourceDirectory>
-                            <includeTestResources>false</includeTestResources>
-                        </configuration>
-                        <goals>
-                            <goal>check</goal>
-                        </goals>
-                    </execution>
-                </executions>
-            </plugin>
+
+
             <plugin>
                 <groupId>org.apache.maven.plugins</groupId>
                 <artifactId>maven-javadoc-plugin</artifactId>
@@ -283,6 +324,7 @@
                     <locale>en</locale>
                 </configuration>
             </plugin>
+
             <!-- Due to the Flink build setup, "mvn spotless:apply" and "mvn spotless:check"
 				don't work. You have to use the fully qualified name, i.e.
 				"mvn com.diffplug.spotless:spotless-maven-plugin:apply" -->
@@ -299,7 +341,7 @@
 
                         <!-- \# refers to the static imports -->
                         <importOrder>
-                            <order>org.apache.rocketmq,org.apache.flink,org.apache.flink.shaded,,javax,java,scala,\#</order>
+                            <order>org.apache.flink,org.apache.flink.shaded,,javax,java,scala,\#</order>
                         </importOrder>
 
                         <removeUnusedImports />
@@ -315,6 +357,7 @@
                     </execution>
                 </executions>
             </plugin>
+
             <plugin>
                 <groupId>org.apache.maven.plugins</groupId>
                 <artifactId>maven-source-plugin</artifactId>
diff --git a/src/main/java/org/apache/rocketmq/flink/legacy/common/selector/TopicSelector.java b/src/main/java/org/apache/flink/connector/rocketmq/MetricUtil.java
similarity index 79%
copy from src/main/java/org/apache/rocketmq/flink/legacy/common/selector/TopicSelector.java
copy to src/main/java/org/apache/flink/connector/rocketmq/MetricUtil.java
index a70c599..528d0fe 100644
--- a/src/main/java/org/apache/rocketmq/flink/legacy/common/selector/TopicSelector.java
+++ b/src/main/java/org/apache/flink/connector/rocketmq/MetricUtil.java
@@ -14,13 +14,7 @@
  * See the License for the specific language governing permissions and
  * limitations under the License.
  */
-package org.apache.rocketmq.flink.legacy.common.selector;
 
-import java.io.Serializable;
+package org.apache.flink.connector.rocketmq;
 
-public interface TopicSelector<T> extends Serializable {
-
-    String getTopic(T tuple);
-
-    String getTag(T tuple);
-}
+public class MetricUtil {}
diff --git a/src/main/java/org/apache/rocketmq/flink/catalog/RocketMQCatalog.java b/src/main/java/org/apache/flink/connector/rocketmq/catalog/RocketMQCatalog.java
similarity index 98%
rename from src/main/java/org/apache/rocketmq/flink/catalog/RocketMQCatalog.java
rename to src/main/java/org/apache/flink/connector/rocketmq/catalog/RocketMQCatalog.java
index 3636995..e56e2e4 100644
--- a/src/main/java/org/apache/rocketmq/flink/catalog/RocketMQCatalog.java
+++ b/src/main/java/org/apache/flink/connector/rocketmq/catalog/RocketMQCatalog.java
@@ -16,18 +16,9 @@
  * limitations under the License.
  */
 
-package org.apache.rocketmq.flink.catalog;
+package org.apache.flink.connector.rocketmq.catalog;
 
-import org.apache.rocketmq.client.exception.MQClientException;
-import org.apache.rocketmq.common.admin.TopicStatsTable;
-import org.apache.rocketmq.flink.common.constant.RocketMqCatalogConstant;
-import org.apache.rocketmq.remoting.protocol.LanguageCode;
-import org.apache.rocketmq.schema.registry.client.SchemaRegistryClient;
-import org.apache.rocketmq.schema.registry.client.SchemaRegistryClientFactory;
-import org.apache.rocketmq.schema.registry.common.dto.GetSchemaResponse;
-import org.apache.rocketmq.schema.registry.common.model.SchemaType;
-import org.apache.rocketmq.tools.admin.DefaultMQAdminExt;
-
+import org.apache.flink.connector.rocketmq.common.constant.RocketMqCatalogConstant;
 import org.apache.flink.formats.avro.typeutils.AvroSchemaConverter;
 import org.apache.flink.table.api.Schema;
 import org.apache.flink.table.catalog.AbstractCatalog;
@@ -62,6 +53,14 @@
 import org.apache.flink.table.types.utils.TypeConversions;
 
 import org.apache.commons.lang3.StringUtils;
+import org.apache.rocketmq.client.exception.MQClientException;
+import org.apache.rocketmq.remoting.protocol.LanguageCode;
+import org.apache.rocketmq.remoting.protocol.admin.TopicStatsTable;
+import org.apache.rocketmq.schema.registry.client.SchemaRegistryClient;
+import org.apache.rocketmq.schema.registry.client.SchemaRegistryClientFactory;
+import org.apache.rocketmq.schema.registry.common.dto.GetSchemaResponse;
+import org.apache.rocketmq.schema.registry.common.model.SchemaType;
+import org.apache.rocketmq.tools.admin.DefaultMQAdminExt;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 
diff --git a/src/main/java/org/apache/rocketmq/flink/catalog/RocketMQCatalogFactory.java b/src/main/java/org/apache/flink/connector/rocketmq/catalog/RocketMQCatalogFactory.java
similarity index 81%
rename from src/main/java/org/apache/rocketmq/flink/catalog/RocketMQCatalogFactory.java
rename to src/main/java/org/apache/flink/connector/rocketmq/catalog/RocketMQCatalogFactory.java
index 609708f..67b048d 100644
--- a/src/main/java/org/apache/rocketmq/flink/catalog/RocketMQCatalogFactory.java
+++ b/src/main/java/org/apache/flink/connector/rocketmq/catalog/RocketMQCatalogFactory.java
@@ -16,7 +16,7 @@
  * limitations under the License.
  */
 
-package org.apache.rocketmq.flink.catalog;
+package org.apache.flink.connector.rocketmq.catalog;
 
 import org.apache.flink.configuration.ConfigOption;
 import org.apache.flink.table.catalog.Catalog;
@@ -27,10 +27,10 @@
 import java.util.HashSet;
 import java.util.Set;
 
-import static org.apache.rocketmq.flink.catalog.RocketMQCatalogFactoryOptions.DEFAULT_DATABASE;
-import static org.apache.rocketmq.flink.catalog.RocketMQCatalogFactoryOptions.IDENTIFIER;
-import static org.apache.rocketmq.flink.catalog.RocketMQCatalogFactoryOptions.NAME_SERVER_ADDR;
-import static org.apache.rocketmq.flink.catalog.RocketMQCatalogFactoryOptions.SCHEMA_REGISTRY_BASE_URL;
+import static org.apache.flink.connector.rocketmq.catalog.RocketMQCatalogFactoryOptions.DEFAULT_DATABASE;
+import static org.apache.flink.connector.rocketmq.catalog.RocketMQCatalogFactoryOptions.IDENTIFIER;
+import static org.apache.flink.connector.rocketmq.catalog.RocketMQCatalogFactoryOptions.NAME_SERVER_ADDR;
+import static org.apache.flink.connector.rocketmq.catalog.RocketMQCatalogFactoryOptions.SCHEMA_REGISTRY_BASE_URL;
 
 /** The {@CatalogFactory} implementation of RocketMQ. */
 public class RocketMQCatalogFactory implements CatalogFactory {
diff --git a/src/main/java/org/apache/rocketmq/flink/catalog/RocketMQCatalogFactoryOptions.java b/src/main/java/org/apache/flink/connector/rocketmq/catalog/RocketMQCatalogFactoryOptions.java
similarity index 91%
rename from src/main/java/org/apache/rocketmq/flink/catalog/RocketMQCatalogFactoryOptions.java
rename to src/main/java/org/apache/flink/connector/rocketmq/catalog/RocketMQCatalogFactoryOptions.java
index 624539b..308a1a3 100644
--- a/src/main/java/org/apache/rocketmq/flink/catalog/RocketMQCatalogFactoryOptions.java
+++ b/src/main/java/org/apache/flink/connector/rocketmq/catalog/RocketMQCatalogFactoryOptions.java
@@ -16,14 +16,13 @@
  * limitations under the License.
  */
 
-package org.apache.rocketmq.flink.catalog;
-
-import org.apache.rocketmq.flink.common.constant.SchemaRegistryConstant;
-import org.apache.rocketmq.flink.legacy.RocketMQConfig;
+package org.apache.flink.connector.rocketmq.catalog;
 
 import org.apache.flink.annotation.Internal;
 import org.apache.flink.configuration.ConfigOption;
 import org.apache.flink.configuration.ConfigOptions;
+import org.apache.flink.connector.rocketmq.common.constant.SchemaRegistryConstant;
+import org.apache.flink.connector.rocketmq.legacy.RocketMQConfig;
 import org.apache.flink.table.catalog.CommonCatalogOptions;
 
 /** {@link ConfigOption}s for {@link RocketMQCatalog}. */
diff --git a/src/main/java/org/apache/flink/connector/rocketmq/common/config/RocketMQConfigBuilder.java b/src/main/java/org/apache/flink/connector/rocketmq/common/config/RocketMQConfigBuilder.java
new file mode 100644
index 0000000..7b610a7
--- /dev/null
+++ b/src/main/java/org/apache/flink/connector/rocketmq/common/config/RocketMQConfigBuilder.java
@@ -0,0 +1,150 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.flink.connector.rocketmq.common.config;
+
+import org.apache.flink.annotation.Internal;
+import org.apache.flink.configuration.ConfigOption;
+import org.apache.flink.configuration.ConfigOptions;
+import org.apache.flink.configuration.Configuration;
+import org.apache.flink.configuration.UnmodifiableConfiguration;
+
+import java.util.ArrayList;
+import java.util.List;
+import java.util.Map;
+import java.util.Objects;
+import java.util.Properties;
+import java.util.function.Function;
+
+import static org.apache.flink.util.Preconditions.checkArgument;
+import static org.apache.flink.util.Preconditions.checkNotNull;
+
+/**
+ * A builder for building the unmodifiable {@link Configuration} instance. Providing the common
+ * validate logic for RocketMQ source & sink.
+ */
+@Internal
+public class RocketMQConfigBuilder {
+
+    private final Configuration configuration = new Configuration();
+
+    /** Validate if the config has a existed option. */
+    public <T> boolean contains(ConfigOption<T> option) {
+        return configuration.contains(option);
+    }
+
+    /**
+     * Get an option-related config value. We would return the default config value defined in the
+     * option if no value existed instead.
+     *
+     * @param key Config option instance.
+     */
+    public <T> T get(ConfigOption<T> key) {
+        return configuration.get(key);
+    }
+
+    /**
+     * Add a config option with a not null value. The config key shouldn't be duplicated.
+     *
+     * @param option Config option instance, contains key & type definition.
+     * @param value The config value which shouldn't be null.
+     */
+    public <T> void set(ConfigOption<T> option, T value) {
+        checkNotNull(option);
+        checkNotNull(value);
+
+        if (configuration.contains(option)) {
+            T oldValue = configuration.get(option);
+            checkArgument(
+                    Objects.equals(oldValue, value),
+                    "This option %s has been set to value %s.",
+                    option.key(),
+                    oldValue);
+        } else {
+            configuration.set(option, value);
+        }
+    }
+
+    /**
+     * Fill in a set of configs which shouldn't be duplicated.
+     *
+     * @param config A set of configs.
+     */
+    public void set(Configuration config) {
+        Map<String, String> existedConfigs = configuration.toMap();
+        List<String> duplicatedKeys = new ArrayList<>();
+        for (Map.Entry<String, String> entry : config.toMap().entrySet()) {
+            String key = entry.getKey();
+            if (existedConfigs.containsKey(key)) {
+                String value2 = existedConfigs.get(key);
+                if (!Objects.equals(value2, entry.getValue())) {
+                    duplicatedKeys.add(key);
+                }
+            }
+        }
+        checkArgument(
+                duplicatedKeys.isEmpty(),
+                "Invalid configuration, these keys %s are already exist with different config value.",
+                duplicatedKeys);
+        configuration.addAll(config);
+    }
+
+    /**
+     * Fill in a set of config properties which shouldn't be duplicated.
+     *
+     * @param properties A config which could be string type.
+     */
+    public void set(Properties properties) {
+        properties.keySet().stream()
+                .map(String::valueOf)
+                .forEach(
+                        key -> {
+                            ConfigOption<String> option =
+                                    ConfigOptions.key(key).stringType().noDefaultValue();
+                            Object value = properties.get(key);
+
+                            if (value != null) {
+                                set(option, value.toString());
+                            }
+                        });
+    }
+
+    /**
+     * Override the option with the given value. It will not check the existed option comparing to
+     * {@link #set(ConfigOption, Object)}.
+     */
+    public <T> void override(ConfigOption<T> option, T value) {
+        checkNotNull(option);
+        checkNotNull(value);
+
+        configuration.set(option, value);
+    }
+
+    public Configuration build(RocketMQConfigValidator validator) {
+        checkNotNull(validator);
+        validator.validate(configuration);
+
+        return new UnmodifiableConfiguration(configuration);
+    }
+
+    /** Validate the current config instance and return a unmodifiable configuration. */
+    public <T extends RocketMQConfiguration> T build(
+            RocketMQConfigValidator validator, Function<Configuration, T> constructor) {
+        validator.validate(configuration);
+        return constructor.apply(configuration);
+    }
+}
diff --git a/src/main/java/org/apache/flink/connector/rocketmq/common/config/RocketMQConfigValidator.java b/src/main/java/org/apache/flink/connector/rocketmq/common/config/RocketMQConfigValidator.java
new file mode 100644
index 0000000..9661042
--- /dev/null
+++ b/src/main/java/org/apache/flink/connector/rocketmq/common/config/RocketMQConfigValidator.java
@@ -0,0 +1,106 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.flink.connector.rocketmq.common.config;
+
+import org.apache.flink.annotation.Internal;
+import org.apache.flink.configuration.ConfigOption;
+import org.apache.flink.configuration.Configuration;
+
+import org.apache.flink.shaded.guava30.com.google.common.collect.ImmutableList;
+import org.apache.flink.shaded.guava30.com.google.common.collect.ImmutableSet;
+
+import java.util.ArrayList;
+import java.util.HashSet;
+import java.util.List;
+import java.util.Set;
+
+import static org.apache.flink.util.Preconditions.checkArgument;
+
+/**
+ * A config validator for building {@link RocketMQConfiguration} in {@link RocketMQConfigBuilder}.
+ * It's used for source & sink builder.
+ *
+ * <p>We would validate:
+ *
+ * <ul>
+ *   <li>If the user has provided the required config options.
+ *   <li>If the user has provided some conflict options.
+ * </ul>
+ */
+@Internal
+public class RocketMQConfigValidator {
+
+    private final List<Set<ConfigOption<?>>> conflictOptions;
+    private final Set<ConfigOption<?>> requiredOptions;
+
+    private RocketMQConfigValidator(
+            List<Set<ConfigOption<?>>> conflictOptions, Set<ConfigOption<?>> requiredOptions) {
+        this.conflictOptions = conflictOptions;
+        this.requiredOptions = requiredOptions;
+    }
+
+    /** Package private validating for using in {@link RocketMQConfigBuilder}. */
+    void validate(Configuration configuration) {
+        requiredOptions.forEach(
+                option ->
+                        checkArgument(
+                                configuration.contains(option),
+                                "Config option %s is not provided for rocketmq client.",
+                                option));
+
+        conflictOptions.forEach(
+                options -> {
+                    long nums = options.stream().filter(configuration::contains).count();
+                    checkArgument(
+                            nums <= 1,
+                            "Conflict config options %s were provided, "
+                                    + "we only support one of them for creating rocketmq client.",
+                            options);
+                });
+    }
+
+    /** Return the builder for building {@link RocketMQConfigValidator}. */
+    public static RocketMQConfigValidatorBuilder builder() {
+        return new RocketMQConfigValidatorBuilder();
+    }
+
+    /** Builder pattern for building {@link RocketMQConfigValidator}. */
+    public static class RocketMQConfigValidatorBuilder {
+
+        private final List<Set<ConfigOption<?>>> conflictOptions = new ArrayList<>();
+        private final Set<ConfigOption<?>> requiredOptions = new HashSet<>();
+
+        public RocketMQConfigValidatorBuilder conflictOptions(ConfigOption<?>... options) {
+            checkArgument(options.length > 1, "You should provide at least two conflict options.");
+            conflictOptions.add(ImmutableSet.copyOf(options));
+            return this;
+        }
+
+        public RocketMQConfigValidatorBuilder requiredOption(ConfigOption<?> option) {
+            requiredOptions.add(option);
+            return this;
+        }
+
+        public RocketMQConfigValidator build() {
+            ImmutableList<Set<ConfigOption<?>>> conflict = ImmutableList.copyOf(conflictOptions);
+            Set<ConfigOption<?>> required = ImmutableSet.copyOf(requiredOptions);
+
+            return new RocketMQConfigValidator(conflict, required);
+        }
+    }
+}
diff --git a/src/main/java/org/apache/flink/connector/rocketmq/common/config/RocketMQConfiguration.java b/src/main/java/org/apache/flink/connector/rocketmq/common/config/RocketMQConfiguration.java
new file mode 100644
index 0000000..a698fd4
--- /dev/null
+++ b/src/main/java/org/apache/flink/connector/rocketmq/common/config/RocketMQConfiguration.java
@@ -0,0 +1,104 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.flink.connector.rocketmq.common.config;
+
+import org.apache.flink.configuration.ConfigOption;
+import org.apache.flink.configuration.ConfigOptions;
+import org.apache.flink.configuration.Configuration;
+import org.apache.flink.configuration.UnmodifiableConfiguration;
+
+import java.util.HashMap;
+import java.util.List;
+import java.util.Map;
+import java.util.function.Consumer;
+import java.util.function.Function;
+
+import static java.util.function.Function.identity;
+import static java.util.stream.Collectors.toList;
+
+/**
+ * An unmodifiable {@link Configuration} for RocketMQ. We provide extra methods for building the
+ * different RocketMQ client instance.
+ */
+public class RocketMQConfiguration extends UnmodifiableConfiguration {
+
+    private static final long serialVersionUID = 3050894147145572345L;
+
+    /**
+     * Creates a new RocketMQConfiguration, which holds a copy of the given configuration that can't
+     * be altered.
+     *
+     * @param config The configuration with the original contents.
+     */
+    protected RocketMQConfiguration(Configuration config) {
+        super(config);
+    }
+
+    /**
+     * Get the option value by a prefix. We would return an empty map if the option doesn't exist.
+     */
+    public Map<String, String> getProperties(ConfigOption<Map<String, String>> option) {
+        Map<String, String> properties = new HashMap<>();
+        if (contains(option)) {
+            Map<String, String> map = get(option);
+            properties.putAll(map);
+        }
+
+        // Filter the sub config option. These options could be provided by SQL.
+        String prefix = option.key() + ".";
+        List<String> keys =
+                keySet().stream()
+                        .filter(key -> key.startsWith(prefix) && key.length() > prefix.length())
+                        .collect(toList());
+
+        // Put these config options' value into return result.
+        for (String key : keys) {
+            ConfigOption<String> o = ConfigOptions.key(key).stringType().noDefaultValue();
+            String value = get(o);
+            properties.put(key.substring(prefix.length()), value);
+        }
+
+        return properties;
+    }
+
+    /** Get an option value from the given config, convert it into a new value instance. */
+    public <F, T> T get(ConfigOption<F> option, Function<F, T> convertor) {
+        F value = get(option);
+        if (value != null) {
+            return convertor.apply(value);
+        } else {
+            return null;
+        }
+    }
+
+    /** Set the config option's value to a given builder. */
+    public <T> void useOption(ConfigOption<T> option, Consumer<T> setter) {
+        useOption(option, identity(), setter);
+    }
+
+    /**
+     * Query the config option's value, convert it into a required type, set it to a given builder.
+     */
+    public <T, V> void useOption(
+            ConfigOption<T> option, Function<T, V> convertor, Consumer<V> setter) {
+        if (contains(option) || option.hasDefaultValue()) {
+            V value = get(option, convertor);
+            setter.accept(value);
+        }
+    }
+}
diff --git a/src/main/java/org/apache/flink/connector/rocketmq/common/config/RocketMQOptions.java b/src/main/java/org/apache/flink/connector/rocketmq/common/config/RocketMQOptions.java
new file mode 100644
index 0000000..444db59
--- /dev/null
+++ b/src/main/java/org/apache/flink/connector/rocketmq/common/config/RocketMQOptions.java
@@ -0,0 +1,169 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.flink.connector.rocketmq.common.config;
+
+import org.apache.flink.annotation.PublicEvolving;
+import org.apache.flink.annotation.docs.ConfigGroup;
+import org.apache.flink.annotation.docs.ConfigGroups;
+import org.apache.flink.configuration.ConfigOption;
+import org.apache.flink.configuration.ConfigOptions;
+
+import org.apache.rocketmq.client.AccessChannel;
+import org.apache.rocketmq.remoting.protocol.LanguageCode;
+
+import static org.apache.flink.connector.rocketmq.common.config.RocketMQOptions.CLIENT_CONFIG_PREFIX;
+
+/**
+ * Configuration for RocketMQ Client, these config options would be used for both source, sink and
+ * table.
+ */
+@PublicEvolving
+@ConfigGroups(
+        groups = {
+            @ConfigGroup(name = "RocketMQClient", keyPrefix = CLIENT_CONFIG_PREFIX),
+        })
+/** <a href="https://rocketmq.apache.org/zh/docs/4.x/parameterConfiguration/01local">...</a> */
+public class RocketMQOptions {
+
+    // --------------------------------------------------------------------------------------------
+    // RocketMQ specific options
+    // --------------------------------------------------------------------------------------------
+
+    public static final String CLIENT_CONFIG_PREFIX = "rocketmq.client.";
+
+    public static final ConfigOption<Boolean> GLOBAL_DEBUG_MODE =
+            ConfigOptions.key(CLIENT_CONFIG_PREFIX + "debug").booleanType().defaultValue(false);
+
+    /**
+     * rocketmq v4 endpoints means nameserver address rocketmq v5 endpoints means proxy server
+     * address
+     */
+    public static final ConfigOption<String> ENDPOINTS =
+            ConfigOptions.key(CLIENT_CONFIG_PREFIX + "endpoints")
+                    .stringType()
+                    .noDefaultValue()
+                    .withDescription("RocketMQ server address");
+
+    public static final ConfigOption<String> NAMESPACE =
+            ConfigOptions.key(CLIENT_CONFIG_PREFIX + "namespace")
+                    .stringType()
+                    .noDefaultValue()
+                    .withDescription("RocketMQ instance namespace");
+
+    /** 这里不知道对轨迹功能有没有影响, 待验证 */
+    public static final ConfigOption<AccessChannel> OPTIONAL_ACCESS_CHANNEL =
+            ConfigOptions.key(CLIENT_CONFIG_PREFIX + "channel")
+                    .enumType(AccessChannel.class)
+                    .defaultValue(AccessChannel.CLOUD)
+                    .withDescription("RocketMQ access channel");
+
+    public static final ConfigOption<Integer> CLIENT_CALLBACK_EXECUTOR_THREADS =
+            ConfigOptions.key(CLIENT_CONFIG_PREFIX + "callback.threads")
+                    .intType()
+                    .defaultValue(Runtime.getRuntime().availableProcessors())
+                    .withDescription(
+                            "The number of processor cores "
+                                    + "when the client communication layer receives a network request");
+
+    public static final ConfigOption<Long> PARTITION_DISCOVERY_INTERVAL_MS =
+            ConfigOptions.key(CLIENT_CONFIG_PREFIX + "partition.discovery.interval.ms")
+                    .longType()
+                    .defaultValue(10000L)
+                    .withDescription(
+                            "Time interval for polling route information from nameserver or proxy");
+
+    public static final ConfigOption<Long> HEARTBEAT_INTERVAL =
+            ConfigOptions.key(CLIENT_CONFIG_PREFIX + "heartbeat.interval.ms")
+                    .longType()
+                    .defaultValue(30000L)
+                    .withDescription(
+                            "Interval for regularly sending registration heartbeats to broker");
+
+    public static final ConfigOption<Boolean> OPTIONAL_UNIT_MODE =
+            ConfigOptions.key(CLIENT_CONFIG_PREFIX + "unitMode").booleanType().defaultValue(false);
+
+    public static final ConfigOption<String> OPTIONAL_UNIT_NAME =
+            ConfigOptions.key(CLIENT_CONFIG_PREFIX + "unitName").stringType().noDefaultValue();
+
+    public static final ConfigOption<Boolean> VIP_CHANNEL_ENABLED =
+            ConfigOptions.key(CLIENT_CONFIG_PREFIX + "channel.vip.enable")
+                    .booleanType()
+                    .defaultValue(false)
+                    .withDescription("Whether to enable vip netty channel for sending messages");
+
+    public static final ConfigOption<Boolean> USE_TLS =
+            ConfigOptions.key(CLIENT_CONFIG_PREFIX + "tls.enable")
+                    .booleanType()
+                    .defaultValue(false)
+                    .withDescription("Whether to use TLS transport.");
+
+    public static final ConfigOption<Long> MQ_CLIENT_API_TIMEOUT =
+            ConfigOptions.key(CLIENT_CONFIG_PREFIX + "network.timeout.ms")
+                    .longType()
+                    .defaultValue(30000L)
+                    .withDescription("RocketMQ client api timeout setting");
+
+    public static final ConfigOption<LanguageCode> LANGUAGE_CODE =
+            ConfigOptions.key(CLIENT_CONFIG_PREFIX + "language")
+                    .enumType(LanguageCode.class)
+                    .defaultValue(LanguageCode.JAVA)
+                    .withDescription("Client implementation language");
+
+    public static final ConfigOption<String> OPTIONAL_TIME_ZONE =
+            ConfigOptions.key(CLIENT_CONFIG_PREFIX + "timeZone").stringType().noDefaultValue();
+
+    // for message payload
+    public static final ConfigOption<String> OPTIONAL_ENCODING =
+            ConfigOptions.key(CLIENT_CONFIG_PREFIX + "message.encoding")
+                    .stringType()
+                    .defaultValue("UTF-8");
+
+    public static final ConfigOption<String> OPTIONAL_FIELD_DELIMITER =
+            ConfigOptions.key(CLIENT_CONFIG_PREFIX + "message.field.delimiter")
+                    .stringType()
+                    .defaultValue("\u0001");
+
+    public static final ConfigOption<String> OPTIONAL_LINE_DELIMITER =
+            ConfigOptions.key(CLIENT_CONFIG_PREFIX + "message.line.delimiter")
+                    .stringType()
+                    .defaultValue("\n");
+
+    public static final ConfigOption<String> OPTIONAL_LENGTH_CHECK =
+            ConfigOptions.key(CLIENT_CONFIG_PREFIX + "message.length.check")
+                    .stringType()
+                    .defaultValue("NONE");
+
+    // the config of session credential
+    public static final ConfigOption<String> OPTIONAL_ACCESS_KEY =
+            ConfigOptions.key(CLIENT_CONFIG_PREFIX + "accessKey").stringType().noDefaultValue();
+
+    public static final ConfigOption<String> OPTIONAL_SECRET_KEY =
+            ConfigOptions.key(CLIENT_CONFIG_PREFIX + "secretKey").stringType().noDefaultValue();
+
+    public static final ConfigOption<Boolean> COMMIT_OFFSETS_ON_CHECKPOINT =
+            ConfigOptions.key("commit.offsets.on.checkpoint")
+                    .booleanType()
+                    .defaultValue(true)
+                    .withDescription("Whether to commit consuming offset on checkpoint.");
+
+    public static final ConfigOption<Long> POLL_TIMEOUT =
+            ConfigOptions.key("poll.timeout")
+                    .longType()
+                    .defaultValue(10L)
+                    .withDescription("how long to wait before giving up, the unit is milliseconds");
+}
diff --git a/src/main/java/org/apache/rocketmq/flink/common/constant/RocketMqCatalogConstant.java b/src/main/java/org/apache/flink/connector/rocketmq/common/constant/RocketMqCatalogConstant.java
similarity index 94%
rename from src/main/java/org/apache/rocketmq/flink/common/constant/RocketMqCatalogConstant.java
rename to src/main/java/org/apache/flink/connector/rocketmq/common/constant/RocketMqCatalogConstant.java
index be3d9da..bed9418 100644
--- a/src/main/java/org/apache/rocketmq/flink/common/constant/RocketMqCatalogConstant.java
+++ b/src/main/java/org/apache/flink/connector/rocketmq/common/constant/RocketMqCatalogConstant.java
@@ -16,7 +16,7 @@
  * limitations under the License.
  */
 
-package org.apache.rocketmq.flink.common.constant;
+package org.apache.flink.connector.rocketmq.common.constant;
 
 /** RocketMqCatalogConstant. */
 public class RocketMqCatalogConstant {
diff --git a/src/main/java/org/apache/rocketmq/flink/common/constant/SchemaRegistryConstant.java b/src/main/java/org/apache/flink/connector/rocketmq/common/constant/SchemaRegistryConstant.java
similarity index 94%
rename from src/main/java/org/apache/rocketmq/flink/common/constant/SchemaRegistryConstant.java
rename to src/main/java/org/apache/flink/connector/rocketmq/common/constant/SchemaRegistryConstant.java
index 8584cb3..416bb0f 100644
--- a/src/main/java/org/apache/rocketmq/flink/common/constant/SchemaRegistryConstant.java
+++ b/src/main/java/org/apache/flink/connector/rocketmq/common/constant/SchemaRegistryConstant.java
@@ -16,7 +16,7 @@
  * limitations under the License.
  */
 
-package org.apache.rocketmq.flink.common.constant;
+package org.apache.flink.connector.rocketmq.common.constant;
 
 /** SchemaRegistryConstant. */
 public class SchemaRegistryConstant {
diff --git a/src/main/java/org/apache/rocketmq/flink/legacy/RocketMQConfig.java b/src/main/java/org/apache/flink/connector/rocketmq/legacy/RocketMQConfig.java
similarity index 95%
rename from src/main/java/org/apache/rocketmq/flink/legacy/RocketMQConfig.java
rename to src/main/java/org/apache/flink/connector/rocketmq/legacy/RocketMQConfig.java
index ecf7a9e..d94188a 100644
--- a/src/main/java/org/apache/rocketmq/flink/legacy/RocketMQConfig.java
+++ b/src/main/java/org/apache/flink/connector/rocketmq/legacy/RocketMQConfig.java
@@ -14,24 +14,23 @@
  * See the License for the specific language governing permissions and
  * limitations under the License.
  */
-package org.apache.rocketmq.flink.legacy;
+package org.apache.flink.connector.rocketmq.legacy;
 
+import org.apache.commons.lang.StringUtils;
+import org.apache.commons.lang.Validate;
 import org.apache.rocketmq.acl.common.AclClientRPCHook;
 import org.apache.rocketmq.acl.common.SessionCredentials;
 import org.apache.rocketmq.client.AccessChannel;
 import org.apache.rocketmq.client.ClientConfig;
 import org.apache.rocketmq.client.consumer.DefaultLitePullConsumer;
 import org.apache.rocketmq.client.producer.DefaultMQProducer;
-import org.apache.rocketmq.common.protocol.heartbeat.MessageModel;
-
-import org.apache.commons.lang.StringUtils;
-import org.apache.commons.lang.Validate;
+import org.apache.rocketmq.remoting.protocol.heartbeat.MessageModel;
 
 import java.util.Properties;
 import java.util.UUID;
 
-import static org.apache.rocketmq.flink.legacy.common.util.RocketMQUtils.getAccessChannel;
-import static org.apache.rocketmq.flink.legacy.common.util.RocketMQUtils.getInteger;
+import static org.apache.flink.connector.rocketmq.legacy.common.util.RocketMQUtils.getAccessChannel;
+import static org.apache.flink.connector.rocketmq.legacy.common.util.RocketMQUtils.getInteger;
 
 /** RocketMQConfig for Consumer/Producer. */
 public class RocketMQConfig {
diff --git a/src/main/java/org/apache/rocketmq/flink/legacy/RocketMQSink.java b/src/main/java/org/apache/flink/connector/rocketmq/legacy/RocketMQSink.java
similarity index 97%
rename from src/main/java/org/apache/rocketmq/flink/legacy/RocketMQSink.java
rename to src/main/java/org/apache/flink/connector/rocketmq/legacy/RocketMQSink.java
index c5ce927..9f87486 100644
--- a/src/main/java/org/apache/rocketmq/flink/legacy/RocketMQSink.java
+++ b/src/main/java/org/apache/flink/connector/rocketmq/legacy/RocketMQSink.java
@@ -14,19 +14,11 @@
  * See the License for the specific language governing permissions and
  * limitations under the License.
  */
-package org.apache.rocketmq.flink.legacy;
-
-import org.apache.rocketmq.client.exception.MQClientException;
-import org.apache.rocketmq.client.producer.DefaultMQProducer;
-import org.apache.rocketmq.client.producer.SendCallback;
-import org.apache.rocketmq.client.producer.SendResult;
-import org.apache.rocketmq.client.producer.SendStatus;
-import org.apache.rocketmq.common.message.Message;
-import org.apache.rocketmq.flink.legacy.common.selector.MessageQueueSelector;
-import org.apache.rocketmq.flink.legacy.common.util.MetricUtils;
-import org.apache.rocketmq.remoting.exception.RemotingException;
+package org.apache.flink.connector.rocketmq.legacy;
 
 import org.apache.flink.configuration.Configuration;
+import org.apache.flink.connector.rocketmq.legacy.common.selector.MessageQueueSelector;
+import org.apache.flink.connector.rocketmq.legacy.common.util.MetricUtils;
 import org.apache.flink.metrics.Meter;
 import org.apache.flink.runtime.state.FunctionInitializationContext;
 import org.apache.flink.runtime.state.FunctionSnapshotContext;
@@ -36,6 +28,13 @@
 import org.apache.flink.util.StringUtils;
 
 import org.apache.commons.lang.Validate;
+import org.apache.rocketmq.client.exception.MQClientException;
+import org.apache.rocketmq.client.producer.DefaultMQProducer;
+import org.apache.rocketmq.client.producer.SendCallback;
+import org.apache.rocketmq.client.producer.SendResult;
+import org.apache.rocketmq.client.producer.SendStatus;
+import org.apache.rocketmq.common.message.Message;
+import org.apache.rocketmq.remoting.exception.RemotingException;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 
diff --git a/src/main/java/org/apache/rocketmq/flink/legacy/RocketMQSourceFunction.java b/src/main/java/org/apache/flink/connector/rocketmq/legacy/RocketMQSourceFunction.java
similarity index 95%
rename from src/main/java/org/apache/rocketmq/flink/legacy/RocketMQSourceFunction.java
rename to src/main/java/org/apache/flink/connector/rocketmq/legacy/RocketMQSourceFunction.java
index fa57839..e445643 100644
--- a/src/main/java/org/apache/rocketmq/flink/legacy/RocketMQSourceFunction.java
+++ b/src/main/java/org/apache/flink/connector/rocketmq/legacy/RocketMQSourceFunction.java
@@ -15,21 +15,7 @@
  * limitations under the License.
  */
 
-package org.apache.rocketmq.flink.legacy;
-
-import org.apache.rocketmq.client.consumer.DefaultLitePullConsumer;
-import org.apache.rocketmq.client.consumer.MessageSelector;
-import org.apache.rocketmq.client.exception.MQClientException;
-import org.apache.rocketmq.common.message.MessageExt;
-import org.apache.rocketmq.common.message.MessageQueue;
-import org.apache.rocketmq.flink.legacy.common.config.OffsetResetStrategy;
-import org.apache.rocketmq.flink.legacy.common.config.StartupMode;
-import org.apache.rocketmq.flink.legacy.common.serialization.KeyValueDeserializationSchema;
-import org.apache.rocketmq.flink.legacy.common.util.MetricUtils;
-import org.apache.rocketmq.flink.legacy.common.util.RetryUtil;
-import org.apache.rocketmq.flink.legacy.common.util.RocketMQUtils;
-import org.apache.rocketmq.flink.legacy.common.watermark.WaterMarkForAll;
-import org.apache.rocketmq.flink.legacy.common.watermark.WaterMarkPerQueue;
+package org.apache.flink.connector.rocketmq.legacy;
 
 import org.apache.flink.api.common.functions.RuntimeContext;
 import org.apache.flink.api.common.state.ListState;
@@ -39,6 +25,14 @@
 import org.apache.flink.api.java.tuple.Tuple2;
 import org.apache.flink.api.java.typeutils.ResultTypeQueryable;
 import org.apache.flink.configuration.Configuration;
+import org.apache.flink.connector.rocketmq.legacy.common.config.OffsetResetStrategy;
+import org.apache.flink.connector.rocketmq.legacy.common.config.StartupMode;
+import org.apache.flink.connector.rocketmq.legacy.common.serialization.KeyValueDeserializationSchema;
+import org.apache.flink.connector.rocketmq.legacy.common.util.MetricUtils;
+import org.apache.flink.connector.rocketmq.legacy.common.util.RetryUtil;
+import org.apache.flink.connector.rocketmq.legacy.common.util.RocketMQUtils;
+import org.apache.flink.connector.rocketmq.legacy.common.watermark.WaterMarkForAll;
+import org.apache.flink.connector.rocketmq.legacy.common.watermark.WaterMarkPerQueue;
 import org.apache.flink.metrics.Counter;
 import org.apache.flink.metrics.Meter;
 import org.apache.flink.metrics.MeterView;
@@ -58,6 +52,11 @@
 import org.apache.commons.collections.map.LinkedMap;
 import org.apache.commons.lang.Validate;
 import org.apache.commons.lang3.StringUtils;
+import org.apache.rocketmq.client.consumer.DefaultLitePullConsumer;
+import org.apache.rocketmq.client.consumer.MessageSelector;
+import org.apache.rocketmq.client.exception.MQClientException;
+import org.apache.rocketmq.common.message.MessageExt;
+import org.apache.rocketmq.common.message.MessageQueue;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 
@@ -77,12 +76,6 @@
 import java.util.concurrent.TimeUnit;
 import java.util.concurrent.locks.ReentrantLock;
 
-import static org.apache.rocketmq.flink.legacy.RocketMQConfig.CONSUMER_BATCH_SIZE;
-import static org.apache.rocketmq.flink.legacy.RocketMQConfig.CONSUMER_TIMEOUT;
-import static org.apache.rocketmq.flink.legacy.RocketMQConfig.DEFAULT_CONSUMER_BATCH_SIZE;
-import static org.apache.rocketmq.flink.legacy.RocketMQConfig.DEFAULT_CONSUMER_TIMEOUT;
-import static org.apache.rocketmq.flink.legacy.common.util.RocketMQUtils.getInteger;
-
 /**
  * The RocketMQSource is based on RocketMQ pull consumer mode, and provides exactly once reliability
  * guarantees when checkpoints are enabled. Otherwise, the source doesn't provide any reliability
@@ -258,7 +251,11 @@
         String sql = props.getProperty(RocketMQConfig.CONSUMER_SQL);
         String tag =
                 props.getProperty(RocketMQConfig.CONSUMER_TAG, RocketMQConfig.DEFAULT_CONSUMER_TAG);
-        int pullBatchSize = getInteger(props, CONSUMER_BATCH_SIZE, DEFAULT_CONSUMER_BATCH_SIZE);
+        int pullBatchSize =
+                RocketMQUtils.getInteger(
+                        props,
+                        RocketMQConfig.CONSUMER_BATCH_SIZE,
+                        RocketMQConfig.DEFAULT_CONSUMER_BATCH_SIZE);
         timer.scheduleAtFixedRate(
                 () -> {
                     // context.emitWatermark(waterMarkPerQueue.getCurrentWatermark());
@@ -286,10 +283,12 @@
                                                 boolean found = false;
                                                 List<MessageExt> messages =
                                                         consumer.poll(
-                                                                getInteger(
+                                                                RocketMQUtils.getInteger(
                                                                         props,
-                                                                        CONSUMER_TIMEOUT,
-                                                                        DEFAULT_CONSUMER_TIMEOUT));
+                                                                        RocketMQConfig
+                                                                                .CONSUMER_TIMEOUT,
+                                                                        RocketMQConfig
+                                                                                .DEFAULT_CONSUMER_TIMEOUT));
                                                 if (CollectionUtils.isNotEmpty(messages)) {
                                                     long fetchTime = System.currentTimeMillis();
                                                     for (MessageExt msg : messages) {
diff --git a/src/main/java/org/apache/rocketmq/flink/legacy/RunningChecker.java b/src/main/java/org/apache/flink/connector/rocketmq/legacy/RunningChecker.java
similarity index 95%
rename from src/main/java/org/apache/rocketmq/flink/legacy/RunningChecker.java
rename to src/main/java/org/apache/flink/connector/rocketmq/legacy/RunningChecker.java
index f36b727..961a302 100644
--- a/src/main/java/org/apache/rocketmq/flink/legacy/RunningChecker.java
+++ b/src/main/java/org/apache/flink/connector/rocketmq/legacy/RunningChecker.java
@@ -14,7 +14,7 @@
  * See the License for the specific language governing permissions and
  * limitations under the License.
  */
-package org.apache.rocketmq.flink.legacy;
+package org.apache.flink.connector.rocketmq.legacy;
 
 import java.io.Serializable;
 
diff --git a/src/main/java/org/apache/rocketmq/flink/legacy/common/config/OffsetResetStrategy.java b/src/main/java/org/apache/flink/connector/rocketmq/legacy/common/config/OffsetResetStrategy.java
similarity index 93%
rename from src/main/java/org/apache/rocketmq/flink/legacy/common/config/OffsetResetStrategy.java
rename to src/main/java/org/apache/flink/connector/rocketmq/legacy/common/config/OffsetResetStrategy.java
index a48e6d4..7a9f13a 100644
--- a/src/main/java/org/apache/rocketmq/flink/legacy/common/config/OffsetResetStrategy.java
+++ b/src/main/java/org/apache/flink/connector/rocketmq/legacy/common/config/OffsetResetStrategy.java
@@ -15,12 +15,13 @@
  * limitations under the License.
  */
 
-package org.apache.rocketmq.flink.legacy.common.config;
+package org.apache.flink.connector.rocketmq.legacy.common.config;
 
 /** Config for #{@link StartupMode#GROUP_OFFSETS}. */
 public enum OffsetResetStrategy {
     /** If group offsets is not found,the latest offset would be set to start consumer */
     LATEST,
+
     /** If group offsets is not found,the earliest offset would be set to start consumer */
     EARLIEST
 }
diff --git a/src/main/java/org/apache/rocketmq/flink/legacy/common/config/StartupMode.java b/src/main/java/org/apache/flink/connector/rocketmq/legacy/common/config/StartupMode.java
similarity index 93%
rename from src/main/java/org/apache/rocketmq/flink/legacy/common/config/StartupMode.java
rename to src/main/java/org/apache/flink/connector/rocketmq/legacy/common/config/StartupMode.java
index 163dae4..d56e605 100644
--- a/src/main/java/org/apache/rocketmq/flink/legacy/common/config/StartupMode.java
+++ b/src/main/java/org/apache/flink/connector/rocketmq/legacy/common/config/StartupMode.java
@@ -15,7 +15,7 @@
  * limitations under the License.
  */
 
-package org.apache.rocketmq.flink.legacy.common.config;
+package org.apache.flink.connector.rocketmq.legacy.common.config;
 
 /** RocketMQ startup mode. */
 public enum StartupMode {
diff --git a/src/main/java/org/apache/rocketmq/flink/legacy/common/selector/DefaultTopicSelector.java b/src/main/java/org/apache/flink/connector/rocketmq/legacy/common/selector/DefaultTopicSelector.java
similarity index 94%
rename from src/main/java/org/apache/rocketmq/flink/legacy/common/selector/DefaultTopicSelector.java
rename to src/main/java/org/apache/flink/connector/rocketmq/legacy/common/selector/DefaultTopicSelector.java
index 128b19e..d98ff71 100644
--- a/src/main/java/org/apache/rocketmq/flink/legacy/common/selector/DefaultTopicSelector.java
+++ b/src/main/java/org/apache/flink/connector/rocketmq/legacy/common/selector/DefaultTopicSelector.java
@@ -14,7 +14,7 @@
  * See the License for the specific language governing permissions and
  * limitations under the License.
  */
-package org.apache.rocketmq.flink.legacy.common.selector;
+package org.apache.flink.connector.rocketmq.legacy.common.selector;
 
 public class DefaultTopicSelector<T> implements TopicSelector<T> {
     private final String topicName;
diff --git a/src/main/java/org/apache/rocketmq/flink/legacy/common/selector/HashMessageQueueSelector.java b/src/main/java/org/apache/flink/connector/rocketmq/legacy/common/selector/HashMessageQueueSelector.java
similarity index 94%
rename from src/main/java/org/apache/rocketmq/flink/legacy/common/selector/HashMessageQueueSelector.java
rename to src/main/java/org/apache/flink/connector/rocketmq/legacy/common/selector/HashMessageQueueSelector.java
index ba81d8f..a3b5db3 100644
--- a/src/main/java/org/apache/rocketmq/flink/legacy/common/selector/HashMessageQueueSelector.java
+++ b/src/main/java/org/apache/flink/connector/rocketmq/legacy/common/selector/HashMessageQueueSelector.java
@@ -16,7 +16,7 @@
  * limitations under the License.
  */
 
-package org.apache.rocketmq.flink.legacy.common.selector;
+package org.apache.flink.connector.rocketmq.legacy.common.selector;
 
 import org.apache.rocketmq.common.message.Message;
 import org.apache.rocketmq.common.message.MessageQueue;
diff --git a/src/main/java/org/apache/rocketmq/flink/legacy/common/selector/MessageQueueSelector.java b/src/main/java/org/apache/flink/connector/rocketmq/legacy/common/selector/MessageQueueSelector.java
similarity index 93%
rename from src/main/java/org/apache/rocketmq/flink/legacy/common/selector/MessageQueueSelector.java
rename to src/main/java/org/apache/flink/connector/rocketmq/legacy/common/selector/MessageQueueSelector.java
index 23c46aa..0eac1bf 100644
--- a/src/main/java/org/apache/rocketmq/flink/legacy/common/selector/MessageQueueSelector.java
+++ b/src/main/java/org/apache/flink/connector/rocketmq/legacy/common/selector/MessageQueueSelector.java
@@ -16,7 +16,7 @@
  * limitations under the License.
  */
 
-package org.apache.rocketmq.flink.legacy.common.selector;
+package org.apache.flink.connector.rocketmq.legacy.common.selector;
 
 import java.io.Serializable;
 
diff --git a/src/main/java/org/apache/rocketmq/flink/legacy/common/selector/RandomMessageQueueSelector.java b/src/main/java/org/apache/flink/connector/rocketmq/legacy/common/selector/RandomMessageQueueSelector.java
similarity index 94%
rename from src/main/java/org/apache/rocketmq/flink/legacy/common/selector/RandomMessageQueueSelector.java
rename to src/main/java/org/apache/flink/connector/rocketmq/legacy/common/selector/RandomMessageQueueSelector.java
index 18fa92c..7580684 100644
--- a/src/main/java/org/apache/rocketmq/flink/legacy/common/selector/RandomMessageQueueSelector.java
+++ b/src/main/java/org/apache/flink/connector/rocketmq/legacy/common/selector/RandomMessageQueueSelector.java
@@ -16,7 +16,7 @@
  * limitations under the License.
  */
 
-package org.apache.rocketmq.flink.legacy.common.selector;
+package org.apache.flink.connector.rocketmq.legacy.common.selector;
 
 import org.apache.rocketmq.common.message.Message;
 import org.apache.rocketmq.common.message.MessageQueue;
diff --git a/src/main/java/org/apache/rocketmq/flink/legacy/common/selector/SimpleTopicSelector.java b/src/main/java/org/apache/flink/connector/rocketmq/legacy/common/selector/SimpleTopicSelector.java
similarity index 97%
rename from src/main/java/org/apache/rocketmq/flink/legacy/common/selector/SimpleTopicSelector.java
rename to src/main/java/org/apache/flink/connector/rocketmq/legacy/common/selector/SimpleTopicSelector.java
index dcdaa2f..d74e52c 100644
--- a/src/main/java/org/apache/rocketmq/flink/legacy/common/selector/SimpleTopicSelector.java
+++ b/src/main/java/org/apache/flink/connector/rocketmq/legacy/common/selector/SimpleTopicSelector.java
@@ -14,7 +14,7 @@
  * See the License for the specific language governing permissions and
  * limitations under the License.
  */
-package org.apache.rocketmq.flink.legacy.common.selector;
+package org.apache.flink.connector.rocketmq.legacy.common.selector;
 
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
diff --git a/src/main/java/org/apache/rocketmq/flink/legacy/common/selector/TopicSelector.java b/src/main/java/org/apache/flink/connector/rocketmq/legacy/common/selector/TopicSelector.java
similarity index 93%
rename from src/main/java/org/apache/rocketmq/flink/legacy/common/selector/TopicSelector.java
rename to src/main/java/org/apache/flink/connector/rocketmq/legacy/common/selector/TopicSelector.java
index a70c599..b24cfb4 100644
--- a/src/main/java/org/apache/rocketmq/flink/legacy/common/selector/TopicSelector.java
+++ b/src/main/java/org/apache/flink/connector/rocketmq/legacy/common/selector/TopicSelector.java
@@ -14,7 +14,7 @@
  * See the License for the specific language governing permissions and
  * limitations under the License.
  */
-package org.apache.rocketmq.flink.legacy.common.selector;
+package org.apache.flink.connector.rocketmq.legacy.common.selector;
 
 import java.io.Serializable;
 
diff --git a/src/main/java/org/apache/rocketmq/flink/legacy/common/serialization/ForwardMessageExtDeserialization.java b/src/main/java/org/apache/flink/connector/rocketmq/legacy/common/serialization/ForwardMessageExtDeserialization.java
similarity index 94%
rename from src/main/java/org/apache/rocketmq/flink/legacy/common/serialization/ForwardMessageExtDeserialization.java
rename to src/main/java/org/apache/flink/connector/rocketmq/legacy/common/serialization/ForwardMessageExtDeserialization.java
index 6c6fa74..598d72a 100644
--- a/src/main/java/org/apache/rocketmq/flink/legacy/common/serialization/ForwardMessageExtDeserialization.java
+++ b/src/main/java/org/apache/flink/connector/rocketmq/legacy/common/serialization/ForwardMessageExtDeserialization.java
@@ -15,12 +15,12 @@
  * limitations under the License.
  */
 
-package org.apache.rocketmq.flink.legacy.common.serialization;
-
-import org.apache.rocketmq.common.message.MessageExt;
+package org.apache.flink.connector.rocketmq.legacy.common.serialization;
 
 import org.apache.flink.api.common.typeinfo.TypeInformation;
 
+import org.apache.rocketmq.common.message.MessageExt;
+
 /** A Forward messageExt deserialization. */
 public class ForwardMessageExtDeserialization
         implements MessageExtDeserializationScheme<MessageExt> {
diff --git a/src/main/java/org/apache/rocketmq/flink/legacy/common/serialization/KeyValueDeserializationSchema.java b/src/main/java/org/apache/flink/connector/rocketmq/legacy/common/serialization/KeyValueDeserializationSchema.java
similarity index 93%
rename from src/main/java/org/apache/rocketmq/flink/legacy/common/serialization/KeyValueDeserializationSchema.java
rename to src/main/java/org/apache/flink/connector/rocketmq/legacy/common/serialization/KeyValueDeserializationSchema.java
index 8d0c778..21aa485 100644
--- a/src/main/java/org/apache/rocketmq/flink/legacy/common/serialization/KeyValueDeserializationSchema.java
+++ b/src/main/java/org/apache/flink/connector/rocketmq/legacy/common/serialization/KeyValueDeserializationSchema.java
@@ -14,7 +14,7 @@
  * See the License for the specific language governing permissions and
  * limitations under the License.
  */
-package org.apache.rocketmq.flink.legacy.common.serialization;
+package org.apache.flink.connector.rocketmq.legacy.common.serialization;
 
 import org.apache.flink.api.java.typeutils.ResultTypeQueryable;
 
diff --git a/src/main/java/org/apache/rocketmq/flink/legacy/common/serialization/KeyValueSerializationSchema.java b/src/main/java/org/apache/flink/connector/rocketmq/legacy/common/serialization/KeyValueSerializationSchema.java
similarity index 92%
rename from src/main/java/org/apache/rocketmq/flink/legacy/common/serialization/KeyValueSerializationSchema.java
rename to src/main/java/org/apache/flink/connector/rocketmq/legacy/common/serialization/KeyValueSerializationSchema.java
index 0000772..a19a445 100644
--- a/src/main/java/org/apache/rocketmq/flink/legacy/common/serialization/KeyValueSerializationSchema.java
+++ b/src/main/java/org/apache/flink/connector/rocketmq/legacy/common/serialization/KeyValueSerializationSchema.java
@@ -14,7 +14,7 @@
  * See the License for the specific language governing permissions and
  * limitations under the License.
  */
-package org.apache.rocketmq.flink.legacy.common.serialization;
+package org.apache.flink.connector.rocketmq.legacy.common.serialization;
 
 import java.io.Serializable;
 
diff --git a/src/main/java/org/apache/rocketmq/flink/legacy/common/serialization/MessageExtDeserializationScheme.java b/src/main/java/org/apache/flink/connector/rocketmq/legacy/common/serialization/MessageExtDeserializationScheme.java
similarity index 94%
rename from src/main/java/org/apache/rocketmq/flink/legacy/common/serialization/MessageExtDeserializationScheme.java
rename to src/main/java/org/apache/flink/connector/rocketmq/legacy/common/serialization/MessageExtDeserializationScheme.java
index 173823e..5ee8da7 100644
--- a/src/main/java/org/apache/rocketmq/flink/legacy/common/serialization/MessageExtDeserializationScheme.java
+++ b/src/main/java/org/apache/flink/connector/rocketmq/legacy/common/serialization/MessageExtDeserializationScheme.java
@@ -15,12 +15,12 @@
  * limitations under the License.
  */
 
-package org.apache.rocketmq.flink.legacy.common.serialization;
-
-import org.apache.rocketmq.common.message.MessageExt;
+package org.apache.flink.connector.rocketmq.legacy.common.serialization;
 
 import org.apache.flink.api.java.typeutils.ResultTypeQueryable;
 
+import org.apache.rocketmq.common.message.MessageExt;
+
 import java.io.Serializable;
 
 /**
diff --git a/src/main/java/org/apache/rocketmq/flink/legacy/common/serialization/RowKeyValueDeserializationSchema.java b/src/main/java/org/apache/flink/connector/rocketmq/legacy/common/serialization/RowKeyValueDeserializationSchema.java
similarity index 97%
rename from src/main/java/org/apache/rocketmq/flink/legacy/common/serialization/RowKeyValueDeserializationSchema.java
rename to src/main/java/org/apache/flink/connector/rocketmq/legacy/common/serialization/RowKeyValueDeserializationSchema.java
index 0ca130d..ca934bd 100644
--- a/src/main/java/org/apache/rocketmq/flink/legacy/common/serialization/RowKeyValueDeserializationSchema.java
+++ b/src/main/java/org/apache/flink/connector/rocketmq/legacy/common/serialization/RowKeyValueDeserializationSchema.java
@@ -14,16 +14,15 @@
  * See the License for the specific language governing permissions and
  * limitations under the License.
  */
-package org.apache.rocketmq.flink.legacy.common.serialization;
-
-import org.apache.rocketmq.flink.source.reader.deserializer.DirtyDataStrategy;
-import org.apache.rocketmq.flink.source.util.ByteSerializer;
-import org.apache.rocketmq.flink.source.util.StringSerializer;
+package org.apache.flink.connector.rocketmq.legacy.common.serialization;
 
 import org.apache.flink.api.common.typeinfo.TypeInformation;
 import org.apache.flink.configuration.ConfigOption;
 import org.apache.flink.configuration.ConfigOptions;
 import org.apache.flink.configuration.Configuration;
+import org.apache.flink.connector.rocketmq.source.reader.deserializer.DirtyDataStrategy;
+import org.apache.flink.connector.rocketmq.source.util.ByteSerializer;
+import org.apache.flink.connector.rocketmq.source.util.StringSerializer;
 import org.apache.flink.table.api.TableSchema;
 import org.apache.flink.table.data.GenericRowData;
 import org.apache.flink.table.data.RowData;
diff --git a/src/main/java/org/apache/rocketmq/flink/legacy/common/serialization/SimpleKeyValueDeserializationSchema.java b/src/main/java/org/apache/flink/connector/rocketmq/legacy/common/serialization/SimpleKeyValueDeserializationSchema.java
similarity index 92%
rename from src/main/java/org/apache/rocketmq/flink/legacy/common/serialization/SimpleKeyValueDeserializationSchema.java
rename to src/main/java/org/apache/flink/connector/rocketmq/legacy/common/serialization/SimpleKeyValueDeserializationSchema.java
index 1177f76..c3e54c9 100644
--- a/src/main/java/org/apache/rocketmq/flink/legacy/common/serialization/SimpleKeyValueDeserializationSchema.java
+++ b/src/main/java/org/apache/flink/connector/rocketmq/legacy/common/serialization/SimpleKeyValueDeserializationSchema.java
@@ -14,16 +14,17 @@
  * See the License for the specific language governing permissions and
  * limitations under the License.
  */
-package org.apache.rocketmq.flink.legacy.common.serialization;
+package org.apache.flink.connector.rocketmq.legacy.common.serialization;
 
 import org.apache.flink.api.common.typeinfo.TypeInformation;
+import org.apache.flink.api.java.typeutils.MapTypeInfo;
 
 import java.nio.charset.StandardCharsets;
 import java.util.HashMap;
 import java.util.Map;
-import org.apache.flink.api.java.typeutils.MapTypeInfo;
 
-public class SimpleKeyValueDeserializationSchema implements KeyValueDeserializationSchema<Map<String, String>> {
+public class SimpleKeyValueDeserializationSchema
+        implements KeyValueDeserializationSchema<Map<String, String>> {
     public static final String DEFAULT_KEY_FIELD = "key";
     public static final String DEFAULT_VALUE_FIELD = "value";
 
diff --git a/src/main/java/org/apache/rocketmq/flink/legacy/common/serialization/SimpleKeyValueSerializationSchema.java b/src/main/java/org/apache/flink/connector/rocketmq/legacy/common/serialization/SimpleKeyValueSerializationSchema.java
similarity index 96%
rename from src/main/java/org/apache/rocketmq/flink/legacy/common/serialization/SimpleKeyValueSerializationSchema.java
rename to src/main/java/org/apache/flink/connector/rocketmq/legacy/common/serialization/SimpleKeyValueSerializationSchema.java
index de01cbc..830cfb6 100644
--- a/src/main/java/org/apache/rocketmq/flink/legacy/common/serialization/SimpleKeyValueSerializationSchema.java
+++ b/src/main/java/org/apache/flink/connector/rocketmq/legacy/common/serialization/SimpleKeyValueSerializationSchema.java
@@ -14,7 +14,7 @@
  * See the License for the specific language governing permissions and
  * limitations under the License.
  */
-package org.apache.rocketmq.flink.legacy.common.serialization;
+package org.apache.flink.connector.rocketmq.legacy.common.serialization;
 
 import java.nio.charset.StandardCharsets;
 import java.util.Map;
diff --git a/src/main/java/org/apache/rocketmq/flink/legacy/common/serialization/SimpleStringDeserializationSchema.java b/src/main/java/org/apache/flink/connector/rocketmq/legacy/common/serialization/SimpleStringDeserializationSchema.java
similarity index 94%
rename from src/main/java/org/apache/rocketmq/flink/legacy/common/serialization/SimpleStringDeserializationSchema.java
rename to src/main/java/org/apache/flink/connector/rocketmq/legacy/common/serialization/SimpleStringDeserializationSchema.java
index f59b961..7f8c4c6 100644
--- a/src/main/java/org/apache/rocketmq/flink/legacy/common/serialization/SimpleStringDeserializationSchema.java
+++ b/src/main/java/org/apache/flink/connector/rocketmq/legacy/common/serialization/SimpleStringDeserializationSchema.java
@@ -15,7 +15,7 @@
  * See the License for the specific language governing permissions and
  * limitations under the License.
  */
-package org.apache.rocketmq.flink.legacy.common.serialization;
+package org.apache.flink.connector.rocketmq.legacy.common.serialization;
 
 import org.apache.flink.api.common.typeinfo.TypeInformation;
 
diff --git a/src/main/java/org/apache/rocketmq/flink/legacy/common/serialization/SimpleTupleDeserializationSchema.java b/src/main/java/org/apache/flink/connector/rocketmq/legacy/common/serialization/SimpleTupleDeserializationSchema.java
similarity index 95%
rename from src/main/java/org/apache/rocketmq/flink/legacy/common/serialization/SimpleTupleDeserializationSchema.java
rename to src/main/java/org/apache/flink/connector/rocketmq/legacy/common/serialization/SimpleTupleDeserializationSchema.java
index 440cb3e..28e94ba 100644
--- a/src/main/java/org/apache/rocketmq/flink/legacy/common/serialization/SimpleTupleDeserializationSchema.java
+++ b/src/main/java/org/apache/flink/connector/rocketmq/legacy/common/serialization/SimpleTupleDeserializationSchema.java
@@ -15,7 +15,7 @@
  * limitations under the License.
  */
 
-package org.apache.rocketmq.flink.legacy.common.serialization;
+package org.apache.flink.connector.rocketmq.legacy.common.serialization;
 
 import org.apache.flink.api.common.typeinfo.TypeHint;
 import org.apache.flink.api.common.typeinfo.TypeInformation;
diff --git a/src/main/java/org/apache/rocketmq/flink/legacy/common/util/MetricUtils.java b/src/main/java/org/apache/flink/connector/rocketmq/legacy/common/util/MetricUtils.java
similarity index 98%
rename from src/main/java/org/apache/rocketmq/flink/legacy/common/util/MetricUtils.java
rename to src/main/java/org/apache/flink/connector/rocketmq/legacy/common/util/MetricUtils.java
index 17b4dda..d98e3c3 100644
--- a/src/main/java/org/apache/rocketmq/flink/legacy/common/util/MetricUtils.java
+++ b/src/main/java/org/apache/flink/connector/rocketmq/legacy/common/util/MetricUtils.java
@@ -14,7 +14,7 @@
  * See the License for the specific language governing permissions and
  * limitations under the License.
  */
-package org.apache.rocketmq.flink.legacy.common.util;
+package org.apache.flink.connector.rocketmq.legacy.common.util;
 
 import org.apache.flink.api.common.functions.RuntimeContext;
 import org.apache.flink.metrics.Counter;
diff --git a/src/main/java/org/apache/rocketmq/flink/legacy/common/util/RetryUtil.java b/src/main/java/org/apache/flink/connector/rocketmq/legacy/common/util/RetryUtil.java
similarity index 83%
rename from src/main/java/org/apache/rocketmq/flink/legacy/common/util/RetryUtil.java
rename to src/main/java/org/apache/flink/connector/rocketmq/legacy/common/util/RetryUtil.java
index aae2148..d11105b 100644
--- a/src/main/java/org/apache/rocketmq/flink/legacy/common/util/RetryUtil.java
+++ b/src/main/java/org/apache/flink/connector/rocketmq/legacy/common/util/RetryUtil.java
@@ -14,9 +14,9 @@
  * See the License for the specific language governing permissions and
  * limitations under the License.
  */
-package org.apache.rocketmq.flink.legacy.common.util;
+package org.apache.flink.connector.rocketmq.legacy.common.util;
 
-import org.apache.rocketmq.flink.legacy.RunningChecker;
+import org.apache.flink.connector.rocketmq.legacy.RunningChecker;
 
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
@@ -24,12 +24,15 @@
 import java.util.concurrent.Callable;
 
 public class RetryUtil {
+
     private static final Logger log = LoggerFactory.getLogger(RetryUtil.class);
 
     private static final long INITIAL_BACKOFF = 200;
     private static final long MAX_BACKOFF = 5000;
     private static final int MAX_ATTEMPTS = 5;
 
+    public static final boolean DEBUG = false;
+
     private RetryUtil() {}
 
     public static void waitForMs(long sleepMs) {
@@ -58,7 +61,11 @@
                     }
                     throw new RuntimeException(ex);
                 }
-                log.error("{}, retry {}/{}", errorMsg, retries, MAX_ATTEMPTS, ex);
+                if (DEBUG) {
+                    log.debug("{}, retry {}/{}", errorMsg, retries, MAX_ATTEMPTS, ex);
+                } else {
+                    log.error("{}, retry {}/{}", errorMsg, retries, MAX_ATTEMPTS, ex);
+                }
                 retries++;
             }
             waitForMs(backoff);
diff --git a/src/main/java/org/apache/rocketmq/flink/legacy/common/util/RocketMQUtils.java b/src/main/java/org/apache/flink/connector/rocketmq/legacy/common/util/RocketMQUtils.java
similarity index 95%
rename from src/main/java/org/apache/rocketmq/flink/legacy/common/util/RocketMQUtils.java
rename to src/main/java/org/apache/flink/connector/rocketmq/legacy/common/util/RocketMQUtils.java
index 1f084a8..9f3d913 100644
--- a/src/main/java/org/apache/rocketmq/flink/legacy/common/util/RocketMQUtils.java
+++ b/src/main/java/org/apache/flink/connector/rocketmq/legacy/common/util/RocketMQUtils.java
@@ -14,7 +14,7 @@
  * See the License for the specific language governing permissions and
  * limitations under the License.
  */
-package org.apache.rocketmq.flink.legacy.common.util;
+package org.apache.flink.connector.rocketmq.legacy.common.util;
 
 import org.apache.rocketmq.client.AccessChannel;
 import org.apache.rocketmq.common.message.MessageQueue;
@@ -54,7 +54,7 @@
 
     /**
      * Average Hashing queue algorithm Refer:
-     * org.apache.rocketmq.client.consumer.rebalance.AllocateMessageQueueAveragely
+     * org.apache.rocketmq.client.consumer.rebalance.AllocateStrategyByAveragely
      */
     public static List<MessageQueue> allocate(
             Collection<MessageQueue> mqSet, int numberOfParallelTasks, int indexOfThisTask) {
diff --git a/src/main/java/org/apache/rocketmq/flink/legacy/common/util/TestUtils.java b/src/main/java/org/apache/flink/connector/rocketmq/legacy/common/util/TestUtils.java
similarity index 95%
rename from src/main/java/org/apache/rocketmq/flink/legacy/common/util/TestUtils.java
rename to src/main/java/org/apache/flink/connector/rocketmq/legacy/common/util/TestUtils.java
index 00a24c9..1bde818 100644
--- a/src/main/java/org/apache/rocketmq/flink/legacy/common/util/TestUtils.java
+++ b/src/main/java/org/apache/flink/connector/rocketmq/legacy/common/util/TestUtils.java
@@ -14,7 +14,7 @@
  * See the License for the specific language governing permissions and
  * limitations under the License.
  */
-package org.apache.rocketmq.flink.legacy.common.util;
+package org.apache.flink.connector.rocketmq.legacy.common.util;
 
 import java.lang.reflect.Field;
 
diff --git a/src/main/java/org/apache/rocketmq/flink/legacy/common/watermark/BoundedOutOfOrdernessGenerator.java b/src/main/java/org/apache/flink/connector/rocketmq/legacy/common/watermark/BoundedOutOfOrdernessGenerator.java
similarity index 96%
rename from src/main/java/org/apache/rocketmq/flink/legacy/common/watermark/BoundedOutOfOrdernessGenerator.java
rename to src/main/java/org/apache/flink/connector/rocketmq/legacy/common/watermark/BoundedOutOfOrdernessGenerator.java
index 2b56f54..68dea07 100644
--- a/src/main/java/org/apache/rocketmq/flink/legacy/common/watermark/BoundedOutOfOrdernessGenerator.java
+++ b/src/main/java/org/apache/flink/connector/rocketmq/legacy/common/watermark/BoundedOutOfOrdernessGenerator.java
@@ -15,13 +15,13 @@
  * limitations under the License.
  */
 
-package org.apache.rocketmq.flink.legacy.common.watermark;
-
-import org.apache.rocketmq.common.message.MessageExt;
+package org.apache.flink.connector.rocketmq.legacy.common.watermark;
 
 import org.apache.flink.streaming.api.functions.AssignerWithPeriodicWatermarks;
 import org.apache.flink.streaming.api.watermark.Watermark;
 
+import org.apache.rocketmq.common.message.MessageExt;
+
 public class BoundedOutOfOrdernessGenerator implements AssignerWithPeriodicWatermarks<MessageExt> {
 
     private long maxOutOfOrderness = 5000; // 5 seconds
diff --git a/src/main/java/org/apache/rocketmq/flink/legacy/common/watermark/BoundedOutOfOrdernessGeneratorPerQueue.java b/src/main/java/org/apache/flink/connector/rocketmq/legacy/common/watermark/BoundedOutOfOrdernessGeneratorPerQueue.java
similarity index 97%
rename from src/main/java/org/apache/rocketmq/flink/legacy/common/watermark/BoundedOutOfOrdernessGeneratorPerQueue.java
rename to src/main/java/org/apache/flink/connector/rocketmq/legacy/common/watermark/BoundedOutOfOrdernessGeneratorPerQueue.java
index ff11251..4aeb171 100644
--- a/src/main/java/org/apache/rocketmq/flink/legacy/common/watermark/BoundedOutOfOrdernessGeneratorPerQueue.java
+++ b/src/main/java/org/apache/flink/connector/rocketmq/legacy/common/watermark/BoundedOutOfOrdernessGeneratorPerQueue.java
@@ -15,13 +15,13 @@
  * limitations under the License.
  */
 
-package org.apache.rocketmq.flink.legacy.common.watermark;
-
-import org.apache.rocketmq.common.message.MessageExt;
+package org.apache.flink.connector.rocketmq.legacy.common.watermark;
 
 import org.apache.flink.streaming.api.functions.AssignerWithPeriodicWatermarks;
 import org.apache.flink.streaming.api.watermark.Watermark;
 
+import org.apache.rocketmq.common.message.MessageExt;
+
 import java.util.Map;
 import java.util.concurrent.ConcurrentHashMap;
 
diff --git a/src/main/java/org/apache/rocketmq/flink/legacy/common/watermark/PunctuatedAssigner.java b/src/main/java/org/apache/flink/connector/rocketmq/legacy/common/watermark/PunctuatedAssigner.java
similarity index 94%
rename from src/main/java/org/apache/rocketmq/flink/legacy/common/watermark/PunctuatedAssigner.java
rename to src/main/java/org/apache/flink/connector/rocketmq/legacy/common/watermark/PunctuatedAssigner.java
index 946e873..4991463 100644
--- a/src/main/java/org/apache/rocketmq/flink/legacy/common/watermark/PunctuatedAssigner.java
+++ b/src/main/java/org/apache/flink/connector/rocketmq/legacy/common/watermark/PunctuatedAssigner.java
@@ -15,14 +15,14 @@
  * limitations under the License.
  */
 
-package org.apache.rocketmq.flink.legacy.common.watermark;
+package org.apache.flink.connector.rocketmq.legacy.common.watermark;
 
-import org.apache.rocketmq.common.message.MessageExt;
-import org.apache.rocketmq.flink.legacy.RocketMQConfig;
-
+import org.apache.flink.connector.rocketmq.legacy.RocketMQConfig;
 import org.apache.flink.streaming.api.functions.AssignerWithPunctuatedWatermarks;
 import org.apache.flink.streaming.api.watermark.Watermark;
 
+import org.apache.rocketmq.common.message.MessageExt;
+
 /**
  * With Punctuated Watermarks To generate watermarks whenever a certain event indicates that a new
  * watermark might be generated, use AssignerWithPunctuatedWatermarks. For this class Flink will
diff --git a/src/main/java/org/apache/rocketmq/flink/legacy/common/watermark/TimeLagWatermarkGenerator.java b/src/main/java/org/apache/flink/connector/rocketmq/legacy/common/watermark/TimeLagWatermarkGenerator.java
similarity index 96%
rename from src/main/java/org/apache/rocketmq/flink/legacy/common/watermark/TimeLagWatermarkGenerator.java
rename to src/main/java/org/apache/flink/connector/rocketmq/legacy/common/watermark/TimeLagWatermarkGenerator.java
index 66cb3cd..de33fcb 100644
--- a/src/main/java/org/apache/rocketmq/flink/legacy/common/watermark/TimeLagWatermarkGenerator.java
+++ b/src/main/java/org/apache/flink/connector/rocketmq/legacy/common/watermark/TimeLagWatermarkGenerator.java
@@ -15,13 +15,13 @@
  * limitations under the License.
  */
 
-package org.apache.rocketmq.flink.legacy.common.watermark;
-
-import org.apache.rocketmq.common.message.MessageExt;
+package org.apache.flink.connector.rocketmq.legacy.common.watermark;
 
 import org.apache.flink.streaming.api.functions.AssignerWithPeriodicWatermarks;
 import org.apache.flink.streaming.api.watermark.Watermark;
 
+import org.apache.rocketmq.common.message.MessageExt;
+
 /**
  * This generator generates watermarks that are lagging behind processing time by a certain amount.
  * It assumes that elements arrive in Flink after at most a certain time.
diff --git a/src/main/java/org/apache/rocketmq/flink/legacy/common/watermark/WaterMarkForAll.java b/src/main/java/org/apache/flink/connector/rocketmq/legacy/common/watermark/WaterMarkForAll.java
similarity index 94%
rename from src/main/java/org/apache/rocketmq/flink/legacy/common/watermark/WaterMarkForAll.java
rename to src/main/java/org/apache/flink/connector/rocketmq/legacy/common/watermark/WaterMarkForAll.java
index 8fadd77..b8d58c5 100644
--- a/src/main/java/org/apache/rocketmq/flink/legacy/common/watermark/WaterMarkForAll.java
+++ b/src/main/java/org/apache/flink/connector/rocketmq/legacy/common/watermark/WaterMarkForAll.java
@@ -15,7 +15,7 @@
  * limitations under the License.
  */
 
-package org.apache.rocketmq.flink.legacy.common.watermark;
+package org.apache.flink.connector.rocketmq.legacy.common.watermark;
 
 import org.apache.flink.streaming.api.watermark.Watermark;
 
diff --git a/src/main/java/org/apache/rocketmq/flink/legacy/common/watermark/WaterMarkPerQueue.java b/src/main/java/org/apache/flink/connector/rocketmq/legacy/common/watermark/WaterMarkPerQueue.java
similarity index 96%
rename from src/main/java/org/apache/rocketmq/flink/legacy/common/watermark/WaterMarkPerQueue.java
rename to src/main/java/org/apache/flink/connector/rocketmq/legacy/common/watermark/WaterMarkPerQueue.java
index 941dec5..fd20322 100644
--- a/src/main/java/org/apache/rocketmq/flink/legacy/common/watermark/WaterMarkPerQueue.java
+++ b/src/main/java/org/apache/flink/connector/rocketmq/legacy/common/watermark/WaterMarkPerQueue.java
@@ -15,12 +15,12 @@
  * limitations under the License.
  */
 
-package org.apache.rocketmq.flink.legacy.common.watermark;
-
-import org.apache.rocketmq.common.message.MessageQueue;
+package org.apache.flink.connector.rocketmq.legacy.common.watermark;
 
 import org.apache.flink.streaming.api.watermark.Watermark;
 
+import org.apache.rocketmq.common.message.MessageQueue;
+
 import java.util.Map;
 import java.util.concurrent.ConcurrentHashMap;
 import java.util.concurrent.ConcurrentMap;
diff --git a/src/main/java/org/apache/rocketmq/flink/legacy/function/SinkMapFunction.java b/src/main/java/org/apache/flink/connector/rocketmq/legacy/function/SinkMapFunction.java
similarity index 96%
rename from src/main/java/org/apache/rocketmq/flink/legacy/function/SinkMapFunction.java
rename to src/main/java/org/apache/flink/connector/rocketmq/legacy/function/SinkMapFunction.java
index f63b636..8cda235 100644
--- a/src/main/java/org/apache/rocketmq/flink/legacy/function/SinkMapFunction.java
+++ b/src/main/java/org/apache/flink/connector/rocketmq/legacy/function/SinkMapFunction.java
@@ -15,15 +15,14 @@
  * limitations under the License.
  */
 
-package org.apache.rocketmq.flink.legacy.function;
-
-import org.apache.rocketmq.common.message.Message;
+package org.apache.flink.connector.rocketmq.legacy.function;
 
 import org.apache.flink.api.java.tuple.Tuple2;
 import org.apache.flink.streaming.api.functions.ProcessFunction;
 import org.apache.flink.util.Collector;
 
 import org.apache.commons.lang.Validate;
+import org.apache.rocketmq.common.message.Message;
 
 public class SinkMapFunction extends ProcessFunction<Tuple2<String, String>, Message> {
 
diff --git a/src/main/java/org/apache/rocketmq/flink/legacy/function/SourceMapFunction.java b/src/main/java/org/apache/flink/connector/rocketmq/legacy/function/SourceMapFunction.java
similarity index 95%
rename from src/main/java/org/apache/rocketmq/flink/legacy/function/SourceMapFunction.java
rename to src/main/java/org/apache/flink/connector/rocketmq/legacy/function/SourceMapFunction.java
index a49df95..1945c70 100644
--- a/src/main/java/org/apache/rocketmq/flink/legacy/function/SourceMapFunction.java
+++ b/src/main/java/org/apache/flink/connector/rocketmq/legacy/function/SourceMapFunction.java
@@ -15,7 +15,7 @@
  * limitations under the License.
  */
 
-package org.apache.rocketmq.flink.legacy.function;
+package org.apache.flink.connector.rocketmq.legacy.function;
 
 import org.apache.flink.api.java.tuple.Tuple2;
 import org.apache.flink.streaming.api.functions.ProcessFunction;
diff --git a/src/main/java/org/apache/flink/connector/rocketmq/sink/InnerProducer.java b/src/main/java/org/apache/flink/connector/rocketmq/sink/InnerProducer.java
new file mode 100644
index 0000000..9d1669b
--- /dev/null
+++ b/src/main/java/org/apache/flink/connector/rocketmq/sink/InnerProducer.java
@@ -0,0 +1,84 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.flink.connector.rocketmq.sink;
+
+import org.apache.flink.connector.rocketmq.sink.committer.SendCommittable;
+
+import org.apache.rocketmq.client.producer.SendResult;
+import org.apache.rocketmq.common.message.Message;
+
+import java.util.concurrent.CompletableFuture;
+
+/**
+ * InnerProducer is an interface that represents a message producer used for sending messages to a
+ * messaging system.
+ *
+ * @see AutoCloseable
+ */
+public interface InnerProducer extends AutoCloseable {
+
+    /** Starts the inner consumer. */
+    void start();
+
+    /**
+     * Gets the consumer group of the consumer.
+     *
+     * @return the consumer group of the consumer
+     */
+    String getProducerGroup();
+
+    /**
+     * Sends the message to the messaging system and returns a Future for the send operation.
+     *
+     * @param message the message to be sent
+     * @return a Future for the send operation that returns a SendResult object
+     * @see CompletableFuture
+     * @see SendResult
+     */
+    CompletableFuture<SendResult> send(Message message);
+
+    /**
+     * Sends the message to the messaging system and returns a Future for the send operation.
+     *
+     * @param message the message to be sent
+     * @return a Future for the send operation that returns a SendResult object
+     * @see CompletableFuture
+     * @see SendResult
+     */
+    CompletableFuture<SendResult> sendMessageInTransaction(Message message);
+
+    /**
+     * Commits the send operation identified by the specified SendCommittable object.
+     *
+     * @param sendCommittable the SendCommittable object identifying the send operation
+     * @return a Future that indicates whether the commit operation was successful
+     * @see CompletableFuture
+     * @see SendCommittable
+     */
+    CompletableFuture<Void> commit(SendCommittable sendCommittable);
+
+    /**
+     * Rolls back the send operation identified by the specified SendCommittable object.
+     *
+     * @param sendCommittable the SendCommittable object identifying the send operation
+     * @return a Future that indicates whether the rollback operation was successful
+     * @see CompletableFuture
+     * @see SendCommittable
+     */
+    CompletableFuture<Void> rollback(SendCommittable sendCommittable);
+}
diff --git a/src/main/java/org/apache/flink/connector/rocketmq/sink/InnerProducerImpl.java b/src/main/java/org/apache/flink/connector/rocketmq/sink/InnerProducerImpl.java
new file mode 100644
index 0000000..24897c7
--- /dev/null
+++ b/src/main/java/org/apache/flink/connector/rocketmq/sink/InnerProducerImpl.java
@@ -0,0 +1,311 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.flink.connector.rocketmq.sink;
+
+import org.apache.flink.configuration.Configuration;
+import org.apache.flink.connector.rocketmq.sink.committer.SendCommittable;
+import org.apache.flink.util.StringUtils;
+
+import com.google.common.util.concurrent.MoreExecutors;
+import org.apache.rocketmq.acl.common.AclClientRPCHook;
+import org.apache.rocketmq.acl.common.SessionCredentials;
+import org.apache.rocketmq.client.exception.MQClientException;
+import org.apache.rocketmq.client.impl.factory.MQClientInstance;
+import org.apache.rocketmq.client.producer.LocalTransactionState;
+import org.apache.rocketmq.client.producer.SendResult;
+import org.apache.rocketmq.client.producer.SendStatus;
+import org.apache.rocketmq.client.producer.TransactionListener;
+import org.apache.rocketmq.client.producer.TransactionMQProducer;
+import org.apache.rocketmq.common.message.Message;
+import org.apache.rocketmq.common.message.MessageAccessor;
+import org.apache.rocketmq.common.message.MessageConst;
+import org.apache.rocketmq.common.message.MessageExt;
+import org.apache.rocketmq.common.sysflag.MessageSysFlag;
+import org.apache.rocketmq.remoting.exception.RemotingException;
+import org.apache.rocketmq.remoting.protocol.NamespaceUtil;
+import org.apache.rocketmq.remoting.protocol.RemotingCommand;
+import org.apache.rocketmq.remoting.protocol.RequestCode;
+import org.apache.rocketmq.remoting.protocol.header.EndTransactionRequestHeader;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import java.lang.management.ManagementFactory;
+import java.util.UUID;
+import java.util.concurrent.ArrayBlockingQueue;
+import java.util.concurrent.CompletableFuture;
+import java.util.concurrent.ThreadPoolExecutor;
+import java.util.concurrent.TimeUnit;
+
+public class InnerProducerImpl implements InnerProducer {
+
+    private static final Logger LOG = LoggerFactory.getLogger(InnerProducerImpl.class);
+
+    private final Configuration configuration;
+    private final TransactionMQProducer producer;
+    private MQClientInstance mqClientInstance;
+
+    private final String endPoints;
+    private final String groupId;
+
+    public InnerProducerImpl(Configuration configuration) {
+        this.configuration = configuration;
+        this.groupId = configuration.getString(RocketMQSinkOptions.PRODUCER_GROUP);
+        this.endPoints = configuration.getString(RocketMQSinkOptions.ENDPOINTS);
+
+        String accessKey = configuration.getString(RocketMQSinkOptions.OPTIONAL_ACCESS_KEY);
+        String secretKey = configuration.getString(RocketMQSinkOptions.OPTIONAL_SECRET_KEY);
+
+        if (!StringUtils.isNullOrWhitespaceOnly(accessKey)
+                && !StringUtils.isNullOrWhitespaceOnly(secretKey)) {
+            AclClientRPCHook aclClientRpcHook =
+                    new AclClientRPCHook(new SessionCredentials(accessKey, secretKey));
+            producer = new TransactionMQProducer(groupId, aclClientRpcHook);
+        } else {
+            producer = new TransactionMQProducer(groupId);
+        }
+
+        producer.setNamesrvAddr(endPoints);
+        producer.setVipChannelEnabled(false);
+        producer.setInstanceName(
+                String.join(
+                        "#",
+                        ManagementFactory.getRuntimeMXBean().getName(),
+                        groupId,
+                        UUID.randomUUID().toString()));
+
+        int corePoolSize = configuration.getInteger(RocketMQSinkOptions.EXECUTOR_NUM);
+        producer.setExecutorService(
+                new ThreadPoolExecutor(
+                        corePoolSize,
+                        corePoolSize,
+                        100,
+                        TimeUnit.SECONDS,
+                        new ArrayBlockingQueue<>(2000),
+                        r -> {
+                            Thread thread = new Thread(r);
+                            thread.setName(groupId);
+                            return thread;
+                        }));
+
+        // always response unknown result
+        producer.setTransactionListener(
+                new TransactionListener() {
+                    @Override
+                    public LocalTransactionState executeLocalTransaction(Message msg, Object arg) {
+                        // no need execute local transaction here
+                        // We will directly call the commit or rollback operation
+                        return LocalTransactionState.UNKNOW;
+                    }
+
+                    @Override
+                    public LocalTransactionState checkLocalTransaction(MessageExt msg) {
+                        long transactionTimeout =
+                                configuration.get(RocketMQSinkOptions.TRANSACTION_TIMEOUT);
+                        if (System.currentTimeMillis() - msg.getBornTimestamp()
+                                > transactionTimeout) {
+                            LOG.info(
+                                    "Exceeded the transaction maximum time, return rollback. topic={}, msgId={}",
+                                    msg.getTopic(),
+                                    msg.getTransactionId());
+                            return LocalTransactionState.ROLLBACK_MESSAGE;
+                        } else {
+                            LOG.info(
+                                    "Not exceeded the transaction maximum time, return unknown. topic={}, msgId={}",
+                                    msg.getTopic(),
+                                    msg.getTransactionId());
+                            return LocalTransactionState.UNKNOW;
+                        }
+                    }
+                });
+    }
+
+    @Override
+    public void start() {
+        try {
+            producer.start();
+            // noinspection deprecation
+            mqClientInstance = producer.getDefaultMQProducerImpl().getMqClientFactory();
+            LOG.info(
+                    "RocketMQ producer in flink sink writer init success, endpoint={}, groupId={}, clientId={}",
+                    endPoints,
+                    groupId,
+                    producer.getInstanceName());
+        } catch (MQClientException e) {
+            LOG.error("RocketMQ producer in flink sink writer init failed", e);
+            throw new RuntimeException(e);
+        }
+    }
+
+    @Override
+    public String getProducerGroup() {
+        return groupId;
+    }
+
+    @Override
+    public CompletableFuture<SendResult> send(Message message) {
+        return CompletableFuture.supplyAsync(
+                () -> {
+                    try {
+                        SendResult sendResult = producer.send(message);
+                        LOG.info(
+                                "Send message successfully, topic={}, messageId={}",
+                                message.getTopic(),
+                                sendResult.getMsgId());
+                        return sendResult;
+                    } catch (Exception e) {
+                        LOG.error("Failed to send message, topic={}", message.getTopic(), e);
+                        throw new RuntimeException(e);
+                    }
+                },
+                MoreExecutors.directExecutor());
+    }
+
+    @Override
+    public CompletableFuture<SendResult> sendMessageInTransaction(Message message) {
+        return CompletableFuture.supplyAsync(
+                () -> {
+                    try {
+                        message.setTopic(
+                                NamespaceUtil.wrapNamespace(
+                                        producer.getNamespace(), message.getTopic()));
+
+                        // Ignore DelayTimeLevel parameter
+                        if (message.getDelayTimeLevel() != 0) {
+                            MessageAccessor.clearProperty(
+                                    message, MessageConst.PROPERTY_DELAY_TIME_LEVEL);
+                        }
+
+                        // In general, message id and transaction id should be the same
+                        long transactionTimeout =
+                                configuration.get(RocketMQSinkOptions.TRANSACTION_TIMEOUT);
+                        message.putUserProperty(
+                                MessageConst.PROPERTY_CHECK_IMMUNITY_TIME_IN_SECONDS,
+                                String.valueOf(transactionTimeout));
+                        MessageAccessor.putProperty(
+                                message,
+                                MessageConst.PROPERTY_TRANSACTION_PREPARED,
+                                Boolean.TRUE.toString().toLowerCase());
+                        MessageAccessor.putProperty(
+                                message, MessageConst.PROPERTY_PRODUCER_GROUP, this.groupId);
+
+                        SendResult sendResult = producer.send(message);
+                        if (SendStatus.SEND_OK.equals(sendResult.getSendStatus())) {
+                            LOG.info(
+                                    "Send transaction message successfully, topic={}, transId={}",
+                                    message.getTopic(),
+                                    sendResult.getTransactionId());
+                        } else {
+                            LOG.warn(
+                                    "Failed to send message, topic={}, message={}",
+                                    message.getTopic(),
+                                    message);
+                        }
+                        return sendResult;
+                    } catch (Exception e) {
+                        LOG.error("Failed to send message, topic={}", message.getTopic(), e);
+                        throw new RuntimeException(e);
+                    }
+                },
+                MoreExecutors.directExecutor());
+    }
+
+    public void endTransaction(
+            final SendCommittable sendCommittable, final TransactionResult transactionResult) {
+
+        try {
+            final String brokerName =
+                    this.mqClientInstance.getBrokerNameFromMessageQueue(
+                            producer.queueWithNamespace(sendCommittable.getMessageQueue()));
+            final String brokerAddress =
+                    this.mqClientInstance.findBrokerAddressInPublish(brokerName);
+
+            EndTransactionRequestHeader requestHeader = new EndTransactionRequestHeader();
+            requestHeader.setTransactionId(sendCommittable.getTransactionId());
+            requestHeader.setCommitLogOffset(sendCommittable.getMessageOffset());
+            requestHeader.setBname(brokerName);
+            requestHeader.setProducerGroup(this.groupId);
+            requestHeader.setTranStateTableOffset(sendCommittable.getQueueOffset());
+            requestHeader.setFromTransactionCheck(true);
+            requestHeader.setMsgId(sendCommittable.getMsgId());
+
+            switch (transactionResult) {
+                case COMMIT:
+                    requestHeader.setCommitOrRollback(MessageSysFlag.TRANSACTION_COMMIT_TYPE);
+                    break;
+                case ROLLBACK:
+                    requestHeader.setCommitOrRollback(MessageSysFlag.TRANSACTION_ROLLBACK_TYPE);
+                    break;
+                case UNKNOWN:
+                    requestHeader.setCommitOrRollback(MessageSysFlag.TRANSACTION_NOT_TYPE);
+                    break;
+                default:
+                    break;
+            }
+
+            if (sendCommittable.getMessageOffset() != -1L) {
+                this.endTransaction(
+                        brokerAddress, requestHeader, "", this.producer.getSendMsgTimeout());
+            } else {
+                LOG.error(
+                        "Convert message physical offset error, msgId={}",
+                        sendCommittable.getMsgId());
+            }
+        } catch (Exception e) {
+            LOG.error("Try end transaction error", e);
+        }
+    }
+
+    public void endTransaction(
+            final String address,
+            final EndTransactionRequestHeader requestHeader,
+            final String remark,
+            final long timeoutMillis)
+            throws RemotingException, InterruptedException {
+        RemotingCommand request =
+                RemotingCommand.createRequestCommand(RequestCode.END_TRANSACTION, requestHeader);
+        request.setRemark(remark);
+        this.mqClientInstance
+                .getMQClientAPIImpl()
+                .getRemotingClient()
+                .invokeSync(address, request, timeoutMillis);
+    }
+
+    @Override
+    public CompletableFuture<Void> commit(SendCommittable sendCommittable) {
+        return CompletableFuture.runAsync(
+                () -> endTransaction(sendCommittable, TransactionResult.COMMIT),
+                producer.getExecutorService());
+    }
+
+    @Override
+    public CompletableFuture<Void> rollback(SendCommittable sendCommittable) {
+        return CompletableFuture.runAsync(
+                () -> endTransaction(sendCommittable, TransactionResult.ROLLBACK),
+                producer.getExecutorService());
+    }
+
+    @Override
+    public void close() throws Exception {
+        if (producer != null) {
+            String clientId = producer.getInstanceName();
+            producer.shutdown();
+            mqClientInstance = null;
+            LOG.info("RocketMQ producer has shutdown, groupId={}, clientId={}", groupId, clientId);
+        }
+    }
+}
diff --git a/src/main/java/org/apache/flink/connector/rocketmq/sink/RocketMQSink.java b/src/main/java/org/apache/flink/connector/rocketmq/sink/RocketMQSink.java
new file mode 100644
index 0000000..5ea2fd8
--- /dev/null
+++ b/src/main/java/org/apache/flink/connector/rocketmq/sink/RocketMQSink.java
@@ -0,0 +1,73 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.flink.connector.rocketmq.sink;
+
+import org.apache.flink.annotation.PublicEvolving;
+import org.apache.flink.api.connector.sink2.Committer;
+import org.apache.flink.api.connector.sink2.TwoPhaseCommittingSink;
+import org.apache.flink.configuration.Configuration;
+import org.apache.flink.connector.rocketmq.legacy.common.selector.MessageQueueSelector;
+import org.apache.flink.connector.rocketmq.sink.committer.RocketMQCommitter;
+import org.apache.flink.connector.rocketmq.sink.committer.SendCommittable;
+import org.apache.flink.connector.rocketmq.sink.committer.SendCommittableSerializer;
+import org.apache.flink.connector.rocketmq.sink.writer.RocketMQWriter;
+import org.apache.flink.connector.rocketmq.sink.writer.serializer.RocketMQSerializationSchema;
+import org.apache.flink.core.io.SimpleVersionedSerializer;
+
+@PublicEvolving
+public class RocketMQSink<IN> implements TwoPhaseCommittingSink<IN, SendCommittable> {
+
+    private final Configuration configuration;
+    private final MessageQueueSelector messageQueueSelector;
+    private final RocketMQSerializationSchema<IN> serializationSchema;
+
+    RocketMQSink(
+            Configuration configuration,
+            MessageQueueSelector messageQueueSelector,
+            RocketMQSerializationSchema<IN> serializationSchema) {
+        this.configuration = configuration;
+        this.messageQueueSelector = messageQueueSelector;
+        this.serializationSchema = serializationSchema;
+    }
+
+    /**
+     * Create a {@link RocketMQSinkBuilder} to construct a new {@link RocketMQSink}.
+     *
+     * @param <IN> type of incoming records
+     * @return {@link RocketMQSinkBuilder}
+     */
+    public static <IN> RocketMQSinkBuilder<IN> builder() {
+        return new RocketMQSinkBuilder<>();
+    }
+
+    @Override
+    public PrecommittingSinkWriter<IN, SendCommittable> createWriter(InitContext context) {
+        return new RocketMQWriter<>(
+                configuration, messageQueueSelector, serializationSchema, context);
+    }
+
+    @Override
+    public Committer<SendCommittable> createCommitter() {
+        return new RocketMQCommitter(configuration);
+    }
+
+    @Override
+    public SimpleVersionedSerializer<SendCommittable> getCommittableSerializer() {
+        return new SendCommittableSerializer();
+    }
+}
diff --git a/src/main/java/org/apache/flink/connector/rocketmq/sink/RocketMQSinkBuilder.java b/src/main/java/org/apache/flink/connector/rocketmq/sink/RocketMQSinkBuilder.java
new file mode 100644
index 0000000..426d43c
--- /dev/null
+++ b/src/main/java/org/apache/flink/connector/rocketmq/sink/RocketMQSinkBuilder.java
@@ -0,0 +1,170 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.flink.connector.rocketmq.sink;
+
+import org.apache.flink.annotation.PublicEvolving;
+import org.apache.flink.api.common.ExecutionConfig;
+import org.apache.flink.api.java.ClosureCleaner;
+import org.apache.flink.configuration.ConfigOption;
+import org.apache.flink.configuration.Configuration;
+import org.apache.flink.connector.base.DeliveryGuarantee;
+import org.apache.flink.connector.rocketmq.common.config.RocketMQConfigBuilder;
+import org.apache.flink.connector.rocketmq.common.config.RocketMQConfigValidator;
+import org.apache.flink.connector.rocketmq.common.config.RocketMQOptions;
+import org.apache.flink.connector.rocketmq.legacy.common.selector.MessageQueueSelector;
+import org.apache.flink.connector.rocketmq.sink.writer.serializer.RocketMQSerializationSchema;
+import org.apache.flink.connector.rocketmq.source.RocketMQSource;
+import org.apache.flink.connector.rocketmq.source.RocketMQSourceOptions;
+
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import java.util.Properties;
+
+import static org.apache.flink.util.Preconditions.checkNotNull;
+
+/**
+ * Builder to construct {@link RocketMQSink}.
+ *
+ * @see RocketMQSink for a more detailed explanation of the different guarantees.
+ */
+@PublicEvolving
+public class RocketMQSinkBuilder<IN> {
+
+    private static final Logger LOG = LoggerFactory.getLogger(RocketMQSinkBuilder.class);
+
+    public static final RocketMQConfigValidator SINK_CONFIG_VALIDATOR =
+            RocketMQConfigValidator.builder().build();
+
+    private final RocketMQConfigBuilder configBuilder;
+    private RocketMQSerializationSchema<IN> serializer;
+    private MessageQueueSelector messageQueueSelector;
+
+    public RocketMQSinkBuilder() {
+        this.configBuilder = new RocketMQConfigBuilder();
+    }
+
+    /**
+     * Configure the access point with which the SDK should communicate.
+     *
+     * @param endpoints address of service.
+     * @return the client configuration builder instance.
+     */
+    public RocketMQSinkBuilder<IN> setEndpoints(String endpoints) {
+        return this.setConfig(RocketMQSinkOptions.ENDPOINTS, endpoints);
+    }
+
+    /**
+     * Sets the consumer group id of the RocketMQSource.
+     *
+     * @param groupId the group id of the RocketMQSource.
+     * @return this RocketMQSourceBuilder.
+     */
+    public RocketMQSinkBuilder<IN> setGroupId(String groupId) {
+        this.configBuilder.set(RocketMQSinkOptions.PRODUCER_GROUP, groupId);
+        return this;
+    }
+
+    /**
+     * Sets the wanted the {@link DeliveryGuarantee}.
+     *
+     * @param deliveryGuarantee target delivery guarantee
+     * @return {@link RocketMQSinkBuilder}
+     */
+    public RocketMQSinkBuilder<IN> setDeliveryGuarantee(DeliveryGuarantee deliveryGuarantee) {
+        checkNotNull(deliveryGuarantee, "delivery guarantee is null");
+        this.configBuilder.set(RocketMQSinkOptions.DELIVERY_GUARANTEE, deliveryGuarantee.name());
+        return this;
+    }
+
+    public RocketMQSinkBuilder<IN> setMessageQueueSelector(
+            MessageQueueSelector messageQueueSelector) {
+        checkNotNull(messageQueueSelector, "message queue selector is null");
+        this.messageQueueSelector = messageQueueSelector;
+        return this;
+    }
+
+    /**
+     * Set an arbitrary property for the RocketMQ source. The valid keys can be found in {@link
+     * RocketMQSourceOptions}.
+     *
+     * <p>Make sure the option could be set only once or with same value.
+     *
+     * @param key the key of the property.
+     * @param value the value of the property.
+     * @return this RocketMQSourceBuilder.
+     */
+    public <T> RocketMQSinkBuilder<IN> setConfig(ConfigOption<T> key, T value) {
+        configBuilder.set(key, value);
+        return this;
+    }
+
+    /**
+     * Set arbitrary properties for the RocketMQSink and RocketMQ Consumer. The valid keys can be
+     * found in {@link RocketMQSinkOptions} and {@link RocketMQOptions}.
+     *
+     * @param config the config to set for the RocketMQSink.
+     * @return this RocketMQSinkBuilder.
+     */
+    public RocketMQSinkBuilder<IN> setConfig(Configuration config) {
+        configBuilder.set(config);
+        return this;
+    }
+
+    /**
+     * Set arbitrary properties for the RocketMQSink and RocketMQ Consumer. The valid keys can be
+     * found in {@link RocketMQSinkOptions} and {@link RocketMQOptions}.
+     *
+     * @param properties the config properties to set for the RocketMQSink.
+     * @return this RocketMQSinkBuilder.
+     */
+    public RocketMQSinkBuilder<IN> setProperties(Properties properties) {
+        configBuilder.set(properties);
+        return this;
+    }
+
+    /**
+     * Sets the {@link RocketMQSerializationSchema} that transforms incoming records to {@link
+     * org.apache.rocketmq.common.message.MessageExt}s.
+     *
+     * @param serializer serialize message
+     * @return {@link RocketMQSinkBuilder}
+     */
+    public RocketMQSinkBuilder<IN> setSerializer(RocketMQSerializationSchema<IN> serializer) {
+        this.serializer = checkNotNull(serializer, "serializer is null");
+        ClosureCleaner.clean(this.serializer, ExecutionConfig.ClosureCleanerLevel.RECURSIVE, true);
+        return this;
+    }
+
+    /**
+     * Build the {@link RocketMQSource}.
+     *
+     * @return a RocketMQSource with the settings made for this builder.
+     */
+    public RocketMQSink<IN> build() {
+        sanityCheck();
+        parseAndSetRequiredProperties();
+        return new RocketMQSink<>(
+                configBuilder.build(SINK_CONFIG_VALIDATOR), messageQueueSelector, serializer);
+    }
+
+    // ------------- private helpers  --------------
+    private void sanityCheck() {}
+
+    private void parseAndSetRequiredProperties() {}
+}
diff --git a/src/main/java/org/apache/flink/connector/rocketmq/sink/RocketMQSinkOptions.java b/src/main/java/org/apache/flink/connector/rocketmq/sink/RocketMQSinkOptions.java
new file mode 100644
index 0000000..0ddd850
--- /dev/null
+++ b/src/main/java/org/apache/flink/connector/rocketmq/sink/RocketMQSinkOptions.java
@@ -0,0 +1,113 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.flink.connector.rocketmq.sink;
+
+import org.apache.flink.configuration.ConfigOption;
+import org.apache.flink.configuration.ConfigOptions;
+import org.apache.flink.connector.base.DeliveryGuarantee;
+import org.apache.flink.connector.rocketmq.common.config.RocketMQOptions;
+
+import java.time.Duration;
+
+public class RocketMQSinkOptions extends RocketMQOptions {
+
+    // rocketmq client API config prefix.
+    public static final String PRODUCER_PREFIX = "rocketmq.sink.";
+
+    public static final ConfigOption<String> PRODUCER_GROUP =
+            ConfigOptions.key(PRODUCER_PREFIX + "group")
+                    .stringType()
+                    .defaultValue("PID-flink-producer");
+
+    public static final ConfigOption<String> TOPIC =
+            ConfigOptions.key(PRODUCER_PREFIX + "topic")
+                    .stringType()
+                    .noDefaultValue()
+                    .withDescription("The name of the persist topic");
+
+    public static final ConfigOption<String> SERIALIZE_FORMAT =
+            ConfigOptions.key(PRODUCER_PREFIX + "serialize.format")
+                    .stringType()
+                    .defaultValue("json");
+
+    public static final ConfigOption<String> DELIVERY_GUARANTEE =
+            ConfigOptions.key(PRODUCER_PREFIX + "delivery.guarantee")
+                    .stringType()
+                    .defaultValue(DeliveryGuarantee.AT_LEAST_ONCE.name())
+                    .withDescription("Optional delivery guarantee when committing.");
+
+    public static final ConfigOption<Long> TRANSACTION_TIMEOUT =
+            ConfigOptions.key(PRODUCER_PREFIX + "transaction.timeout")
+                    .longType()
+                    .defaultValue(Duration.ofMinutes(15).getSeconds());
+
+    public static final ConfigOption<Integer> SEND_RETRY_TIMES =
+            ConfigOptions.key(PRODUCER_PREFIX + "send.retry.times").intType().defaultValue(3);
+
+    public static final ConfigOption<Integer> SEND_PENDING_MAX =
+            ConfigOptions.key(PRODUCER_PREFIX + "send.pending.max")
+                    .intType()
+                    .defaultValue(1000)
+                    .withDescription("Message send timeout in ms.");
+
+    public static final ConfigOption<Long> SEND_TIMEOUT =
+            ConfigOptions.key(PRODUCER_PREFIX + "send.timeout")
+                    .longType()
+                    .defaultValue(5000L)
+                    .withDescription("Message send timeout in ms.");
+
+    public static final ConfigOption<Integer> EXECUTOR_NUM =
+            ConfigOptions.key(PRODUCER_PREFIX + "executor.num")
+                    .intType()
+                    .defaultValue(4)
+                    .withDescription("Message send timeout in ms.");
+
+    public static final ConfigOption<String> TAG =
+            ConfigOptions.key(PRODUCER_PREFIX + "tag")
+                    .stringType()
+                    .noDefaultValue()
+                    .withDescription("The name of the persist topic");
+
+    public static final ConfigOption<String> KEY =
+            ConfigOptions.key(PRODUCER_PREFIX + "key")
+                    .stringType()
+                    .noDefaultValue()
+                    .withDescription("The name of the persist topic");
+
+    // old config
+    public static final ConfigOption<Integer> OPTIONAL_WRITE_RETRY_TIMES =
+            ConfigOptions.key("retryTimes").intType().defaultValue(10);
+
+    public static final ConfigOption<Long> OPTIONAL_WRITE_SLEEP_TIME_MS =
+            ConfigOptions.key("sleepTimeMs").longType().defaultValue(5000L);
+
+    public static final ConfigOption<Boolean> OPTIONAL_WRITE_IS_DYNAMIC_TAG =
+            ConfigOptions.key("isDynamicTag").booleanType().defaultValue(false);
+
+    public static final ConfigOption<String> OPTIONAL_WRITE_DYNAMIC_TAG_COLUMN =
+            ConfigOptions.key("dynamicTagColumn").stringType().noDefaultValue();
+
+    public static final ConfigOption<Boolean> OPTIONAL_WRITE_DYNAMIC_TAG_COLUMN_WRITE_INCLUDED =
+            ConfigOptions.key("dynamicTagColumnWriteIncluded").booleanType().defaultValue(true);
+
+    public static final ConfigOption<String> OPTIONAL_WRITE_KEY_COLUMNS =
+            ConfigOptions.key("keyColumns").stringType().noDefaultValue();
+
+    public static final ConfigOption<Boolean> OPTIONAL_WRITE_KEYS_TO_BODY =
+            ConfigOptions.key("writeKeysToBody").booleanType().defaultValue(false);
+}
diff --git a/src/main/java/org/apache/rocketmq/flink/legacy/common/serialization/KeyValueDeserializationSchema.java b/src/main/java/org/apache/flink/connector/rocketmq/sink/TransactionResult.java
similarity index 62%
copy from src/main/java/org/apache/rocketmq/flink/legacy/common/serialization/KeyValueDeserializationSchema.java
copy to src/main/java/org/apache/flink/connector/rocketmq/sink/TransactionResult.java
index 8d0c778..da72ac8 100644
--- a/src/main/java/org/apache/rocketmq/flink/legacy/common/serialization/KeyValueDeserializationSchema.java
+++ b/src/main/java/org/apache/flink/connector/rocketmq/sink/TransactionResult.java
@@ -14,12 +14,20 @@
  * See the License for the specific language governing permissions and
  * limitations under the License.
  */
-package org.apache.rocketmq.flink.legacy.common.serialization;
 
-import org.apache.flink.api.java.typeutils.ResultTypeQueryable;
+package org.apache.flink.connector.rocketmq.sink;
 
-import java.io.Serializable;
+public enum TransactionResult {
 
-public interface KeyValueDeserializationSchema<T> extends ResultTypeQueryable<T>, Serializable {
-    T deserializeKeyAndValue(byte[] key, byte[] value);
+    /** Notify server that current transaction should be committed. */
+    COMMIT,
+
+    /** Notify server that current transaction should be roll-backed. */
+    ROLLBACK,
+
+    /**
+     * Notify the server that the state of this transaction is not sure. You should be cautious
+     * before return unknown because the examination from the server will be performed periodically.
+     */
+    UNKNOWN
 }
diff --git a/src/main/java/org/apache/flink/connector/rocketmq/sink/committer/RocketMQCommitter.java b/src/main/java/org/apache/flink/connector/rocketmq/sink/committer/RocketMQCommitter.java
new file mode 100644
index 0000000..55bacca
--- /dev/null
+++ b/src/main/java/org/apache/flink/connector/rocketmq/sink/committer/RocketMQCommitter.java
@@ -0,0 +1,112 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.flink.connector.rocketmq.sink.committer;
+
+import org.apache.flink.api.connector.sink2.Committer;
+import org.apache.flink.configuration.Configuration;
+import org.apache.flink.connector.rocketmq.sink.InnerProducer;
+import org.apache.flink.connector.rocketmq.sink.InnerProducerImpl;
+import org.apache.flink.connector.rocketmq.sink.RocketMQSink;
+
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import java.util.ArrayList;
+import java.util.Collection;
+import java.util.List;
+import java.util.concurrent.CompletableFuture;
+
+import static com.google.common.base.Preconditions.checkNotNull;
+
+/**
+ * Committer implementation for {@link RocketMQSink}
+ *
+ * <p>The committer is responsible to finalize the RocketMQ transactions by committing them.
+ */
+public class RocketMQCommitter implements Committer<SendCommittable>, Cloneable {
+
+    private static final Logger LOG = LoggerFactory.getLogger(RocketMQCommitter.class);
+    private InnerProducer producer;
+    private final Configuration configuration;
+
+    public RocketMQCommitter(Configuration configuration) {
+        this.configuration = configuration;
+    }
+
+    @Override
+    public void commit(Collection<CommitRequest<SendCommittable>> requests) {
+        List<CompletableFuture<Void>> futures = new ArrayList<>();
+        try {
+            for (CommitRequest<SendCommittable> request : requests) {
+                final SendCommittable committable = request.getCommittable();
+                LOG.info("Commit transaction message, send committable={}", committable);
+                try {
+                    CompletableFuture<Void> future =
+                            this.getTransactionProducer()
+                                    .commit(committable)
+                                    .thenAccept(unused -> request.signalAlreadyCommitted())
+                                    .exceptionally(
+                                            throwable -> {
+                                                LOG.error(
+                                                        "Commit message error, committable={}",
+                                                        committable);
+                                                request.signalFailedWithKnownReason(throwable);
+                                                return null;
+                                            });
+                    futures.add(future);
+                } catch (Throwable e) {
+                    LOG.error("Commit message error, committable={}", committable, e);
+                    request.signalFailedWithKnownReason(e);
+                }
+            }
+            CompletableFuture<Void> allFuture =
+                    CompletableFuture.allOf(futures.toArray(new CompletableFuture[0]));
+            allFuture.get();
+        } catch (Exception e) {
+            LOG.error("Commit message error", e);
+        }
+    }
+
+    /** Lazy initialize this backend transaction client. */
+    private InnerProducer getTransactionProducer() {
+        if (producer == null) {
+            this.producer = new InnerProducerImpl(configuration);
+            this.producer.start();
+            checkNotNull(producer, "You haven't enable rocketmq transaction client.");
+        }
+        return producer;
+    }
+
+    @Override
+    public void close() throws Exception {
+        if (producer != null) {
+            producer.close();
+        }
+    }
+
+    @Override
+    public RocketMQCommitter clone() {
+        try {
+            // TODO: copy mutable state here, so the clone can't change the internals of the
+            // original
+            return (RocketMQCommitter) super.clone();
+        } catch (CloneNotSupportedException e) {
+            throw new AssertionError();
+        }
+    }
+}
diff --git a/src/main/java/org/apache/flink/connector/rocketmq/sink/committer/SendCommittable.java b/src/main/java/org/apache/flink/connector/rocketmq/sink/committer/SendCommittable.java
new file mode 100644
index 0000000..a69e737
--- /dev/null
+++ b/src/main/java/org/apache/flink/connector/rocketmq/sink/committer/SendCommittable.java
@@ -0,0 +1,153 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.flink.connector.rocketmq.sink.committer;
+
+import org.apache.flink.annotation.Internal;
+
+import org.apache.commons.lang3.StringUtils;
+import org.apache.rocketmq.client.producer.SendResult;
+import org.apache.rocketmq.common.message.MessageDecoder;
+import org.apache.rocketmq.common.message.MessageQueue;
+
+/** The writer state for RocketMQ connector. We would use in RocketMQ committer. */
+@Internal
+public class SendCommittable {
+
+    private String topic;
+
+    private String brokerName;
+
+    private Integer queueId;
+
+    private Long queueOffset;
+
+    private String msgId;
+
+    private String offsetMsgId;
+
+    private String transactionId;
+
+    public SendCommittable() {}
+
+    public SendCommittable(SendResult sendResult) {
+        this.topic = sendResult.getMessageQueue().getTopic();
+        this.brokerName = sendResult.getMessageQueue().getBrokerName();
+        this.queueId = sendResult.getMessageQueue().getQueueId();
+        this.queueOffset = sendResult.getQueueOffset();
+        this.msgId = sendResult.getMsgId();
+        this.offsetMsgId = sendResult.getOffsetMsgId();
+        this.transactionId = sendResult.getTransactionId();
+    }
+
+    public String getTopic() {
+        return topic;
+    }
+
+    public void setTopic(String topic) {
+        this.topic = topic;
+    }
+
+    public String getBrokerName() {
+        return brokerName;
+    }
+
+    public void setBrokerName(String brokerName) {
+        this.brokerName = brokerName;
+    }
+
+    public Integer getQueueId() {
+        return queueId;
+    }
+
+    public void setQueueId(Integer queueId) {
+        this.queueId = queueId;
+    }
+
+    public Long getQueueOffset() {
+        return queueOffset;
+    }
+
+    public void setQueueOffset(Long queueOffset) {
+        this.queueOffset = queueOffset;
+    }
+
+    public String getMsgId() {
+        return msgId;
+    }
+
+    public void setMsgId(String msgId) {
+        this.msgId = msgId;
+    }
+
+    public String getOffsetMsgId() {
+        return offsetMsgId != null ? offsetMsgId : "";
+    }
+
+    public void setOffsetMsgId(String offsetMsgId) {
+        this.offsetMsgId = offsetMsgId;
+    }
+
+    public String getTransactionId() {
+        return transactionId;
+    }
+
+    public void setTransactionId(String transactionId) {
+        this.transactionId = transactionId;
+    }
+
+    public MessageQueue getMessageQueue() {
+        return new MessageQueue(topic, brokerName, queueId);
+    }
+
+    public long getMessageOffset() {
+        try {
+            if (StringUtils.isNotBlank(offsetMsgId)) {
+                return MessageDecoder.decodeMessageId(offsetMsgId).getOffset();
+            } else {
+                return MessageDecoder.decodeMessageId(msgId).getOffset();
+            }
+        } catch (Exception e) {
+            return -1L;
+        }
+    }
+
+    @Override
+    public String toString() {
+        return "SendCommittable{"
+                + "topic='"
+                + topic
+                + '\''
+                + ", brokerName='"
+                + brokerName
+                + '\''
+                + ", queueId="
+                + queueId
+                + ", queueOffset="
+                + queueOffset
+                + ", msgId='"
+                + msgId
+                + '\''
+                + ", offsetMsgId='"
+                + offsetMsgId
+                + '\''
+                + ", transactionId='"
+                + transactionId
+                + '\''
+                + '}';
+    }
+}
diff --git a/src/main/java/org/apache/flink/connector/rocketmq/sink/committer/SendCommittableSerializer.java b/src/main/java/org/apache/flink/connector/rocketmq/sink/committer/SendCommittableSerializer.java
new file mode 100644
index 0000000..4b64873
--- /dev/null
+++ b/src/main/java/org/apache/flink/connector/rocketmq/sink/committer/SendCommittableSerializer.java
@@ -0,0 +1,69 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.flink.connector.rocketmq.sink.committer;
+
+import org.apache.flink.core.io.SimpleVersionedSerializer;
+
+import java.io.ByteArrayInputStream;
+import java.io.ByteArrayOutputStream;
+import java.io.DataInputStream;
+import java.io.DataOutputStream;
+import java.io.IOException;
+
+/** A serializer used to serialize {@link SendCommittable}. */
+public class SendCommittableSerializer implements SimpleVersionedSerializer<SendCommittable> {
+
+    private static final int CURRENT_VERSION = 1;
+
+    @Override
+    public int getVersion() {
+        return CURRENT_VERSION;
+    }
+
+    @Override
+    public byte[] serialize(SendCommittable obj) throws IOException {
+        try (final ByteArrayOutputStream stream = new ByteArrayOutputStream();
+                final DataOutputStream out = new DataOutputStream(stream)) {
+            out.writeUTF(obj.getTopic());
+            out.writeUTF(obj.getBrokerName());
+            out.writeInt(obj.getQueueId());
+            out.writeLong(obj.getQueueOffset());
+            out.writeUTF(obj.getMsgId());
+            out.writeUTF(obj.getOffsetMsgId());
+            out.writeUTF(obj.getTransactionId());
+            out.flush();
+            return stream.toByteArray();
+        }
+    }
+
+    @Override
+    public SendCommittable deserialize(int version, byte[] serialized) throws IOException {
+        try (final ByteArrayInputStream bis = new ByteArrayInputStream(serialized);
+                final DataInputStream in = new DataInputStream(bis)) {
+            SendCommittable sendCommittable = new SendCommittable();
+            sendCommittable.setTopic(in.readUTF());
+            sendCommittable.setBrokerName(in.readUTF());
+            sendCommittable.setQueueId(in.readInt());
+            sendCommittable.setQueueOffset(in.readLong());
+            sendCommittable.setMsgId(in.readUTF());
+            sendCommittable.setOffsetMsgId(in.readUTF());
+            sendCommittable.setTransactionId(in.readUTF());
+            return sendCommittable;
+        }
+    }
+}
diff --git a/src/main/java/org/apache/rocketmq/flink/sink/table/RocketMQDynamicTableSink.java b/src/main/java/org/apache/flink/connector/rocketmq/sink/table/RocketMQDynamicTableSink.java
similarity index 94%
rename from src/main/java/org/apache/rocketmq/flink/sink/table/RocketMQDynamicTableSink.java
rename to src/main/java/org/apache/flink/connector/rocketmq/sink/table/RocketMQDynamicTableSink.java
index 366991d..73e8af7 100644
--- a/src/main/java/org/apache/rocketmq/flink/sink/table/RocketMQDynamicTableSink.java
+++ b/src/main/java/org/apache/flink/connector/rocketmq/sink/table/RocketMQDynamicTableSink.java
@@ -15,13 +15,12 @@
  * limitations under the License.
  */
 
-package org.apache.rocketmq.flink.sink.table;
-
-import org.apache.rocketmq.flink.legacy.RocketMQConfig;
-import org.apache.rocketmq.flink.legacy.RocketMQSink;
+package org.apache.flink.connector.rocketmq.sink.table;
 
 import org.apache.flink.api.common.typeinfo.TypeInformation;
 import org.apache.flink.api.java.typeutils.RowTypeInfo;
+import org.apache.flink.connector.rocketmq.legacy.RocketMQConfig;
+import org.apache.flink.connector.rocketmq.legacy.RocketMQSink;
 import org.apache.flink.table.api.DataTypes;
 import org.apache.flink.table.api.TableSchema;
 import org.apache.flink.table.connector.ChangelogMode;
@@ -41,8 +40,6 @@
 import java.util.Properties;
 import java.util.stream.Stream;
 
-import static org.apache.rocketmq.flink.sink.table.RocketMQRowDataConverter.MetadataConverter;
-
 /** Defines the dynamic table sink of RocketMQ. */
 public class RocketMQDynamicTableSink implements DynamicTableSink, SupportsWritingMetadata {
 
@@ -266,7 +263,7 @@
         KEYS(
                 "keys",
                 DataTypes.STRING().nullable(),
-                new MetadataConverter() {
+                new RocketMQRowDataConverter.MetadataConverter() {
                     private static final long serialVersionUID = 1L;
 
                     @Override
@@ -281,7 +278,7 @@
         TAGS(
                 "tags",
                 DataTypes.STRING().nullable(),
-                new MetadataConverter() {
+                new RocketMQRowDataConverter.MetadataConverter() {
                     private static final long serialVersionUID = 1L;
 
                     @Override
@@ -297,9 +294,12 @@
 
         final DataType dataType;
 
-        final MetadataConverter converter;
+        final RocketMQRowDataConverter.MetadataConverter converter;
 
-        WritableMetadata(String key, DataType dataType, MetadataConverter converter) {
+        WritableMetadata(
+                String key,
+                DataType dataType,
+                RocketMQRowDataConverter.MetadataConverter converter) {
             this.key = key;
             this.dataType = dataType;
             this.converter = converter;
diff --git a/src/main/java/org/apache/rocketmq/flink/sink/table/RocketMQDynamicTableSinkFactory.java b/src/main/java/org/apache/flink/connector/rocketmq/sink/table/RocketMQDynamicTableSinkFactory.java
similarity index 69%
rename from src/main/java/org/apache/rocketmq/flink/sink/table/RocketMQDynamicTableSinkFactory.java
rename to src/main/java/org/apache/flink/connector/rocketmq/sink/table/RocketMQDynamicTableSinkFactory.java
index 73d383d..03a89b1 100644
--- a/src/main/java/org/apache/rocketmq/flink/sink/table/RocketMQDynamicTableSinkFactory.java
+++ b/src/main/java/org/apache/flink/connector/rocketmq/sink/table/RocketMQDynamicTableSinkFactory.java
@@ -16,7 +16,7 @@
  * limitations under the License.
  */
 
-package org.apache.rocketmq.flink.sink.table;
+package org.apache.flink.connector.rocketmq.sink.table;
 
 import org.apache.flink.configuration.ConfigOption;
 import org.apache.flink.configuration.Configuration;
@@ -31,22 +31,22 @@
 import java.util.Map;
 import java.util.Set;
 
+import static org.apache.flink.connector.rocketmq.common.config.RocketMQOptions.ENDPOINTS;
+import static org.apache.flink.connector.rocketmq.sink.RocketMQSinkOptions.OPTIONAL_ACCESS_KEY;
+import static org.apache.flink.connector.rocketmq.sink.RocketMQSinkOptions.OPTIONAL_ENCODING;
+import static org.apache.flink.connector.rocketmq.sink.RocketMQSinkOptions.OPTIONAL_FIELD_DELIMITER;
+import static org.apache.flink.connector.rocketmq.sink.RocketMQSinkOptions.OPTIONAL_SECRET_KEY;
+import static org.apache.flink.connector.rocketmq.sink.RocketMQSinkOptions.OPTIONAL_WRITE_DYNAMIC_TAG_COLUMN;
+import static org.apache.flink.connector.rocketmq.sink.RocketMQSinkOptions.OPTIONAL_WRITE_DYNAMIC_TAG_COLUMN_WRITE_INCLUDED;
+import static org.apache.flink.connector.rocketmq.sink.RocketMQSinkOptions.OPTIONAL_WRITE_IS_DYNAMIC_TAG;
+import static org.apache.flink.connector.rocketmq.sink.RocketMQSinkOptions.OPTIONAL_WRITE_KEYS_TO_BODY;
+import static org.apache.flink.connector.rocketmq.sink.RocketMQSinkOptions.OPTIONAL_WRITE_KEY_COLUMNS;
+import static org.apache.flink.connector.rocketmq.sink.RocketMQSinkOptions.OPTIONAL_WRITE_RETRY_TIMES;
+import static org.apache.flink.connector.rocketmq.sink.RocketMQSinkOptions.OPTIONAL_WRITE_SLEEP_TIME_MS;
+import static org.apache.flink.connector.rocketmq.sink.RocketMQSinkOptions.PRODUCER_GROUP;
+import static org.apache.flink.connector.rocketmq.sink.RocketMQSinkOptions.TAG;
+import static org.apache.flink.connector.rocketmq.sink.RocketMQSinkOptions.TOPIC;
 import static org.apache.flink.table.factories.FactoryUtil.createTableFactoryHelper;
-import static org.apache.rocketmq.flink.common.RocketMQOptions.NAME_SERVER_ADDRESS;
-import static org.apache.rocketmq.flink.common.RocketMQOptions.OPTIONAL_ACCESS_KEY;
-import static org.apache.rocketmq.flink.common.RocketMQOptions.OPTIONAL_ENCODING;
-import static org.apache.rocketmq.flink.common.RocketMQOptions.OPTIONAL_FIELD_DELIMITER;
-import static org.apache.rocketmq.flink.common.RocketMQOptions.OPTIONAL_SECRET_KEY;
-import static org.apache.rocketmq.flink.common.RocketMQOptions.OPTIONAL_TAG;
-import static org.apache.rocketmq.flink.common.RocketMQOptions.OPTIONAL_WRITE_DYNAMIC_TAG_COLUMN;
-import static org.apache.rocketmq.flink.common.RocketMQOptions.OPTIONAL_WRITE_DYNAMIC_TAG_COLUMN_WRITE_INCLUDED;
-import static org.apache.rocketmq.flink.common.RocketMQOptions.OPTIONAL_WRITE_IS_DYNAMIC_TAG;
-import static org.apache.rocketmq.flink.common.RocketMQOptions.OPTIONAL_WRITE_KEYS_TO_BODY;
-import static org.apache.rocketmq.flink.common.RocketMQOptions.OPTIONAL_WRITE_KEY_COLUMNS;
-import static org.apache.rocketmq.flink.common.RocketMQOptions.OPTIONAL_WRITE_RETRY_TIMES;
-import static org.apache.rocketmq.flink.common.RocketMQOptions.OPTIONAL_WRITE_SLEEP_TIME_MS;
-import static org.apache.rocketmq.flink.common.RocketMQOptions.PRODUCER_GROUP;
-import static org.apache.rocketmq.flink.common.RocketMQOptions.TOPIC;
 
 /**
  * Defines the {@link DynamicTableSinkFactory} implementation to create {@link
@@ -64,21 +64,14 @@
         Set<ConfigOption<?>> requiredOptions = new HashSet<>();
         requiredOptions.add(TOPIC);
         requiredOptions.add(PRODUCER_GROUP);
-        requiredOptions.add(NAME_SERVER_ADDRESS);
+        // requiredOptions.add(PERSIST_OFFSET_INTERVAL);
         return requiredOptions;
     }
 
     @Override
     public Set<ConfigOption<?>> optionalOptions() {
         Set<ConfigOption<?>> optionalOptions = new HashSet<>();
-        optionalOptions.add(OPTIONAL_TAG);
-        optionalOptions.add(OPTIONAL_WRITE_RETRY_TIMES);
-        optionalOptions.add(OPTIONAL_WRITE_SLEEP_TIME_MS);
-        optionalOptions.add(OPTIONAL_WRITE_IS_DYNAMIC_TAG);
-        optionalOptions.add(OPTIONAL_WRITE_DYNAMIC_TAG_COLUMN);
-        optionalOptions.add(OPTIONAL_WRITE_DYNAMIC_TAG_COLUMN_WRITE_INCLUDED);
-        optionalOptions.add(OPTIONAL_WRITE_KEYS_TO_BODY);
-        optionalOptions.add(OPTIONAL_WRITE_KEY_COLUMNS);
+        optionalOptions.add(TAG);
         optionalOptions.add(OPTIONAL_ENCODING);
         optionalOptions.add(OPTIONAL_FIELD_DELIMITER);
         optionalOptions.add(OPTIONAL_ACCESS_KEY);
@@ -94,8 +87,8 @@
         Configuration properties = Configuration.fromMap(rawProperties);
         String topicName = properties.getString(TOPIC);
         String producerGroup = properties.getString(PRODUCER_GROUP);
-        String nameServerAddress = properties.getString(NAME_SERVER_ADDRESS);
-        String tag = properties.getString(OPTIONAL_TAG);
+        String nameServerAddress = properties.getString(ENDPOINTS);
+        String tag = properties.getString(TAG);
         String accessKey = properties.getString(OPTIONAL_ACCESS_KEY);
         String secretKey = properties.getString(OPTIONAL_SECRET_KEY);
         String dynamicColumn = properties.getString(OPTIONAL_WRITE_DYNAMIC_TAG_COLUMN);
diff --git a/src/main/java/org/apache/rocketmq/flink/sink/table/RocketMQRowDataConverter.java b/src/main/java/org/apache/flink/connector/rocketmq/sink/table/RocketMQRowDataConverter.java
similarity index 95%
rename from src/main/java/org/apache/rocketmq/flink/sink/table/RocketMQRowDataConverter.java
rename to src/main/java/org/apache/flink/connector/rocketmq/sink/table/RocketMQRowDataConverter.java
index 9c5c8af..dc14363 100644
--- a/src/main/java/org/apache/rocketmq/flink/sink/table/RocketMQRowDataConverter.java
+++ b/src/main/java/org/apache/flink/connector/rocketmq/sink/table/RocketMQRowDataConverter.java
@@ -16,9 +16,7 @@
  * limitations under the License.
  */
 
-package org.apache.rocketmq.flink.sink.table;
-
-import org.apache.rocketmq.common.message.Message;
+package org.apache.flink.connector.rocketmq.sink.table;
 
 import org.apache.flink.api.java.typeutils.RowTypeInfo;
 import org.apache.flink.table.data.RowData;
@@ -26,6 +24,7 @@
 import org.apache.flink.types.RowKind;
 
 import org.apache.commons.lang3.StringUtils;
+import org.apache.rocketmq.common.message.Message;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 
@@ -36,10 +35,10 @@
 import java.util.List;
 import java.util.Set;
 
+import static org.apache.flink.connector.rocketmq.sink.table.RocketMQDynamicTableSink.WritableMetadata;
+import static org.apache.flink.connector.rocketmq.sink.table.RocketMQDynamicTableSink.WritableMetadata.KEYS;
+import static org.apache.flink.connector.rocketmq.sink.table.RocketMQDynamicTableSink.WritableMetadata.TAGS;
 import static org.apache.flink.util.Preconditions.checkState;
-import static org.apache.rocketmq.flink.sink.table.RocketMQDynamicTableSink.WritableMetadata;
-import static org.apache.rocketmq.flink.sink.table.RocketMQDynamicTableSink.WritableMetadata.KEYS;
-import static org.apache.rocketmq.flink.sink.table.RocketMQDynamicTableSink.WritableMetadata.TAGS;
 
 /** RocketMQRowDataConverter converts the row data of table to RocketMQ message pattern. */
 public class RocketMQRowDataConverter implements Serializable {
diff --git a/src/main/java/org/apache/rocketmq/flink/sink/table/RocketMQRowDataSink.java b/src/main/java/org/apache/flink/connector/rocketmq/sink/table/RocketMQRowDataSink.java
similarity index 94%
rename from src/main/java/org/apache/rocketmq/flink/sink/table/RocketMQRowDataSink.java
rename to src/main/java/org/apache/flink/connector/rocketmq/sink/table/RocketMQRowDataSink.java
index ac01829..c7d9a71 100644
--- a/src/main/java/org/apache/rocketmq/flink/sink/table/RocketMQRowDataSink.java
+++ b/src/main/java/org/apache/flink/connector/rocketmq/sink/table/RocketMQRowDataSink.java
@@ -12,16 +12,16 @@
  * express or implied. See the License for the specific language governing permissions and
  * limitations under the License.
  */
-package org.apache.rocketmq.flink.sink.table;
-
-import org.apache.rocketmq.common.message.Message;
-import org.apache.rocketmq.flink.legacy.RocketMQSink;
+package org.apache.flink.connector.rocketmq.sink.table;
 
 import org.apache.flink.api.common.functions.RuntimeContext;
 import org.apache.flink.configuration.Configuration;
+import org.apache.flink.connector.rocketmq.legacy.RocketMQSink;
 import org.apache.flink.streaming.api.functions.sink.RichSinkFunction;
 import org.apache.flink.table.data.RowData;
 
+import org.apache.rocketmq.common.message.Message;
+
 /** RocketMQRowDataSink helps for writing the converted row data of table to RocketMQ messages. */
 public class RocketMQRowDataSink extends RichSinkFunction<RowData> {
 
diff --git a/src/main/java/org/apache/flink/connector/rocketmq/sink/writer/RocketMQWriter.java b/src/main/java/org/apache/flink/connector/rocketmq/sink/writer/RocketMQWriter.java
new file mode 100644
index 0000000..5118e0b
--- /dev/null
+++ b/src/main/java/org/apache/flink/connector/rocketmq/sink/writer/RocketMQWriter.java
@@ -0,0 +1,135 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.flink.connector.rocketmq.sink.writer;
+
+import org.apache.flink.api.common.eventtime.Watermark;
+import org.apache.flink.api.connector.sink2.Sink;
+import org.apache.flink.api.connector.sink2.TwoPhaseCommittingSink;
+import org.apache.flink.configuration.Configuration;
+import org.apache.flink.connector.base.DeliveryGuarantee;
+import org.apache.flink.connector.rocketmq.legacy.common.selector.MessageQueueSelector;
+import org.apache.flink.connector.rocketmq.sink.InnerProducer;
+import org.apache.flink.connector.rocketmq.sink.InnerProducerImpl;
+import org.apache.flink.connector.rocketmq.sink.RocketMQSinkOptions;
+import org.apache.flink.connector.rocketmq.sink.committer.SendCommittable;
+import org.apache.flink.connector.rocketmq.sink.writer.context.RocketMQSinkContext;
+import org.apache.flink.connector.rocketmq.sink.writer.context.RocketMQSinkContextImpl;
+import org.apache.flink.connector.rocketmq.sink.writer.serializer.RocketMQSerializationSchema;
+
+import org.apache.rocketmq.client.producer.SendResult;
+import org.apache.rocketmq.common.message.Message;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import java.io.IOException;
+import java.util.ArrayList;
+import java.util.Collection;
+import java.util.Collections;
+import java.util.List;
+import java.util.Map;
+import java.util.concurrent.ConcurrentHashMap;
+
+public class RocketMQWriter<IN>
+        implements TwoPhaseCommittingSink.PrecommittingSinkWriter<IN, SendCommittable> {
+
+    private static final Logger LOG = LoggerFactory.getLogger(RocketMQWriter.class);
+    private static final String RMQ_PRODUCER_METRIC_NAME = "RocketMQProducer";
+    private static final long METRIC_UPDATE_INTERVAL_MILLIS = 500;
+
+    private static final String KEY_DISABLE_METRICS = "flink.disable-metrics";
+    private static final String KEY_REGISTER_METRICS = "register.producer.metrics";
+    private static final String ROCKETMQ_PRODUCER_METRICS = "producer-metrics";
+
+    private final transient InnerProducer producer;
+
+    private final DeliveryGuarantee deliveryGuarantee;
+    private final MessageQueueSelector messageQueueSelector;
+    private final RocketMQSinkContext rocketmqSinkContext;
+    private final RocketMQSerializationSchema<IN> serializationSchema;
+    private final Map<String, SendResult> sendResultMap;
+
+    public RocketMQWriter(
+            Configuration configuration,
+            MessageQueueSelector messageQueueSelector,
+            RocketMQSerializationSchema<IN> serializationSchema,
+            Sink.InitContext initContext) {
+
+        this.deliveryGuarantee =
+                DeliveryGuarantee.valueOf(
+                        configuration.getString(RocketMQSinkOptions.DELIVERY_GUARANTEE));
+        this.messageQueueSelector = messageQueueSelector;
+        this.serializationSchema = serializationSchema;
+        this.rocketmqSinkContext = new RocketMQSinkContextImpl(initContext, configuration);
+        this.sendResultMap = new ConcurrentHashMap<>();
+        this.producer = new InnerProducerImpl(configuration);
+        this.producer.start();
+    }
+
+    @Override
+    public void write(IN element, Context context) throws IOException {
+        try {
+            Message message =
+                    serializationSchema.serialize(
+                            element, rocketmqSinkContext, System.currentTimeMillis());
+            if (deliveryGuarantee == DeliveryGuarantee.EXACTLY_ONCE) {
+                producer.sendMessageInTransaction(message)
+                        .whenComplete(
+                                (sendResult, throwable) -> {
+                                    sendResultMap.put(sendResult.getTransactionId(), sendResult);
+                                });
+            } else {
+                producer.send(message)
+                        .whenComplete(
+                                (sendResult, throwable) -> {
+                                    sendResultMap.put(sendResult.getTransactionId(), sendResult);
+                                });
+            }
+        } catch (Exception e) {
+            LOG.error("Send message error", e);
+            throw new IOException(e);
+        }
+    }
+
+    @Override
+    public void flush(boolean endOfInput) throws IOException, InterruptedException {
+        // rocketmq client send message to server immediately, no need flush here
+    }
+
+    @Override
+    public Collection<SendCommittable> prepareCommit() throws IOException, InterruptedException {
+        LOG.info("Prepare commit");
+        if (deliveryGuarantee == DeliveryGuarantee.EXACTLY_ONCE) {
+            final List<SendCommittable> committables = new ArrayList<>();
+            for (SendResult sendResult : sendResultMap.values()) {
+                committables.add(new SendCommittable(sendResult));
+            }
+            LOG.info("Committable size={}.", committables.size());
+            sendResultMap.clear();
+            return committables;
+        }
+        return Collections.emptyList();
+    }
+
+    @Override
+    public void writeWatermark(Watermark watermark) throws IOException, InterruptedException {
+        TwoPhaseCommittingSink.PrecommittingSinkWriter.super.writeWatermark(watermark);
+    }
+
+    @Override
+    public void close() throws Exception {}
+}
diff --git a/src/main/java/org/apache/flink/connector/rocketmq/sink/writer/context/RocketMQSinkContext.java b/src/main/java/org/apache/flink/connector/rocketmq/sink/writer/context/RocketMQSinkContext.java
new file mode 100644
index 0000000..24573eb
--- /dev/null
+++ b/src/main/java/org/apache/flink/connector/rocketmq/sink/writer/context/RocketMQSinkContext.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.flink.connector.rocketmq.sink.writer.context;
+
+import org.apache.flink.annotation.Experimental;
+import org.apache.flink.annotation.PublicEvolving;
+import org.apache.flink.api.common.operators.MailboxExecutor;
+
+/**
+ * This context provides information on the rocketmq record target location. An implementation that
+ * would contain all the required context.
+ */
+@PublicEvolving
+public interface RocketMQSinkContext {
+
+    /**
+     * Get the number of the subtask that RocketMQSink is running on. The numbering starts from 0
+     * and goes up to parallelism-1. (parallelism as returned by {@link
+     * #getNumberOfParallelInstances()}
+     *
+     * @return number of subtask
+     */
+    int getParallelInstanceId();
+
+    /** @return number of parallel RocketMQSink tasks. */
+    int getNumberOfParallelInstances();
+
+    /**
+     * RocketMQ can check the schema and upgrade the schema automatically. If you enable this
+     * option, we wouldn't serialize the record into bytes, we send and serialize it in the client.
+     */
+    @Experimental
+    boolean isEnableSchemaEvolution();
+
+    /** Returns the current process time in flink. */
+    long processTime();
+
+    MailboxExecutor getMailboxExecutor();
+}
diff --git a/src/main/java/org/apache/flink/connector/rocketmq/sink/writer/context/RocketMQSinkContextImpl.java b/src/main/java/org/apache/flink/connector/rocketmq/sink/writer/context/RocketMQSinkContextImpl.java
new file mode 100644
index 0000000..032c1fe
--- /dev/null
+++ b/src/main/java/org/apache/flink/connector/rocketmq/sink/writer/context/RocketMQSinkContextImpl.java
@@ -0,0 +1,67 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.flink.connector.rocketmq.sink.writer.context;
+
+import org.apache.flink.annotation.PublicEvolving;
+import org.apache.flink.api.common.operators.MailboxExecutor;
+import org.apache.flink.api.common.operators.ProcessingTimeService;
+import org.apache.flink.api.connector.sink2.Sink.InitContext;
+import org.apache.flink.configuration.Configuration;
+
+@PublicEvolving
+public class RocketMQSinkContextImpl implements RocketMQSinkContext {
+
+    private final int numberOfParallelSubtasks;
+    private final int parallelInstanceId;
+    private final ProcessingTimeService processingTimeService;
+    private final MailboxExecutor mailboxExecutor;
+    private final boolean enableSchemaEvolution;
+
+    public RocketMQSinkContextImpl(InitContext initContext, Configuration configuration) {
+        this.parallelInstanceId = initContext.getSubtaskId();
+        this.numberOfParallelSubtasks = initContext.getNumberOfParallelSubtasks();
+        this.processingTimeService = initContext.getProcessingTimeService();
+        this.mailboxExecutor = initContext.getMailboxExecutor();
+        this.enableSchemaEvolution = false;
+    }
+
+    @Override
+    public int getParallelInstanceId() {
+        return parallelInstanceId;
+    }
+
+    @Override
+    public int getNumberOfParallelInstances() {
+        return numberOfParallelSubtasks;
+    }
+
+    @Override
+    public boolean isEnableSchemaEvolution() {
+        return enableSchemaEvolution;
+    }
+
+    @Override
+    public long processTime() {
+        return processingTimeService.getCurrentProcessingTime();
+    }
+
+    @Override
+    public MailboxExecutor getMailboxExecutor() {
+        return mailboxExecutor;
+    }
+}
diff --git a/src/main/java/org/apache/flink/connector/rocketmq/sink/writer/serializer/RocketMQSerializationSchema.java b/src/main/java/org/apache/flink/connector/rocketmq/sink/writer/serializer/RocketMQSerializationSchema.java
new file mode 100644
index 0000000..14591fd
--- /dev/null
+++ b/src/main/java/org/apache/flink/connector/rocketmq/sink/writer/serializer/RocketMQSerializationSchema.java
@@ -0,0 +1,44 @@
+package org.apache.flink.connector.rocketmq.sink.writer.serializer;
+
+import org.apache.flink.annotation.PublicEvolving;
+import org.apache.flink.api.common.serialization.SerializationSchema;
+import org.apache.flink.connector.rocketmq.sink.writer.context.RocketMQSinkContext;
+
+import org.apache.rocketmq.common.message.Message;
+import org.apache.rocketmq.common.message.MessageExt;
+
+import java.io.Serializable;
+
+/**
+ * The serialization schema for how to serialize records into RocketMQ. A serialization schema which
+ * defines how to convert a value of type {@code T} to {@link MessageExt}.
+ *
+ * @param <T> the type of values being serialized
+ */
+@PublicEvolving
+public interface RocketMQSerializationSchema<T> extends Serializable {
+
+    /**
+     * Initialization method for the schema. It is called before the actual working methods {@link
+     * #serialize(Object, RocketMQSinkContext, Long)} and thus suitable for one time setup work.
+     *
+     * <p>The provided {@link SerializationSchema.InitializationContext} can be used to access
+     * additional features such as e.g. registering user metrics.
+     *
+     * @param context Contextual information that can be used during initialization.
+     * @param sinkContext runtime information i.e. partitions, subtaskId
+     */
+    default void open(
+            SerializationSchema.InitializationContext context, RocketMQSinkContext sinkContext)
+            throws Exception {}
+
+    /**
+     * Serializes given element and returns it as a {@link MessageExt}.
+     *
+     * @param element element to be serialized
+     * @param context context to possibly determine target partition
+     * @param timestamp timestamp
+     * @return RocketMQ {@link MessageExt}
+     */
+    Message serialize(T element, RocketMQSinkContext context, Long timestamp);
+}
diff --git a/src/main/java/org/apache/rocketmq/flink/legacy/common/selector/TopicSelector.java b/src/main/java/org/apache/flink/connector/rocketmq/sink/writer/serializer/RocketMQSerializerWrapper.java
similarity index 72%
copy from src/main/java/org/apache/rocketmq/flink/legacy/common/selector/TopicSelector.java
copy to src/main/java/org/apache/flink/connector/rocketmq/sink/writer/serializer/RocketMQSerializerWrapper.java
index a70c599..67b5089 100644
--- a/src/main/java/org/apache/rocketmq/flink/legacy/common/selector/TopicSelector.java
+++ b/src/main/java/org/apache/flink/connector/rocketmq/sink/writer/serializer/RocketMQSerializerWrapper.java
@@ -14,13 +14,14 @@
  * See the License for the specific language governing permissions and
  * limitations under the License.
  */
-package org.apache.rocketmq.flink.legacy.common.selector;
 
-import java.io.Serializable;
+package org.apache.flink.connector.rocketmq.sink.writer.serializer;
 
-public interface TopicSelector<T> extends Serializable {
+import org.apache.flink.annotation.Internal;
 
-    String getTopic(T tuple);
-
-    String getTag(T tuple);
-}
+/**
+ * Wrap the RocketMQ Schema into RocketMQSerializationSchema. We support schema evolution out of box
+ * by this implementation.
+ */
+@Internal
+public class RocketMQSerializerWrapper {}
diff --git a/src/main/java/org/apache/flink/connector/rocketmq/source/InnerConsumer.java b/src/main/java/org/apache/flink/connector/rocketmq/source/InnerConsumer.java
new file mode 100644
index 0000000..09bc777
--- /dev/null
+++ b/src/main/java/org/apache/flink/connector/rocketmq/source/InnerConsumer.java
@@ -0,0 +1,137 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.flink.connector.rocketmq.source;
+
+import org.apache.flink.connector.rocketmq.source.reader.MessageView;
+
+import org.apache.rocketmq.common.message.MessageQueue;
+
+import java.time.Duration;
+import java.util.Collection;
+import java.util.List;
+import java.util.Set;
+import java.util.concurrent.CompletableFuture;
+
+public interface InnerConsumer extends AutoCloseable {
+
+    /** start inner consumer */
+    void start();
+
+    /** Get the consumer group of the consumer. */
+    String getConsumerGroup();
+
+    /**
+     * Fetch message queues of the topic.
+     *
+     * @param topic topic list
+     * @return key is topic, values are message queue collections
+     */
+    CompletableFuture<Collection<MessageQueue>> fetchMessageQueues(String topic);
+
+    /**
+     * Manually assign a list of message queues to this consumer. This interface does not allow for
+     * incremental assignment and will replace the previous assignment (if there is one).
+     *
+     * @param messageQueues Message queues that needs to be assigned.
+     */
+    void assign(Collection<MessageQueue> messageQueues);
+
+    /**
+     * Returns a set of message queues that are assigned to the current consumer. The assignment is
+     * typically performed by a message broker and may change dynamically based on various factors
+     * such as load balancing and consumer group membership.
+     *
+     * @return A set of message queues that are currently assigned to the consumer.
+     */
+    Set<MessageQueue> assignment();
+
+    /**
+     * Fetch data for the topics or partitions specified using assign API
+     *
+     * @return list of message, can be null.
+     */
+    List<MessageView> poll(Duration timeout);
+
+    /** interrupt poll message */
+    void wakeup();
+
+    /**
+     * Suspending message pulling from the message queues.
+     *
+     * @param messageQueues message queues that need to be suspended.
+     */
+    void pause(Collection<MessageQueue> messageQueues);
+
+    /**
+     * Resuming message pulling from the message queues.
+     *
+     * @param messageQueues message queues that need to be resumed.
+     */
+    void resume(Collection<MessageQueue> messageQueues);
+
+    /**
+     * Overrides the fetch offsets that the consumer will use on the next poll. If this method is
+     * invoked for the same message queue more than once, the latest offset will be used on the next
+     * {@link #poll(Duration)}.
+     *
+     * @param messageQueue the message queue to override the fetch offset.
+     * @param offset message offset.
+     */
+    void seek(MessageQueue messageQueue, long offset);
+
+    /**
+     * Seek consumer group previously committed offset
+     *
+     * @param messageQueue rocketmq queue to locate single queue
+     * @return offset for message queue
+     */
+    CompletableFuture<Long /*offset*/> seekCommittedOffset(MessageQueue messageQueue);
+
+    /**
+     * Seek consumer group previously committed offset
+     *
+     * @param messageQueue rocketmq queue to locate single queue
+     * @return offset for message queue
+     */
+    CompletableFuture<Long /*offset*/> seekMinOffset(MessageQueue messageQueue);
+
+    /**
+     * Seek consumer group previously committed offset
+     *
+     * @param messageQueue rocketmq queue to locate single queue
+     * @return offset for message queue
+     */
+    CompletableFuture<Long /*offset*/> seekMaxOffset(MessageQueue messageQueue);
+
+    /**
+     * Seek consumer group previously committed offset
+     *
+     * @param messageQueue rocketmq queue to locate single queue
+     * @return offset for message queue
+     */
+    CompletableFuture<Long /*offset*/> seekOffsetByTimestamp(
+            MessageQueue messageQueue, long timestamp);
+
+    /**
+     * Seek consumer group previously committed offset
+     *
+     * @param messageQueue rocketmq queue to locate single queue
+     * @return offset for message queue
+     */
+    CompletableFuture<Void> commitOffset(MessageQueue messageQueue, long offset);
+}
diff --git a/src/main/java/org/apache/flink/connector/rocketmq/source/InnerConsumerImpl.java b/src/main/java/org/apache/flink/connector/rocketmq/source/InnerConsumerImpl.java
new file mode 100644
index 0000000..edc438b
--- /dev/null
+++ b/src/main/java/org/apache/flink/connector/rocketmq/source/InnerConsumerImpl.java
@@ -0,0 +1,471 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.flink.connector.rocketmq.source;
+
+import org.apache.flink.annotation.VisibleForTesting;
+import org.apache.flink.configuration.Configuration;
+import org.apache.flink.connector.rocketmq.source.enumerator.offset.OffsetsSelector;
+import org.apache.flink.connector.rocketmq.source.reader.MessageView;
+import org.apache.flink.connector.rocketmq.source.reader.MessageViewExt;
+import org.apache.flink.connector.rocketmq.source.util.UtilAll;
+import org.apache.flink.util.FlinkRuntimeException;
+import org.apache.flink.util.StringUtils;
+
+import com.alibaba.fastjson.JSON;
+import org.apache.rocketmq.acl.common.AclClientRPCHook;
+import org.apache.rocketmq.acl.common.SessionCredentials;
+import org.apache.rocketmq.client.consumer.DefaultLitePullConsumer;
+import org.apache.rocketmq.client.consumer.store.ReadOffsetType;
+import org.apache.rocketmq.client.exception.MQClientException;
+import org.apache.rocketmq.common.ThreadFactoryImpl;
+import org.apache.rocketmq.common.message.MessageExt;
+import org.apache.rocketmq.common.message.MessageQueue;
+import org.apache.rocketmq.tools.admin.DefaultMQAdminExt;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import java.lang.management.ManagementFactory;
+import java.time.Duration;
+import java.util.Collection;
+import java.util.Collections;
+import java.util.List;
+import java.util.Map;
+import java.util.Set;
+import java.util.UUID;
+import java.util.concurrent.CompletableFuture;
+import java.util.concurrent.ConcurrentHashMap;
+import java.util.concurrent.ExecutorService;
+import java.util.concurrent.LinkedBlockingQueue;
+import java.util.concurrent.ThreadPoolExecutor;
+import java.util.concurrent.TimeUnit;
+import java.util.function.Function;
+import java.util.stream.Collectors;
+
+public class InnerConsumerImpl implements InnerConsumer {
+
+    private static final Logger LOG = LoggerFactory.getLogger(InnerConsumerImpl.class);
+
+    private final Configuration configuration;
+    private final DefaultMQAdminExt adminExt;
+    private final DefaultLitePullConsumer consumer;
+    private final ExecutorService commonExecutorService;
+
+    public InnerConsumerImpl(Configuration configuration) {
+        this.configuration = configuration;
+        this.commonExecutorService = buildExecutorService(configuration);
+
+        String accessKey = configuration.getString(RocketMQSourceOptions.OPTIONAL_ACCESS_KEY);
+        String secretKey = configuration.getString(RocketMQSourceOptions.OPTIONAL_SECRET_KEY);
+
+        // Note: sync pull thread num may not enough
+        if (!StringUtils.isNullOrWhitespaceOnly(accessKey)
+                && !StringUtils.isNullOrWhitespaceOnly(secretKey)) {
+            AclClientRPCHook aclClientRpcHook =
+                    new AclClientRPCHook(new SessionCredentials(accessKey, secretKey));
+            this.adminExt = new DefaultMQAdminExt(aclClientRpcHook);
+            this.consumer = new DefaultLitePullConsumer(aclClientRpcHook);
+        } else {
+            this.adminExt = new DefaultMQAdminExt();
+            this.consumer = new DefaultLitePullConsumer();
+        }
+
+        String groupId = configuration.getString(RocketMQSourceOptions.CONSUMER_GROUP);
+        String endPoints = configuration.getString(RocketMQSourceOptions.ENDPOINTS);
+
+        this.consumer.setNamesrvAddr(endPoints);
+        this.consumer.setConsumerGroup(groupId);
+        this.consumer.setAutoCommit(false);
+        this.consumer.setVipChannelEnabled(false);
+        this.consumer.setInstanceName(
+                String.join(
+                        "#",
+                        ManagementFactory.getRuntimeMXBean().getName(),
+                        groupId,
+                        UUID.randomUUID().toString()));
+
+        this.adminExt.setNamesrvAddr(endPoints);
+        this.adminExt.setAdminExtGroup(groupId);
+        this.adminExt.setVipChannelEnabled(false);
+        this.adminExt.setInstanceName(
+                String.join(
+                        "#",
+                        ManagementFactory.getRuntimeMXBean().getName(),
+                        groupId,
+                        UUID.randomUUID().toString()));
+    }
+
+    @Override
+    public void start() {
+        try {
+            this.adminExt.start();
+            this.consumer.start();
+            LOG.info(
+                    "RocketMQ consumer started success, group={}, consumerId={}",
+                    this.consumer.getConsumerGroup(),
+                    this.consumer.getInstanceName());
+        } catch (Throwable t) {
+            LOG.error("RocketMQ consumer started failed", t);
+            throw new FlinkRuntimeException("RocketMQ consumer started failed.", t);
+        }
+    }
+
+    @Override
+    public void close() throws Exception {
+        this.commonExecutorService.shutdown();
+        this.adminExt.shutdown();
+        this.consumer.shutdown();
+    }
+
+    private ExecutorService buildExecutorService(Configuration configuration) {
+        int processors = Runtime.getRuntime().availableProcessors();
+        int threadNum =
+                configuration.getInteger(RocketMQSourceOptions.PULL_THREADS_NUM, processors);
+        return new ThreadPoolExecutor(
+                threadNum,
+                threadNum,
+                TimeUnit.MINUTES.toMillis(1),
+                TimeUnit.MILLISECONDS,
+                new LinkedBlockingQueue<>(1024),
+                new ThreadFactoryImpl("RocketMQCommonExecutorThread_"));
+    }
+
+    @Override
+    public String getConsumerGroup() {
+        return configuration.getString(RocketMQSourceOptions.CONSUMER_GROUP);
+    }
+
+    @Override
+    public CompletableFuture<Collection<MessageQueue>> fetchMessageQueues(String topic) {
+        return CompletableFuture.supplyAsync(
+                () -> {
+                    try {
+                        Collection<MessageQueue> result = consumer.fetchMessageQueues(topic);
+                        LOG.info(
+                                "Consumer request topic route for service discovery, topic={}, route={}",
+                                topic,
+                                JSON.toJSONString(result));
+                        return result;
+                    } catch (Exception e) {
+                        LOG.error(
+                                "Consumer request topic route for service discovery, topic={}, nsAddress={}",
+                                topic,
+                                this.consumer.getNamesrvAddr(),
+                                e);
+                    }
+                    return Collections.emptyList();
+                },
+                commonExecutorService);
+    }
+
+    @Override
+    public void assign(Collection<MessageQueue> messageQueues) {
+        this.consumer.assign(messageQueues);
+    }
+
+    @Override
+    public Set<MessageQueue> assignment() {
+        try {
+            return this.consumer.assignment();
+        } catch (MQClientException e) {
+            throw new FlinkRuntimeException(e);
+        }
+    }
+
+    @Override
+    public List<MessageView> poll(Duration timeout) {
+        return this.consumer.poll(timeout.toMillis()).stream()
+                .map((Function<MessageExt, MessageView>) MessageViewExt::new)
+                .collect(Collectors.toList());
+    }
+
+    @Override
+    public void wakeup() {
+        // wakeup long polling
+        try {
+            Set<MessageQueue> assignment = this.consumer.assignment();
+            if (assignment != null) {
+                this.consumer.pause(assignment);
+            }
+        } catch (MQClientException e) {
+            LOG.warn("Consume wakeup long polling failed", e);
+            throw new RuntimeException(e);
+        }
+    }
+
+    @Override
+    public void seek(MessageQueue messageQueue, long offset) {
+        try {
+            this.consumer.seek(messageQueue, offset);
+            LOG.info(
+                    "Consumer current offset has been reset, mq={}, next poll will start from offset={}",
+                    UtilAll.getQueueDescription(messageQueue),
+                    offset);
+        } catch (MQClientException e) {
+            throw new FlinkRuntimeException(e);
+        }
+    }
+
+    @Override
+    public void pause(Collection<MessageQueue> messageQueues) {
+        this.consumer.pause(messageQueues);
+        LOG.info("Consumer pause fetch messages, mq(s)={}", messageQueues);
+    }
+
+    @Override
+    public void resume(Collection<MessageQueue> messageQueues) {
+        this.consumer.resume(messageQueues);
+        LOG.info("Consumer resume fetch messages, mq(s)={}", messageQueues);
+    }
+
+    @Override
+    public CompletableFuture<Long> seekCommittedOffset(MessageQueue messageQueue) {
+        return CompletableFuture.supplyAsync(
+                () -> {
+                    try {
+                        long offset =
+                                consumer.getOffsetStore()
+                                        .readOffset(messageQueue, ReadOffsetType.READ_FROM_STORE);
+                        LOG.error(
+                                "Consumer seek committed offset from remote, mq={}, offset={}",
+                                UtilAll.getQueueDescription(messageQueue),
+                                offset);
+                        return offset;
+                    } catch (Exception e) {
+                        LOG.error(
+                                "Consumer seek committed offset from remote error, mq={}",
+                                UtilAll.getQueueDescription(messageQueue),
+                                e);
+                        throw new RuntimeException(e);
+                    }
+                },
+                commonExecutorService);
+    }
+
+    @Override
+    public CompletableFuture<Long> seekMinOffset(MessageQueue messageQueue) {
+        return CompletableFuture.supplyAsync(
+                () -> {
+                    try {
+                        long offset = adminExt.minOffset(messageQueue);
+                        LOG.info(
+                                "Consumer seek min offset from remote, mq={}, offset={}",
+                                UtilAll.getQueueDescription(messageQueue),
+                                offset);
+                        return offset;
+                    } catch (Exception e) {
+                        LOG.info(
+                                "Consumer seek min offset from remote error, mq={}",
+                                UtilAll.getQueueDescription(messageQueue),
+                                e);
+                        throw new RuntimeException(e);
+                    }
+                },
+                commonExecutorService);
+    }
+
+    @Override
+    public CompletableFuture<Long> seekMaxOffset(MessageQueue messageQueue) {
+        return CompletableFuture.supplyAsync(
+                () -> {
+                    try {
+                        long offset = adminExt.maxOffset(messageQueue);
+                        LOG.info(
+                                "Consumer seek max offset from remote, mq={}, offset={}",
+                                UtilAll.getQueueDescription(messageQueue),
+                                offset);
+                        return offset;
+                    } catch (Exception e) {
+                        LOG.info(
+                                "Consumer seek max offset from remote error, mq={}",
+                                UtilAll.getQueueDescription(messageQueue),
+                                e);
+                        throw new RuntimeException(e);
+                    }
+                },
+                commonExecutorService);
+    }
+
+    @Override
+    public CompletableFuture<Long> seekOffsetByTimestamp(
+            MessageQueue messageQueue, long timestamp) {
+        return CompletableFuture.supplyAsync(
+                () -> {
+                    try {
+                        long offset = adminExt.searchOffset(messageQueue, timestamp);
+                        LOG.info(
+                                "Consumer seek offset by timestamp from remote, mq={}, timestamp={}, offset={}",
+                                UtilAll.getQueueDescription(messageQueue),
+                                timestamp,
+                                offset);
+                        return offset;
+                    } catch (MQClientException e) {
+                        LOG.info(
+                                "Consumer seek offset by timestamp from remote error, mq={}, timestamp={}",
+                                UtilAll.getQueueDescription(messageQueue),
+                                timestamp,
+                                e);
+                        throw new RuntimeException(e);
+                    }
+                },
+                commonExecutorService);
+    }
+
+    @Override
+    public CompletableFuture<Void> commitOffset(MessageQueue messageQueue, long offset) {
+        return CompletableFuture.runAsync(
+                () -> consumer.getOffsetStore().updateOffset(messageQueue, offset, true),
+                commonExecutorService);
+    }
+
+    /** The implementation for offsets retriever with a consumer and an admin client. */
+    @VisibleForTesting
+    public static class RemotingOffsetsRetrieverImpl
+            implements OffsetsSelector.MessageQueueOffsetsRetriever, AutoCloseable {
+
+        private final InnerConsumer innerConsumer;
+
+        public RemotingOffsetsRetrieverImpl(InnerConsumer innerConsumer) {
+            this.innerConsumer = innerConsumer;
+        }
+
+        @Override
+        public void close() throws Exception {
+            this.innerConsumer.close();
+        }
+
+        @Override
+        public Map<MessageQueue, Long> committedOffsets(Collection<MessageQueue> messageQueues) {
+            Map<MessageQueue, Long> result = new ConcurrentHashMap<>();
+            CompletableFuture.allOf(
+                            messageQueues.stream()
+                                    .map(
+                                            messageQueue ->
+                                                    CompletableFuture.supplyAsync(
+                                                                    () ->
+                                                                            innerConsumer
+                                                                                    .seekCommittedOffset(
+                                                                                            messageQueue))
+                                                            .thenAccept(
+                                                                    future -> {
+                                                                        try {
+                                                                            result.put(
+                                                                                    messageQueue,
+                                                                                    future.get());
+                                                                        } catch (Exception e) {
+                                                                            LOG.error(
+                                                                                    "Consumer offsets retriever fetch committed offset error",
+                                                                                    e);
+                                                                        }
+                                                                    }))
+                                    .toArray(CompletableFuture[]::new))
+                    .join();
+            return result;
+        }
+
+        @Override
+        public Map<MessageQueue, Long> minOffsets(Collection<MessageQueue> messageQueues) {
+            Map<MessageQueue, Long> result = new ConcurrentHashMap<>();
+            CompletableFuture.allOf(
+                            messageQueues.stream()
+                                    .map(
+                                            messageQueue ->
+                                                    CompletableFuture.supplyAsync(
+                                                                    () ->
+                                                                            innerConsumer
+                                                                                    .seekMinOffset(
+                                                                                            messageQueue))
+                                                            .thenAccept(
+                                                                    future -> {
+                                                                        try {
+                                                                            result.put(
+                                                                                    messageQueue,
+                                                                                    future.get());
+                                                                        } catch (Exception e) {
+                                                                            LOG.error(
+                                                                                    "Consumer offsets retriever fetch min offset error",
+                                                                                    e);
+                                                                        }
+                                                                    }))
+                                    .toArray(CompletableFuture[]::new))
+                    .join();
+            return result;
+        }
+
+        @Override
+        public Map<MessageQueue, Long> maxOffsets(Collection<MessageQueue> messageQueues) {
+            Map<MessageQueue, Long> result = new ConcurrentHashMap<>();
+            CompletableFuture.allOf(
+                            messageQueues.stream()
+                                    .map(
+                                            messageQueue ->
+                                                    CompletableFuture.supplyAsync(
+                                                                    () ->
+                                                                            innerConsumer
+                                                                                    .seekMaxOffset(
+                                                                                            messageQueue))
+                                                            .thenAccept(
+                                                                    future -> {
+                                                                        try {
+                                                                            result.put(
+                                                                                    messageQueue,
+                                                                                    future.get());
+                                                                        } catch (Exception e) {
+                                                                            LOG.error(
+                                                                                    "Consumer offsets retriever fetch committed offset error",
+                                                                                    e);
+                                                                        }
+                                                                    }))
+                                    .toArray(CompletableFuture[]::new))
+                    .join();
+            return result;
+        }
+
+        @Override
+        public Map<MessageQueue, Long> offsetsForTimes(
+                Map<MessageQueue, Long> messageQueueWithTimeMap) {
+            Map<MessageQueue, Long> result = new ConcurrentHashMap<>();
+            CompletableFuture.allOf(
+                            messageQueueWithTimeMap.entrySet().stream()
+                                    .map(
+                                            entry ->
+                                                    CompletableFuture.supplyAsync(
+                                                                    () ->
+                                                                            innerConsumer
+                                                                                    .seekOffsetByTimestamp(
+                                                                                            entry
+                                                                                                    .getKey(),
+                                                                                            entry
+                                                                                                    .getValue()))
+                                                            .thenAccept(
+                                                                    future -> {
+                                                                        try {
+                                                                            result.put(
+                                                                                    entry.getKey(),
+                                                                                    future.get());
+                                                                        } catch (Exception e) {
+                                                                            LOG.error(
+                                                                                    "Consumer offsets retriever fetch offset by timestamp error",
+                                                                                    e);
+                                                                        }
+                                                                    }))
+                                    .toArray(CompletableFuture[]::new))
+                    .join();
+            return result;
+        }
+    }
+}
diff --git a/src/main/java/org/apache/flink/connector/rocketmq/source/RocketMQSource.java b/src/main/java/org/apache/flink/connector/rocketmq/source/RocketMQSource.java
new file mode 100644
index 0000000..25fd8c7
--- /dev/null
+++ b/src/main/java/org/apache/flink/connector/rocketmq/source/RocketMQSource.java
@@ -0,0 +1,191 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.flink.connector.rocketmq.source;
+
+import org.apache.flink.api.common.serialization.DeserializationSchema;
+import org.apache.flink.api.common.typeinfo.TypeInformation;
+import org.apache.flink.api.connector.source.Boundedness;
+import org.apache.flink.api.connector.source.Source;
+import org.apache.flink.api.connector.source.SourceReader;
+import org.apache.flink.api.connector.source.SourceReaderContext;
+import org.apache.flink.api.connector.source.SplitEnumerator;
+import org.apache.flink.api.connector.source.SplitEnumeratorContext;
+import org.apache.flink.api.java.typeutils.ResultTypeQueryable;
+import org.apache.flink.configuration.Configuration;
+import org.apache.flink.connector.base.source.reader.RecordsWithSplitIds;
+import org.apache.flink.connector.base.source.reader.splitreader.SplitReader;
+import org.apache.flink.connector.base.source.reader.synchronization.FutureCompletingBlockingQueue;
+import org.apache.flink.connector.rocketmq.source.enumerator.RocketMQSourceEnumState;
+import org.apache.flink.connector.rocketmq.source.enumerator.RocketMQSourceEnumStateSerializer;
+import org.apache.flink.connector.rocketmq.source.enumerator.RocketMQSourceEnumerator;
+import org.apache.flink.connector.rocketmq.source.enumerator.offset.OffsetsSelector;
+import org.apache.flink.connector.rocketmq.source.metrics.RocketMQSourceReaderMetrics;
+import org.apache.flink.connector.rocketmq.source.reader.MessageView;
+import org.apache.flink.connector.rocketmq.source.reader.RocketMQRecordEmitter;
+import org.apache.flink.connector.rocketmq.source.reader.RocketMQSourceFetcherManager;
+import org.apache.flink.connector.rocketmq.source.reader.RocketMQSourceReader;
+import org.apache.flink.connector.rocketmq.source.reader.RocketMQSplitReader;
+import org.apache.flink.connector.rocketmq.source.reader.deserializer.RocketMQDeserializationSchema;
+import org.apache.flink.connector.rocketmq.source.split.RocketMQPartitionSplitSerializer;
+import org.apache.flink.connector.rocketmq.source.split.RocketMQSourceSplit;
+import org.apache.flink.core.io.SimpleVersionedSerializer;
+import org.apache.flink.metrics.MetricGroup;
+import org.apache.flink.util.UserCodeClassLoader;
+
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import java.util.function.Supplier;
+
+/** The Source implementation of RocketMQ. */
+public class RocketMQSource<OUT>
+        implements Source<OUT, RocketMQSourceSplit, RocketMQSourceEnumState>,
+                ResultTypeQueryable<OUT> {
+
+    private static final long serialVersionUID = -1L;
+    private static final Logger log = LoggerFactory.getLogger(RocketMQSource.class);
+
+    // Users can specify the starting / stopping offset initializer.
+    private final OffsetsSelector startingOffsetsSelector;
+    private final OffsetsSelector stoppingOffsetsSelector;
+
+    // The configurations.
+    private final Configuration configuration;
+
+    // Boundedness
+    private final Boundedness boundedness;
+
+    // RocketMQ DeserializationSchema
+    private final RocketMQDeserializationSchema<OUT> deserializationSchema;
+
+    public RocketMQSource(
+            OffsetsSelector startingOffsetsSelector,
+            OffsetsSelector stoppingOffsetsSelector,
+            Boundedness boundedness,
+            RocketMQDeserializationSchema<OUT> deserializationSchema,
+            Configuration configuration) {
+        this.startingOffsetsSelector = startingOffsetsSelector;
+        this.stoppingOffsetsSelector = stoppingOffsetsSelector;
+        this.boundedness = boundedness;
+        this.deserializationSchema = deserializationSchema;
+        this.configuration = configuration;
+    }
+
+    /**
+     * Get a RocketMQSourceBuilder to build a {@link RocketMQSourceBuilder}.
+     *
+     * @return a RocketMQ source builder.
+     */
+    public static <OUT> RocketMQSourceBuilder<OUT> builder() {
+        return new RocketMQSourceBuilder<>();
+    }
+
+    @Override
+    public Boundedness getBoundedness() {
+        return this.boundedness;
+    }
+
+    @Override
+    public SourceReader<OUT, RocketMQSourceSplit> createReader(SourceReaderContext readerContext)
+            throws Exception {
+
+        FutureCompletingBlockingQueue<RecordsWithSplitIds<MessageView>> elementsQueue =
+                new FutureCompletingBlockingQueue<>();
+
+        deserializationSchema.open(
+                new DeserializationSchema.InitializationContext() {
+                    @Override
+                    public MetricGroup getMetricGroup() {
+                        return readerContext.metricGroup().addGroup("deserializer");
+                    }
+
+                    @Override
+                    public UserCodeClassLoader getUserCodeClassLoader() {
+                        return readerContext.getUserCodeClassLoader();
+                    }
+                });
+
+        final RocketMQSourceReaderMetrics rocketMQSourceReaderMetrics =
+                new RocketMQSourceReaderMetrics(readerContext.metricGroup());
+
+        Supplier<SplitReader<MessageView, RocketMQSourceSplit>> splitReaderSupplier =
+                () ->
+                        new RocketMQSplitReader<>(
+                                configuration,
+                                readerContext,
+                                deserializationSchema,
+                                rocketMQSourceReaderMetrics);
+
+        RocketMQSourceFetcherManager rocketmqSourceFetcherManager =
+                new RocketMQSourceFetcherManager(
+                        elementsQueue, splitReaderSupplier, (ignore) -> {});
+
+        RocketMQRecordEmitter<OUT> recordEmitter =
+                new RocketMQRecordEmitter<>(deserializationSchema);
+
+        return new RocketMQSourceReader<>(
+                elementsQueue,
+                rocketmqSourceFetcherManager,
+                recordEmitter,
+                configuration,
+                readerContext,
+                rocketMQSourceReaderMetrics);
+    }
+
+    @Override
+    public SplitEnumerator<RocketMQSourceSplit, RocketMQSourceEnumState> createEnumerator(
+            SplitEnumeratorContext<RocketMQSourceSplit> enumContext) {
+
+        return new RocketMQSourceEnumerator(
+                startingOffsetsSelector,
+                stoppingOffsetsSelector,
+                boundedness,
+                configuration,
+                enumContext);
+    }
+
+    @Override
+    public SplitEnumerator<RocketMQSourceSplit, RocketMQSourceEnumState> restoreEnumerator(
+            SplitEnumeratorContext<RocketMQSourceSplit> enumContext,
+            RocketMQSourceEnumState checkpoint) {
+
+        return new RocketMQSourceEnumerator(
+                startingOffsetsSelector,
+                stoppingOffsetsSelector,
+                boundedness,
+                configuration,
+                enumContext,
+                checkpoint.getCurrentSplitAssignment());
+    }
+
+    @Override
+    public SimpleVersionedSerializer<RocketMQSourceSplit> getSplitSerializer() {
+        return new RocketMQPartitionSplitSerializer();
+    }
+
+    @Override
+    public SimpleVersionedSerializer<RocketMQSourceEnumState> getEnumeratorCheckpointSerializer() {
+        return new RocketMQSourceEnumStateSerializer();
+    }
+
+    @Override
+    public TypeInformation<OUT> getProducedType() {
+        return deserializationSchema.getProducedType();
+    }
+}
diff --git a/src/main/java/org/apache/flink/connector/rocketmq/source/RocketMQSourceBuilder.java b/src/main/java/org/apache/flink/connector/rocketmq/source/RocketMQSourceBuilder.java
new file mode 100644
index 0000000..00237a4
--- /dev/null
+++ b/src/main/java/org/apache/flink/connector/rocketmq/source/RocketMQSourceBuilder.java
@@ -0,0 +1,201 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.flink.connector.rocketmq.source;
+
+import org.apache.flink.annotation.PublicEvolving;
+import org.apache.flink.api.common.serialization.DeserializationSchema;
+import org.apache.flink.api.connector.source.Boundedness;
+import org.apache.flink.configuration.ConfigOption;
+import org.apache.flink.configuration.Configuration;
+import org.apache.flink.connector.rocketmq.common.config.RocketMQConfigBuilder;
+import org.apache.flink.connector.rocketmq.source.enumerator.offset.OffsetsSelector;
+import org.apache.flink.connector.rocketmq.source.enumerator.offset.OffsetsSelectorNoStopping;
+import org.apache.flink.connector.rocketmq.source.reader.deserializer.RocketMQDeserializationSchema;
+
+import org.apache.commons.lang3.StringUtils;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import java.util.Arrays;
+import java.util.List;
+import java.util.Properties;
+
+import static org.apache.flink.connector.rocketmq.source.RocketMQSourceOptions.SOURCE_CONFIG_VALIDATOR;
+
+@PublicEvolving
+public class RocketMQSourceBuilder<OUT> {
+
+    private static final Logger log = LoggerFactory.getLogger(RocketMQSourceBuilder.class);
+
+    // The configurations.
+    protected final RocketMQConfigBuilder configBuilder;
+
+    // Users can specify the starting / stopping offset initializer.
+    private OffsetsSelector minOffsetsSelector;
+    private OffsetsSelector maxOffsetsSelector;
+
+    // Boundedness
+    private Boundedness boundedness;
+
+    // Deserialization Schema
+    private RocketMQDeserializationSchema<OUT> deserializationSchema;
+
+    public RocketMQSourceBuilder() {
+        this.configBuilder = new RocketMQConfigBuilder();
+        this.setMinOffsets(OffsetsSelector.committedOffsets());
+        this.setUnbounded(new OffsetsSelectorNoStopping());
+    }
+
+    /**
+     * Configure the access point with which the SDK should communicate.
+     *
+     * @param endpoints address of service.
+     * @return this RocketMQSourceBuilder.
+     */
+    public RocketMQSourceBuilder<OUT> setEndpoints(String endpoints) {
+        this.configBuilder.set(RocketMQSourceOptions.ENDPOINTS, endpoints);
+        return this;
+    }
+
+    /**
+     * Sets the consumer group id of the RocketMQSource.
+     *
+     * @param groupId the group id of the RocketMQSource.
+     * @return this RocketMQSourceBuilder.
+     */
+    public RocketMQSourceBuilder<OUT> setGroupId(String groupId) {
+        this.configBuilder.set(RocketMQSourceOptions.CONSUMER_GROUP, groupId);
+        return this;
+    }
+
+    /**
+     * Set a list of topics the RocketMQSource should consume from. All the topics in the list
+     * should have existed in the RocketMQ cluster. Otherwise, an exception will be thrown.
+     *
+     * @param topics the list of topics to consume from.
+     * @return this RocketMQSourceBuilder.
+     */
+    public RocketMQSourceBuilder<OUT> setTopics(List<String> topics) {
+        this.configBuilder.set(
+                RocketMQSourceOptions.TOPIC,
+                StringUtils.join(topics, RocketMQSourceOptions.TOPIC_SEPARATOR));
+        return this;
+    }
+
+    /**
+     * Set a list of topics the RocketMQSource should consume from. All the topics in the list
+     * should have existed in the RocketMQ cluster. Otherwise, an exception will be thrown.
+     *
+     * @param topics the list of topics to consume from.
+     * @return this RocketMQSourceBuilder.
+     */
+    public RocketMQSourceBuilder<OUT> setTopics(String... topics) {
+        return this.setTopics(Arrays.asList(topics));
+    }
+
+    public RocketMQSourceBuilder<OUT> setMinOffsets(OffsetsSelector offsetsSelector) {
+        this.minOffsetsSelector = offsetsSelector;
+        return this;
+    }
+
+    public RocketMQSourceBuilder<OUT> setUnbounded(OffsetsSelector stoppingOffsetsSelector) {
+        this.boundedness = Boundedness.CONTINUOUS_UNBOUNDED;
+        this.maxOffsetsSelector = stoppingOffsetsSelector;
+        return this;
+    }
+
+    public RocketMQSourceBuilder<OUT> setBounded(OffsetsSelector stoppingOffsetsSelector) {
+        this.boundedness = Boundedness.BOUNDED;
+        this.maxOffsetsSelector = stoppingOffsetsSelector;
+        return this;
+    }
+
+    public RocketMQSourceBuilder<OUT> setDeserializer(
+            RocketMQDeserializationSchema<OUT> recordDeserializer) {
+        this.deserializationSchema = recordDeserializer;
+        return this;
+    }
+
+    public RocketMQSourceBuilder<OUT> setBodyOnlyDeserializer(
+            DeserializationSchema<OUT> deserializationSchema) {
+        this.deserializationSchema =
+                RocketMQDeserializationSchema.flinkBodyOnlySchema(deserializationSchema);
+        return this;
+    }
+
+    /**
+     * Set an arbitrary property for the RocketMQ source. The valid keys can be found in {@link
+     * RocketMQSourceOptions}. Make sure the option could be set only once or with same value.
+     *
+     * @param key the key of the property.
+     * @param value the value of the property.
+     * @return this RocketMQSourceBuilder.
+     */
+    public <T> RocketMQSourceBuilder<OUT> setConfig(ConfigOption<T> key, T value) {
+        configBuilder.set(key, value);
+        return this;
+    }
+
+    /**
+     * Set arbitrary properties for the RocketMQ source. The valid keys can be found in {@link
+     * RocketMQSourceOptions}.
+     *
+     * @param config the config to set for the RocketMQSourceBuilder.
+     * @return this RocketMQSourceBuilder.
+     */
+    public RocketMQSourceBuilder<OUT> setConfig(Configuration config) {
+        configBuilder.set(config);
+        return this;
+    }
+
+    /**
+     * Set arbitrary properties for the RocketMQ source. This method is mainly used for future flink
+     * SQL binding.
+     *
+     * @param properties the config properties to set for the RocketMQSource.
+     * @return this RocketMQSourceBuilder.
+     */
+    public RocketMQSourceBuilder<OUT> setProperties(Properties properties) {
+        configBuilder.set(properties);
+        return this;
+    }
+
+    /**
+     * Build the {@link RocketMQSource}.
+     *
+     * @return a RocketMQSource with the settings made for this builder.
+     */
+    public RocketMQSource<OUT> build() {
+        sanityCheck();
+        parseAndSetRequiredProperties();
+
+        Configuration configuration = configBuilder.build(SOURCE_CONFIG_VALIDATOR);
+
+        return new RocketMQSource<>(
+                minOffsetsSelector,
+                maxOffsetsSelector,
+                boundedness,
+                deserializationSchema,
+                configuration);
+    }
+
+    // ------------- private helpers  --------------
+    private void sanityCheck() {}
+
+    private void parseAndSetRequiredProperties() {}
+}
diff --git a/src/main/java/org/apache/flink/connector/rocketmq/source/RocketMQSourceOptions.java b/src/main/java/org/apache/flink/connector/rocketmq/source/RocketMQSourceOptions.java
new file mode 100644
index 0000000..85eb71b
--- /dev/null
+++ b/src/main/java/org/apache/flink/connector/rocketmq/source/RocketMQSourceOptions.java
@@ -0,0 +1,187 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.flink.connector.rocketmq.source;
+
+import org.apache.flink.configuration.ConfigOption;
+import org.apache.flink.configuration.ConfigOptions;
+import org.apache.flink.connector.rocketmq.common.config.RocketMQConfigValidator;
+import org.apache.flink.connector.rocketmq.common.config.RocketMQOptions;
+import org.apache.flink.connector.rocketmq.legacy.RocketMQConfig;
+import org.apache.flink.connector.rocketmq.source.enumerator.allocate.AllocateStrategyFactory;
+
+/** Includes config options of RocketMQ connector type. */
+public class RocketMQSourceOptions extends RocketMQOptions {
+
+    public static final RocketMQConfigValidator SOURCE_CONFIG_VALIDATOR =
+            RocketMQConfigValidator.builder().build();
+
+    // rocketmq source connector config prefix.
+    public static final String CONSUMER_PREFIX = "rocketmq.source.";
+
+    public static final String TOPIC_SEPARATOR = ";";
+
+    public static final ConfigOption<String> CONSUMER_GROUP =
+            ConfigOptions.key(CONSUMER_PREFIX + "group")
+                    .stringType()
+                    .noDefaultValue()
+                    .withDescription(
+                            "The name of the consumer group, used to identify a type of consumer");
+
+    public static final ConfigOption<String> TOPIC =
+            ConfigOptions.key(CONSUMER_PREFIX + "topic")
+                    .stringType()
+                    .noDefaultValue()
+                    .withDescription("The name of the subscribe topic");
+
+    public static final ConfigOption<Boolean> OPTIONAL_USE_NEW_API =
+            ConfigOptions.key(CONSUMER_PREFIX + "api.new.enable").booleanType().defaultValue(true);
+
+    public static final ConfigOption<String> OPTIONAL_TAG =
+            ConfigOptions.key(CONSUMER_PREFIX + "filter.tag")
+                    .stringType()
+                    .defaultValue("*")
+                    .withDescription(
+                            "for message filter, rocketmq only support single filter option");
+
+    public static final ConfigOption<String> OPTIONAL_SQL =
+            ConfigOptions.key(CONSUMER_PREFIX + "filter.sql").stringType().noDefaultValue();
+
+    public static final ConfigOption<String> OPTIONAL_STARTUP_SCAN_MODE =
+            ConfigOptions.key(CONSUMER_PREFIX + "startup.scan.mode")
+                    .stringType()
+                    .defaultValue("latest");
+
+    /** for initialization consume offset */
+    public static final ConfigOption<Long> OPTIONAL_STARTUP_OFFSET_SPECIFIC =
+            ConfigOptions.key(CONSUMER_PREFIX + "startup.offset.specific")
+                    .longType()
+                    .defaultValue(RocketMQConfig.DEFAULT_START_MESSAGE_OFFSET);
+
+    public static final ConfigOption<String> OPTIONAL_STARTUP_OFFSET_STRATEGY =
+            ConfigOptions.key(CONSUMER_PREFIX + "startup.offset.strategy")
+                    .stringType()
+                    .noDefaultValue();
+
+    public static final ConfigOption<String> OPTIONAL_STARTUP_OFFSET_DATE =
+            ConfigOptions.key(CONSUMER_PREFIX + "startup.offset.date")
+                    .stringType()
+                    .noDefaultValue();
+
+    public static final ConfigOption<Long> OPTIONAL_STARTUP_OFFSET_TIMESTAMP =
+            ConfigOptions.key(CONSUMER_PREFIX + "startup.offset.timestamp")
+                    .longType()
+                    .noDefaultValue();
+
+    public static final ConfigOption<String> OPTIONAL_STOP_OFFSET_TIMESTAMP =
+            ConfigOptions.key(CONSUMER_PREFIX + "stop.offset.timestamp")
+                    .stringType()
+                    .noDefaultValue();
+
+    public static final ConfigOption<Boolean> OPTIONAL_COLUMN_ERROR_DEBUG =
+            ConfigOptions.key(CONSUMER_PREFIX + "column.error.debug")
+                    .booleanType()
+                    .defaultValue(true)
+                    .withDescription("If object deserialize failed, would print error message");
+
+    public static final ConfigOption<String> ALLOCATE_MESSAGE_QUEUE_STRATEGY =
+            ConfigOptions.key(CONSUMER_PREFIX + "allocate.strategy")
+                    .stringType()
+                    .defaultValue(AllocateStrategyFactory.STRATEGY_NAME_CONSISTENT_HASH)
+                    .withDescription("The load balancing strategy algorithm");
+
+    // pull message limit
+    public static final ConfigOption<Integer> PULL_THREADS_NUM =
+            ConfigOptions.key(CONSUMER_PREFIX + "pull.threads.num")
+                    .intType()
+                    .defaultValue(20)
+                    .withDescription("The number of pull threads set");
+
+    public static final ConfigOption<Long> PULL_BATCH_SIZE =
+            ConfigOptions.key(CONSUMER_PREFIX + "pull.batch.size")
+                    .longType()
+                    .defaultValue(32L)
+                    .withDescription("The maximum number of messages pulled each time");
+
+    public static final ConfigOption<Long> PULL_THRESHOLD_FOR_QUEUE =
+            ConfigOptions.key(CONSUMER_PREFIX + "pull.threshold.queue")
+                    .longType()
+                    .defaultValue(1000L)
+                    .withDescription("The queue level flow control threshold");
+
+    public static final ConfigOption<Long> PULL_THRESHOLD_FOR_ALL =
+            ConfigOptions.key(CONSUMER_PREFIX + "pull.threshold.all")
+                    .longType()
+                    .defaultValue(10 * 1000L)
+                    .withDescription("The threshold for flow control of consumed requests");
+
+    public static final ConfigOption<Long> PULL_TIMEOUT_MILLIS =
+            ConfigOptions.key(CONSUMER_PREFIX + "pull.rpc.timeout")
+                    .longType()
+                    .defaultValue(20 * 1000L)
+                    .withDescription("The polling timeout setting");
+
+    public static final ConfigOption<Long> PULL_TIME_DELAY_MILLS_WHEN_EXCEPTION =
+            ConfigOptions.key(CONSUMER_PREFIX + "pull.rpc.exception.delay")
+                    .longType()
+                    .defaultValue(3 * 1000L)
+                    .withDescription(
+                            "The maximum time that a connection will be suspended "
+                                    + "for in long polling by the broker");
+
+    public static final ConfigOption<Long> PULL_TIMEOUT_LONG_POLLING_SUSPEND =
+            ConfigOptions.key(CONSUMER_PREFIX + "pull.suspend.timeout")
+                    .longType()
+                    .defaultValue(30 * 1000L)
+                    .withDescription(
+                            "The maximum wait time for a response from the broker "
+                                    + "in long polling by the client");
+
+    /** for auto commit offset to rocketmq server */
+    public static final ConfigOption<Boolean> AUTO_COMMIT_OFFSET =
+            ConfigOptions.key(CONSUMER_PREFIX + "offset.commit.auto")
+                    .booleanType()
+                    .defaultValue(true)
+                    .withDescription("The setting for automatic commit of offset");
+
+    public static final ConfigOption<Long> AUTO_COMMIT_OFFSET_INTERVAL =
+            ConfigOptions.key(CONSUMER_PREFIX + "offset.commit.interval")
+                    .longType()
+                    .defaultValue(5 * 1000L)
+                    .withDescription(
+                            "Applies to Consumer, the interval for persisting consumption progress");
+
+    public static final ConfigOption<Boolean> COMMIT_OFFSETS_ON_CHECKPOINT =
+            ConfigOptions.key(CONSUMER_PREFIX + "offset.commit.checkpoint")
+                    .booleanType()
+                    .defaultValue(true)
+                    .withDescription("Whether to commit consuming offset on checkpoint.");
+
+    /** for message trace, suggest not enable when heavy traffic */
+    public static final ConfigOption<Boolean> ENABLE_MESSAGE_TRACE =
+            ConfigOptions.key(CONSUMER_PREFIX + "trace.enable")
+                    .booleanType()
+                    .defaultValue(true)
+                    .withDescription("The flag for message tracing");
+
+    public static final ConfigOption<String> CUSTOMIZED_TRACE_TOPIC =
+            ConfigOptions.key(CONSUMER_PREFIX + "trace.topic")
+                    .stringType()
+                    .noDefaultValue()
+                    .withDescription("The name of the topic for message tracing");
+}
diff --git a/src/main/java/org/apache/flink/connector/rocketmq/source/config/OffsetVerification.java b/src/main/java/org/apache/flink/connector/rocketmq/source/config/OffsetVerification.java
new file mode 100644
index 0000000..4fee0d6
--- /dev/null
+++ b/src/main/java/org/apache/flink/connector/rocketmq/source/config/OffsetVerification.java
@@ -0,0 +1,49 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.flink.connector.rocketmq.source.config;
+
+import org.apache.flink.annotation.Internal;
+import org.apache.flink.annotation.PublicEvolving;
+import org.apache.flink.configuration.DescribedEnum;
+import org.apache.flink.configuration.description.InlineElement;
+
+import static org.apache.flink.configuration.description.TextElement.text;
+
+/** The enum class for defining the offset verify behavior. */
+@PublicEvolving
+public enum OffsetVerification implements DescribedEnum {
+
+    /** We would just fail the consuming. */
+    FAIL_ON_MISMATCH(
+            text("Fail the consuming from RocketMQ when we don't find the related offset.")),
+
+    /** Print one warn message and start consuming from the valid offset. */
+    WARN_ON_MISMATCH(text("Print a warn message and start consuming from the valid offset."));
+
+    private final InlineElement desc;
+
+    OffsetVerification(InlineElement desc) {
+        this.desc = desc;
+    }
+
+    @Internal
+    @Override
+    public InlineElement getDescription() {
+        return desc;
+    }
+}
diff --git a/src/main/java/org/apache/rocketmq/flink/source/enumerator/RocketMQSourceEnumState.java b/src/main/java/org/apache/flink/connector/rocketmq/source/enumerator/RocketMQSourceEnumState.java
similarity index 64%
rename from src/main/java/org/apache/rocketmq/flink/source/enumerator/RocketMQSourceEnumState.java
rename to src/main/java/org/apache/flink/connector/rocketmq/source/enumerator/RocketMQSourceEnumState.java
index a23139f..aa29578 100644
--- a/src/main/java/org/apache/rocketmq/flink/source/enumerator/RocketMQSourceEnumState.java
+++ b/src/main/java/org/apache/flink/connector/rocketmq/source/enumerator/RocketMQSourceEnumState.java
@@ -16,23 +16,25 @@
  * limitations under the License.
  */
 
-package org.apache.rocketmq.flink.source.enumerator;
+package org.apache.flink.connector.rocketmq.source.enumerator;
 
-import org.apache.rocketmq.flink.source.split.RocketMQPartitionSplit;
+import org.apache.flink.annotation.Internal;
 
-import java.util.List;
-import java.util.Map;
+import org.apache.rocketmq.common.message.MessageQueue;
+
+import java.util.Set;
 
 /** The state of RocketMQ source enumerator. */
+@Internal
 public class RocketMQSourceEnumState {
 
-    private final Map<Integer, List<RocketMQPartitionSplit>> currentAssignment;
+    private final Set<MessageQueue> currentSplitAssignment;
 
-    RocketMQSourceEnumState(Map<Integer, List<RocketMQPartitionSplit>> currentAssignment) {
-        this.currentAssignment = currentAssignment;
+    public RocketMQSourceEnumState(Set<MessageQueue> currentSplitAssignment) {
+        this.currentSplitAssignment = currentSplitAssignment;
     }
 
-    public Map<Integer, List<RocketMQPartitionSplit>> getCurrentAssignment() {
-        return currentAssignment;
+    public Set<MessageQueue> getCurrentSplitAssignment() {
+        return currentSplitAssignment;
     }
 }
diff --git a/src/main/java/org/apache/flink/connector/rocketmq/source/enumerator/RocketMQSourceEnumStateSerializer.java b/src/main/java/org/apache/flink/connector/rocketmq/source/enumerator/RocketMQSourceEnumStateSerializer.java
new file mode 100644
index 0000000..7589ba4
--- /dev/null
+++ b/src/main/java/org/apache/flink/connector/rocketmq/source/enumerator/RocketMQSourceEnumStateSerializer.java
@@ -0,0 +1,103 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.flink.connector.rocketmq.source.enumerator;
+
+import org.apache.flink.core.io.SimpleVersionedSerializer;
+
+import com.alibaba.fastjson.JSON;
+import org.apache.rocketmq.common.message.MessageQueue;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import java.io.ByteArrayInputStream;
+import java.io.ByteArrayOutputStream;
+import java.io.DataInputStream;
+import java.io.DataOutputStream;
+import java.io.IOException;
+import java.util.HashSet;
+import java.util.Set;
+
+/** The {@link SimpleVersionedSerializer Serializer} for the enumerator state of RocketMQ source. */
+public class RocketMQSourceEnumStateSerializer
+        implements SimpleVersionedSerializer<RocketMQSourceEnumState> {
+
+    private static final Logger LOG =
+            LoggerFactory.getLogger(RocketMQSourceEnumStateSerializer.class);
+
+    private static final int CURRENT_VERSION = 0;
+
+    @Override
+    public int getVersion() {
+        return CURRENT_VERSION;
+    }
+
+    @Override
+    public byte[] serialize(RocketMQSourceEnumState enumState) throws IOException {
+        Set<MessageQueue> assignments = enumState.getCurrentSplitAssignment();
+
+        LOG.info("Serializer enumerator state, {}", JSON.toJSONString(enumState));
+
+        try (ByteArrayOutputStream baos = new ByteArrayOutputStream();
+                DataOutputStream out = new DataOutputStream(baos)) {
+
+            out.writeInt(assignments.size());
+            for (MessageQueue assignment : assignments) {
+                out.writeUTF(assignment.getBrokerName());
+                out.writeUTF(assignment.getTopic());
+                out.writeInt(assignment.getQueueId());
+            }
+            out.flush();
+
+            return baos.toByteArray();
+        }
+    }
+
+    @Override
+    public RocketMQSourceEnumState deserialize(int version, byte[] serialized) throws IOException {
+        // Check whether the version of serialized bytes is supported.
+        if (version == CURRENT_VERSION) {
+            Set<MessageQueue> queues = deserializeMessageQueue(serialized);
+            return new RocketMQSourceEnumState(queues);
+        }
+        throw new IOException(
+                String.format(
+                        "The bytes are serialized with version %d, "
+                                + "while this deserializer only supports version up to %d",
+                        version, getVersion()));
+    }
+
+    private Set<MessageQueue> deserializeMessageQueue(byte[] serialized) throws IOException {
+        try (ByteArrayInputStream bais = new ByteArrayInputStream(serialized);
+                DataInputStream in = new DataInputStream(bais)) {
+
+            int size = in.readInt();
+            HashSet<MessageQueue> result = new HashSet<>();
+            for (int i = 0; i < size; i++) {
+                String brokerName = in.readUTF();
+                String topic = in.readUTF();
+                int queueId = in.readInt();
+
+                MessageQueue queue = new MessageQueue(brokerName, topic, queueId);
+                result.add(queue);
+            }
+
+            return result;
+        }
+    }
+}
diff --git a/src/main/java/org/apache/flink/connector/rocketmq/source/enumerator/RocketMQSourceEnumerator.java b/src/main/java/org/apache/flink/connector/rocketmq/source/enumerator/RocketMQSourceEnumerator.java
new file mode 100644
index 0000000..6103444
--- /dev/null
+++ b/src/main/java/org/apache/flink/connector/rocketmq/source/enumerator/RocketMQSourceEnumerator.java
@@ -0,0 +1,429 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.flink.connector.rocketmq.source.enumerator;
+
+import org.apache.flink.annotation.Internal;
+import org.apache.flink.annotation.VisibleForTesting;
+import org.apache.flink.api.connector.source.Boundedness;
+import org.apache.flink.api.connector.source.SourceReader;
+import org.apache.flink.api.connector.source.SplitEnumerator;
+import org.apache.flink.api.connector.source.SplitEnumeratorContext;
+import org.apache.flink.api.connector.source.SplitsAssignment;
+import org.apache.flink.configuration.Configuration;
+import org.apache.flink.connector.rocketmq.source.InnerConsumer;
+import org.apache.flink.connector.rocketmq.source.InnerConsumerImpl;
+import org.apache.flink.connector.rocketmq.source.RocketMQSourceOptions;
+import org.apache.flink.connector.rocketmq.source.enumerator.allocate.AllocateStrategy;
+import org.apache.flink.connector.rocketmq.source.enumerator.allocate.AllocateStrategyFactory;
+import org.apache.flink.connector.rocketmq.source.enumerator.offset.OffsetsSelector;
+import org.apache.flink.connector.rocketmq.source.split.RocketMQSourceSplit;
+import org.apache.flink.util.FlinkRuntimeException;
+
+import com.alibaba.fastjson.JSON;
+import com.google.common.collect.Sets;
+import org.apache.rocketmq.common.message.MessageQueue;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import javax.annotation.Nullable;
+
+import java.util.ArrayList;
+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;
+import java.util.stream.Collectors;
+import java.util.stream.Stream;
+
+/** The enumerator class for RocketMQ source. */
+@Internal
+public class RocketMQSourceEnumerator
+        implements SplitEnumerator<RocketMQSourceSplit, RocketMQSourceEnumState> {
+
+    private static final Logger log = LoggerFactory.getLogger(RocketMQSourceEnumerator.class);
+
+    private final Configuration configuration;
+    private final SplitEnumeratorContext<RocketMQSourceSplit> context;
+    private final Boundedness boundedness;
+
+    private InnerConsumer consumer;
+
+    // Users can specify the starting / stopping offset initializer.
+    private final AllocateStrategy allocateStrategy;
+    private final OffsetsSelector startingOffsetsSelector;
+    private final OffsetsSelector stoppingOffsetsSelector;
+
+    // The internal states of the enumerator.
+    // This set is only accessed by the partition discovery callable in the callAsync() method.
+    // The current assignment by reader id. Only accessed by the coordinator thread.
+    // The discovered and initialized partition splits that are waiting for owner reader to be
+    // ready.
+    private final Set<MessageQueue> allocatedSet;
+    private final Map<Integer, Set<RocketMQSourceSplit>> pendingSplitAssignmentMap;
+
+    // Param from configuration
+    private final String groupId;
+    private final long partitionDiscoveryIntervalMs;
+
+    public RocketMQSourceEnumerator(
+            OffsetsSelector startingOffsetsSelector,
+            OffsetsSelector stoppingOffsetsSelector,
+            Boundedness boundedness,
+            Configuration configuration,
+            SplitEnumeratorContext<RocketMQSourceSplit> context) {
+
+        this(
+                startingOffsetsSelector,
+                stoppingOffsetsSelector,
+                boundedness,
+                configuration,
+                context,
+                new HashSet<>());
+    }
+
+    public RocketMQSourceEnumerator(
+            OffsetsSelector startingOffsetsSelector,
+            OffsetsSelector stoppingOffsetsSelector,
+            Boundedness boundedness,
+            Configuration configuration,
+            SplitEnumeratorContext<RocketMQSourceSplit> context,
+            Set<MessageQueue> currentSplitAssignment) {
+        this.configuration = configuration;
+        this.context = context;
+        this.boundedness = boundedness;
+
+        // Support allocate splits to reader
+        this.pendingSplitAssignmentMap = new ConcurrentHashMap<>();
+        this.allocatedSet = new HashSet<>(currentSplitAssignment);
+        this.allocateStrategy =
+                AllocateStrategyFactory.getStrategy(
+                        configuration, context, new RocketMQSourceEnumState(allocatedSet));
+
+        // For rocketmq setting
+        this.groupId = configuration.getString(RocketMQSourceOptions.CONSUMER_GROUP);
+        this.startingOffsetsSelector = startingOffsetsSelector;
+        this.stoppingOffsetsSelector = stoppingOffsetsSelector;
+        this.partitionDiscoveryIntervalMs =
+                configuration.getLong(RocketMQSourceOptions.PARTITION_DISCOVERY_INTERVAL_MS);
+    }
+
+    @Override
+    public void start() {
+        consumer = new InnerConsumerImpl(configuration);
+        consumer.start();
+
+        if (partitionDiscoveryIntervalMs > 0) {
+            log.info(
+                    "Starting the RocketMQSourceEnumerator for consumer group {} "
+                            + "with partition discovery interval of {} ms.",
+                    groupId,
+                    partitionDiscoveryIntervalMs);
+
+            context.callAsync(
+                    this::requestServiceDiscovery,
+                    this::handleSourceQueueChange,
+                    0,
+                    partitionDiscoveryIntervalMs);
+        } else {
+            log.info(
+                    "Starting the RocketMQSourceEnumerator for consumer group {} "
+                            + "without periodic partition discovery.",
+                    groupId);
+
+            context.callAsync(this::requestServiceDiscovery, this::handleSourceQueueChange);
+        }
+    }
+
+    @Override
+    public void handleSplitRequest(int subtaskId, @Nullable String requesterHostname) {
+        // the rocketmq source pushes splits eagerly, rather than act upon split requests
+    }
+
+    /**
+     * Add a split back to the split enumerator. It will only happen when a {@link SourceReader}
+     * fails and there are splits assigned to it after the last successful checkpoint.
+     *
+     * @param splits The split to add back to the enumerator for reassignment.
+     * @param subtaskId The id of the subtask to which the returned splits belong.
+     */
+    @Override
+    public void addSplitsBack(List<RocketMQSourceSplit> splits, int subtaskId) {
+        // If the failed subtask has already restarted, we need to assign splits to it
+        if (context.registeredReaders().containsKey(subtaskId)) {
+            sendSplitChangesToRemote(Collections.singleton(subtaskId));
+        }
+    }
+
+    @Override
+    public void addReader(int subtaskId) {
+        log.debug(
+                "Adding reader {} to RocketMQSourceEnumerator for consumer group {}.",
+                subtaskId,
+                groupId);
+        sendSplitChangesToRemote(Collections.singleton(subtaskId));
+        if (this.boundedness == Boundedness.BOUNDED) {
+            // for RocketMQ bounded source,
+            // send this signal to ensure the task can end after all the splits assigned are
+            // completed.
+            context.signalNoMoreSplits(subtaskId);
+        }
+    }
+
+    @Override
+    public RocketMQSourceEnumState snapshotState(long checkpointId) {
+        return new RocketMQSourceEnumState(allocatedSet);
+    }
+
+    @Override
+    public void close() {
+        if (consumer != null) {
+            try {
+                consumer.close();
+                consumer = null;
+            } catch (Exception e) {
+                log.error("Shutdown rocketmq internal consumer error", e);
+            }
+        }
+    }
+
+    // ----------------- private methods -------------------
+
+    private Set<MessageQueue> requestServiceDiscovery() {
+        Set<String> topicSet =
+                Sets.newHashSet(
+                        configuration
+                                .getString(RocketMQSourceOptions.TOPIC)
+                                .split(RocketMQSourceOptions.TOPIC_SEPARATOR));
+
+        return topicSet.stream()
+                .flatMap(
+                        topic -> {
+                            try {
+                                return consumer.fetchMessageQueues(topic).get().stream();
+                            } catch (Exception e) {
+                                log.error(
+                                        "Request topic route for service discovery error, topic={}",
+                                        topic,
+                                        e);
+                            }
+                            return Stream.empty();
+                        })
+                .collect(Collectors.toSet());
+    }
+
+    // This method should only be invoked in the coordinator executor thread.
+    private void handleSourceQueueChange(Set<MessageQueue> latestSet, Throwable t) {
+        if (t != null) {
+            throw new FlinkRuntimeException("Failed to handle source splits change due to ", t);
+        }
+
+        final SourceChangeResult sourceChangeResult = getSourceChangeResult(latestSet);
+        if (sourceChangeResult.isEmpty()) {
+            log.debug("Skip handle source allocated due to not queue change");
+            return;
+        }
+
+        context.callAsync(
+                () -> initializeSourceSplits(sourceChangeResult), this::handleSplitChanges);
+    }
+
+    // This method should only be invoked in the coordinator executor thread.
+    private SourceSplitChangeResult initializeSourceSplits(SourceChangeResult sourceChangeResult) {
+        Set<MessageQueue> increaseSet = sourceChangeResult.getIncreaseSet();
+
+        OffsetsSelector.MessageQueueOffsetsRetriever offsetsRetriever =
+                new InnerConsumerImpl.RemotingOffsetsRetrieverImpl(consumer);
+
+        Map<MessageQueue, Long> startingOffsets =
+                startingOffsetsSelector.getMessageQueueOffsets(increaseSet, offsetsRetriever);
+        Map<MessageQueue, Long> stoppingOffsets =
+                stoppingOffsetsSelector.getMessageQueueOffsets(increaseSet, offsetsRetriever);
+
+        Set<RocketMQSourceSplit> increaseSplitSet =
+                increaseSet.stream()
+                        .map(
+                                mq -> {
+                                    long startingOffset = startingOffsets.get(mq);
+                                    long stoppingOffset =
+                                            stoppingOffsets.getOrDefault(
+                                                    mq, RocketMQSourceSplit.NO_STOPPING_OFFSET);
+                                    return new RocketMQSourceSplit(
+                                            mq, startingOffset, stoppingOffset);
+                                })
+                        .collect(Collectors.toSet());
+
+        return new SourceSplitChangeResult(increaseSplitSet, sourceChangeResult.getDecreaseSet());
+    }
+
+    /**
+     * Mark partition splits initialized by {@link
+     * RocketMQSourceEnumerator#initializeSourceSplits(SourceChangeResult)} as pending and try to
+     * assign pending splits to registered readers.
+     *
+     * <p>NOTE: This method should only be invoked in the coordinator executor thread.
+     *
+     * @param sourceSplitChangeResult Partition split changes
+     * @param t Exception in worker thread
+     */
+    private void handleSplitChanges(SourceSplitChangeResult sourceSplitChangeResult, Throwable t) {
+        if (t != null) {
+            throw new FlinkRuntimeException("Failed to initialize partition splits due to ", t);
+        }
+        if (partitionDiscoveryIntervalMs <= 0) {
+            log.info("Split changes, but dynamic partition discovery is disabled.");
+        }
+        this.calculateSplitAssignment(sourceSplitChangeResult);
+        this.sendSplitChangesToRemote(context.registeredReaders().keySet());
+    }
+
+    /** Calculate new split assignment according allocate strategy */
+    private void calculateSplitAssignment(SourceSplitChangeResult sourceSplitChangeResult) {
+        Map<Integer, Set<RocketMQSourceSplit>> newSourceSplitAllocateMap =
+                this.allocateStrategy.allocate(
+                        sourceSplitChangeResult.getIncreaseSet(), context.currentParallelism());
+
+        for (Map.Entry<Integer, Set<RocketMQSourceSplit>> entry :
+                newSourceSplitAllocateMap.entrySet()) {
+            this.pendingSplitAssignmentMap
+                    .computeIfAbsent(entry.getKey(), r -> new HashSet<>())
+                    .addAll(entry.getValue());
+        }
+    }
+
+    // This method should only be invoked in the coordinator executor thread.
+    private void sendSplitChangesToRemote(Set<Integer> pendingReaders) {
+        Map<Integer, List<RocketMQSourceSplit>> incrementalSplit = new ConcurrentHashMap<>();
+
+        for (Integer pendingReader : pendingReaders) {
+            if (!context.registeredReaders().containsKey(pendingReader)) {
+                throw new IllegalStateException(
+                        String.format(
+                                "Reader %d is not registered to source coordinator",
+                                pendingReader));
+            }
+
+            final Set<RocketMQSourceSplit> pendingAssignmentForReader =
+                    this.pendingSplitAssignmentMap.get(pendingReader);
+
+            // Put pending assignment into incremental assignment
+            if (pendingAssignmentForReader != null && !pendingAssignmentForReader.isEmpty()) {
+                incrementalSplit
+                        .computeIfAbsent(pendingReader, k -> new ArrayList<>())
+                        .addAll(pendingAssignmentForReader);
+                pendingAssignmentForReader.forEach(
+                        split -> this.allocatedSet.add(split.getMessageQueue()));
+            }
+        }
+
+        // Assign pending splits to readers
+        if (!incrementalSplit.isEmpty()) {
+            log.info(
+                    "Enumerator assigning split(s) to readers {}",
+                    JSON.toJSONString(incrementalSplit, false));
+            context.assignSplits(new SplitsAssignment<>(incrementalSplit));
+        }
+
+        // Sends NoMoreSplitsEvent to the readers if there is no more partition.
+        if (partitionDiscoveryIntervalMs <= 0 && this.boundedness == Boundedness.BOUNDED) {
+            log.info(
+                    "No more rocketmq partition to assign. "
+                            + "Sending NoMoreSplitsEvent to the readers in consumer group {}.",
+                    groupId);
+            pendingReaders.forEach(this.context::signalNoMoreSplits);
+        }
+    }
+
+    /** A container class to hold the newly added partitions and removed partitions. */
+    @VisibleForTesting
+    private static class SourceChangeResult {
+        private final Set<MessageQueue> increaseSet;
+        private final Set<MessageQueue> decreaseSet;
+
+        public SourceChangeResult(Set<MessageQueue> increaseSet, Set<MessageQueue> decreaseSet) {
+            this.increaseSet = increaseSet;
+            this.decreaseSet = decreaseSet;
+        }
+
+        public Set<MessageQueue> getIncreaseSet() {
+            return increaseSet;
+        }
+
+        public Set<MessageQueue> getDecreaseSet() {
+            return decreaseSet;
+        }
+
+        public boolean isEmpty() {
+            return increaseSet.isEmpty() && decreaseSet.isEmpty();
+        }
+    }
+
+    @VisibleForTesting
+    public static class SourceSplitChangeResult {
+
+        private final Set<RocketMQSourceSplit> increaseSet;
+        private final Set<MessageQueue> decreaseSet;
+
+        private SourceSplitChangeResult(Set<RocketMQSourceSplit> increaseSet) {
+            this.increaseSet = Collections.unmodifiableSet(increaseSet);
+            this.decreaseSet = Sets.newHashSet();
+        }
+
+        private SourceSplitChangeResult(
+                Set<RocketMQSourceSplit> increaseSet, Set<MessageQueue> decreaseSet) {
+            this.increaseSet = Collections.unmodifiableSet(increaseSet);
+            this.decreaseSet = Collections.unmodifiableSet(decreaseSet);
+        }
+
+        public Set<RocketMQSourceSplit> getIncreaseSet() {
+            return increaseSet;
+        }
+
+        public Set<MessageQueue> getDecreaseSet() {
+            return decreaseSet;
+        }
+    }
+
+    @VisibleForTesting
+    private SourceChangeResult getSourceChangeResult(Set<MessageQueue> latestSet) {
+        Set<MessageQueue> currentSet = Collections.unmodifiableSet(this.allocatedSet);
+        Set<MessageQueue> increaseSet = Sets.difference(latestSet, currentSet);
+        Set<MessageQueue> decreaseSet = Sets.difference(currentSet, latestSet);
+
+        SourceChangeResult changeResult = new SourceChangeResult(increaseSet, decreaseSet);
+
+        // Current topic route is same as before
+        if (changeResult.isEmpty()) {
+            log.info(
+                    "Request topic route for service discovery, current allocated queues size={}",
+                    currentSet.size());
+        } else {
+            log.info(
+                    "Request topic route for service discovery, current allocated queues size: {}. "
+                            + "Changed details, current={}, latest={}, increase={}, decrease={}",
+                    currentSet.size(),
+                    currentSet,
+                    latestSet,
+                    increaseSet,
+                    decreaseSet);
+        }
+        return changeResult;
+    }
+}
diff --git a/src/main/java/org/apache/flink/connector/rocketmq/source/enumerator/allocate/AllocateStrategy.java b/src/main/java/org/apache/flink/connector/rocketmq/source/enumerator/allocate/AllocateStrategy.java
new file mode 100644
index 0000000..bfd814e
--- /dev/null
+++ b/src/main/java/org/apache/flink/connector/rocketmq/source/enumerator/allocate/AllocateStrategy.java
@@ -0,0 +1,47 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.flink.connector.rocketmq.source.enumerator.allocate;
+
+import org.apache.flink.annotation.PublicEvolving;
+import org.apache.flink.connector.rocketmq.source.split.RocketMQSourceSplit;
+
+import java.util.Collection;
+import java.util.Map;
+import java.util.Set;
+
+/** This interface defines a strategy for allocating RocketMQ source splits to Flink tasks. */
+@PublicEvolving
+public interface AllocateStrategy {
+
+    /**
+     * Allocate strategy name
+     *
+     * @return Current strategy name
+     */
+    String getStrategyName();
+
+    /**
+     * Allocates RocketMQ source splits to Flink tasks based on the selected allocation strategy.
+     *
+     * @param mqAll a collection of all available RocketMQ source splits
+     * @param parallelism the desired parallelism for the Flink tasks
+     * @return a map of task indices to sets of corresponding RocketMQ source splits
+     */
+    Map<Integer, Set<RocketMQSourceSplit>> allocate(
+            final Collection<RocketMQSourceSplit> mqAll, final int parallelism);
+}
diff --git a/src/main/java/org/apache/flink/connector/rocketmq/source/enumerator/allocate/AllocateStrategyFactory.java b/src/main/java/org/apache/flink/connector/rocketmq/source/enumerator/allocate/AllocateStrategyFactory.java
new file mode 100644
index 0000000..6c9d723
--- /dev/null
+++ b/src/main/java/org/apache/flink/connector/rocketmq/source/enumerator/allocate/AllocateStrategyFactory.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.flink.connector.rocketmq.source.enumerator.allocate;
+
+import org.apache.flink.api.connector.source.SplitEnumeratorContext;
+import org.apache.flink.configuration.Configuration;
+import org.apache.flink.connector.rocketmq.source.RocketMQSourceOptions;
+import org.apache.flink.connector.rocketmq.source.enumerator.RocketMQSourceEnumState;
+import org.apache.flink.connector.rocketmq.source.split.RocketMQSourceSplit;
+
+public class AllocateStrategyFactory {
+
+    public static final String STRATEGY_NAME_BROADCAST = "broadcast";
+    public static final String STRATEGY_NAME_CONSISTENT_HASH = "hash";
+
+    private AllocateStrategyFactory() {
+        // No public constructor.
+    }
+
+    public static AllocateStrategy getStrategy(
+            Configuration rocketmqSourceOptions,
+            SplitEnumeratorContext<RocketMQSourceSplit> context,
+            RocketMQSourceEnumState enumState) {
+
+        String allocateStrategyName =
+                rocketmqSourceOptions.getString(
+                        RocketMQSourceOptions.ALLOCATE_MESSAGE_QUEUE_STRATEGY);
+
+        switch (allocateStrategyName) {
+            case STRATEGY_NAME_CONSISTENT_HASH:
+                return new ConsistentHashAllocateStrategy();
+            case STRATEGY_NAME_BROADCAST:
+                return new BroadcastAllocateStrategy();
+            default:
+                throw new IllegalArgumentException(
+                        "We don't support this allocate strategy: " + allocateStrategyName);
+        }
+    }
+}
diff --git a/src/main/java/org/apache/flink/connector/rocketmq/source/enumerator/allocate/BroadcastAllocateStrategy.java b/src/main/java/org/apache/flink/connector/rocketmq/source/enumerator/allocate/BroadcastAllocateStrategy.java
new file mode 100644
index 0000000..2e46419
--- /dev/null
+++ b/src/main/java/org/apache/flink/connector/rocketmq/source/enumerator/allocate/BroadcastAllocateStrategy.java
@@ -0,0 +1,43 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.flink.connector.rocketmq.source.enumerator.allocate;
+
+import org.apache.flink.connector.rocketmq.source.split.RocketMQSourceSplit;
+
+import java.util.Collection;
+import java.util.HashMap;
+import java.util.HashSet;
+import java.util.Map;
+import java.util.Set;
+
+public class BroadcastAllocateStrategy implements AllocateStrategy {
+
+    @Override
+    public String getStrategyName() {
+        return AllocateStrategyFactory.STRATEGY_NAME_BROADCAST;
+    }
+
+    @Override
+    public Map<Integer, Set<RocketMQSourceSplit>> allocate(
+            final Collection<RocketMQSourceSplit> mqAll, final int parallelism) {
+        Map<Integer, Set<RocketMQSourceSplit>> result = new HashMap<>(parallelism);
+        for (int i = 0; i < parallelism; i++) {
+            result.computeIfAbsent(i, k -> new HashSet<>()).addAll(mqAll);
+        }
+        return result;
+    }
+}
diff --git a/src/main/java/org/apache/flink/connector/rocketmq/source/enumerator/allocate/ConsistentHashAllocateStrategy.java b/src/main/java/org/apache/flink/connector/rocketmq/source/enumerator/allocate/ConsistentHashAllocateStrategy.java
new file mode 100644
index 0000000..6a3ad2c
--- /dev/null
+++ b/src/main/java/org/apache/flink/connector/rocketmq/source/enumerator/allocate/ConsistentHashAllocateStrategy.java
@@ -0,0 +1,51 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.flink.connector.rocketmq.source.enumerator.allocate;
+
+import org.apache.flink.connector.rocketmq.source.split.RocketMQSourceSplit;
+
+import java.util.Collection;
+import java.util.HashMap;
+import java.util.HashSet;
+import java.util.Map;
+import java.util.Set;
+
+public class ConsistentHashAllocateStrategy implements AllocateStrategy {
+
+    @Override
+    public String getStrategyName() {
+        return AllocateStrategyFactory.STRATEGY_NAME_CONSISTENT_HASH;
+    }
+
+    /** Returns the index of the target subtask that a specific queue should be assigned to. */
+    private int getSplitOwner(RocketMQSourceSplit sourceSplit, int parallelism) {
+        int startIndex =
+                ((sourceSplit.getMessageQueue().hashCode() * 31) & Integer.MAX_VALUE) % parallelism;
+        return startIndex % parallelism;
+    }
+
+    @Override
+    public Map<Integer, Set<RocketMQSourceSplit>> allocate(
+            final Collection<RocketMQSourceSplit> mqAll, final int parallelism) {
+        Map<Integer, Set<RocketMQSourceSplit>> result = new HashMap<>();
+        for (RocketMQSourceSplit mq : mqAll) {
+            int readerIndex = this.getSplitOwner(mq, parallelism);
+            result.computeIfAbsent(readerIndex, k -> new HashSet<>()).add(mq);
+        }
+        return result;
+    }
+}
diff --git a/src/main/java/org/apache/flink/connector/rocketmq/source/enumerator/offset/OffsetsSelector.java b/src/main/java/org/apache/flink/connector/rocketmq/source/enumerator/offset/OffsetsSelector.java
new file mode 100644
index 0000000..14ada53
--- /dev/null
+++ b/src/main/java/org/apache/flink/connector/rocketmq/source/enumerator/offset/OffsetsSelector.java
@@ -0,0 +1,168 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.flink.connector.rocketmq.source.enumerator.offset;
+
+import org.apache.flink.annotation.PublicEvolving;
+import org.apache.flink.connector.rocketmq.legacy.common.config.OffsetResetStrategy;
+import org.apache.flink.connector.rocketmq.source.RocketMQSource;
+import org.apache.flink.connector.rocketmq.source.split.RocketMQSourceSplit;
+
+import org.apache.rocketmq.common.consumer.ConsumeFromWhere;
+import org.apache.rocketmq.common.message.MessageQueue;
+
+import java.io.Serializable;
+import java.util.Collection;
+import java.util.Map;
+
+/**
+ * An interface for users to specify the starting / stopping offset of a {@link
+ * RocketMQSourceSplit}.
+ */
+@PublicEvolving
+public interface OffsetsSelector extends Serializable {
+
+    /**
+     * This method retrieves the current offsets for a collection of {@link MessageQueue}s using a
+     * provided {@link MessageQueueOffsetsRetriever}.
+     *
+     * @param messageQueues the collection of queues for which to retrieve the offsets
+     * @param offsetsRetriever the offsets retriever to use for obtaining the offsets
+     * @return a mapping of {@link MessageQueue}s to their current offsets
+     */
+    Map<MessageQueue, Long> getMessageQueueOffsets(
+            Collection<MessageQueue> messageQueues, MessageQueueOffsetsRetriever offsetsRetriever);
+
+    /**
+     * Returns the strategy for automatically resetting the offset when there is no initial offset
+     * in RocketMQ or if the current offset does not exist in RocketMQ.
+     *
+     * @return strategy for automatically resetting the offset
+     */
+    OffsetResetStrategy getAutoOffsetResetStrategy();
+
+    /**
+     * An interface that provides necessary information to the {@link OffsetsSelector} to get the
+     * initial offsets of the RocketMQ message queues.
+     */
+    interface MessageQueueOffsetsRetriever {
+
+        /**
+         * The group id should be the set for {@link RocketMQSource } before invoking this method.
+         * Otherwise, an {@code IllegalStateException} will be thrown.
+         */
+        Map<MessageQueue, Long> committedOffsets(Collection<MessageQueue> messageQueues);
+
+        /** List min offsets for the specified MessageQueues. */
+        Map<MessageQueue, Long> minOffsets(Collection<MessageQueue> messageQueues);
+
+        /** List max offsets for the specified MessageQueues. */
+        Map<MessageQueue, Long> maxOffsets(Collection<MessageQueue> messageQueues);
+
+        /** List max offsets for the specified MessageQueues. */
+        Map<MessageQueue, Long> offsetsForTimes(Map<MessageQueue, Long> messageQueueWithTimeMap);
+    }
+
+    // --------------- factory methods ---------------
+
+    /**
+     * Get an {@link OffsetsSelector} which initializes the offsets to the committed offsets. An
+     * exception will be thrown at runtime if there is no committed offsets.
+     *
+     * @return an offset initializer which initialize the offsets to the committed offsets.
+     */
+    static OffsetsSelector committedOffsets() {
+        return committedOffsets(OffsetResetStrategy.LATEST);
+    }
+
+    /**
+     * Get an {@link OffsetsSelector} which initializes the offsets to the committed offsets. Use
+     * the given {@link OffsetResetStrategy} to initialize the offsets if the committed offsets does
+     * not exist.
+     *
+     * @param offsetResetStrategy the offset reset strategy to use when the committed offsets do not
+     *     exist.
+     * @return an {@link OffsetsSelector} which initializes the offsets to the committed offsets.
+     */
+    static OffsetsSelector committedOffsets(OffsetResetStrategy offsetResetStrategy) {
+        return new OffsetsSelectorByStrategy(
+                ConsumeFromWhere.CONSUME_FROM_LAST_OFFSET, offsetResetStrategy);
+    }
+
+    /**
+     * Get an {@link OffsetsSelector} which initializes the offsets in each partition so that the
+     * initialized offset is the offset of the first record whose record timestamp is greater than
+     * or equals the give timestamp (milliseconds).
+     *
+     * @param timestamp the timestamp (milliseconds) to start the consumption.
+     * @return an {@link OffsetsSelector} which initializes the offsets based on the given
+     *     timestamp.
+     */
+    static OffsetsSelector timestamp(long timestamp) {
+        return new OffsetsSelectorByTimestamp(timestamp);
+    }
+
+    /**
+     * Get an {@link OffsetsSelector} which initializes the offsets to the earliest available
+     * offsets of each partition.
+     *
+     * @return an {@link OffsetsSelector} which initializes the offsets to the earliest available
+     *     offsets.
+     */
+    static OffsetsSelector earliest() {
+        return new OffsetsSelectorByStrategy(
+                ConsumeFromWhere.CONSUME_FROM_FIRST_OFFSET, OffsetResetStrategy.EARLIEST);
+    }
+
+    /**
+     * Get an {@link OffsetsSelector} which initializes the offsets to the latest offsets of each
+     * partition.
+     *
+     * @return an {@link OffsetsSelector} which initializes the offsets to the latest offsets.
+     */
+    @SuppressWarnings("deprecation")
+    static OffsetsSelector latest() {
+        return new OffsetsSelectorByStrategy(
+                ConsumeFromWhere.CONSUME_FROM_MAX_OFFSET, OffsetResetStrategy.LATEST);
+    }
+
+    /**
+     * Get an {@link OffsetsSelector} which initializes the offsets to the specified offsets.
+     *
+     * @param offsets the specified offsets for each partition.
+     * @return an {@link OffsetsSelector} which initializes the offsets to the specified offsets.
+     */
+    static OffsetsSelector offsets(Map<MessageQueue, Long> offsets) {
+        return new OffsetsSelectorBySpecified(offsets, OffsetResetStrategy.EARLIEST);
+    }
+
+    /**
+     * Get an {@link OffsetsSelector} which initializes the offsets to the specified offsets. Use
+     * the given {@link OffsetResetStrategy} to initialize the offsets in case the specified offset
+     * is out of range.
+     *
+     * @param offsets the specified offsets for each partition.
+     * @param offsetResetStrategy the {@link OffsetResetStrategy} to use when the specified offset
+     *     is out of range.
+     * @return an {@link OffsetsSelector} which initializes the offsets to the specified offsets.
+     */
+    static OffsetsSelector offsets(
+            Map<MessageQueue, Long> offsets, OffsetResetStrategy offsetResetStrategy) {
+        return new OffsetsSelectorBySpecified(offsets, offsetResetStrategy);
+    }
+}
diff --git a/src/main/java/org/apache/flink/connector/rocketmq/source/enumerator/offset/OffsetsSelectorBySpecified.java b/src/main/java/org/apache/flink/connector/rocketmq/source/enumerator/offset/OffsetsSelectorBySpecified.java
new file mode 100644
index 0000000..edf7010
--- /dev/null
+++ b/src/main/java/org/apache/flink/connector/rocketmq/source/enumerator/offset/OffsetsSelectorBySpecified.java
@@ -0,0 +1,84 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.flink.connector.rocketmq.source.enumerator.offset;
+
+import org.apache.flink.connector.rocketmq.legacy.common.config.OffsetResetStrategy;
+
+import org.apache.rocketmq.common.message.MessageQueue;
+
+import java.util.ArrayList;
+import java.util.Collection;
+import java.util.Collections;
+import java.util.HashMap;
+import java.util.List;
+import java.util.Map;
+import java.util.Properties;
+
+public class OffsetsSelectorBySpecified implements OffsetsSelector, OffsetsValidator {
+
+    private final Map<MessageQueue, Long> initialOffsets;
+    private final OffsetResetStrategy offsetResetStrategy;
+
+    OffsetsSelectorBySpecified(
+            Map<MessageQueue, Long> initialOffsets, OffsetResetStrategy offsetResetStrategy) {
+        this.initialOffsets = Collections.unmodifiableMap(initialOffsets);
+        this.offsetResetStrategy = offsetResetStrategy;
+    }
+
+    @Override
+    public Map<MessageQueue, Long> getMessageQueueOffsets(
+            Collection<MessageQueue> messageQueues, MessageQueueOffsetsRetriever offsetsRetriever) {
+        Map<MessageQueue, Long> offsets = new HashMap<>();
+        List<MessageQueue> toLookup = new ArrayList<>();
+        for (MessageQueue tp : messageQueues) {
+            Long offset = initialOffsets.get(tp);
+            if (offset == null) {
+                toLookup.add(tp);
+            } else {
+                offsets.put(tp, offset);
+            }
+        }
+        if (!toLookup.isEmpty()) {
+            // First check the committed offsets.
+            Map<MessageQueue, Long> committedOffsets = offsetsRetriever.committedOffsets(toLookup);
+            offsets.putAll(committedOffsets);
+            toLookup.removeAll(committedOffsets.keySet());
+
+            switch (offsetResetStrategy) {
+                case EARLIEST:
+                    offsets.putAll(offsetsRetriever.minOffsets(toLookup));
+                    break;
+                case LATEST:
+                    offsets.putAll(offsetsRetriever.maxOffsets(toLookup));
+                    break;
+                default:
+                    throw new IllegalStateException(
+                            "Cannot find initial offsets for partitions: " + toLookup);
+            }
+        }
+        return offsets;
+    }
+
+    @Override
+    public OffsetResetStrategy getAutoOffsetResetStrategy() {
+        return offsetResetStrategy;
+    }
+
+    @Override
+    public void validate(Properties rocketmqSourceProperties) {}
+}
diff --git a/src/main/java/org/apache/flink/connector/rocketmq/source/enumerator/offset/OffsetsSelectorByStrategy.java b/src/main/java/org/apache/flink/connector/rocketmq/source/enumerator/offset/OffsetsSelectorByStrategy.java
new file mode 100644
index 0000000..b6be92d
--- /dev/null
+++ b/src/main/java/org/apache/flink/connector/rocketmq/source/enumerator/offset/OffsetsSelectorByStrategy.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.flink.connector.rocketmq.source.enumerator.offset;
+
+import org.apache.flink.connector.rocketmq.legacy.common.config.OffsetResetStrategy;
+
+import org.apache.rocketmq.common.consumer.ConsumeFromWhere;
+import org.apache.rocketmq.common.message.MessageQueue;
+
+import java.util.Collection;
+import java.util.Map;
+import java.util.Properties;
+
+public class OffsetsSelectorByStrategy implements OffsetsSelector, OffsetsValidator {
+
+    private final ConsumeFromWhere consumeFromWhere;
+    private final OffsetResetStrategy offsetResetStrategy;
+
+    OffsetsSelectorByStrategy(
+            ConsumeFromWhere consumeFromWhere, OffsetResetStrategy offsetResetStrategy) {
+        this.consumeFromWhere = consumeFromWhere;
+        this.offsetResetStrategy = offsetResetStrategy;
+    }
+
+    @Override
+    @SuppressWarnings("deprecation")
+    public Map<MessageQueue, Long> getMessageQueueOffsets(
+            Collection<MessageQueue> messageQueues, MessageQueueOffsetsRetriever offsetsRetriever) {
+
+        switch (consumeFromWhere) {
+            case CONSUME_FROM_FIRST_OFFSET:
+                return offsetsRetriever.minOffsets(messageQueues);
+            case CONSUME_FROM_MAX_OFFSET:
+                return offsetsRetriever.maxOffsets(messageQueues);
+            case CONSUME_FROM_LAST_OFFSET:
+            default:
+                return offsetsRetriever.committedOffsets(messageQueues);
+        }
+    }
+
+    @Override
+    public OffsetResetStrategy getAutoOffsetResetStrategy() {
+        return offsetResetStrategy;
+    }
+
+    @Override
+    public void validate(Properties rocketmqSourceProperties) throws IllegalStateException {}
+}
diff --git a/src/main/java/org/apache/flink/connector/rocketmq/source/enumerator/offset/OffsetsSelectorByTimestamp.java b/src/main/java/org/apache/flink/connector/rocketmq/source/enumerator/offset/OffsetsSelectorByTimestamp.java
new file mode 100644
index 0000000..b685661
--- /dev/null
+++ b/src/main/java/org/apache/flink/connector/rocketmq/source/enumerator/offset/OffsetsSelectorByTimestamp.java
@@ -0,0 +1,67 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.flink.connector.rocketmq.source.enumerator.offset;
+
+import org.apache.flink.connector.rocketmq.legacy.common.config.OffsetResetStrategy;
+
+import org.apache.rocketmq.common.message.MessageQueue;
+
+import java.util.Collection;
+import java.util.HashMap;
+import java.util.Map;
+
+class OffsetsSelectorByTimestamp implements OffsetsSelector {
+    private static final long serialVersionUID = 2932230571773627233L;
+    private final long startingTimestamp;
+
+    OffsetsSelectorByTimestamp(long startingTimestamp) {
+        this.startingTimestamp = startingTimestamp;
+    }
+
+    @Override
+    public Map<MessageQueue, Long> getMessageQueueOffsets(
+            Collection<MessageQueue> messageQueues, MessageQueueOffsetsRetriever offsetsRetriever) {
+        Map<MessageQueue, Long> startingTimestamps = new HashMap<>();
+        Map<MessageQueue, Long> initialOffsets = new HashMap<>();
+
+        // First get the current end offsets of the partitions. This is going to be used
+        // in case we cannot find a suitable offsets based on the timestamp, i.e. the message
+        // meeting the requirement of the timestamp have not been produced to RocketMQ yet,
+        // in this case, we just use the latest offset.
+        // We need to get the latest offsets before querying offsets by time to ensure that
+        // no message is going to be missed.
+        Map<MessageQueue, Long> endOffsets = offsetsRetriever.maxOffsets(messageQueues);
+        messageQueues.forEach(tp -> startingTimestamps.put(tp, startingTimestamp));
+        offsetsRetriever
+                .offsetsForTimes(startingTimestamps)
+                .forEach(
+                        (mq, offsetByTimestamp) -> {
+                            long offset =
+                                    offsetByTimestamp != null
+                                            ? offsetByTimestamp
+                                            : endOffsets.get(mq);
+                            initialOffsets.put(mq, offset);
+                        });
+        return initialOffsets;
+    }
+
+    @Override
+    public OffsetResetStrategy getAutoOffsetResetStrategy() {
+        return OffsetResetStrategy.LATEST;
+    }
+}
diff --git a/src/main/java/org/apache/flink/connector/rocketmq/source/enumerator/offset/OffsetsSelectorNoStopping.java b/src/main/java/org/apache/flink/connector/rocketmq/source/enumerator/offset/OffsetsSelectorNoStopping.java
new file mode 100644
index 0000000..ccd916f
--- /dev/null
+++ b/src/main/java/org/apache/flink/connector/rocketmq/source/enumerator/offset/OffsetsSelectorNoStopping.java
@@ -0,0 +1,46 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.flink.connector.rocketmq.source.enumerator.offset;
+
+import org.apache.flink.annotation.Internal;
+import org.apache.flink.connector.rocketmq.legacy.common.config.OffsetResetStrategy;
+
+import org.apache.rocketmq.common.message.MessageQueue;
+
+import java.util.Collection;
+import java.util.Collections;
+import java.util.Map;
+
+@Internal
+public class OffsetsSelectorNoStopping implements OffsetsSelector {
+
+    @Override
+    public Map<MessageQueue, Long> getMessageQueueOffsets(
+            Collection<MessageQueue> messageQueues, MessageQueueOffsetsRetriever offsetsRetriever) {
+
+        return Collections.emptyMap();
+    }
+
+    @Override
+    public OffsetResetStrategy getAutoOffsetResetStrategy() {
+
+        throw new UnsupportedOperationException(
+                "The OffsetsSelectorNoStopping does not have an OffsetResetStrategy. "
+                        + "It should only be used to end offset.");
+    }
+}
diff --git a/src/main/java/org/apache/flink/connector/rocketmq/source/enumerator/offset/OffsetsValidator.java b/src/main/java/org/apache/flink/connector/rocketmq/source/enumerator/offset/OffsetsValidator.java
new file mode 100644
index 0000000..18581c8
--- /dev/null
+++ b/src/main/java/org/apache/flink/connector/rocketmq/source/enumerator/offset/OffsetsValidator.java
@@ -0,0 +1,17 @@
+package org.apache.flink.connector.rocketmq.source.enumerator.offset;
+
+import org.apache.flink.annotation.Internal;
+
+import java.util.Properties;
+
+@Internal
+public interface OffsetsValidator {
+
+    /**
+     * Validate offsets initializer with properties of RocketMQ source.
+     *
+     * @param properties Properties of RocketMQ source
+     * @throws IllegalStateException if validation fails
+     */
+    void validate(Properties properties) throws IllegalStateException;
+}
diff --git a/src/main/java/org/apache/flink/connector/rocketmq/source/metrics/RocketMQSourceReaderMetrics.java b/src/main/java/org/apache/flink/connector/rocketmq/source/metrics/RocketMQSourceReaderMetrics.java
new file mode 100644
index 0000000..09c1049
--- /dev/null
+++ b/src/main/java/org/apache/flink/connector/rocketmq/source/metrics/RocketMQSourceReaderMetrics.java
@@ -0,0 +1,48 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.flink.connector.rocketmq.source.metrics;
+
+import org.apache.flink.annotation.PublicEvolving;
+import org.apache.flink.metrics.groups.SourceReaderMetricGroup;
+
+import org.apache.rocketmq.common.message.MessageQueue;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+@PublicEvolving
+public class RocketMQSourceReaderMetrics {
+
+    private static final Logger LOG = LoggerFactory.getLogger(RocketMQSourceReaderMetrics.class);
+
+    public static final String ROCKETMQ_SOURCE_READER_METRIC_GROUP = "RocketmqSourceReader";
+    public static final String TOPIC_GROUP = "topic";
+    public static final String QUEUE_GROUP = "queue";
+    public static final String CURRENT_OFFSET_METRIC_GAUGE = "currentOffset";
+    public static final String COMMITTED_OFFSET_METRIC_GAUGE = "committedOffset";
+    public static final String COMMITS_SUCCEEDED_METRIC_COUNTER = "commitsSucceeded";
+    public static final String COMMITS_FAILED_METRIC_COUNTER = "commitsFailed";
+    public static final String ROCKETMQ_CONSUMER_METRIC_GROUP = "RocketMQConsumer";
+
+    public static final String CONSUMER_FETCH_MANAGER_GROUP = "consumer-fetch-manager-metrics";
+    public static final String BYTES_CONSUMED_TOTAL = "bytes-consumed-total";
+    public static final String RECORDS_LAG = "records-lag";
+
+    public RocketMQSourceReaderMetrics(SourceReaderMetricGroup sourceReaderMetricGroup) {}
+
+    public void registerNewMessageQueue(MessageQueue messageQueue) {}
+}
diff --git a/src/main/java/org/apache/flink/connector/rocketmq/source/reader/MessageView.java b/src/main/java/org/apache/flink/connector/rocketmq/source/reader/MessageView.java
new file mode 100644
index 0000000..84ce64b
--- /dev/null
+++ b/src/main/java/org/apache/flink/connector/rocketmq/source/reader/MessageView.java
@@ -0,0 +1,117 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.flink.connector.rocketmq.source.reader;
+
+import java.util.Collection;
+import java.util.Map;
+
+/** This interface defines the methods for obtaining information about a message in RocketMQ. */
+public interface MessageView {
+
+    /**
+     * Get the unique message ID.
+     *
+     * @return the message ID
+     */
+    String getMessageId();
+
+    /**
+     * Get the topic that the message belongs to.
+     *
+     * @return the topic
+     */
+    String getTopic();
+
+    /**
+     * Get the name of the broker that handles the message.
+     *
+     * @return the broker name
+     */
+    String getBrokerName();
+
+    /**
+     * Get the ID of the queue that the message is stored in.
+     *
+     * @return the queue ID
+     */
+    int getQueueId();
+
+    /**
+     * Get the offset of the message within the queue.
+     *
+     * @return the queue offset
+     */
+    long getQueueOffset();
+
+    /**
+     * Get the tag of the message, which is used for filtering.
+     *
+     * @return the message tag
+     */
+    String getTag();
+
+    /**
+     * Get the keys of the message, which are used for partitioning and indexing.
+     *
+     * @return the message keys
+     */
+    Collection<String> getKeys();
+
+    /**
+     * Get the size of the message in bytes.
+     *
+     * @return the message size
+     */
+    int getStoreSize();
+
+    /**
+     * Get the body of the message.
+     *
+     * @return the message body
+     */
+    byte[] getBody();
+
+    /**
+     * Get the number of times that the message has been attempted to be delivered.
+     *
+     * @return the delivery attempt count
+     */
+    int getDeliveryAttempt();
+
+    /**
+     * Get the event time of the message, which is used for filtering and sorting.
+     *
+     * @return the event time
+     */
+    long getEventTime();
+
+    /**
+     * Get the ingestion time of the message, which is the time that the message was received by the
+     * broker.
+     *
+     * @return the ingestion time
+     */
+    long getIngestionTime();
+
+    /**
+     * Get the properties of the message, which are set by the producer.
+     *
+     * @return the message properties
+     */
+    Map<String, String> getProperties();
+}
diff --git a/src/main/java/org/apache/flink/connector/rocketmq/source/reader/MessageViewExt.java b/src/main/java/org/apache/flink/connector/rocketmq/source/reader/MessageViewExt.java
new file mode 100644
index 0000000..b289f20
--- /dev/null
+++ b/src/main/java/org/apache/flink/connector/rocketmq/source/reader/MessageViewExt.java
@@ -0,0 +1,174 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.flink.connector.rocketmq.source.reader;
+
+import org.apache.rocketmq.common.message.MessageExt;
+
+import java.util.ArrayList;
+import java.util.Arrays;
+import java.util.Collection;
+import java.util.Map;
+
+public class MessageViewExt implements MessageView {
+
+    private static final String KEY_SEPARATE = "";
+
+    private final String messageId;
+
+    private final String topic;
+
+    private final String brokerName;
+
+    private final int queueId;
+
+    private final long queueOffset;
+
+    private final int storeSize;
+
+    private final String tag;
+
+    private final Collection<String> keys;
+
+    private final byte[] body;
+
+    /** message consume times */
+    private final int deliveryAttempt;
+
+    /** trust message born timestamp, we don't care store timestamp */
+    private final long eventTime;
+
+    private final long ingestionTime;
+
+    private final Map<String, String> properties;
+
+    public MessageViewExt(MessageExt messageExt) {
+        this.messageId = messageExt.getMsgId();
+        this.topic = messageExt.getTopic();
+        this.brokerName = messageExt.getBrokerName();
+        this.queueId = messageExt.getQueueId();
+        this.queueOffset = messageExt.getQueueOffset();
+        this.storeSize = messageExt.getStoreSize();
+        this.tag = messageExt.getTags();
+        this.keys =
+                messageExt.getKeys() != null
+                        ? Arrays.asList(messageExt.getKeys().split(KEY_SEPARATE))
+                        : new ArrayList<>();
+        this.body = messageExt.getBody();
+        this.deliveryAttempt = messageExt.getReconsumeTimes();
+        this.eventTime = messageExt.getBornTimestamp();
+        this.ingestionTime = System.currentTimeMillis();
+        this.properties = messageExt.getProperties();
+    }
+
+    @Override
+    public String getMessageId() {
+        return messageId;
+    }
+
+    @Override
+    public String getTopic() {
+        return topic;
+    }
+
+    @Override
+    public String getBrokerName() {
+        return brokerName;
+    }
+
+    @Override
+    public int getQueueId() {
+        return queueId;
+    }
+
+    @Override
+    public long getQueueOffset() {
+        return queueOffset;
+    }
+
+    @Override
+    public String getTag() {
+        return tag;
+    }
+
+    @Override
+    public Collection<String> getKeys() {
+        return keys;
+    }
+
+    @Override
+    public int getStoreSize() {
+        return storeSize;
+    }
+
+    @Override
+    public byte[] getBody() {
+        return body;
+    }
+
+    @Override
+    public int getDeliveryAttempt() {
+        return deliveryAttempt;
+    }
+
+    @Override
+    public long getEventTime() {
+        return eventTime;
+    }
+
+    @Override
+    public long getIngestionTime() {
+        return ingestionTime;
+    }
+
+    @Override
+    public Map<String, String> getProperties() {
+        return properties;
+    }
+
+    @Override
+    public String toString() {
+        return "MessageViewExt{"
+                + "messageId='"
+                + messageId
+                + '\''
+                + ", topic='"
+                + topic
+                + '\''
+                + ", brokerName='"
+                + brokerName
+                + '\''
+                + ", queueId="
+                + queueId
+                + ", queueOffset="
+                + queueOffset
+                + ", storeSize="
+                + storeSize
+                + ", tag='"
+                + tag
+                + '\''
+                + ", keys="
+                + keys
+                + ", deliveryAttempt="
+                + deliveryAttempt
+                + ", eventTime="
+                + eventTime
+                + ", ingestionTime="
+                + ingestionTime
+                + '}';
+    }
+}
diff --git a/src/main/java/org/apache/flink/connector/rocketmq/source/reader/RocketMQRecordEmitter.java b/src/main/java/org/apache/flink/connector/rocketmq/source/reader/RocketMQRecordEmitter.java
new file mode 100644
index 0000000..7ba42d6
--- /dev/null
+++ b/src/main/java/org/apache/flink/connector/rocketmq/source/reader/RocketMQRecordEmitter.java
@@ -0,0 +1,77 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.flink.connector.rocketmq.source.reader;
+
+import org.apache.flink.api.connector.source.SourceOutput;
+import org.apache.flink.connector.base.source.reader.RecordEmitter;
+import org.apache.flink.connector.rocketmq.source.reader.deserializer.RocketMQDeserializationSchema;
+import org.apache.flink.connector.rocketmq.source.split.RocketMQSourceSplitState;
+import org.apache.flink.util.Collector;
+
+import java.io.IOException;
+
+/** The {@link RecordEmitter} implementation for {@link RocketMQSourceReader}. */
+public class RocketMQRecordEmitter<T>
+        implements RecordEmitter<MessageView, T, RocketMQSourceSplitState> {
+
+    private final RocketMQDeserializationSchema<T> deserializationSchema;
+    private final SourceOutputWrapper<T> sourceOutputWrapper = new SourceOutputWrapper<>();
+
+    public RocketMQRecordEmitter(RocketMQDeserializationSchema<T> deserializationSchema) {
+        this.deserializationSchema = deserializationSchema;
+    }
+
+    @Override
+    public void emitRecord(
+            MessageView element, SourceOutput<T> output, RocketMQSourceSplitState splitState)
+            throws IOException {
+
+        try {
+            sourceOutputWrapper.setSourceOutput(output);
+            sourceOutputWrapper.setTimestamp(element.getEventTime());
+            deserializationSchema.deserialize(element, sourceOutputWrapper);
+            splitState.setCurrentOffset(element.getQueueOffset() + 1);
+        } catch (Exception e) {
+            throw new IOException("Failed to deserialize message due to", e);
+        }
+    }
+
+    private static class SourceOutputWrapper<T> implements Collector<T> {
+
+        private SourceOutput<T> sourceOutput;
+
+        private long timestamp;
+
+        @Override
+        public void collect(T record) {
+            sourceOutput.collect(record, timestamp);
+        }
+
+        @Override
+        public void close() {}
+
+        private void setSourceOutput(SourceOutput<T> sourceOutput) {
+            this.sourceOutput = sourceOutput;
+        }
+
+        private void setTimestamp(long timestamp) {
+            this.timestamp = timestamp;
+        }
+    }
+}
diff --git a/src/main/java/org/apache/flink/connector/rocketmq/source/reader/RocketMQSourceFetcherManager.java b/src/main/java/org/apache/flink/connector/rocketmq/source/reader/RocketMQSourceFetcherManager.java
new file mode 100644
index 0000000..a635b17
--- /dev/null
+++ b/src/main/java/org/apache/flink/connector/rocketmq/source/reader/RocketMQSourceFetcherManager.java
@@ -0,0 +1,98 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.flink.connector.rocketmq.source.reader;
+
+import org.apache.flink.annotation.Internal;
+import org.apache.flink.connector.base.source.reader.RecordsWithSplitIds;
+import org.apache.flink.connector.base.source.reader.SourceReaderBase;
+import org.apache.flink.connector.base.source.reader.fetcher.SingleThreadFetcherManager;
+import org.apache.flink.connector.base.source.reader.fetcher.SplitFetcher;
+import org.apache.flink.connector.base.source.reader.fetcher.SplitFetcherTask;
+import org.apache.flink.connector.base.source.reader.splitreader.SplitReader;
+import org.apache.flink.connector.base.source.reader.synchronization.FutureCompletingBlockingQueue;
+import org.apache.flink.connector.rocketmq.source.split.RocketMQSourceSplit;
+
+import org.apache.rocketmq.common.message.MessageQueue;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import java.util.Collection;
+import java.util.Map;
+import java.util.function.Consumer;
+import java.util.function.Supplier;
+
+@Internal
+public class RocketMQSourceFetcherManager
+        extends SingleThreadFetcherManager<MessageView, RocketMQSourceSplit> {
+
+    private static final Logger LOG = LoggerFactory.getLogger(RocketMQSourceFetcherManager.class);
+
+    /**
+     * Creates a new SplitFetcherManager with a single I/O threads.
+     *
+     * @param elementsQueue The queue that is used to hand over data from the I/O thread (the
+     *     fetchers) to the reader (which emits the records and book-keeps the state. This must be
+     *     the same queue instance that is also passed to the {@link SourceReaderBase}.
+     * @param splitReaderSupplier The factory for the split reader that connects to the source
+     *     system.
+     * @param splitFinishedHook Hook for handling finished splits in split fetchers.
+     */
+    public RocketMQSourceFetcherManager(
+            FutureCompletingBlockingQueue<RecordsWithSplitIds<MessageView>> elementsQueue,
+            Supplier<SplitReader<MessageView, RocketMQSourceSplit>> splitReaderSupplier,
+            Consumer<Collection<String>> splitFinishedHook) {
+
+        super(elementsQueue, splitReaderSupplier, splitFinishedHook);
+    }
+
+    public void commitOffsets(Map<MessageQueue, Long> offsetsToCommit) {
+        if (offsetsToCommit.isEmpty()) {
+            return;
+        }
+
+        LOG.info("Consumer commit offsets {}", offsetsToCommit);
+        SplitFetcher<MessageView, RocketMQSourceSplit> splitFetcher = fetchers.get(0);
+        if (splitFetcher != null) {
+            // The fetcher thread is still running. This should be the majority of the cases.
+            enqueueOffsetsCommitTask(splitFetcher, offsetsToCommit);
+        } else {
+            splitFetcher = createSplitFetcher();
+            enqueueOffsetsCommitTask(splitFetcher, offsetsToCommit);
+            startFetcher(splitFetcher);
+        }
+    }
+
+    private void enqueueOffsetsCommitTask(
+            SplitFetcher<MessageView, RocketMQSourceSplit> splitFetcher,
+            Map<MessageQueue, Long> offsetsToCommit) {
+
+        RocketMQSplitReader<?> splitReader = (RocketMQSplitReader<?>) splitFetcher.getSplitReader();
+
+        splitFetcher.enqueueTask(
+                new SplitFetcherTask() {
+                    @Override
+                    public boolean run() {
+                        splitReader.notifyCheckpointComplete(offsetsToCommit);
+                        return true;
+                    }
+
+                    @Override
+                    public void wakeUp() {}
+                });
+    }
+}
diff --git a/src/main/java/org/apache/flink/connector/rocketmq/source/reader/RocketMQSourceReader.java b/src/main/java/org/apache/flink/connector/rocketmq/source/reader/RocketMQSourceReader.java
new file mode 100644
index 0000000..9303f35
--- /dev/null
+++ b/src/main/java/org/apache/flink/connector/rocketmq/source/reader/RocketMQSourceReader.java
@@ -0,0 +1,153 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.flink.connector.rocketmq.source.reader;
+
+import org.apache.flink.annotation.VisibleForTesting;
+import org.apache.flink.api.connector.source.SourceReaderContext;
+import org.apache.flink.configuration.Configuration;
+import org.apache.flink.connector.base.source.reader.RecordEmitter;
+import org.apache.flink.connector.base.source.reader.RecordsWithSplitIds;
+import org.apache.flink.connector.base.source.reader.SingleThreadMultiplexSourceReaderBase;
+import org.apache.flink.connector.base.source.reader.synchronization.FutureCompletingBlockingQueue;
+import org.apache.flink.connector.rocketmq.source.RocketMQSourceOptions;
+import org.apache.flink.connector.rocketmq.source.metrics.RocketMQSourceReaderMetrics;
+import org.apache.flink.connector.rocketmq.source.split.RocketMQSourceSplit;
+import org.apache.flink.connector.rocketmq.source.split.RocketMQSourceSplitState;
+import org.apache.flink.connector.rocketmq.source.util.UtilAll;
+
+import org.apache.rocketmq.common.message.MessageQueue;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import java.util.Collections;
+import java.util.HashMap;
+import java.util.List;
+import java.util.Map;
+import java.util.SortedMap;
+import java.util.TreeMap;
+import java.util.concurrent.ConcurrentHashMap;
+import java.util.concurrent.ConcurrentMap;
+
+/** The source reader for RocketMQ partitions. */
+public class RocketMQSourceReader<T>
+        extends SingleThreadMultiplexSourceReaderBase<
+                MessageView, T, RocketMQSourceSplit, RocketMQSourceSplitState> {
+
+    private static final Logger LOG = LoggerFactory.getLogger(RocketMQSourceReader.class);
+
+    // These maps need to be concurrent because it will be accessed by both the main thread
+    // and the split fetcher thread in the callback.
+    private final boolean commitOffsetsOnCheckpoint;
+    private final SortedMap<Long, Map<MessageQueue, Long>> offsetsToCommit;
+    private final ConcurrentMap<MessageQueue, Long> offsetsOfFinishedSplits;
+    private final RocketMQSourceReaderMetrics rocketmqSourceReaderMetrics;
+
+    public RocketMQSourceReader(
+            FutureCompletingBlockingQueue<RecordsWithSplitIds<MessageView>> elementsQueue,
+            RocketMQSourceFetcherManager rocketmqSourceFetcherManager,
+            RecordEmitter<MessageView, T, RocketMQSourceSplitState> recordEmitter,
+            Configuration config,
+            SourceReaderContext context,
+            RocketMQSourceReaderMetrics rocketMQSourceReaderMetrics) {
+
+        super(elementsQueue, rocketmqSourceFetcherManager, recordEmitter, config, context);
+        this.offsetsToCommit = Collections.synchronizedSortedMap(new TreeMap<>());
+        this.offsetsOfFinishedSplits = new ConcurrentHashMap<>();
+        this.commitOffsetsOnCheckpoint =
+                config.get(RocketMQSourceOptions.COMMIT_OFFSETS_ON_CHECKPOINT);
+        this.rocketmqSourceReaderMetrics = rocketMQSourceReaderMetrics;
+    }
+
+    @Override
+    protected void onSplitFinished(Map<String, RocketMQSourceSplitState> finishedSplitIds) {
+        finishedSplitIds.forEach(
+                (ignored, splitState) -> {
+                    if (splitState.getCurrentOffset() >= 0) {
+                        offsetsOfFinishedSplits.put(
+                                splitState.getMessageQueue(), splitState.getCurrentOffset());
+                    }
+                });
+    }
+
+    @Override
+    public List<RocketMQSourceSplit> snapshotState(long checkpointId) {
+        List<RocketMQSourceSplit> splits = super.snapshotState(checkpointId);
+        if (!commitOffsetsOnCheckpoint) {
+            return splits;
+        }
+
+        if (splits.isEmpty() && offsetsOfFinishedSplits.isEmpty()) {
+            offsetsToCommit.put(checkpointId, Collections.emptyMap());
+        } else {
+            Map<MessageQueue, Long> offsetsMap =
+                    offsetsToCommit.computeIfAbsent(checkpointId, id -> new HashMap<>());
+            // Put the offsets of the active splits.
+            for (RocketMQSourceSplit split : splits) {
+                // If the checkpoint is triggered before the queue min offsets
+                // is retrieved, do not commit the offsets for those partitions.
+                if (split.getStartingOffset() >= 0) {
+                    offsetsMap.put(UtilAll.getMessageQueue(split), split.getStartingOffset());
+                }
+            }
+            // Put offsets of all the finished splits.
+            offsetsMap.putAll(offsetsOfFinishedSplits);
+        }
+        return splits;
+    }
+
+    @Override
+    public void notifyCheckpointComplete(long checkpointId) {
+        LOG.debug("Committing offsets for checkpoint {}", checkpointId);
+        if (!commitOffsetsOnCheckpoint) {
+            return;
+        }
+
+        Map<MessageQueue, Long> committedPartitions = offsetsToCommit.get(checkpointId);
+        if (committedPartitions == null) {
+            LOG.info(
+                    "Offsets for checkpoint {} either do not exist or have already been committed.",
+                    checkpointId);
+            return;
+        }
+
+        ((RocketMQSourceFetcherManager) splitFetcherManager).commitOffsets(committedPartitions);
+    }
+
+    @Override
+    protected RocketMQSourceSplitState initializedState(RocketMQSourceSplit partitionSplit) {
+        return new RocketMQSourceSplitState(partitionSplit);
+    }
+
+    @Override
+    protected RocketMQSourceSplit toSplitType(String splitId, RocketMQSourceSplitState splitState) {
+        return splitState.getSourceSplit();
+    }
+
+    // ------------------------
+
+    @VisibleForTesting
+    SortedMap<Long, Map<MessageQueue, Long>> getOffsetsToCommit() {
+        return offsetsToCommit;
+    }
+
+    @VisibleForTesting
+    int getNumAliveFetchers() {
+        return splitFetcherManager.getNumAliveFetchers();
+    }
+}
diff --git a/src/main/java/org/apache/flink/connector/rocketmq/source/reader/RocketMQSplitReader.java b/src/main/java/org/apache/flink/connector/rocketmq/source/reader/RocketMQSplitReader.java
new file mode 100644
index 0000000..b0bca4a
--- /dev/null
+++ b/src/main/java/org/apache/flink/connector/rocketmq/source/reader/RocketMQSplitReader.java
@@ -0,0 +1,297 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.flink.connector.rocketmq.source.reader;
+
+import org.apache.flink.annotation.Internal;
+import org.apache.flink.api.connector.source.SourceReaderContext;
+import org.apache.flink.api.java.tuple.Tuple2;
+import org.apache.flink.configuration.Configuration;
+import org.apache.flink.connector.base.source.reader.RecordsWithSplitIds;
+import org.apache.flink.connector.base.source.reader.splitreader.SplitReader;
+import org.apache.flink.connector.base.source.reader.splitreader.SplitsAddition;
+import org.apache.flink.connector.base.source.reader.splitreader.SplitsChange;
+import org.apache.flink.connector.rocketmq.common.config.RocketMQOptions;
+import org.apache.flink.connector.rocketmq.source.InnerConsumer;
+import org.apache.flink.connector.rocketmq.source.InnerConsumerImpl;
+import org.apache.flink.connector.rocketmq.source.RocketMQSourceOptions;
+import org.apache.flink.connector.rocketmq.source.metrics.RocketMQSourceReaderMetrics;
+import org.apache.flink.connector.rocketmq.source.reader.deserializer.RocketMQDeserializationSchema;
+import org.apache.flink.connector.rocketmq.source.split.RocketMQSourceSplit;
+import org.apache.flink.connector.rocketmq.source.util.UtilAll;
+import org.apache.flink.util.FlinkRuntimeException;
+import org.apache.flink.util.Preconditions;
+
+import org.apache.rocketmq.common.message.MessageQueue;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import javax.annotation.Nullable;
+
+import java.io.IOException;
+import java.time.Duration;
+import java.util.ArrayList;
+import java.util.Collection;
+import java.util.HashMap;
+import java.util.HashSet;
+import java.util.Iterator;
+import java.util.List;
+import java.util.Map;
+import java.util.Set;
+import java.util.SortedMap;
+import java.util.TreeMap;
+import java.util.concurrent.ConcurrentHashMap;
+import java.util.concurrent.ConcurrentMap;
+
+/**
+ * A {@link SplitReader} implementation that reads records from RocketMQ partitions.
+ *
+ * <p>The returned type are in the format of {@code tuple3(record, offset and timestamp}.
+ */
+@Internal
+public class RocketMQSplitReader<T> implements SplitReader<MessageView, RocketMQSourceSplit> {
+
+    private static final Logger LOG = LoggerFactory.getLogger(RocketMQSplitReader.class);
+
+    private volatile boolean wakeup = false;
+
+    private final InnerConsumer consumer;
+    private final Configuration configuration;
+    private final SourceReaderContext sourceReaderContext;
+    private final RocketMQDeserializationSchema<T> deserializationSchema;
+
+    // These maps need to be concurrent because it will be accessed by both the main thread
+    // and the split fetcher thread in the callback.
+    private final boolean commitOffsetsOnCheckpoint;
+    private final SortedMap<Long, Map<MessageQueue, Long>> offsetsToCommit;
+
+    private final ConcurrentMap<MessageQueue, Tuple2<Long, Long>> currentOffsetTable;
+    private final RocketMQSourceReaderMetrics rocketmqSourceReaderMetrics;
+
+    public RocketMQSplitReader(
+            Configuration configuration,
+            SourceReaderContext sourceReaderContext,
+            RocketMQDeserializationSchema<T> deserializationSchema,
+            RocketMQSourceReaderMetrics rocketmqSourceReaderMetrics) {
+
+        this.configuration = configuration;
+        this.sourceReaderContext = sourceReaderContext;
+        this.deserializationSchema = deserializationSchema;
+        this.offsetsToCommit = new TreeMap<>();
+        this.currentOffsetTable = new ConcurrentHashMap<>();
+
+        this.consumer = new InnerConsumerImpl(configuration);
+        this.consumer.start();
+
+        this.rocketmqSourceReaderMetrics = rocketmqSourceReaderMetrics;
+        this.commitOffsetsOnCheckpoint =
+                configuration.getBoolean(RocketMQOptions.COMMIT_OFFSETS_ON_CHECKPOINT);
+    }
+
+    @Override
+    public RecordsWithSplitIds<MessageView> fetch() throws IOException {
+        wakeup = false;
+        RocketMQRecordsWithSplitIds<MessageView> recordsWithSplitIds =
+                new RocketMQRecordsWithSplitIds<>(rocketmqSourceReaderMetrics);
+        try {
+            Duration duration =
+                    Duration.ofMillis(this.configuration.getLong(RocketMQOptions.POLL_TIMEOUT));
+            List<MessageView> messageExtList = consumer.poll(duration);
+            for (MessageView messageView : messageExtList) {
+                String splitId =
+                        UtilAll.getSplitId(
+                                new MessageQueue(
+                                        messageView.getTopic(),
+                                        messageView.getBrokerName(),
+                                        messageView.getQueueId()));
+                recordsWithSplitIds.recordsForSplit(splitId).add(messageView);
+                if (this.configuration.getBoolean(RocketMQSourceOptions.GLOBAL_DEBUG_MODE)) {
+                    LOG.info(
+                            "Reader fetch splitId: {}, messageId: {}",
+                            splitId,
+                            messageView.getMessageId());
+                }
+            }
+            recordsWithSplitIds.prepareForRead();
+        } catch (Exception e) {
+            LOG.error("Reader fetch split error", e);
+        }
+        return recordsWithSplitIds;
+    }
+
+    @Override
+    public void handleSplitsChanges(SplitsChange<RocketMQSourceSplit> splitsChange) {
+        // Current not support assign addition splits.
+        if (!(splitsChange instanceof SplitsAddition)) {
+            throw new UnsupportedOperationException(
+                    String.format(
+                            "The SplitChange type of %s is not supported.",
+                            splitsChange.getClass()));
+        }
+
+        // Assignment.
+        ConcurrentMap<MessageQueue, Tuple2<Long, Long>> newOffsetTable = new ConcurrentHashMap<>();
+
+        // Set up the stopping timestamps.
+        splitsChange
+                .splits()
+                .forEach(
+                        split -> {
+                            MessageQueue messageQueue =
+                                    new MessageQueue(
+                                            split.getTopic(),
+                                            split.getBrokerName(),
+                                            split.getQueueId());
+                            newOffsetTable.put(
+                                    messageQueue,
+                                    new Tuple2<>(
+                                            split.getStartingOffset(), split.getStoppingOffset()));
+                            rocketmqSourceReaderMetrics.registerNewMessageQueue(messageQueue);
+                        });
+
+        // todo: log message queue change
+
+        // It will replace the previous assignment
+        Set<MessageQueue> incrementalSplits = newOffsetTable.keySet();
+        consumer.assign(incrementalSplits);
+
+        // set offset to consumer
+        for (Map.Entry<MessageQueue, Tuple2<Long, Long>> entry : newOffsetTable.entrySet()) {
+            MessageQueue messageQueue = entry.getKey();
+            Long startingOffset = entry.getValue().f0;
+            try {
+                consumer.seek(messageQueue, startingOffset);
+            } catch (Exception e) {
+                String info =
+                        String.format(
+                                "messageQueue:%s, seek to starting offset:%s",
+                                messageQueue, startingOffset);
+                throw new FlinkRuntimeException(info, e);
+            }
+        }
+    }
+
+    @Override
+    public void wakeUp() {
+        LOG.debug("Wake up the split reader in case the fetcher thread is blocking in fetch().");
+        wakeup = true;
+        this.consumer.wakeup();
+    }
+
+    @Override
+    public void close() {
+        try {
+            consumer.close();
+        } catch (Exception e) {
+            LOG.error("close consumer error", e);
+        }
+    }
+
+    public void notifyCheckpointComplete(Map<MessageQueue, Long> offsetsToCommit) {
+        if (offsetsToCommit != null) {
+            for (Map.Entry<MessageQueue, Long> entry : offsetsToCommit.entrySet()) {
+                consumer.commitOffset(entry.getKey(), entry.getValue());
+            }
+        }
+    }
+
+    private void finishSplitAtRecord(
+            MessageQueue messageQueue,
+            long currentOffset,
+            RocketMQRecordsWithSplitIds<MessageView> recordsBySplits) {
+
+        LOG.info("message queue {} has reached stopping offset {}", messageQueue, currentOffset);
+        // recordsBySplits.addFinishedSplit(getSplitId(messageQueue));
+        this.currentOffsetTable.remove(messageQueue);
+    }
+
+    // ---------------- private helper class ------------------------
+
+    private static class RocketMQRecordsWithSplitIds<T> implements RecordsWithSplitIds<T> {
+
+        // Mark split message queue identifier as current split id
+        private String currentSplitId;
+
+        private final Set<String> finishedSplits = new HashSet<>();
+        private Iterator<T> recordIterator;
+
+        private final Map<String, List<T>> recordsBySplits = new HashMap<>();
+        private Iterator<Map.Entry<String, List<T>>> splitIterator;
+
+        private final RocketMQSourceReaderMetrics readerMetrics;
+
+        public RocketMQRecordsWithSplitIds(RocketMQSourceReaderMetrics readerMetrics) {
+            this.readerMetrics = readerMetrics;
+        }
+
+        /** return records container */
+        private Collection<T> recordsForSplit(String splitId) {
+            return this.recordsBySplits.computeIfAbsent(splitId, id -> new ArrayList<>());
+        }
+
+        private void addFinishedSplit(String splitId) {
+            this.finishedSplits.add(splitId);
+        }
+
+        private void prepareForRead() {
+            this.splitIterator = recordsBySplits.entrySet().iterator();
+        }
+
+        /**
+         * Moves to the next split. This method is also called initially to move to the first split.
+         * Returns null, if no splits are left.
+         */
+        @Nullable
+        @Override
+        public String nextSplit() {
+            if (splitIterator.hasNext()) {
+                Map.Entry<String, List<T>> entry = splitIterator.next();
+                currentSplitId = entry.getKey();
+                recordIterator = entry.getValue().iterator();
+                return currentSplitId;
+            } else {
+                currentSplitId = null;
+                recordIterator = null;
+                return null;
+            }
+        }
+
+        /**
+         * Gets the next record from the current split. Returns null if no more records are left in
+         * this split.
+         */
+        @Nullable
+        @Override
+        public T nextRecordFromSplit() {
+            Preconditions.checkNotNull(
+                    currentSplitId,
+                    "Make sure nextSplit() did not return null before "
+                            + "iterate over the records split.");
+            if (recordIterator.hasNext()) {
+                return recordIterator.next();
+                // todo: support metrics here
+            }
+            return null;
+        }
+
+        @Override
+        public Set<String> finishedSplits() {
+            return finishedSplits;
+        }
+    }
+}
diff --git a/src/main/java/org/apache/rocketmq/flink/source/reader/deserializer/BytesMessage.java b/src/main/java/org/apache/flink/connector/rocketmq/source/reader/deserializer/BytesMessage.java
similarity index 95%
rename from src/main/java/org/apache/rocketmq/flink/source/reader/deserializer/BytesMessage.java
rename to src/main/java/org/apache/flink/connector/rocketmq/source/reader/deserializer/BytesMessage.java
index d109a7f..d3173e0 100644
--- a/src/main/java/org/apache/rocketmq/flink/source/reader/deserializer/BytesMessage.java
+++ b/src/main/java/org/apache/flink/connector/rocketmq/source/reader/deserializer/BytesMessage.java
@@ -16,7 +16,7 @@
  * limitations under the License.
  */
 
-package org.apache.rocketmq.flink.source.reader.deserializer;
+package org.apache.flink.connector.rocketmq.source.reader.deserializer;
 
 import java.util.HashMap;
 import java.util.Map;
@@ -25,6 +25,7 @@
 public class BytesMessage {
 
     private byte[] data;
+
     private Map<String, String> properties = new HashMap<>();
 
     public byte[] getData() {
diff --git a/src/main/java/org/apache/rocketmq/flink/source/reader/deserializer/DirtyDataStrategy.java b/src/main/java/org/apache/flink/connector/rocketmq/source/reader/deserializer/DirtyDataStrategy.java
similarity index 92%
rename from src/main/java/org/apache/rocketmq/flink/source/reader/deserializer/DirtyDataStrategy.java
rename to src/main/java/org/apache/flink/connector/rocketmq/source/reader/deserializer/DirtyDataStrategy.java
index 06a0c2d..2770e8a 100644
--- a/src/main/java/org/apache/rocketmq/flink/source/reader/deserializer/DirtyDataStrategy.java
+++ b/src/main/java/org/apache/flink/connector/rocketmq/source/reader/deserializer/DirtyDataStrategy.java
@@ -15,7 +15,7 @@
  * limitations under the License.
  */
 
-package org.apache.rocketmq.flink.source.reader.deserializer;
+package org.apache.flink.connector.rocketmq.source.reader.deserializer;
 
 /** Dirty data process strategy. */
 public enum DirtyDataStrategy {
diff --git a/src/main/java/org/apache/rocketmq/flink/source/reader/deserializer/DeserializationSchema.java b/src/main/java/org/apache/flink/connector/rocketmq/source/reader/deserializer/QueryableSchema.java
similarity index 92%
rename from src/main/java/org/apache/rocketmq/flink/source/reader/deserializer/DeserializationSchema.java
rename to src/main/java/org/apache/flink/connector/rocketmq/source/reader/deserializer/QueryableSchema.java
index f0f47f3..4dfcebe 100644
--- a/src/main/java/org/apache/rocketmq/flink/source/reader/deserializer/DeserializationSchema.java
+++ b/src/main/java/org/apache/flink/connector/rocketmq/source/reader/deserializer/QueryableSchema.java
@@ -16,7 +16,7 @@
  * limitations under the License.
  */
 
-package org.apache.rocketmq.flink.source.reader.deserializer;
+package org.apache.flink.connector.rocketmq.source.reader.deserializer;
 
 import org.apache.flink.annotation.PublicEvolving;
 import org.apache.flink.api.common.serialization.DeserializationSchema.InitializationContext;
@@ -27,7 +27,7 @@
 import java.io.Serializable;
 
 /** An interface for the deserialization of records. */
-public interface DeserializationSchema<IN, OUT> extends Serializable, ResultTypeQueryable<OUT> {
+public interface QueryableSchema<IN, OUT> extends Serializable, ResultTypeQueryable<OUT> {
 
     /**
      * Initialization method for the schema. It is called before the actual working methods {@link
diff --git a/src/main/java/org/apache/flink/connector/rocketmq/source/reader/deserializer/RocketMQDeserializationSchema.java b/src/main/java/org/apache/flink/connector/rocketmq/source/reader/deserializer/RocketMQDeserializationSchema.java
new file mode 100644
index 0000000..5985ada
--- /dev/null
+++ b/src/main/java/org/apache/flink/connector/rocketmq/source/reader/deserializer/RocketMQDeserializationSchema.java
@@ -0,0 +1,88 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.flink.connector.rocketmq.source.reader.deserializer;
+
+import org.apache.flink.annotation.PublicEvolving;
+import org.apache.flink.api.common.serialization.DeserializationSchema;
+import org.apache.flink.api.common.serialization.DeserializationSchema.InitializationContext;
+import org.apache.flink.api.java.typeutils.ResultTypeQueryable;
+import org.apache.flink.connector.rocketmq.source.reader.MessageView;
+import org.apache.flink.util.Collector;
+
+import java.io.IOException;
+import java.io.Serializable;
+
+/** An interface for the deserialization of RocketMQ records. */
+@PublicEvolving
+public interface RocketMQDeserializationSchema<T> extends Serializable, ResultTypeQueryable<T> {
+
+    /**
+     * Initialization method for the schema. It is called before the actual working methods {@link
+     * #deserialize} and thus suitable for one time setup work.
+     *
+     * <p>The provided {@link InitializationContext} can be used to access additional features such
+     * as e.g. registering user metrics.
+     *
+     * @param context Contextual information that can be used during initialization.
+     */
+    default void open(DeserializationSchema.InitializationContext context) throws Exception {
+        // Nothing to do here for the default implementation.
+    }
+
+    /**
+     * Deserializes the byte message.
+     *
+     * <p>Can output multiple records through the {@link Collector}. Note that number and size of
+     * the produced records should be relatively small. Depending on the source implementation
+     * records can be buffered in memory or collecting records might delay emitting checkpoint
+     * barrier.
+     *
+     * @param messageView The MessageView to deserialize.
+     * @param out The collector to put the resulting messages.
+     */
+    void deserialize(MessageView messageView, Collector<T> out) throws IOException;
+
+    /**
+     * Create a RocketMQDeserializationSchema by using the flink's {@link DeserializationSchema}. It
+     * would consume the rocketmq message as byte array and decode the message by using flink's
+     * logic.
+     */
+    static <T> RocketMQDeserializationSchema<T> flinkSchema(
+            DeserializationSchema<T> deserializationSchema) {
+        return new RocketMQDeserializationSchemaWrapper<>(deserializationSchema);
+    }
+
+    /**
+     * Wraps a {@link DeserializationSchema} as the value deserialization schema. The other fields
+     * such as key, headers, timestamp are ignored.
+     *
+     * @param deserializationSchema the {@link DeserializationSchema} used to deserialize the value
+     *     of a {@link RocketMQDeserializationSchemaWrapper}.
+     * @param <T> the type of the deserialized record.
+     */
+    static <T> RocketMQDeserializationSchema<T> flinkBodyOnlySchema(
+            DeserializationSchema<T> deserializationSchema) {
+        return new RocketMQDeserializationSchemaWrapper<>(deserializationSchema);
+    }
+
+    static <T> RocketMQDeserializationSchema<T> rocketMQSchema(
+            DeserializationSchema<T> valueDeserializationSchema) {
+        return new RocketMQDeserializationSchemaWrapper<>(valueDeserializationSchema);
+    }
+}
diff --git a/src/main/java/org/apache/flink/connector/rocketmq/source/reader/deserializer/RocketMQDeserializationSchemaWrapper.java b/src/main/java/org/apache/flink/connector/rocketmq/source/reader/deserializer/RocketMQDeserializationSchemaWrapper.java
new file mode 100644
index 0000000..bbfe879
--- /dev/null
+++ b/src/main/java/org/apache/flink/connector/rocketmq/source/reader/deserializer/RocketMQDeserializationSchemaWrapper.java
@@ -0,0 +1,57 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.flink.connector.rocketmq.source.reader.deserializer;
+
+import org.apache.flink.annotation.Internal;
+import org.apache.flink.api.common.serialization.DeserializationSchema;
+import org.apache.flink.api.common.typeinfo.TypeInformation;
+import org.apache.flink.connector.rocketmq.source.reader.MessageView;
+import org.apache.flink.util.Collector;
+
+import java.io.IOException;
+
+/**
+ * A {@link RocketMQDeserializationSchema} implementation which based on the given flink's {@link
+ * DeserializationSchema}. We would consume the message as byte array from rocketmq and deserialize
+ * it by using flink serialization logic.
+ *
+ * @param <T> The output type of the message.
+ */
+@Internal
+public class RocketMQDeserializationSchemaWrapper<T> implements RocketMQDeserializationSchema<T> {
+
+    private static final long serialVersionUID = 1L;
+    private final DeserializationSchema<T> deserializationSchema;
+
+    RocketMQDeserializationSchemaWrapper(DeserializationSchema<T> deserializationSchema) {
+        this.deserializationSchema = deserializationSchema;
+    }
+
+    @Override
+    public void open(DeserializationSchema.InitializationContext context) throws Exception {
+        RocketMQDeserializationSchema.super.open(context);
+    }
+
+    @Override
+    public void deserialize(MessageView messageView, Collector<T> out) throws IOException {}
+
+    @Override
+    public TypeInformation<T> getProducedType() {
+        return deserializationSchema.getProducedType();
+    }
+}
diff --git a/src/main/java/org/apache/rocketmq/flink/source/reader/deserializer/RocketMQRowDeserializationSchema.java b/src/main/java/org/apache/flink/connector/rocketmq/source/reader/deserializer/RocketMQRowDeserializationSchema.java
similarity index 86%
rename from src/main/java/org/apache/rocketmq/flink/source/reader/deserializer/RocketMQRowDeserializationSchema.java
rename to src/main/java/org/apache/flink/connector/rocketmq/source/reader/deserializer/RocketMQRowDeserializationSchema.java
index 5bd990e..c4cefca 100644
--- a/src/main/java/org/apache/rocketmq/flink/source/reader/deserializer/RocketMQRowDeserializationSchema.java
+++ b/src/main/java/org/apache/flink/connector/rocketmq/source/reader/deserializer/RocketMQRowDeserializationSchema.java
@@ -16,18 +16,20 @@
  * limitations under the License.
  */
 
-package org.apache.rocketmq.flink.source.reader.deserializer;
-
-import org.apache.rocketmq.common.message.MessageExt;
-import org.apache.rocketmq.flink.source.reader.deserializer.RowDeserializationSchema.MetadataConverter;
+package org.apache.flink.connector.rocketmq.source.reader.deserializer;
 
 import org.apache.flink.annotation.VisibleForTesting;
 import org.apache.flink.api.common.serialization.DeserializationSchema.InitializationContext;
 import org.apache.flink.api.common.typeinfo.TypeInformation;
+import org.apache.flink.connector.rocketmq.source.reader.MessageView;
+import org.apache.flink.connector.rocketmq.source.reader.deserializer.RowDeserializationSchema.MetadataConverter;
 import org.apache.flink.table.api.TableSchema;
 import org.apache.flink.table.data.RowData;
 import org.apache.flink.util.Collector;
 
+import org.apache.rocketmq.common.message.MessageExt;
+
+import java.io.IOException;
 import java.util.ArrayList;
 import java.util.List;
 import java.util.Map;
@@ -65,11 +67,17 @@
     }
 
     @Override
-    public void deserialize(List<MessageExt> input, Collector<RowData> collector) {
-        extractMessages(input);
-        deserializationSchema.deserialize(bytesMessages, collector);
+    public void deserialize(MessageView messageView, Collector<RowData> collector)
+            throws IOException {
+
+        // extractMessages(input);
+        // deserializationSchema.deserialize(bytesMessages, collector);
     }
 
+    // @Override
+    // public void deserialize(List<MessageExt> input, Collector<RowData> collector) {
+    // }
+
     @Override
     public TypeInformation<RowData> getProducedType() {
         return deserializationSchema.getProducedType();
diff --git a/src/main/java/org/apache/rocketmq/flink/legacy/common/serialization/ForwardMessageExtDeserialization.java b/src/main/java/org/apache/flink/connector/rocketmq/source/reader/deserializer/RocketMQSchemaWrapper.java
similarity index 64%
copy from src/main/java/org/apache/rocketmq/flink/legacy/common/serialization/ForwardMessageExtDeserialization.java
copy to src/main/java/org/apache/flink/connector/rocketmq/source/reader/deserializer/RocketMQSchemaWrapper.java
index 6c6fa74..408c6a3 100644
--- a/src/main/java/org/apache/rocketmq/flink/legacy/common/serialization/ForwardMessageExtDeserialization.java
+++ b/src/main/java/org/apache/flink/connector/rocketmq/source/reader/deserializer/RocketMQSchemaWrapper.java
@@ -15,23 +15,20 @@
  * limitations under the License.
  */
 
-package org.apache.rocketmq.flink.legacy.common.serialization;
-
-import org.apache.rocketmq.common.message.MessageExt;
+package org.apache.flink.connector.rocketmq.source.reader.deserializer;
 
 import org.apache.flink.api.common.typeinfo.TypeInformation;
+import org.apache.flink.connector.rocketmq.source.reader.MessageView;
+import org.apache.flink.util.Collector;
 
-/** A Forward messageExt deserialization. */
-public class ForwardMessageExtDeserialization
-        implements MessageExtDeserializationScheme<MessageExt> {
+import java.io.IOException;
+
+public class RocketMQSchemaWrapper<T> implements RocketMQDeserializationSchema<T> {
+    @Override
+    public void deserialize(MessageView messageView, Collector<T> out) throws IOException {}
 
     @Override
-    public MessageExt deserializeMessageExt(MessageExt messageExt) {
-        return messageExt;
-    }
-
-    @Override
-    public TypeInformation<MessageExt> getProducedType() {
-        return TypeInformation.of(MessageExt.class);
+    public TypeInformation<T> getProducedType() {
+        return null;
     }
 }
diff --git a/src/main/java/org/apache/rocketmq/flink/source/reader/deserializer/RowDeserializationSchema.java b/src/main/java/org/apache/flink/connector/rocketmq/source/reader/deserializer/RowDeserializationSchema.java
similarity index 96%
rename from src/main/java/org/apache/rocketmq/flink/source/reader/deserializer/RowDeserializationSchema.java
rename to src/main/java/org/apache/flink/connector/rocketmq/source/reader/deserializer/RowDeserializationSchema.java
index 8beaaa2..98baef7 100644
--- a/src/main/java/org/apache/rocketmq/flink/source/reader/deserializer/RowDeserializationSchema.java
+++ b/src/main/java/org/apache/flink/connector/rocketmq/source/reader/deserializer/RowDeserializationSchema.java
@@ -16,17 +16,15 @@
  * limitations under the License.
  */
 
-package org.apache.rocketmq.flink.source.reader.deserializer;
-
-import org.apache.rocketmq.flink.source.util.ByteSerializer;
-import org.apache.rocketmq.flink.source.util.ByteSerializer.ValueType;
-import org.apache.rocketmq.flink.source.util.StringSerializer;
+package org.apache.flink.connector.rocketmq.source.reader.deserializer;
 
 import org.apache.flink.api.common.serialization.DeserializationSchema.InitializationContext;
 import org.apache.flink.api.common.typeinfo.TypeInformation;
 import org.apache.flink.configuration.ConfigOption;
 import org.apache.flink.configuration.ConfigOptions;
 import org.apache.flink.configuration.Configuration;
+import org.apache.flink.connector.rocketmq.source.util.ByteSerializer;
+import org.apache.flink.connector.rocketmq.source.util.StringSerializer;
 import org.apache.flink.table.api.TableSchema;
 import org.apache.flink.table.data.GenericRowData;
 import org.apache.flink.table.data.RowData;
@@ -50,11 +48,8 @@
 import java.util.Map;
 import java.util.Set;
 
-/**
- * The row based implementation of {@link DeserializationSchema} for the deserialization of records.
- */
-public class RowDeserializationSchema
-        implements DeserializationSchema<List<BytesMessage>, RowData> {
+/** The row based implementation of {@link QueryableSchema} for the deserialization of records. */
+public class RowDeserializationSchema implements QueryableSchema<List<BytesMessage>, RowData> {
 
     private static final long serialVersionUID = -1L;
     private static final Logger LOGGER = LoggerFactory.getLogger(RowDeserializationSchema.class);
@@ -70,7 +65,7 @@
     private final MetadataCollector metadataCollector;
     private final int totalColumnSize;
     private final int dataColumnSize;
-    private final ValueType[] fieldTypes;
+    private final ByteSerializer.ValueType[] fieldTypes;
     private transient DataType[] fieldDataTypes;
     private final Set<String> headerFields;
     private final Map<String, String> properties;
@@ -107,14 +102,14 @@
         this.properties = properties;
         this.totalColumnSize = tableSchema.getFieldNames().length;
         int dataColumnSize = 0;
-        this.fieldTypes = new ValueType[totalColumnSize];
+        this.fieldTypes = new ByteSerializer.ValueType[totalColumnSize];
         this.columnIndexMapping = new HashMap<>();
         this.dataIndexMapping = new HashMap<>();
         for (int index = 0; index < totalColumnSize; index++) {
             this.columnIndexMapping.put(tableSchema.getFieldNames()[index], index);
         }
         for (int index = 0; index < totalColumnSize; index++) {
-            ValueType type =
+            ByteSerializer.ValueType type =
                     ByteSerializer.getTypeIndex(tableSchema.getFieldTypes()[index].getTypeClass());
             this.fieldTypes[index] = type;
             if (!isHeaderField(index)) {
@@ -275,8 +270,9 @@
         TypeInformation<?> typeInformation =
                 tableSchema.getFieldTypes()[columnIndexMapping.get(fieldName)];
         if (typeInformation != null) {
-            ValueType valueType = ByteSerializer.getTypeIndex(typeInformation.getTypeClass());
-            return valueType == ValueType.V_ByteArray;
+            ByteSerializer.ValueType valueType =
+                    ByteSerializer.getTypeIndex(typeInformation.getTypeClass());
+            return valueType == ByteSerializer.ValueType.V_ByteArray;
         }
         return false;
     }
diff --git a/src/main/java/org/apache/flink/connector/rocketmq/source/split/RocketMQPartitionSplitSerializer.java b/src/main/java/org/apache/flink/connector/rocketmq/source/split/RocketMQPartitionSplitSerializer.java
new file mode 100644
index 0000000..36c7a0f
--- /dev/null
+++ b/src/main/java/org/apache/flink/connector/rocketmq/source/split/RocketMQPartitionSplitSerializer.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.flink.connector.rocketmq.source.split;
+
+import org.apache.flink.core.io.SimpleVersionedSerializer;
+
+import java.io.ByteArrayInputStream;
+import java.io.ByteArrayOutputStream;
+import java.io.DataInputStream;
+import java.io.DataOutputStream;
+import java.io.IOException;
+
+/** The {@link SimpleVersionedSerializer serializer} for {@link RocketMQSourceSplit}. */
+public class RocketMQPartitionSplitSerializer
+        implements SimpleVersionedSerializer<RocketMQSourceSplit> {
+
+    private static final int SNAPSHOT_VERSION = 0;
+    private static final int CURRENT_VERSION = 1;
+
+    @Override
+    public int getVersion() {
+        return CURRENT_VERSION;
+    }
+
+    @Override
+    public byte[] serialize(RocketMQSourceSplit split) throws IOException {
+        try (ByteArrayOutputStream byteArrayOutputStream = new ByteArrayOutputStream();
+                DataOutputStream out = new DataOutputStream(byteArrayOutputStream)) {
+            out.writeUTF(split.getTopic());
+            out.writeUTF(split.getBrokerName());
+            out.writeInt(split.getQueueId());
+            out.writeLong(split.getStartingOffset());
+            out.writeLong(split.getStoppingOffset());
+            out.flush();
+            return byteArrayOutputStream.toByteArray();
+        }
+    }
+
+    @Override
+    public RocketMQSourceSplit deserialize(int version, byte[] serialized) throws IOException {
+        try (ByteArrayInputStream byteArrayInputStream = new ByteArrayInputStream(serialized);
+                DataInputStream in = new DataInputStream(byteArrayInputStream)) {
+            String topic = in.readUTF();
+            String broker = in.readUTF();
+            int partition = in.readInt();
+            long startingOffset = in.readLong();
+            long stoppingOffset = in.readLong();
+            return new RocketMQSourceSplit(
+                    topic, broker, partition, startingOffset, stoppingOffset);
+        }
+    }
+}
diff --git a/src/main/java/org/apache/flink/connector/rocketmq/source/split/RocketMQSourceSplit.java b/src/main/java/org/apache/flink/connector/rocketmq/source/split/RocketMQSourceSplit.java
new file mode 100644
index 0000000..7124086
--- /dev/null
+++ b/src/main/java/org/apache/flink/connector/rocketmq/source/split/RocketMQSourceSplit.java
@@ -0,0 +1,117 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.flink.connector.rocketmq.source.split;
+
+import org.apache.flink.api.connector.source.SourceSplit;
+
+import org.apache.rocketmq.common.message.MessageQueue;
+
+import java.util.Objects;
+
+import static org.apache.flink.connector.rocketmq.source.util.UtilAll.SEPARATOR;
+
+/** A {@link SourceSplit} for a RocketMQ partition. */
+public class RocketMQSourceSplit implements SourceSplit {
+
+    // -1 means Long.MAX_VALUE
+    public static final long NO_STOPPING_OFFSET = -1L;
+
+    private final String topic;
+    private final String brokerName;
+    private final int queueId;
+    private final long startingOffset;
+    private final long stoppingOffset;
+
+    public RocketMQSourceSplit(
+            MessageQueue messageQueue, long startingOffset, long stoppingOffset) {
+        this(
+                messageQueue.getTopic(),
+                messageQueue.getBrokerName(),
+                messageQueue.getQueueId(),
+                startingOffset,
+                stoppingOffset);
+    }
+
+    public RocketMQSourceSplit(
+            String topic,
+            String brokerName,
+            int queueId,
+            long startingOffset,
+            long stoppingOffset) {
+        this.topic = topic;
+        this.brokerName = brokerName;
+        this.queueId = queueId;
+        this.startingOffset = startingOffset;
+        this.stoppingOffset = stoppingOffset;
+    }
+
+    public String getTopic() {
+        return topic;
+    }
+
+    public String getBrokerName() {
+        return brokerName;
+    }
+
+    public int getQueueId() {
+        return queueId;
+    }
+
+    public long getStartingOffset() {
+        return startingOffset;
+    }
+
+    public long getStoppingOffset() {
+        return stoppingOffset;
+    }
+
+    public MessageQueue getMessageQueue() {
+        return new MessageQueue(topic, brokerName, queueId);
+    }
+
+    @Override
+    public String splitId() {
+        return topic + SEPARATOR + brokerName + SEPARATOR + queueId;
+    }
+
+    @Override
+    public String toString() {
+        return String.format(
+                "(Topic: %s, BrokerName: %s, QueueId: %d, MinOffset: %d, MaxOffset: %d)",
+                topic, brokerName, queueId, startingOffset, stoppingOffset);
+    }
+
+    @Override
+    public int hashCode() {
+        return Objects.hash(topic, brokerName, queueId, startingOffset, stoppingOffset);
+    }
+
+    @Override
+    public boolean equals(Object obj) {
+        if (!(obj instanceof RocketMQSourceSplit)) {
+            return false;
+        }
+        RocketMQSourceSplit other = (RocketMQSourceSplit) obj;
+        return topic.equals(other.topic)
+                && brokerName.equals(other.brokerName)
+                && queueId == other.queueId
+                && startingOffset == other.startingOffset
+                && stoppingOffset == other.stoppingOffset;
+    }
+}
diff --git a/src/main/java/org/apache/rocketmq/flink/source/split/RocketMQPartitionSplitState.java b/src/main/java/org/apache/flink/connector/rocketmq/source/split/RocketMQSourceSplitState.java
similarity index 60%
rename from src/main/java/org/apache/rocketmq/flink/source/split/RocketMQPartitionSplitState.java
rename to src/main/java/org/apache/flink/connector/rocketmq/source/split/RocketMQSourceSplitState.java
index 4fbb3da..ca74d58 100644
--- a/src/main/java/org/apache/rocketmq/flink/source/split/RocketMQPartitionSplitState.java
+++ b/src/main/java/org/apache/flink/connector/rocketmq/source/split/RocketMQSourceSplitState.java
@@ -16,20 +16,20 @@
  * limitations under the License.
  */
 
-package org.apache.rocketmq.flink.source.split;
+package org.apache.flink.connector.rocketmq.source.split;
 
-/** This class extends RocketMQPartitionSplit to track a mutable current offset. */
-public class RocketMQPartitionSplitState extends RocketMQPartitionSplit {
+/** This class extends RocketMQSourceSplit to track a mutable current offset. */
+public class RocketMQSourceSplitState extends RocketMQSourceSplit {
 
     private long currentOffset;
 
-    public RocketMQPartitionSplitState(RocketMQPartitionSplit partitionSplit) {
+    public RocketMQSourceSplitState(RocketMQSourceSplit partitionSplit) {
         super(
                 partitionSplit.getTopic(),
-                partitionSplit.getBroker(),
-                partitionSplit.getPartition(),
+                partitionSplit.getBrokerName(),
+                partitionSplit.getQueueId(),
                 partitionSplit.getStartingOffset(),
-                partitionSplit.getStoppingTimestamp());
+                partitionSplit.getStoppingOffset());
         this.currentOffset = partitionSplit.getStartingOffset();
     }
 
@@ -42,16 +42,12 @@
     }
 
     /**
-     * Use the current offset as the starting offset to create a new RocketMQPartitionSplit.
+     * Use the current offset as the starting offset to create a new RocketMQSourceSplit.
      *
-     * @return a new RocketMQPartitionSplit which uses the current offset as its starting offset.
+     * @return a new RocketMQSourceSplit which uses the current offset as its starting offset.
      */
-    public RocketMQPartitionSplit toRocketMQPartitionSplit() {
-        return new RocketMQPartitionSplit(
-                getTopic(),
-                getBroker(),
-                getPartition(),
-                getCurrentOffset(),
-                getStoppingTimestamp());
+    public RocketMQSourceSplit getSourceSplit() {
+        return new RocketMQSourceSplit(
+                getTopic(), getBrokerName(), getQueueId(), getCurrentOffset(), getStoppingOffset());
     }
 }
diff --git a/src/main/java/org/apache/flink/connector/rocketmq/source/table/RocketMQDynamicTableSourceFactory.java b/src/main/java/org/apache/flink/connector/rocketmq/source/table/RocketMQDynamicTableSourceFactory.java
new file mode 100644
index 0000000..2161385
--- /dev/null
+++ b/src/main/java/org/apache/flink/connector/rocketmq/source/table/RocketMQDynamicTableSourceFactory.java
@@ -0,0 +1,189 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.flink.connector.rocketmq.source.table;
+
+import org.apache.flink.configuration.ConfigOption;
+import org.apache.flink.configuration.Configuration;
+import org.apache.flink.connector.rocketmq.legacy.RocketMQConfig;
+import org.apache.flink.connector.rocketmq.source.RocketMQSourceOptions;
+import org.apache.flink.table.api.TableSchema;
+import org.apache.flink.table.connector.source.DynamicTableSource;
+import org.apache.flink.table.descriptors.DescriptorProperties;
+import org.apache.flink.table.factories.DynamicTableSourceFactory;
+import org.apache.flink.table.factories.FactoryUtil;
+import org.apache.flink.table.utils.TableSchemaUtils;
+import org.apache.flink.util.Preconditions;
+import org.apache.flink.util.StringUtils;
+
+import org.apache.commons.lang3.time.FastDateFormat;
+
+import java.text.ParseException;
+import java.util.HashSet;
+import java.util.Map;
+import java.util.Set;
+import java.util.TimeZone;
+
+import static org.apache.flink.table.factories.FactoryUtil.createTableFactoryHelper;
+
+/**
+ * Defines the {@link DynamicTableSourceFactory} implementation to create {@link
+ * RocketMQScanTableSource}.
+ */
+public class RocketMQDynamicTableSourceFactory implements DynamicTableSourceFactory {
+
+    private static final String DATE_FORMAT = "yyyy-MM-dd HH:mm:ss";
+
+    @Override
+    public String factoryIdentifier() {
+        return "rocketmq";
+    }
+
+    @Override
+    public Set<ConfigOption<?>> requiredOptions() {
+        Set<ConfigOption<?>> requiredOptions = new HashSet<>();
+        requiredOptions.add(RocketMQSourceOptions.TOPIC);
+        requiredOptions.add(RocketMQSourceOptions.CONSUMER_GROUP);
+        return requiredOptions;
+    }
+
+    @Override
+    public Set<ConfigOption<?>> optionalOptions() {
+        Set<ConfigOption<?>> optionalOptions = new HashSet<>();
+        optionalOptions.add(RocketMQSourceOptions.OPTIONAL_TAG);
+        optionalOptions.add(RocketMQSourceOptions.OPTIONAL_SQL);
+        optionalOptions.add(RocketMQSourceOptions.OPTIONAL_STARTUP_OFFSET_SPECIFIC);
+        optionalOptions.add(RocketMQSourceOptions.OPTIONAL_STARTUP_OFFSET_TIMESTAMP);
+        optionalOptions.add(RocketMQSourceOptions.OPTIONAL_STOP_OFFSET_TIMESTAMP);
+        optionalOptions.add(RocketMQSourceOptions.OPTIONAL_TIME_ZONE);
+        optionalOptions.add(RocketMQSourceOptions.PULL_TIMEOUT_MILLIS);
+        optionalOptions.add(RocketMQSourceOptions.OPTIONAL_ENCODING);
+        optionalOptions.add(RocketMQSourceOptions.OPTIONAL_FIELD_DELIMITER);
+        optionalOptions.add(RocketMQSourceOptions.OPTIONAL_LINE_DELIMITER);
+        optionalOptions.add(RocketMQSourceOptions.OPTIONAL_COLUMN_ERROR_DEBUG);
+        optionalOptions.add(RocketMQSourceOptions.OPTIONAL_LENGTH_CHECK);
+        optionalOptions.add(RocketMQSourceOptions.OPTIONAL_ACCESS_KEY);
+        optionalOptions.add(RocketMQSourceOptions.OPTIONAL_SECRET_KEY);
+        optionalOptions.add(RocketMQSourceOptions.OPTIONAL_STARTUP_SCAN_MODE);
+        return optionalOptions;
+    }
+
+    @Override
+    public DynamicTableSource createDynamicTableSource(Context context) {
+        FactoryUtil.TableFactoryHelper helper = createTableFactoryHelper(this, context);
+        helper.validate();
+        Map<String, String> rawProperties = context.getCatalogTable().getOptions();
+        Configuration configuration = Configuration.fromMap(rawProperties);
+        String topic = configuration.getString(RocketMQSourceOptions.TOPIC);
+        String consumerGroup = configuration.getString(RocketMQSourceOptions.CONSUMER_GROUP);
+        String nameServerAddress = configuration.getString(RocketMQSourceOptions.ENDPOINTS);
+        String tag = configuration.getString(RocketMQSourceOptions.OPTIONAL_TAG);
+        String sql = configuration.getString(RocketMQSourceOptions.OPTIONAL_SQL);
+        if (configuration.contains(RocketMQSourceOptions.OPTIONAL_STARTUP_SCAN_MODE)
+                && (configuration.contains(RocketMQSourceOptions.OPTIONAL_STARTUP_OFFSET_TIMESTAMP)
+                        || configuration.contains(
+                                RocketMQSourceOptions.OPTIONAL_STARTUP_OFFSET_DATE))) {
+            throw new IllegalArgumentException(
+                    String.format(
+                            "cannot support these configs when %s has been set: [%s] !",
+                            RocketMQSourceOptions.OPTIONAL_STARTUP_SCAN_MODE.key(),
+                            RocketMQSourceOptions.OPTIONAL_STARTUP_OFFSET_SPECIFIC.key()));
+        }
+        long startMessageOffset =
+                configuration.getLong(RocketMQSourceOptions.OPTIONAL_STARTUP_OFFSET_SPECIFIC);
+        long startTimeMs =
+                configuration.getLong(RocketMQSourceOptions.OPTIONAL_STARTUP_OFFSET_TIMESTAMP);
+        String startDateTime =
+                configuration.getString(RocketMQSourceOptions.OPTIONAL_STARTUP_OFFSET_DATE);
+        String timeZone = configuration.getString(RocketMQSourceOptions.OPTIONAL_TIME_ZONE);
+        String accessKey = configuration.getString(RocketMQSourceOptions.OPTIONAL_ACCESS_KEY);
+        String secretKey = configuration.getString(RocketMQSourceOptions.OPTIONAL_SECRET_KEY);
+        long startTime = startTimeMs;
+        if (startTime == -1) {
+            if (!StringUtils.isNullOrWhitespaceOnly(startDateTime)) {
+                try {
+                    startTime = parseDateString(startDateTime, timeZone);
+                } catch (ParseException e) {
+                    throw new RuntimeException(
+                            String.format(
+                                    "Incorrect datetime format: %s, pls use ISO-8601 "
+                                            + "complete date plus hours, minutes and seconds format:%s.",
+                                    startDateTime, DATE_FORMAT),
+                            e);
+                }
+            }
+        }
+        long stopInMs = Long.MAX_VALUE;
+        String endDateTime =
+                configuration.getString(RocketMQSourceOptions.OPTIONAL_STOP_OFFSET_TIMESTAMP);
+        if (!StringUtils.isNullOrWhitespaceOnly(endDateTime)) {
+            try {
+                stopInMs = parseDateString(endDateTime, timeZone);
+            } catch (ParseException e) {
+                throw new RuntimeException(
+                        String.format(
+                                "Incorrect datetime format: %s, pls use ISO-8601 "
+                                        + "complete date plus hours, minutes and seconds format:%s.",
+                                endDateTime, DATE_FORMAT),
+                        e);
+            }
+            Preconditions.checkArgument(
+                    stopInMs >= startTime, "Start time should be less than stop time.");
+        }
+        long partitionDiscoveryIntervalMs =
+                configuration.getLong(RocketMQSourceOptions.PARTITION_DISCOVERY_INTERVAL_MS);
+        boolean useNewApi = configuration.getBoolean(RocketMQSourceOptions.OPTIONAL_USE_NEW_API);
+        DescriptorProperties descriptorProperties = new DescriptorProperties();
+        descriptorProperties.putProperties(rawProperties);
+        TableSchema physicalSchema =
+                TableSchemaUtils.getPhysicalSchema(context.getCatalogTable().getSchema());
+        descriptorProperties.putTableSchema("schema", physicalSchema);
+        String consumerOffsetMode =
+                configuration.getString(
+                        RocketMQSourceOptions.OPTIONAL_STARTUP_SCAN_MODE,
+                        RocketMQConfig.CONSUMER_OFFSET_LATEST);
+        long consumerOffsetTimestamp =
+                configuration.getLong(
+                        RocketMQSourceOptions.OPTIONAL_STARTUP_OFFSET_TIMESTAMP,
+                        System.currentTimeMillis());
+        return new RocketMQScanTableSource(
+                configuration.getLong(RocketMQSourceOptions.PULL_TIMEOUT_LONG_POLLING_SUSPEND),
+                descriptorProperties,
+                physicalSchema,
+                topic,
+                consumerGroup,
+                nameServerAddress,
+                accessKey,
+                secretKey,
+                tag,
+                sql,
+                stopInMs,
+                startMessageOffset,
+                startMessageOffset < 0 ? startTime : -1L,
+                partitionDiscoveryIntervalMs,
+                consumerOffsetMode,
+                consumerOffsetTimestamp,
+                useNewApi);
+    }
+
+    private Long parseDateString(String dateString, String timeZone) throws ParseException {
+        FastDateFormat simpleDateFormat =
+                FastDateFormat.getInstance(DATE_FORMAT, TimeZone.getTimeZone(timeZone));
+        return simpleDateFormat.parse(dateString).getTime();
+    }
+}
diff --git a/src/main/java/org/apache/rocketmq/flink/source/table/RocketMQScanTableSource.java b/src/main/java/org/apache/flink/connector/rocketmq/source/table/RocketMQScanTableSource.java
similarity index 82%
rename from src/main/java/org/apache/rocketmq/flink/source/table/RocketMQScanTableSource.java
rename to src/main/java/org/apache/flink/connector/rocketmq/source/table/RocketMQScanTableSource.java
index 3eb68df..21ccdb9 100644
--- a/src/main/java/org/apache/rocketmq/flink/source/table/RocketMQScanTableSource.java
+++ b/src/main/java/org/apache/flink/connector/rocketmq/source/table/RocketMQScanTableSource.java
@@ -15,18 +15,17 @@
  * limitations under the License.
  */
 
-package org.apache.rocketmq.flink.source.table;
+package org.apache.flink.connector.rocketmq.source.table;
 
-import org.apache.rocketmq.flink.legacy.RocketMQConfig;
-import org.apache.rocketmq.flink.legacy.RocketMQSourceFunction;
-import org.apache.rocketmq.flink.legacy.common.serialization.KeyValueDeserializationSchema;
-import org.apache.rocketmq.flink.legacy.common.serialization.RowKeyValueDeserializationSchema;
-import org.apache.rocketmq.flink.source.RocketMQSource;
-import org.apache.rocketmq.flink.source.reader.deserializer.BytesMessage;
-import org.apache.rocketmq.flink.source.reader.deserializer.RocketMQDeserializationSchema;
-import org.apache.rocketmq.flink.source.reader.deserializer.RocketMQRowDeserializationSchema;
-import org.apache.rocketmq.flink.source.reader.deserializer.RowDeserializationSchema.MetadataConverter;
-
+import org.apache.flink.connector.rocketmq.legacy.RocketMQConfig;
+import org.apache.flink.connector.rocketmq.legacy.RocketMQSourceFunction;
+import org.apache.flink.connector.rocketmq.legacy.common.serialization.KeyValueDeserializationSchema;
+import org.apache.flink.connector.rocketmq.legacy.common.serialization.RowKeyValueDeserializationSchema;
+import org.apache.flink.connector.rocketmq.source.RocketMQSource;
+import org.apache.flink.connector.rocketmq.source.reader.deserializer.BytesMessage;
+import org.apache.flink.connector.rocketmq.source.reader.deserializer.RocketMQDeserializationSchema;
+import org.apache.flink.connector.rocketmq.source.reader.deserializer.RocketMQRowDeserializationSchema;
+import org.apache.flink.connector.rocketmq.source.reader.deserializer.RowDeserializationSchema.MetadataConverter;
 import org.apache.flink.table.api.DataTypes;
 import org.apache.flink.table.api.TableSchema;
 import org.apache.flink.table.connector.ChangelogMode;
@@ -47,9 +46,6 @@
 import java.util.Properties;
 import java.util.stream.Stream;
 
-import static org.apache.flink.api.connector.source.Boundedness.BOUNDED;
-import static org.apache.flink.api.connector.source.Boundedness.CONTINUOUS_UNBOUNDED;
-
 /** Defines the scan table source of RocketMQ. */
 public class RocketMQScanTableSource implements ScanTableSource, SupportsReadingMetadata {
 
@@ -123,24 +119,26 @@
     @Override
     public ScanRuntimeProvider getScanRuntimeProvider(ScanContext scanContext) {
         if (useNewApi) {
-            return SourceProvider.of(
-                    new RocketMQSource<>(
-                            pollTime,
-                            topic,
-                            consumerGroup,
-                            nameServerAddress,
-                            accessKey,
-                            secretKey,
-                            tag,
-                            sql,
-                            stopInMs,
-                            startTime,
-                            startMessageOffset < 0 ? 0 : startMessageOffset,
-                            partitionDiscoveryIntervalMs,
-                            isBounded() ? BOUNDED : CONTINUOUS_UNBOUNDED,
-                            createRocketMQDeserializationSchema(),
-                            consumerOffsetMode,
-                            consumerOffsetTimestamp));
+            // return SourceProvider.of(
+            //        new RocketMQSource<>(
+            //                pollTime,
+            //                topic,
+            //                consumerGroup,
+            //                nameServerAddress,
+            //                accessKey,
+            //                secretKey,
+            //                tag,
+            //                sql,
+            //                stopInMs,
+            //                startTime,
+            //                startMessageOffset < 0 ? 0 : startMessageOffset,
+            //                partitionDiscoveryIntervalMs,
+            //                isBounded() ? BOUNDED : CONTINUOUS_UNBOUNDED,
+            //                createRocketMQDeserializationSchema(),
+            //                consumerOffsetMode,
+            //                consumerOffsetTimestamp));
+
+            return SourceProvider.of(new RocketMQSource<>(null, null, null, null, null));
         } else {
             return SourceFunctionProvider.of(
                     new RocketMQSourceFunction<>(
diff --git a/src/main/java/org/apache/rocketmq/flink/source/util/ByteSerializer.java b/src/main/java/org/apache/flink/connector/rocketmq/source/util/ByteSerializer.java
similarity index 98%
rename from src/main/java/org/apache/rocketmq/flink/source/util/ByteSerializer.java
rename to src/main/java/org/apache/flink/connector/rocketmq/source/util/ByteSerializer.java
index 358cb84..efaac6e 100644
--- a/src/main/java/org/apache/rocketmq/flink/source/util/ByteSerializer.java
+++ b/src/main/java/org/apache/flink/connector/rocketmq/source/util/ByteSerializer.java
@@ -16,7 +16,7 @@
  * limitations under the License.
  */
 
-package org.apache.rocketmq.flink.source.util;
+package org.apache.flink.connector.rocketmq.source.util;
 
 import java.math.BigDecimal;
 import java.math.BigInteger;
diff --git a/src/main/java/org/apache/rocketmq/flink/source/util/ByteUtils.java b/src/main/java/org/apache/flink/connector/rocketmq/source/util/ByteUtils.java
similarity index 98%
rename from src/main/java/org/apache/rocketmq/flink/source/util/ByteUtils.java
rename to src/main/java/org/apache/flink/connector/rocketmq/source/util/ByteUtils.java
index 6e223a3..6ec0768 100644
--- a/src/main/java/org/apache/rocketmq/flink/source/util/ByteUtils.java
+++ b/src/main/java/org/apache/flink/connector/rocketmq/source/util/ByteUtils.java
@@ -16,7 +16,7 @@
  * limitations under the License.
  */
 
-package org.apache.rocketmq.flink.source.util;
+package org.apache.flink.connector.rocketmq.source.util;
 
 import java.math.BigDecimal;
 import java.math.BigInteger;
diff --git a/src/main/java/org/apache/rocketmq/flink/source/util/StringSerializer.java b/src/main/java/org/apache/flink/connector/rocketmq/source/util/StringSerializer.java
similarity index 98%
rename from src/main/java/org/apache/rocketmq/flink/source/util/StringSerializer.java
rename to src/main/java/org/apache/flink/connector/rocketmq/source/util/StringSerializer.java
index cdbc78a..d89eb98 100644
--- a/src/main/java/org/apache/rocketmq/flink/source/util/StringSerializer.java
+++ b/src/main/java/org/apache/flink/connector/rocketmq/source/util/StringSerializer.java
@@ -15,7 +15,7 @@
  * limitations under the License.
  */
 
-package org.apache.rocketmq.flink.source.util;
+package org.apache.flink.connector.rocketmq.source.util;
 
 import org.apache.flink.table.data.DecimalData;
 import org.apache.flink.table.data.binary.BinaryStringData;
diff --git a/src/main/java/org/apache/flink/connector/rocketmq/source/util/UtilAll.java b/src/main/java/org/apache/flink/connector/rocketmq/source/util/UtilAll.java
new file mode 100644
index 0000000..567ceca
--- /dev/null
+++ b/src/main/java/org/apache/flink/connector/rocketmq/source/util/UtilAll.java
@@ -0,0 +1,41 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.flink.connector.rocketmq.source.util;
+
+import org.apache.flink.connector.rocketmq.source.split.RocketMQSourceSplit;
+
+import org.apache.rocketmq.common.message.MessageQueue;
+
+public class UtilAll {
+
+    public static final String SEPARATOR = "#";
+
+    public static String getSplitId(MessageQueue mq) {
+        return mq.getTopic() + SEPARATOR + mq.getBrokerName() + SEPARATOR + mq.getQueueId();
+    }
+
+    public static String getQueueDescription(MessageQueue mq) {
+        return String.format(
+                "(Topic: %s, BrokerName: %s, QueueId: %d)",
+                mq.getTopic(), mq.getBrokerName(), mq.getQueueId());
+    }
+
+    public static MessageQueue getMessageQueue(RocketMQSourceSplit split) {
+        return new MessageQueue(split.getTopic(), split.getBrokerName(), split.getQueueId());
+    }
+}
diff --git a/src/main/java/org/apache/rocketmq/flink/common/RocketMQOptions.java b/src/main/java/org/apache/rocketmq/flink/common/RocketMQOptions.java
deleted file mode 100644
index 50a0883..0000000
--- a/src/main/java/org/apache/rocketmq/flink/common/RocketMQOptions.java
+++ /dev/null
@@ -1,120 +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.flink.common;
-
-import org.apache.flink.configuration.ConfigOption;
-import org.apache.flink.configuration.ConfigOptions;
-
-import static org.apache.rocketmq.flink.legacy.RocketMQConfig.DEFAULT_START_MESSAGE_OFFSET;
-
-/** Includes config options of RocketMQ connector type. */
-public class RocketMQOptions {
-
-    public static final ConfigOption<String> TOPIC =
-            ConfigOptions.key("topic").stringType().noDefaultValue();
-
-    public static final ConfigOption<String> CONSUMER_GROUP =
-            ConfigOptions.key("consumerGroup").stringType().noDefaultValue();
-
-    public static final ConfigOption<String> PRODUCER_GROUP =
-            ConfigOptions.key("producerGroup").stringType().noDefaultValue();
-
-    public static final ConfigOption<String> NAME_SERVER_ADDRESS =
-            ConfigOptions.key("nameServerAddress").stringType().noDefaultValue();
-
-    public static final ConfigOption<String> OPTIONAL_TAG =
-            ConfigOptions.key("tag").stringType().noDefaultValue();
-
-    public static final ConfigOption<String> OPTIONAL_SQL =
-            ConfigOptions.key("sql").stringType().noDefaultValue();
-
-    public static final ConfigOption<Long> OPTIONAL_START_MESSAGE_OFFSET =
-            ConfigOptions.key("startMessageOffset")
-                    .longType()
-                    .defaultValue(DEFAULT_START_MESSAGE_OFFSET);
-
-    public static final ConfigOption<Long> OPTIONAL_START_TIME_MILLS =
-            ConfigOptions.key("startTimeMs").longType().defaultValue(-1L);
-
-    public static final ConfigOption<String> OPTIONAL_START_TIME =
-            ConfigOptions.key("startTime").stringType().noDefaultValue();
-
-    public static final ConfigOption<String> OPTIONAL_END_TIME =
-            ConfigOptions.key("endTime").stringType().noDefaultValue();
-
-    public static final ConfigOption<String> OPTIONAL_TIME_ZONE =
-            ConfigOptions.key("timeZone").stringType().noDefaultValue();
-
-    public static final ConfigOption<Long> OPTIONAL_PARTITION_DISCOVERY_INTERVAL_MS =
-            ConfigOptions.key("partitionDiscoveryIntervalMs").longType().defaultValue(30000L);
-
-    public static final ConfigOption<Long> OPTIONAL_CONSUMER_POLL_MS =
-            ConfigOptions.key("consumer.timeout").longType().defaultValue(3000L);
-
-    public static final ConfigOption<Boolean> OPTIONAL_USE_NEW_API =
-            ConfigOptions.key("useNewApi").booleanType().defaultValue(true);
-
-    public static final ConfigOption<String> OPTIONAL_ENCODING =
-            ConfigOptions.key("encoding").stringType().defaultValue("UTF-8");
-
-    public static final ConfigOption<String> OPTIONAL_FIELD_DELIMITER =
-            ConfigOptions.key("fieldDelimiter").stringType().defaultValue("\u0001");
-
-    public static final ConfigOption<String> OPTIONAL_LINE_DELIMITER =
-            ConfigOptions.key("lineDelimiter").stringType().defaultValue("\n");
-
-    public static final ConfigOption<Boolean> OPTIONAL_COLUMN_ERROR_DEBUG =
-            ConfigOptions.key("columnErrorDebug").booleanType().defaultValue(true);
-
-    public static final ConfigOption<String> OPTIONAL_LENGTH_CHECK =
-            ConfigOptions.key("lengthCheck").stringType().defaultValue("NONE");
-
-    public static final ConfigOption<Integer> OPTIONAL_WRITE_RETRY_TIMES =
-            ConfigOptions.key("retryTimes").intType().defaultValue(10);
-
-    public static final ConfigOption<Long> OPTIONAL_WRITE_SLEEP_TIME_MS =
-            ConfigOptions.key("sleepTimeMs").longType().defaultValue(5000L);
-
-    public static final ConfigOption<Boolean> OPTIONAL_WRITE_IS_DYNAMIC_TAG =
-            ConfigOptions.key("isDynamicTag").booleanType().defaultValue(false);
-
-    public static final ConfigOption<String> OPTIONAL_WRITE_DYNAMIC_TAG_COLUMN =
-            ConfigOptions.key("dynamicTagColumn").stringType().noDefaultValue();
-
-    public static final ConfigOption<Boolean> OPTIONAL_WRITE_DYNAMIC_TAG_COLUMN_WRITE_INCLUDED =
-            ConfigOptions.key("dynamicTagColumnWriteIncluded").booleanType().defaultValue(true);
-
-    public static final ConfigOption<String> OPTIONAL_WRITE_KEY_COLUMNS =
-            ConfigOptions.key("keyColumns").stringType().noDefaultValue();
-
-    public static final ConfigOption<Boolean> OPTIONAL_WRITE_KEYS_TO_BODY =
-            ConfigOptions.key("writeKeysToBody").booleanType().defaultValue(false);
-
-    public static final ConfigOption<String> OPTIONAL_ACCESS_KEY =
-            ConfigOptions.key("accessKey").stringType().noDefaultValue();
-
-    public static final ConfigOption<String> OPTIONAL_SECRET_KEY =
-            ConfigOptions.key("secretKey").stringType().noDefaultValue();
-
-    public static final ConfigOption<String> OPTIONAL_SCAN_STARTUP_MODE =
-            ConfigOptions.key("scanStartupMode").stringType().defaultValue("latest");
-
-    public static final ConfigOption<Long> OPTIONAL_OFFSET_FROM_TIMESTAMP =
-            ConfigOptions.key("offsetFromTimestamp").longType().noDefaultValue();
-}
diff --git a/src/main/java/org/apache/rocketmq/flink/legacy/example/SimpleConsumer.java b/src/main/java/org/apache/rocketmq/flink/legacy/example/SimpleConsumer.java
deleted file mode 100644
index dfc6559..0000000
--- a/src/main/java/org/apache/rocketmq/flink/legacy/example/SimpleConsumer.java
+++ /dev/null
@@ -1,88 +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.flink.legacy.example;
-
-import org.apache.rocketmq.acl.common.AclClientRPCHook;
-import org.apache.rocketmq.acl.common.SessionCredentials;
-import org.apache.rocketmq.client.AccessChannel;
-import org.apache.rocketmq.client.consumer.DefaultMQPushConsumer;
-import org.apache.rocketmq.client.consumer.listener.ConsumeConcurrentlyStatus;
-import org.apache.rocketmq.client.consumer.listener.MessageListenerConcurrently;
-import org.apache.rocketmq.client.consumer.rebalance.AllocateMessageQueueAveragely;
-import org.apache.rocketmq.client.exception.MQClientException;
-import org.apache.rocketmq.common.consumer.ConsumeFromWhere;
-import org.apache.rocketmq.common.message.MessageExt;
-import org.apache.rocketmq.remoting.RPCHook;
-
-import org.slf4j.Logger;
-import org.slf4j.LoggerFactory;
-
-public class SimpleConsumer {
-
-    private static final Logger LOGGER = LoggerFactory.getLogger(SimpleConsumer.class);
-
-    // Consumer config
-    private static final String NAME_SERVER_ADDR =
-            "http://${instanceId}.${region}.mq-internal.aliyuncs.com:8080";
-    private static final String GROUP = "GID_SIMPLE_CONSUMER";
-    private static final String TOPIC = "SINK_TOPIC";
-    private static final String TAGS = "*";
-
-    private static RPCHook getAclRPCHook() {
-        final String accessKey = "${AccessKey}";
-        final String secretKey = "${SecretKey}";
-        return new AclClientRPCHook(new SessionCredentials(accessKey, secretKey));
-    }
-
-    public static void main(String[] args) {
-        DefaultMQPushConsumer consumer =
-                new DefaultMQPushConsumer(
-                        GROUP, getAclRPCHook(), new AllocateMessageQueueAveragely());
-        consumer.setNamesrvAddr(NAME_SERVER_ADDR);
-
-        // When using aliyun products, you need to set up channels
-        consumer.setAccessChannel(AccessChannel.CLOUD);
-
-        try {
-            consumer.subscribe(TOPIC, TAGS);
-        } catch (MQClientException e) {
-            e.printStackTrace();
-        }
-
-        consumer.setConsumeFromWhere(ConsumeFromWhere.CONSUME_FROM_FIRST_OFFSET);
-        consumer.registerMessageListener(
-                (MessageListenerConcurrently)
-                        (msgs, context) -> {
-                            for (MessageExt msg : msgs) {
-                                System.out.printf(
-                                        "%s %s %d %s\n",
-                                        msg.getMsgId(),
-                                        msg.getBrokerName(),
-                                        msg.getQueueId(),
-                                        new String(msg.getBody()));
-                            }
-                            return ConsumeConcurrentlyStatus.CONSUME_SUCCESS;
-                        });
-
-        try {
-            consumer.start();
-        } catch (MQClientException e) {
-            LOGGER.info("send message failed. {}", e.toString());
-        }
-    }
-}
diff --git a/src/main/java/org/apache/rocketmq/flink/legacy/example/SimpleProducer.java b/src/main/java/org/apache/rocketmq/flink/legacy/example/SimpleProducer.java
deleted file mode 100644
index 24e86ac..0000000
--- a/src/main/java/org/apache/rocketmq/flink/legacy/example/SimpleProducer.java
+++ /dev/null
@@ -1,81 +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.flink.legacy.example;
-
-import org.apache.rocketmq.acl.common.AclClientRPCHook;
-import org.apache.rocketmq.acl.common.SessionCredentials;
-import org.apache.rocketmq.client.AccessChannel;
-import org.apache.rocketmq.client.exception.MQClientException;
-import org.apache.rocketmq.client.producer.DefaultMQProducer;
-import org.apache.rocketmq.client.producer.SendResult;
-import org.apache.rocketmq.common.message.Message;
-import org.apache.rocketmq.remoting.RPCHook;
-
-import org.slf4j.Logger;
-import org.slf4j.LoggerFactory;
-
-public class SimpleProducer {
-
-    private static final Logger LOGGER = LoggerFactory.getLogger(SimpleProducer.class);
-
-    private static final int MESSAGE_NUM = 10000;
-
-    // Producer config
-    private static final String NAME_SERVER_ADDR =
-            "http://${instanceId}.${region}.mq-internal.aliyuncs.com:8080";
-    private static final String PRODUCER_GROUP = "GID_SIMPLE_PRODUCER";
-    private static final String TOPIC = "SOURCE_TOPIC";
-    private static final String TAGS = "*";
-    private static final String KEY_PREFIX = "KEY";
-
-    private static RPCHook getAclRPCHook() {
-        final String accessKey = "${AccessKey}";
-        final String secretKey = "${SecretKey}";
-        return new AclClientRPCHook(new SessionCredentials(accessKey, secretKey));
-    }
-
-    public static void main(String[] args) {
-        DefaultMQProducer producer =
-                new DefaultMQProducer(PRODUCER_GROUP, getAclRPCHook(), true, null);
-        producer.setNamesrvAddr(NAME_SERVER_ADDR);
-
-        // When using aliyun products, you need to set up channels
-        producer.setAccessChannel(AccessChannel.CLOUD);
-
-        try {
-            producer.start();
-        } catch (MQClientException e) {
-            e.printStackTrace();
-        }
-
-        for (int i = 0; i < MESSAGE_NUM; i++) {
-            String content = "Test Message " + i;
-            Message msg = new Message(TOPIC, TAGS, KEY_PREFIX + i, content.getBytes());
-            try {
-                SendResult sendResult = producer.send(msg);
-                assert sendResult != null;
-                System.out.printf(
-                        "send result: %s %s\n",
-                        sendResult.getMsgId(), sendResult.getMessageQueue().toString());
-                Thread.sleep(50);
-            } catch (Exception e) {
-                LOGGER.info("send message failed. {}", e.toString());
-            }
-        }
-    }
-}
diff --git a/src/main/java/org/apache/rocketmq/flink/source/RocketMQSource.java b/src/main/java/org/apache/rocketmq/flink/source/RocketMQSource.java
deleted file mode 100644
index 8d98d2e..0000000
--- a/src/main/java/org/apache/rocketmq/flink/source/RocketMQSource.java
+++ /dev/null
@@ -1,222 +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.flink.source;
-
-import org.apache.rocketmq.flink.legacy.RocketMQConfig;
-import org.apache.rocketmq.flink.source.enumerator.RocketMQSourceEnumState;
-import org.apache.rocketmq.flink.source.enumerator.RocketMQSourceEnumStateSerializer;
-import org.apache.rocketmq.flink.source.enumerator.RocketMQSourceEnumerator;
-import org.apache.rocketmq.flink.source.reader.RocketMQPartitionSplitReader;
-import org.apache.rocketmq.flink.source.reader.RocketMQRecordEmitter;
-import org.apache.rocketmq.flink.source.reader.RocketMQSourceReader;
-import org.apache.rocketmq.flink.source.reader.deserializer.RocketMQDeserializationSchema;
-import org.apache.rocketmq.flink.source.split.RocketMQPartitionSplit;
-import org.apache.rocketmq.flink.source.split.RocketMQPartitionSplitSerializer;
-
-import org.apache.flink.api.common.serialization.DeserializationSchema;
-import org.apache.flink.api.common.typeinfo.TypeInformation;
-import org.apache.flink.api.connector.source.Boundedness;
-import org.apache.flink.api.connector.source.Source;
-import org.apache.flink.api.connector.source.SourceReader;
-import org.apache.flink.api.connector.source.SourceReaderContext;
-import org.apache.flink.api.connector.source.SplitEnumerator;
-import org.apache.flink.api.connector.source.SplitEnumeratorContext;
-import org.apache.flink.api.java.tuple.Tuple3;
-import org.apache.flink.api.java.typeutils.ResultTypeQueryable;
-import org.apache.flink.configuration.Configuration;
-import org.apache.flink.connector.base.source.reader.RecordsWithSplitIds;
-import org.apache.flink.connector.base.source.reader.splitreader.SplitReader;
-import org.apache.flink.connector.base.source.reader.synchronization.FutureCompletingBlockingQueue;
-import org.apache.flink.core.io.SimpleVersionedSerializer;
-import org.apache.flink.metrics.MetricGroup;
-import org.apache.flink.util.UserCodeClassLoader;
-
-import org.apache.commons.lang.Validate;
-import org.apache.commons.lang3.StringUtils;
-
-import java.util.function.Supplier;
-
-/** The Source implementation of RocketMQ. */
-public class RocketMQSource<OUT>
-        implements Source<OUT, RocketMQPartitionSplit, RocketMQSourceEnumState>,
-                ResultTypeQueryable<OUT> {
-    private static final long serialVersionUID = -1L;
-
-    private final String consumerOffsetMode;
-    private final long consumerOffsetTimestamp;
-    private final long pollTime;
-    private final String topic;
-    private final String consumerGroup;
-    private final String nameServerAddress;
-    private final String tag;
-    private final String sql;
-
-    private final String accessKey;
-    private final String secretKey;
-
-    private final long stopInMs;
-    private final long startTime;
-    private final long startOffset;
-    private final long partitionDiscoveryIntervalMs;
-
-    // Boundedness
-    private final Boundedness boundedness;
-    private final RocketMQDeserializationSchema<OUT> deserializationSchema;
-
-    public RocketMQSource(
-            long pollTime,
-            String topic,
-            String consumerGroup,
-            String nameServerAddress,
-            String accessKey,
-            String secretKey,
-            String tag,
-            String sql,
-            long stopInMs,
-            long startTime,
-            long startOffset,
-            long partitionDiscoveryIntervalMs,
-            Boundedness boundedness,
-            RocketMQDeserializationSchema<OUT> deserializationSchema,
-            String cosumerOffsetMode,
-            long consumerOffsetTimestamp) {
-        Validate.isTrue(
-                !(StringUtils.isNotEmpty(tag) && StringUtils.isNotEmpty(sql)),
-                "Consumer tag and sql can not set value at the same time");
-        this.pollTime = pollTime;
-        this.topic = topic;
-        this.consumerGroup = consumerGroup;
-        this.nameServerAddress = nameServerAddress;
-        this.accessKey = accessKey;
-        this.secretKey = secretKey;
-        this.tag = StringUtils.isEmpty(tag) ? RocketMQConfig.DEFAULT_CONSUMER_TAG : tag;
-        this.sql = sql;
-        this.stopInMs = stopInMs;
-        this.startTime = startTime;
-        this.startOffset = startOffset > 0 ? startOffset : startTime;
-        this.partitionDiscoveryIntervalMs = partitionDiscoveryIntervalMs;
-        this.boundedness = boundedness;
-        this.deserializationSchema = deserializationSchema;
-        this.consumerOffsetMode = cosumerOffsetMode;
-        this.consumerOffsetTimestamp = consumerOffsetTimestamp;
-    }
-
-    @Override
-    public Boundedness getBoundedness() {
-        return this.boundedness;
-    }
-
-    @Override
-    public SourceReader<OUT, RocketMQPartitionSplit> createReader(
-            SourceReaderContext readerContext) {
-        FutureCompletingBlockingQueue<RecordsWithSplitIds<Tuple3<OUT, Long, Long>>> elementsQueue =
-                new FutureCompletingBlockingQueue<>();
-        deserializationSchema.open(
-                new DeserializationSchema.InitializationContext() {
-                    @Override
-                    public MetricGroup getMetricGroup() {
-                        return readerContext.metricGroup();
-                    }
-
-                    @Override
-                    public UserCodeClassLoader getUserCodeClassLoader() {
-                        return null;
-                    }
-                });
-
-        Supplier<SplitReader<Tuple3<OUT, Long, Long>, RocketMQPartitionSplit>> splitReaderSupplier =
-                () ->
-                        new RocketMQPartitionSplitReader<>(
-                                pollTime,
-                                topic,
-                                consumerGroup,
-                                nameServerAddress,
-                                accessKey,
-                                secretKey,
-                                tag,
-                                sql,
-                                stopInMs,
-                                startTime,
-                                startOffset,
-                                deserializationSchema);
-        RocketMQRecordEmitter<OUT> recordEmitter = new RocketMQRecordEmitter<>();
-
-        return new RocketMQSourceReader<>(
-                elementsQueue,
-                splitReaderSupplier,
-                recordEmitter,
-                new Configuration(),
-                readerContext);
-    }
-
-    @Override
-    public SplitEnumerator<RocketMQPartitionSplit, RocketMQSourceEnumState> createEnumerator(
-            SplitEnumeratorContext<RocketMQPartitionSplit> enumContext) {
-
-        return new RocketMQSourceEnumerator(
-                topic,
-                consumerGroup,
-                nameServerAddress,
-                accessKey,
-                secretKey,
-                stopInMs,
-                startOffset,
-                partitionDiscoveryIntervalMs,
-                boundedness,
-                enumContext,
-                consumerOffsetMode,
-                consumerOffsetTimestamp);
-    }
-
-    @Override
-    public SplitEnumerator<RocketMQPartitionSplit, RocketMQSourceEnumState> restoreEnumerator(
-            SplitEnumeratorContext<RocketMQPartitionSplit> enumContext,
-            RocketMQSourceEnumState checkpoint) {
-
-        return new RocketMQSourceEnumerator(
-                topic,
-                consumerGroup,
-                nameServerAddress,
-                accessKey,
-                secretKey,
-                stopInMs,
-                startOffset,
-                partitionDiscoveryIntervalMs,
-                boundedness,
-                enumContext,
-                checkpoint.getCurrentAssignment(),
-                consumerOffsetMode,
-                consumerOffsetTimestamp);
-    }
-
-    @Override
-    public SimpleVersionedSerializer<RocketMQPartitionSplit> getSplitSerializer() {
-        return new RocketMQPartitionSplitSerializer();
-    }
-
-    @Override
-    public SimpleVersionedSerializer<RocketMQSourceEnumState> getEnumeratorCheckpointSerializer() {
-        return new RocketMQSourceEnumStateSerializer();
-    }
-
-    @Override
-    public TypeInformation<OUT> getProducedType() {
-        return deserializationSchema.getProducedType();
-    }
-}
diff --git a/src/main/java/org/apache/rocketmq/flink/source/enumerator/RocketMQSourceEnumStateSerializer.java b/src/main/java/org/apache/rocketmq/flink/source/enumerator/RocketMQSourceEnumStateSerializer.java
deleted file mode 100644
index ce45b51..0000000
--- a/src/main/java/org/apache/rocketmq/flink/source/enumerator/RocketMQSourceEnumStateSerializer.java
+++ /dev/null
@@ -1,64 +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.flink.source.enumerator;
-
-import org.apache.rocketmq.flink.source.split.RocketMQPartitionSplit;
-import org.apache.rocketmq.flink.source.split.RocketMQPartitionSplitSerializer;
-
-import org.apache.flink.connector.base.source.utils.SerdeUtils;
-import org.apache.flink.core.io.SimpleVersionedSerializer;
-
-import java.io.IOException;
-import java.util.ArrayList;
-import java.util.List;
-import java.util.Map;
-
-/** The {@link SimpleVersionedSerializer Serializer} for the enumerator state of RocketMQ source. */
-public class RocketMQSourceEnumStateSerializer
-        implements SimpleVersionedSerializer<RocketMQSourceEnumState> {
-
-    private static final int CURRENT_VERSION = 0;
-
-    @Override
-    public int getVersion() {
-        return CURRENT_VERSION;
-    }
-
-    @Override
-    public byte[] serialize(RocketMQSourceEnumState enumState) throws IOException {
-        return SerdeUtils.serializeSplitAssignments(
-                enumState.getCurrentAssignment(), new RocketMQPartitionSplitSerializer());
-    }
-
-    @Override
-    public RocketMQSourceEnumState deserialize(int version, byte[] serialized) throws IOException {
-        // Check whether the version of serialized bytes is supported.
-        if (version == getVersion()) {
-            Map<Integer, List<RocketMQPartitionSplit>> currentPartitionAssignment =
-                    SerdeUtils.deserializeSplitAssignments(
-                            serialized, new RocketMQPartitionSplitSerializer(), ArrayList::new);
-            return new RocketMQSourceEnumState(currentPartitionAssignment);
-        }
-        throw new IOException(
-                String.format(
-                        "The bytes are serialized with version %d, "
-                                + "while this deserializer only supports version up to %d",
-                        version, getVersion()));
-    }
-}
diff --git a/src/main/java/org/apache/rocketmq/flink/source/enumerator/RocketMQSourceEnumerator.java b/src/main/java/org/apache/rocketmq/flink/source/enumerator/RocketMQSourceEnumerator.java
deleted file mode 100644
index bf489bb..0000000
--- a/src/main/java/org/apache/rocketmq/flink/source/enumerator/RocketMQSourceEnumerator.java
+++ /dev/null
@@ -1,417 +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.flink.source.enumerator;
-
-import org.apache.rocketmq.acl.common.AclClientRPCHook;
-import org.apache.rocketmq.acl.common.SessionCredentials;
-import org.apache.rocketmq.client.consumer.DefaultLitePullConsumer;
-import org.apache.rocketmq.client.exception.MQClientException;
-import org.apache.rocketmq.common.message.MessageQueue;
-import org.apache.rocketmq.flink.source.split.RocketMQPartitionSplit;
-
-import org.apache.flink.annotation.Internal;
-import org.apache.flink.annotation.VisibleForTesting;
-import org.apache.flink.api.connector.source.Boundedness;
-import org.apache.flink.api.connector.source.SplitEnumerator;
-import org.apache.flink.api.connector.source.SplitEnumeratorContext;
-import org.apache.flink.api.connector.source.SplitsAssignment;
-import org.apache.flink.api.java.tuple.Tuple3;
-import org.apache.flink.util.FlinkRuntimeException;
-import org.apache.flink.util.StringUtils;
-
-import org.slf4j.Logger;
-import org.slf4j.LoggerFactory;
-
-import javax.annotation.Nullable;
-
-import java.lang.management.ManagementFactory;
-import java.util.ArrayList;
-import java.util.Collection;
-import java.util.Collections;
-import java.util.HashMap;
-import java.util.HashSet;
-import java.util.List;
-import java.util.Map;
-import java.util.Set;
-
-import static org.apache.rocketmq.flink.legacy.RocketMQConfig.CONSUMER_OFFSET_EARLIEST;
-import static org.apache.rocketmq.flink.legacy.RocketMQConfig.CONSUMER_OFFSET_LATEST;
-import static org.apache.rocketmq.flink.legacy.RocketMQConfig.CONSUMER_OFFSET_TIMESTAMP;
-
-/** The enumerator class for RocketMQ source. */
-@Internal
-public class RocketMQSourceEnumerator
-        implements SplitEnumerator<RocketMQPartitionSplit, RocketMQSourceEnumState> {
-
-    private static final Logger LOG = LoggerFactory.getLogger(RocketMQSourceEnumerator.class);
-    private final Map<MessageQueue, Long> offsetTable = new HashMap<>();
-    private final String consumerOffsetMode;
-    private final long consumerOffsetTimestamp;
-    /** The topic used for this RocketMQSource. */
-    private final String topic;
-    /** The consumer group used for this RocketMQSource. */
-    private final String consumerGroup;
-    /** The name server address used for this RocketMQSource. */
-    private final String nameServerAddress;
-    /** The stop timestamp for this RocketMQSource. */
-    private final long stopInMs;
-    /** The start offset for this RocketMQSource. */
-    private final long startOffset;
-    /** The partition discovery interval for this RocketMQSource. */
-    private final long partitionDiscoveryIntervalMs;
-    /** The boundedness of this RocketMQSource. */
-    private final Boundedness boundedness;
-
-    /** The accessKey used for this RocketMQSource. */
-    private final String accessKey;
-    /** The secretKey used for this RocketMQSource. */
-    private final String secretKey;
-
-    private final SplitEnumeratorContext<RocketMQPartitionSplit> context;
-
-    // The internal states of the enumerator.
-    /**
-     * This set is only accessed by the partition discovery callable in the callAsync() method, i.e
-     * worker thread.
-     */
-    private final Set<Tuple3<String, String, Integer>> discoveredPartitions;
-    /** The current assignment by reader id. Only accessed by the coordinator thread. */
-    private final Map<Integer, List<RocketMQPartitionSplit>> readerIdToSplitAssignments;
-    /**
-     * The discovered and initialized partition splits that are waiting for owner reader to be
-     * ready.
-     */
-    private final Map<Integer, Set<RocketMQPartitionSplit>> pendingPartitionSplitAssignment;
-
-    // Lazily instantiated or mutable fields.
-    private DefaultLitePullConsumer consumer;
-
-    private boolean noMoreNewPartitionSplits = false;
-
-    public RocketMQSourceEnumerator(
-            String topic,
-            String consumerGroup,
-            String nameServerAddress,
-            String accessKey,
-            String secretKey,
-            long stopInMs,
-            long startOffset,
-            long partitionDiscoveryIntervalMs,
-            Boundedness boundedness,
-            SplitEnumeratorContext<RocketMQPartitionSplit> context,
-            String consumerOffsetMode,
-            long consumerOffsetTimestamp) {
-        this(
-                topic,
-                consumerGroup,
-                nameServerAddress,
-                accessKey,
-                secretKey,
-                stopInMs,
-                startOffset,
-                partitionDiscoveryIntervalMs,
-                boundedness,
-                context,
-                new HashMap<>(),
-                consumerOffsetMode,
-                consumerOffsetTimestamp);
-    }
-
-    public RocketMQSourceEnumerator(
-            String topic,
-            String consumerGroup,
-            String nameServerAddress,
-            String accessKey,
-            String secretKey,
-            long stopInMs,
-            long startOffset,
-            long partitionDiscoveryIntervalMs,
-            Boundedness boundedness,
-            SplitEnumeratorContext<RocketMQPartitionSplit> context,
-            Map<Integer, List<RocketMQPartitionSplit>> currentSplitsAssignments,
-            String consumerOffsetMode,
-            long consumerOffsetTimestamp) {
-        this.topic = topic;
-        this.consumerGroup = consumerGroup;
-        this.nameServerAddress = nameServerAddress;
-        this.accessKey = accessKey;
-        this.secretKey = secretKey;
-        this.stopInMs = stopInMs;
-        this.startOffset = startOffset;
-        this.partitionDiscoveryIntervalMs = partitionDiscoveryIntervalMs;
-        this.boundedness = boundedness;
-        this.context = context;
-
-        this.discoveredPartitions = new HashSet<>();
-        this.readerIdToSplitAssignments = new HashMap<>(currentSplitsAssignments);
-        this.readerIdToSplitAssignments.forEach(
-                (reader, splits) ->
-                        splits.forEach(
-                                s ->
-                                        discoveredPartitions.add(
-                                                new Tuple3<>(
-                                                        s.getTopic(),
-                                                        s.getBroker(),
-                                                        s.getPartition()))));
-        this.pendingPartitionSplitAssignment = new HashMap<>();
-        this.consumerOffsetMode = consumerOffsetMode;
-        this.consumerOffsetTimestamp = consumerOffsetTimestamp;
-    }
-
-    @Override
-    public void start() {
-        initialRocketMQConsumer();
-        LOG.info(
-                "Starting the RocketMQSourceEnumerator for consumer group {} "
-                        + "with partition discovery interval of {} ms.",
-                consumerGroup,
-                partitionDiscoveryIntervalMs);
-        context.callAsync(
-                this::discoverAndInitializePartitionSplit,
-                this::handlePartitionSplitChanges,
-                0,
-                partitionDiscoveryIntervalMs);
-    }
-
-    @Override
-    public void handleSplitRequest(int subtaskId, @Nullable String requesterHostname) {
-        // the RocketMQ source pushes splits eagerly, rather than act upon split requests
-    }
-
-    @Override
-    public void addSplitsBack(List<RocketMQPartitionSplit> splits, int subtaskId) {
-        addPartitionSplitChangeToPendingAssignments(splits);
-        assignPendingPartitionSplits();
-    }
-
-    @Override
-    public void addReader(int subtaskId) {
-        LOG.debug(
-                "Adding reader {} to RocketMQSourceEnumerator for consumer group {}.",
-                subtaskId,
-                consumerGroup);
-        assignPendingPartitionSplits();
-        if (boundedness == Boundedness.BOUNDED) {
-            // for RocketMQ bounded source, send this signal to ensure the task can end after all
-            // the
-            // splits assigned are completed.
-            context.signalNoMoreSplits(subtaskId);
-        }
-    }
-
-    @Override
-    public RocketMQSourceEnumState snapshotState(long checkpointId) {
-        return new RocketMQSourceEnumState(readerIdToSplitAssignments);
-    }
-
-    @Override
-    public void close() {
-        if (consumer != null) {
-            consumer.shutdown();
-        }
-    }
-
-    // ----------------- private methods -------------------
-
-    private Set<RocketMQPartitionSplit> discoverAndInitializePartitionSplit()
-            throws MQClientException {
-        Set<Tuple3<String, String, Integer>> newPartitions = new HashSet<>();
-        Set<Tuple3<String, String, Integer>> removedPartitions =
-                new HashSet<>(Collections.unmodifiableSet(discoveredPartitions));
-
-        Collection<MessageQueue> messageQueues = consumer.fetchMessageQueues(topic);
-        Set<RocketMQPartitionSplit> result = new HashSet<>();
-        for (MessageQueue messageQueue : messageQueues) {
-            Tuple3<String, String, Integer> topicPartition =
-                    new Tuple3<>(
-                            messageQueue.getTopic(),
-                            messageQueue.getBrokerName(),
-                            messageQueue.getQueueId());
-            if (!removedPartitions.remove(topicPartition)) {
-                newPartitions.add(topicPartition);
-                result.add(
-                        new RocketMQPartitionSplit(
-                                topicPartition.f0,
-                                topicPartition.f1,
-                                topicPartition.f2,
-                                getOffsetByMessageQueue(messageQueue),
-                                stopInMs));
-            }
-        }
-        discoveredPartitions.addAll(Collections.unmodifiableSet(newPartitions));
-        return result;
-    }
-
-    // This method should only be invoked in the coordinator executor thread.
-    private void handlePartitionSplitChanges(
-            Set<RocketMQPartitionSplit> partitionSplits, Throwable t) {
-        if (t != null) {
-            throw new FlinkRuntimeException("Failed to handle partition splits change due to ", t);
-        }
-        if (partitionDiscoveryIntervalMs < 0) {
-            LOG.debug("");
-            noMoreNewPartitionSplits = true;
-        }
-        addPartitionSplitChangeToPendingAssignments(partitionSplits);
-        assignPendingPartitionSplits();
-    }
-
-    // This method should only be invoked in the coordinator executor thread.
-    private void addPartitionSplitChangeToPendingAssignments(
-            Collection<RocketMQPartitionSplit> newPartitionSplits) {
-        int numReaders = context.currentParallelism();
-        for (RocketMQPartitionSplit split : newPartitionSplits) {
-            int ownerReader =
-                    getSplitOwner(
-                            split.getTopic(), split.getBroker(), split.getPartition(), numReaders);
-            pendingPartitionSplitAssignment
-                    .computeIfAbsent(ownerReader, r -> new HashSet<>())
-                    .add(split);
-        }
-        LOG.debug(
-                "Assigned {} to {} readers of consumer group {}.",
-                newPartitionSplits,
-                numReaders,
-                consumerGroup);
-    }
-
-    // This method should only be invoked in the coordinator executor thread.
-    private void assignPendingPartitionSplits() {
-        Map<Integer, List<RocketMQPartitionSplit>> incrementalAssignment = new HashMap<>();
-        pendingPartitionSplitAssignment.forEach(
-                (ownerReader, pendingSplits) -> {
-                    if (!pendingSplits.isEmpty()
-                            && context.registeredReaders().containsKey(ownerReader)) {
-                        // The owner reader is ready, assign the split to the owner reader.
-                        incrementalAssignment
-                                .computeIfAbsent(ownerReader, r -> new ArrayList<>())
-                                .addAll(pendingSplits);
-                    }
-                });
-        if (incrementalAssignment.isEmpty()) {
-            // No assignment is made.
-            return;
-        }
-
-        LOG.info("Assigning splits to readers {}", incrementalAssignment);
-        context.assignSplits(new SplitsAssignment<>(incrementalAssignment));
-        incrementalAssignment.forEach(
-                (readerOwner, newPartitionSplits) -> {
-                    // Update the split assignment.
-                    readerIdToSplitAssignments
-                            .computeIfAbsent(readerOwner, r -> new ArrayList<>())
-                            .addAll(newPartitionSplits);
-                    // Clear the pending splits for the reader owner.
-                    pendingPartitionSplitAssignment.remove(readerOwner);
-                    // Sends NoMoreSplitsEvent to the readers if there is no more partition splits
-                    // to be assigned.
-                    if (noMoreNewPartitionSplits) {
-                        LOG.debug(
-                                "No more RocketMQPartitionSplits to assign. Sending NoMoreSplitsEvent to the readers "
-                                        + "in consumer group {}.",
-                                consumerGroup);
-                        context.signalNoMoreSplits(readerOwner);
-                    }
-                });
-    }
-
-    private long getOffsetByMessageQueue(MessageQueue mq) throws MQClientException {
-        Long offset = offsetTable.get(mq);
-        if (offset == null) {
-            if (startOffset > 0) {
-                offset = startOffset;
-            } else {
-                switch (consumerOffsetMode) {
-                    case CONSUMER_OFFSET_EARLIEST:
-                        consumer.seekToBegin(mq);
-                        return -1;
-                    case CONSUMER_OFFSET_LATEST:
-                        consumer.seekToEnd(mq);
-                        return -1;
-                    case CONSUMER_OFFSET_TIMESTAMP:
-                        offset = consumer.offsetForTimestamp(mq, consumerOffsetTimestamp);
-                        break;
-                    default:
-                        offset = consumer.committed(mq);
-                        if (offset < 0) {
-                            throw new IllegalArgumentException(
-                                    "Unknown value for CONSUMER_OFFSET_RESET_TO.");
-                        }
-                }
-            }
-        }
-        offsetTable.put(mq, offset);
-        return offsetTable.get(mq);
-    }
-
-    private void initialRocketMQConsumer() {
-        try {
-            if (!StringUtils.isNullOrWhitespaceOnly(accessKey)
-                    && !StringUtils.isNullOrWhitespaceOnly(secretKey)) {
-                AclClientRPCHook aclClientRPCHook =
-                        new AclClientRPCHook(new SessionCredentials(accessKey, secretKey));
-                consumer = new DefaultLitePullConsumer(consumerGroup, aclClientRPCHook);
-            } else {
-                consumer = new DefaultLitePullConsumer(consumerGroup);
-            }
-            consumer.setNamesrvAddr(nameServerAddress);
-            consumer.setInstanceName(
-                    String.join(
-                            "||",
-                            ManagementFactory.getRuntimeMXBean().getName(),
-                            topic,
-                            consumerGroup,
-                            "" + System.nanoTime()));
-            consumer.start();
-        } catch (MQClientException e) {
-            LOG.error("Failed to initial RocketMQ consumer.", e);
-            consumer.shutdown();
-        }
-    }
-
-    /**
-     * Returns the index of the target subtask that a specific RocketMQ partition should be assigned
-     * to.
-     *
-     * <p>The resulting distribution of partitions of a single topic has the following contract:
-     *
-     * <ul>
-     *   <li>1. Uniformly distributed across subtasks
-     *   <li>2. Partitions are round-robin distributed (strictly clockwise w.r.t. ascending subtask
-     *       indices) by using the partition id as the offset from a starting index (i.e., the index
-     *       of the subtask which partition 0 of the topic will be assigned to, determined using the
-     *       topic name).
-     * </ul>
-     *
-     * @param topic the RocketMQ topic assigned.
-     * @param broker the RocketMQ broker assigned.
-     * @param partition the RocketMQ partition to assign.
-     * @param numReaders the total number of readers.
-     * @return the id of the subtask that owns the split.
-     */
-    @VisibleForTesting
-    static int getSplitOwner(String topic, String broker, int partition, int numReaders) {
-        int startIndex = (((topic + "-" + broker).hashCode() * 31) & 0x7FFFFFFF) % numReaders;
-
-        // here, the assumption is that the id of RocketMQ partitions are always ascending
-        // starting from 0, and therefore can be used directly as the offset clockwise from the
-        // start index
-        return (startIndex + partition) % numReaders;
-    }
-}
diff --git a/src/main/java/org/apache/rocketmq/flink/source/reader/RocketMQPartitionSplitReader.java b/src/main/java/org/apache/rocketmq/flink/source/reader/RocketMQPartitionSplitReader.java
deleted file mode 100644
index 72fd96e..0000000
--- a/src/main/java/org/apache/rocketmq/flink/source/reader/RocketMQPartitionSplitReader.java
+++ /dev/null
@@ -1,425 +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.flink.source.reader;
-
-import org.apache.rocketmq.acl.common.AclClientRPCHook;
-import org.apache.rocketmq.acl.common.SessionCredentials;
-import org.apache.rocketmq.client.consumer.DefaultLitePullConsumer;
-import org.apache.rocketmq.client.consumer.MessageSelector;
-import org.apache.rocketmq.client.exception.MQClientException;
-import org.apache.rocketmq.common.message.MessageExt;
-import org.apache.rocketmq.common.message.MessageQueue;
-import org.apache.rocketmq.flink.source.reader.deserializer.RocketMQDeserializationSchema;
-import org.apache.rocketmq.flink.source.split.RocketMQPartitionSplit;
-
-import org.apache.flink.api.java.tuple.Tuple3;
-import org.apache.flink.connector.base.source.reader.RecordsWithSplitIds;
-import org.apache.flink.connector.base.source.reader.splitreader.SplitReader;
-import org.apache.flink.connector.base.source.reader.splitreader.SplitsAddition;
-import org.apache.flink.connector.base.source.reader.splitreader.SplitsChange;
-import org.apache.flink.util.Collector;
-import org.apache.flink.util.Preconditions;
-
-import org.apache.commons.lang3.StringUtils;
-import org.slf4j.Logger;
-import org.slf4j.LoggerFactory;
-
-import javax.annotation.Nullable;
-
-import java.io.IOException;
-import java.lang.management.ManagementFactory;
-import java.util.ArrayList;
-import java.util.Collection;
-import java.util.Collections;
-import java.util.HashMap;
-import java.util.HashSet;
-import java.util.Iterator;
-import java.util.List;
-import java.util.Map;
-import java.util.Set;
-
-/**
- * A {@link SplitReader} implementation that reads records from RocketMQ partitions.
- *
- * <p>The returned type are in the format of {@code tuple3(record, offset and timestamp}.
- */
-public class RocketMQPartitionSplitReader<T>
-        implements SplitReader<Tuple3<T, Long, Long>, RocketMQPartitionSplit> {
-    private static final Logger LOG = LoggerFactory.getLogger(RocketMQPartitionSplitReader.class);
-
-    private final String topic;
-    private final String tag;
-    private final String sql;
-    private final long stopInMs;
-    private final long startTime;
-    private final long startOffset;
-
-    private final long pollTime;
-
-    private final String accessKey;
-    private final String secretKey;
-
-    private final RocketMQDeserializationSchema<T> deserializationSchema;
-    private final Map<Tuple3<String, String, Integer>, Long> startingOffsets;
-    private final Map<Tuple3<String, String, Integer>, Long> stoppingTimestamps;
-    private final SimpleCollector<T> collector;
-
-    private DefaultLitePullConsumer consumer;
-
-    private volatile boolean wakeup = false;
-
-    private static final int MAX_MESSAGE_NUMBER_PER_BLOCK = 64;
-
-    public RocketMQPartitionSplitReader(
-            long pollTime,
-            String topic,
-            String consumerGroup,
-            String nameServerAddress,
-            String accessKey,
-            String secretKey,
-            String tag,
-            String sql,
-            long stopInMs,
-            long startTime,
-            long startOffset,
-            RocketMQDeserializationSchema<T> deserializationSchema) {
-        this.pollTime = pollTime;
-        this.topic = topic;
-        this.tag = tag;
-        this.sql = sql;
-        this.accessKey = accessKey;
-        this.secretKey = secretKey;
-        this.stopInMs = stopInMs;
-        this.startTime = startTime;
-        this.startOffset = startOffset;
-        this.deserializationSchema = deserializationSchema;
-        this.startingOffsets = new HashMap<>();
-        this.stoppingTimestamps = new HashMap<>();
-        this.collector = new SimpleCollector<>();
-        initialRocketMQConsumer(consumerGroup, nameServerAddress, accessKey, secretKey);
-    }
-
-    @Override
-    public RecordsWithSplitIds<Tuple3<T, Long, Long>> fetch() throws IOException {
-        RocketMQPartitionSplitRecords<Tuple3<T, Long, Long>> recordsBySplits =
-                new RocketMQPartitionSplitRecords<>();
-        Collection<MessageQueue> messageQueues;
-        try {
-            messageQueues = consumer.fetchMessageQueues(topic);
-        } catch (MQClientException e) {
-            LOG.error(
-                    String.format(
-                            "Fetch RocketMQ subscribe message queues of topic[%s] exception.",
-                            topic),
-                    e);
-            recordsBySplits.prepareForRead();
-            return recordsBySplits;
-        }
-        for (MessageQueue messageQueue : messageQueues) {
-            Tuple3<String, String, Integer> topicPartition =
-                    new Tuple3<>(
-                            messageQueue.getTopic(),
-                            messageQueue.getBrokerName(),
-                            messageQueue.getQueueId());
-            if (startingOffsets.containsKey(topicPartition)) {
-                long messageOffset = startingOffsets.get(topicPartition);
-                if (messageOffset == 0) {
-                    try {
-                        messageOffset =
-                                startTime > 0
-                                        ? consumer.offsetForTimestamp(messageQueue, startTime)
-                                        : startOffset;
-                    } catch (MQClientException e) {
-                        LOG.warn(
-                                String.format(
-                                        "Search RocketMQ message offset of topic[%s] broker[%s] queue[%d] exception.",
-                                        messageQueue.getTopic(),
-                                        messageQueue.getBrokerName(),
-                                        messageQueue.getQueueId()),
-                                e);
-                    }
-                    messageOffset = messageOffset > -1 ? messageOffset : 0;
-                }
-                List<MessageExt> messageExts = null;
-                try {
-                    if (wakeup) {
-                        LOG.info(
-                                String.format(
-                                        "Wake up pulling messages of topic[%s] broker[%s] queue[%d] tag[%s] sql[%s] from offset[%d].",
-                                        messageQueue.getTopic(),
-                                        messageQueue.getBrokerName(),
-                                        messageQueue.getQueueId(),
-                                        tag,
-                                        sql,
-                                        messageOffset));
-                        wakeup = false;
-                        recordsBySplits.prepareForRead();
-                        return recordsBySplits;
-                    }
-
-                    consumer.setPullBatchSize(MAX_MESSAGE_NUMBER_PER_BLOCK);
-                    consumer.seek(messageQueue, messageOffset);
-                    messageExts = consumer.poll(pollTime);
-                } catch (MQClientException e) {
-                    LOG.warn(
-                            String.format(
-                                    "Pull RocketMQ messages of topic[%s] broker[%s] queue[%d] tag[%s] sql[%s] from offset[%d] exception.",
-                                    messageQueue.getTopic(),
-                                    messageQueue.getBrokerName(),
-                                    messageQueue.getQueueId(),
-                                    tag,
-                                    sql,
-                                    messageOffset),
-                            e);
-                }
-                try {
-                    startingOffsets.put(
-                            topicPartition,
-                            messageExts == null ? messageOffset : consumer.committed(messageQueue));
-                } catch (MQClientException e) {
-                    LOG.warn(
-                            String.format(
-                                    "Pull RocketMQ messages of topic[%s] broker[%s] queue[%d] tag[%s] sql[%s] from offset[%d] exception.",
-                                    messageQueue.getTopic(),
-                                    messageQueue.getBrokerName(),
-                                    messageQueue.getQueueId(),
-                                    tag,
-                                    sql,
-                                    messageOffset),
-                            e);
-                }
-                if (messageExts != null) {
-                    Collection<Tuple3<T, Long, Long>> recordsForSplit =
-                            recordsBySplits.recordsForSplit(
-                                    messageQueue.getTopic()
-                                            + "-"
-                                            + messageQueue.getBrokerName()
-                                            + "-"
-                                            + messageQueue.getQueueId());
-                    for (MessageExt messageExt : messageExts) {
-                        long stoppingTimestamp = getStoppingTimestamp(topicPartition);
-                        long storeTimestamp = messageExt.getStoreTimestamp();
-                        if (storeTimestamp > stoppingTimestamp) {
-                            finishSplitAtRecord(
-                                    topicPartition,
-                                    stoppingTimestamp,
-                                    messageExt.getQueueOffset(),
-                                    recordsBySplits);
-                            break;
-                        }
-                        // Add the record to the partition collector.
-                        try {
-                            deserializationSchema.deserialize(
-                                    Collections.singletonList(messageExt), collector);
-                            collector
-                                    .getRecords()
-                                    .forEach(
-                                            r ->
-                                                    recordsForSplit.add(
-                                                            new Tuple3<>(
-                                                                    r,
-                                                                    messageExt.getQueueOffset(),
-                                                                    messageExt
-                                                                            .getStoreTimestamp())));
-                        } catch (Exception e) {
-                            throw new IOException(
-                                    "Failed to deserialize consumer record due to", e);
-                        } finally {
-                            collector.reset();
-                        }
-                    }
-                }
-            }
-        }
-        recordsBySplits.prepareForRead();
-        LOG.debug(
-                String.format(
-                        "Fetch record splits for MetaQ subscribe message queues of topic[%s].",
-                        topic));
-        return recordsBySplits;
-    }
-
-    @Override
-    public void handleSplitsChanges(SplitsChange<RocketMQPartitionSplit> splitsChange) {
-        // Get all the partition assignments and stopping timestamps..
-        if (!(splitsChange instanceof SplitsAddition)) {
-            throw new UnsupportedOperationException(
-                    String.format(
-                            "The SplitChange type of %s is not supported.",
-                            splitsChange.getClass()));
-        }
-        // Set up the stopping timestamps.
-        splitsChange
-                .splits()
-                .forEach(
-                        split -> {
-                            Tuple3<String, String, Integer> topicPartition =
-                                    new Tuple3<>(
-                                            split.getTopic(),
-                                            split.getBroker(),
-                                            split.getPartition());
-                            startingOffsets.put(topicPartition, split.getStartingOffset());
-                            stoppingTimestamps.put(topicPartition, split.getStoppingTimestamp());
-                        });
-    }
-
-    @Override
-    public void wakeUp() {
-        LOG.debug("Wake up the split reader in case the fetcher thread is blocking in fetch().");
-        wakeup = true;
-    }
-
-    @Override
-    public void close() {
-        consumer.shutdown();
-    }
-
-    private void finishSplitAtRecord(
-            Tuple3<String, String, Integer> topicPartition,
-            long stoppingTimestamp,
-            long currentOffset,
-            RocketMQPartitionSplitRecords<Tuple3<T, Long, Long>> recordsBySplits) {
-        LOG.debug(
-                "{} has reached stopping timestamp {}, current offset is {}",
-                topicPartition.f0 + "-" + topicPartition.f1,
-                stoppingTimestamp,
-                currentOffset);
-        recordsBySplits.addFinishedSplit(RocketMQPartitionSplit.toSplitId(topicPartition));
-        startingOffsets.remove(topicPartition);
-        stoppingTimestamps.remove(topicPartition);
-    }
-
-    private long getStoppingTimestamp(Tuple3<String, String, Integer> topicPartition) {
-        return stoppingTimestamps.getOrDefault(topicPartition, stopInMs);
-    }
-
-    // --------------- private helper method ----------------------
-
-    private void initialRocketMQConsumer(
-            String consumerGroup, String nameServerAddress, String accessKey, String secretKey) {
-
-        try {
-            if (StringUtils.isNotBlank(accessKey) && StringUtils.isNotBlank(secretKey)) {
-                AclClientRPCHook aclClientRPCHook =
-                        new AclClientRPCHook(new SessionCredentials(accessKey, secretKey));
-                consumer = new DefaultLitePullConsumer(consumerGroup, aclClientRPCHook);
-            } else {
-                consumer = new DefaultLitePullConsumer(consumerGroup);
-            }
-            consumer.setNamesrvAddr(nameServerAddress);
-            consumer.setInstanceName(
-                    String.join(
-                            "||",
-                            ManagementFactory.getRuntimeMXBean().getName(),
-                            topic,
-                            consumerGroup,
-                            "" + System.nanoTime()));
-            consumer.start();
-            if (StringUtils.isNotEmpty(sql)) {
-                consumer.subscribe(topic, MessageSelector.bySql(sql));
-            } else {
-                consumer.subscribe(topic, tag);
-            }
-        } catch (MQClientException e) {
-            LOG.error("Failed to initial RocketMQ consumer.", e);
-            consumer.shutdown();
-        }
-    }
-
-    // ---------------- private helper class ------------------------
-
-    private static class RocketMQPartitionSplitRecords<T> implements RecordsWithSplitIds<T> {
-        private final Map<String, Collection<T>> recordsBySplits;
-        private final Set<String> finishedSplits;
-        private Iterator<Map.Entry<String, Collection<T>>> splitIterator;
-        private String currentSplitId;
-        private Iterator<T> recordIterator;
-
-        public RocketMQPartitionSplitRecords() {
-            this.recordsBySplits = new HashMap<>();
-            this.finishedSplits = new HashSet<>();
-        }
-
-        private Collection<T> recordsForSplit(String splitId) {
-            return recordsBySplits.computeIfAbsent(splitId, id -> new ArrayList<>());
-        }
-
-        private void addFinishedSplit(String splitId) {
-            finishedSplits.add(splitId);
-        }
-
-        private void prepareForRead() {
-            splitIterator = recordsBySplits.entrySet().iterator();
-        }
-
-        @Override
-        @Nullable
-        public String nextSplit() {
-            if (splitIterator.hasNext()) {
-                Map.Entry<String, Collection<T>> entry = splitIterator.next();
-                currentSplitId = entry.getKey();
-                recordIterator = entry.getValue().iterator();
-                return currentSplitId;
-            } else {
-                currentSplitId = null;
-                recordIterator = null;
-                return null;
-            }
-        }
-
-        @Override
-        @Nullable
-        public T nextRecordFromSplit() {
-            Preconditions.checkNotNull(
-                    currentSplitId,
-                    "Make sure nextSplit() did not return null before "
-                            + "iterate over the records split.");
-            if (recordIterator.hasNext()) {
-                return recordIterator.next();
-            } else {
-                return null;
-            }
-        }
-
-        @Override
-        public Set<String> finishedSplits() {
-            return finishedSplits;
-        }
-    }
-
-    private static class SimpleCollector<T> implements Collector<T> {
-        private final List<T> records = new ArrayList<>();
-
-        @Override
-        public void collect(T record) {
-            records.add(record);
-        }
-
-        @Override
-        public void close() {}
-
-        private List<T> getRecords() {
-            return records;
-        }
-
-        private void reset() {
-            records.clear();
-        }
-    }
-}
diff --git a/src/main/java/org/apache/rocketmq/flink/source/reader/RocketMQRecordEmitter.java b/src/main/java/org/apache/rocketmq/flink/source/reader/RocketMQRecordEmitter.java
deleted file mode 100644
index 25270b5..0000000
--- a/src/main/java/org/apache/rocketmq/flink/source/reader/RocketMQRecordEmitter.java
+++ /dev/null
@@ -1,39 +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.flink.source.reader;
-
-import org.apache.rocketmq.flink.source.split.RocketMQPartitionSplitState;
-
-import org.apache.flink.api.connector.source.SourceOutput;
-import org.apache.flink.api.java.tuple.Tuple3;
-import org.apache.flink.connector.base.source.reader.RecordEmitter;
-
-/** The {@link RecordEmitter} implementation for {@link RocketMQSourceReader}. */
-public class RocketMQRecordEmitter<T>
-        implements RecordEmitter<Tuple3<T, Long, Long>, T, RocketMQPartitionSplitState> {
-
-    @Override
-    public void emitRecord(
-            Tuple3<T, Long, Long> element,
-            SourceOutput<T> output,
-            RocketMQPartitionSplitState splitState) {
-        output.collect(element.f0, element.f2);
-        splitState.setCurrentOffset(element.f1 + 1);
-    }
-}
diff --git a/src/main/java/org/apache/rocketmq/flink/source/reader/RocketMQSourceReader.java b/src/main/java/org/apache/rocketmq/flink/source/reader/RocketMQSourceReader.java
deleted file mode 100644
index 0257e34..0000000
--- a/src/main/java/org/apache/rocketmq/flink/source/reader/RocketMQSourceReader.java
+++ /dev/null
@@ -1,64 +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.flink.source.reader;
-
-import org.apache.rocketmq.flink.source.split.RocketMQPartitionSplit;
-import org.apache.rocketmq.flink.source.split.RocketMQPartitionSplitState;
-
-import org.apache.flink.api.connector.source.SourceReaderContext;
-import org.apache.flink.api.java.tuple.Tuple3;
-import org.apache.flink.configuration.Configuration;
-import org.apache.flink.connector.base.source.reader.RecordEmitter;
-import org.apache.flink.connector.base.source.reader.RecordsWithSplitIds;
-import org.apache.flink.connector.base.source.reader.SingleThreadMultiplexSourceReaderBase;
-import org.apache.flink.connector.base.source.reader.splitreader.SplitReader;
-import org.apache.flink.connector.base.source.reader.synchronization.FutureCompletingBlockingQueue;
-
-import java.util.Map;
-import java.util.function.Supplier;
-
-/** The source reader for RocketMQ partitions. */
-public class RocketMQSourceReader<T>
-        extends SingleThreadMultiplexSourceReaderBase<
-                Tuple3<T, Long, Long>, T, RocketMQPartitionSplit, RocketMQPartitionSplitState> {
-
-    public RocketMQSourceReader(
-            FutureCompletingBlockingQueue<RecordsWithSplitIds<Tuple3<T, Long, Long>>> elementsQueue,
-            Supplier<SplitReader<Tuple3<T, Long, Long>, RocketMQPartitionSplit>>
-                    splitReaderSupplier,
-            RecordEmitter<Tuple3<T, Long, Long>, T, RocketMQPartitionSplitState> recordEmitter,
-            Configuration config,
-            SourceReaderContext context) {
-        super(elementsQueue, splitReaderSupplier, recordEmitter, config, context);
-    }
-
-    @Override
-    protected void onSplitFinished(Map<String, RocketMQPartitionSplitState> map) {}
-
-    @Override
-    protected RocketMQPartitionSplitState initializedState(RocketMQPartitionSplit partitionSplit) {
-        return new RocketMQPartitionSplitState(partitionSplit);
-    }
-
-    @Override
-    protected RocketMQPartitionSplit toSplitType(
-            String splitId, RocketMQPartitionSplitState splitState) {
-        return splitState.toRocketMQPartitionSplit();
-    }
-}
diff --git a/src/main/java/org/apache/rocketmq/flink/source/reader/deserializer/RocketMQDeserializationSchema.java b/src/main/java/org/apache/rocketmq/flink/source/reader/deserializer/RocketMQDeserializationSchema.java
deleted file mode 100644
index e50b702..0000000
--- a/src/main/java/org/apache/rocketmq/flink/source/reader/deserializer/RocketMQDeserializationSchema.java
+++ /dev/null
@@ -1,61 +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.flink.source.reader.deserializer;
-
-import org.apache.rocketmq.common.message.MessageExt;
-
-import org.apache.flink.annotation.PublicEvolving;
-import org.apache.flink.api.common.serialization.DeserializationSchema.InitializationContext;
-import org.apache.flink.util.Collector;
-
-import java.io.IOException;
-import java.util.List;
-
-/** An interface for the deserialization of RocketMQ records. */
-public interface RocketMQDeserializationSchema<T>
-        extends DeserializationSchema<List<MessageExt>, T> {
-
-    /**
-     * Initialization method for the schema. It is called before the actual working methods {@link
-     * #deserialize} and thus suitable for one time setup work.
-     *
-     * <p>The provided {@link InitializationContext} can be used to access additional features such
-     * as e.g. registering user metrics.
-     *
-     * @param context Contextual information that can be used during initialization.
-     */
-    @Override
-    @PublicEvolving
-    default void open(InitializationContext context) {}
-
-    /**
-     * Deserializes the byte message.
-     *
-     * <p>Can output multiple records through the {@link Collector}. Note that number and size of
-     * the produced records should be relatively small. Depending on the source implementation
-     * records can be buffered in memory or collecting records might delay emitting checkpoint
-     * barrier.
-     *
-     * @param record The MessageExts to deserialize.
-     * @param out The collector to put the resulting messages.
-     */
-    @Override
-    @PublicEvolving
-    void deserialize(List<MessageExt> record, Collector<T> out) throws IOException;
-}
diff --git a/src/main/java/org/apache/rocketmq/flink/source/split/RocketMQPartitionSplit.java b/src/main/java/org/apache/rocketmq/flink/source/split/RocketMQPartitionSplit.java
deleted file mode 100644
index 5717767..0000000
--- a/src/main/java/org/apache/rocketmq/flink/source/split/RocketMQPartitionSplit.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.flink.source.split;
-
-import org.apache.flink.api.connector.source.SourceSplit;
-import org.apache.flink.api.java.tuple.Tuple3;
-
-import java.util.Objects;
-
-/** A {@link SourceSplit} for a RocketMQ partition. */
-public class RocketMQPartitionSplit implements SourceSplit {
-
-    private final String topic;
-    private final String broker;
-    private final int partition;
-    private final long startingOffset;
-    private final long stoppingTimestamp;
-
-    public RocketMQPartitionSplit(
-            String topic,
-            String broker,
-            int partition,
-            long startingOffset,
-            long stoppingTimestamp) {
-        this.topic = topic;
-        this.broker = broker;
-        this.partition = partition;
-        this.startingOffset = startingOffset;
-        this.stoppingTimestamp = stoppingTimestamp;
-    }
-
-    public String getTopic() {
-        return topic;
-    }
-
-    public String getBroker() {
-        return broker;
-    }
-
-    public int getPartition() {
-        return partition;
-    }
-
-    public long getStartingOffset() {
-        return startingOffset;
-    }
-
-    public long getStoppingTimestamp() {
-        return stoppingTimestamp;
-    }
-
-    @Override
-    public String splitId() {
-        return topic + "-" + broker + "-" + partition;
-    }
-
-    @Override
-    public String toString() {
-        return String.format(
-                "[Topic: %s, Broker: %s, Partition: %s, StartingOffset: %d, StoppingTimestamp: %d]",
-                topic, broker, partition, startingOffset, stoppingTimestamp);
-    }
-
-    @Override
-    public int hashCode() {
-        return Objects.hash(topic, broker, partition, startingOffset, stoppingTimestamp);
-    }
-
-    @Override
-    public boolean equals(Object obj) {
-        if (!(obj instanceof RocketMQPartitionSplit)) {
-            return false;
-        }
-        RocketMQPartitionSplit other = (RocketMQPartitionSplit) obj;
-        return topic.equals(other.topic)
-                && broker.equals(other.broker)
-                && partition == other.partition
-                && startingOffset == other.startingOffset
-                && stoppingTimestamp == other.stoppingTimestamp;
-    }
-
-    public static String toSplitId(Tuple3<String, String, Integer> topicPartition) {
-        return topicPartition.f0 + "-" + topicPartition.f1 + "-" + topicPartition.f2;
-    }
-}
diff --git a/src/main/java/org/apache/rocketmq/flink/source/split/RocketMQPartitionSplitSerializer.java b/src/main/java/org/apache/rocketmq/flink/source/split/RocketMQPartitionSplitSerializer.java
deleted file mode 100644
index 2363257..0000000
--- a/src/main/java/org/apache/rocketmq/flink/source/split/RocketMQPartitionSplitSerializer.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.flink.source.split;
-
-import org.apache.flink.core.io.SimpleVersionedSerializer;
-
-import java.io.ByteArrayInputStream;
-import java.io.ByteArrayOutputStream;
-import java.io.DataInputStream;
-import java.io.DataOutputStream;
-import java.io.IOException;
-
-/** The {@link SimpleVersionedSerializer serializer} for {@link RocketMQPartitionSplit}. */
-public class RocketMQPartitionSplitSerializer
-        implements SimpleVersionedSerializer<RocketMQPartitionSplit> {
-
-    private static final int CURRENT_VERSION = 0;
-
-    @Override
-    public int getVersion() {
-        return CURRENT_VERSION;
-    }
-
-    @Override
-    public byte[] serialize(RocketMQPartitionSplit split) throws IOException {
-        try (ByteArrayOutputStream baos = new ByteArrayOutputStream();
-                DataOutputStream out = new DataOutputStream(baos)) {
-            out.writeUTF(split.getTopic());
-            out.writeUTF(split.getBroker());
-            out.writeInt(split.getPartition());
-            out.writeLong(split.getStartingOffset());
-            out.writeLong(split.getStoppingTimestamp());
-            out.flush();
-            return baos.toByteArray();
-        }
-    }
-
-    @Override
-    public RocketMQPartitionSplit deserialize(int version, byte[] serialized) throws IOException {
-        try (ByteArrayInputStream bais = new ByteArrayInputStream(serialized);
-                DataInputStream in = new DataInputStream(bais)) {
-            String topic = in.readUTF();
-            String broker = in.readUTF();
-            int partition = in.readInt();
-            long offset = in.readLong();
-            long timestamp = in.readLong();
-            return new RocketMQPartitionSplit(topic, broker, partition, offset, timestamp);
-        }
-    }
-}
diff --git a/src/main/java/org/apache/rocketmq/flink/source/table/RocketMQDynamicTableSourceFactory.java b/src/main/java/org/apache/rocketmq/flink/source/table/RocketMQDynamicTableSourceFactory.java
deleted file mode 100644
index 8b4fd52..0000000
--- a/src/main/java/org/apache/rocketmq/flink/source/table/RocketMQDynamicTableSourceFactory.java
+++ /dev/null
@@ -1,211 +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.flink.source.table;
-
-import org.apache.rocketmq.flink.common.RocketMQOptions;
-
-import org.apache.flink.configuration.ConfigOption;
-import org.apache.flink.configuration.Configuration;
-import org.apache.flink.table.api.TableSchema;
-import org.apache.flink.table.connector.source.DynamicTableSource;
-import org.apache.flink.table.descriptors.DescriptorProperties;
-import org.apache.flink.table.factories.DynamicTableSourceFactory;
-import org.apache.flink.table.factories.FactoryUtil;
-import org.apache.flink.table.utils.TableSchemaUtils;
-import org.apache.flink.util.Preconditions;
-import org.apache.flink.util.StringUtils;
-
-import org.apache.commons.lang3.time.FastDateFormat;
-
-import java.text.ParseException;
-import java.util.HashSet;
-import java.util.Map;
-import java.util.Set;
-import java.util.TimeZone;
-
-import static org.apache.flink.table.factories.FactoryUtil.createTableFactoryHelper;
-import static org.apache.rocketmq.flink.common.RocketMQOptions.CONSUMER_GROUP;
-import static org.apache.rocketmq.flink.common.RocketMQOptions.NAME_SERVER_ADDRESS;
-import static org.apache.rocketmq.flink.common.RocketMQOptions.OPTIONAL_ACCESS_KEY;
-import static org.apache.rocketmq.flink.common.RocketMQOptions.OPTIONAL_COLUMN_ERROR_DEBUG;
-import static org.apache.rocketmq.flink.common.RocketMQOptions.OPTIONAL_CONSUMER_POLL_MS;
-import static org.apache.rocketmq.flink.common.RocketMQOptions.OPTIONAL_ENCODING;
-import static org.apache.rocketmq.flink.common.RocketMQOptions.OPTIONAL_END_TIME;
-import static org.apache.rocketmq.flink.common.RocketMQOptions.OPTIONAL_FIELD_DELIMITER;
-import static org.apache.rocketmq.flink.common.RocketMQOptions.OPTIONAL_LENGTH_CHECK;
-import static org.apache.rocketmq.flink.common.RocketMQOptions.OPTIONAL_LINE_DELIMITER;
-import static org.apache.rocketmq.flink.common.RocketMQOptions.OPTIONAL_PARTITION_DISCOVERY_INTERVAL_MS;
-import static org.apache.rocketmq.flink.common.RocketMQOptions.OPTIONAL_SCAN_STARTUP_MODE;
-import static org.apache.rocketmq.flink.common.RocketMQOptions.OPTIONAL_SECRET_KEY;
-import static org.apache.rocketmq.flink.common.RocketMQOptions.OPTIONAL_SQL;
-import static org.apache.rocketmq.flink.common.RocketMQOptions.OPTIONAL_START_MESSAGE_OFFSET;
-import static org.apache.rocketmq.flink.common.RocketMQOptions.OPTIONAL_START_TIME;
-import static org.apache.rocketmq.flink.common.RocketMQOptions.OPTIONAL_START_TIME_MILLS;
-import static org.apache.rocketmq.flink.common.RocketMQOptions.OPTIONAL_TAG;
-import static org.apache.rocketmq.flink.common.RocketMQOptions.OPTIONAL_TIME_ZONE;
-import static org.apache.rocketmq.flink.common.RocketMQOptions.OPTIONAL_USE_NEW_API;
-import static org.apache.rocketmq.flink.common.RocketMQOptions.TOPIC;
-import static org.apache.rocketmq.flink.legacy.RocketMQConfig.CONSUMER_OFFSET_LATEST;
-
-/**
- * Defines the {@link DynamicTableSourceFactory} implementation to create {@link
- * RocketMQScanTableSource}.
- */
-public class RocketMQDynamicTableSourceFactory implements DynamicTableSourceFactory {
-
-    private static final String DATE_FORMAT = "yyyy-MM-dd HH:mm:ss";
-
-    @Override
-    public String factoryIdentifier() {
-        return "rocketmq";
-    }
-
-    @Override
-    public Set<ConfigOption<?>> requiredOptions() {
-        Set<ConfigOption<?>> requiredOptions = new HashSet<>();
-        requiredOptions.add(TOPIC);
-        requiredOptions.add(CONSUMER_GROUP);
-        requiredOptions.add(NAME_SERVER_ADDRESS);
-        return requiredOptions;
-    }
-
-    @Override
-    public Set<ConfigOption<?>> optionalOptions() {
-        Set<ConfigOption<?>> optionalOptions = new HashSet<>();
-        optionalOptions.add(OPTIONAL_TAG);
-        optionalOptions.add(OPTIONAL_SQL);
-        optionalOptions.add(OPTIONAL_START_MESSAGE_OFFSET);
-        optionalOptions.add(OPTIONAL_START_TIME_MILLS);
-        optionalOptions.add(OPTIONAL_START_TIME);
-        optionalOptions.add(OPTIONAL_END_TIME);
-        optionalOptions.add(OPTIONAL_TIME_ZONE);
-        optionalOptions.add(OPTIONAL_PARTITION_DISCOVERY_INTERVAL_MS);
-        optionalOptions.add(OPTIONAL_USE_NEW_API);
-        optionalOptions.add(OPTIONAL_ENCODING);
-        optionalOptions.add(OPTIONAL_FIELD_DELIMITER);
-        optionalOptions.add(OPTIONAL_LINE_DELIMITER);
-        optionalOptions.add(OPTIONAL_COLUMN_ERROR_DEBUG);
-        optionalOptions.add(OPTIONAL_LENGTH_CHECK);
-        optionalOptions.add(OPTIONAL_ACCESS_KEY);
-        optionalOptions.add(OPTIONAL_SECRET_KEY);
-        optionalOptions.add(OPTIONAL_SCAN_STARTUP_MODE);
-        optionalOptions.add(OPTIONAL_CONSUMER_POLL_MS);
-        return optionalOptions;
-    }
-
-    @Override
-    public DynamicTableSource createDynamicTableSource(Context context) {
-        FactoryUtil.TableFactoryHelper helper = createTableFactoryHelper(this, context);
-        helper.validate();
-        Map<String, String> rawProperties = context.getCatalogTable().getOptions();
-        Configuration configuration = Configuration.fromMap(rawProperties);
-        String topic = configuration.getString(TOPIC);
-        String consumerGroup = configuration.getString(CONSUMER_GROUP);
-        String nameServerAddress = configuration.getString(NAME_SERVER_ADDRESS);
-        String tag = configuration.getString(OPTIONAL_TAG);
-        String sql = configuration.getString(OPTIONAL_SQL);
-        if (configuration.contains(OPTIONAL_SCAN_STARTUP_MODE)
-                && (configuration.contains(OPTIONAL_START_MESSAGE_OFFSET)
-                        || configuration.contains(OPTIONAL_START_TIME_MILLS)
-                        || configuration.contains(OPTIONAL_START_TIME))) {
-            throw new IllegalArgumentException(
-                    String.format(
-                            "cannot support these configs when %s has been set: [%s, %s, %s] !",
-                            OPTIONAL_SCAN_STARTUP_MODE.key(),
-                            OPTIONAL_START_MESSAGE_OFFSET.key(),
-                            OPTIONAL_START_TIME.key(),
-                            OPTIONAL_START_TIME_MILLS.key()));
-        }
-        long startMessageOffset = configuration.getLong(OPTIONAL_START_MESSAGE_OFFSET);
-        long startTimeMs = configuration.getLong(OPTIONAL_START_TIME_MILLS);
-        String startDateTime = configuration.getString(OPTIONAL_START_TIME);
-        String timeZone = configuration.getString(OPTIONAL_TIME_ZONE);
-        String accessKey = configuration.getString(OPTIONAL_ACCESS_KEY);
-        String secretKey = configuration.getString(OPTIONAL_SECRET_KEY);
-        long startTime = startTimeMs;
-        if (startTime == -1) {
-            if (!StringUtils.isNullOrWhitespaceOnly(startDateTime)) {
-                try {
-                    startTime = parseDateString(startDateTime, timeZone);
-                } catch (ParseException e) {
-                    throw new RuntimeException(
-                            String.format(
-                                    "Incorrect datetime format: %s, pls use ISO-8601 "
-                                            + "complete date plus hours, minutes and seconds format:%s.",
-                                    startDateTime, DATE_FORMAT),
-                            e);
-                }
-            }
-        }
-        long stopInMs = Long.MAX_VALUE;
-        String endDateTime = configuration.getString(OPTIONAL_END_TIME);
-        if (!StringUtils.isNullOrWhitespaceOnly(endDateTime)) {
-            try {
-                stopInMs = parseDateString(endDateTime, timeZone);
-            } catch (ParseException e) {
-                throw new RuntimeException(
-                        String.format(
-                                "Incorrect datetime format: %s, pls use ISO-8601 "
-                                        + "complete date plus hours, minutes and seconds format:%s.",
-                                endDateTime, DATE_FORMAT),
-                        e);
-            }
-            Preconditions.checkArgument(
-                    stopInMs >= startTime, "Start time should be less than stop time.");
-        }
-        long partitionDiscoveryIntervalMs =
-                configuration.getLong(OPTIONAL_PARTITION_DISCOVERY_INTERVAL_MS);
-        boolean useNewApi = configuration.getBoolean(OPTIONAL_USE_NEW_API);
-        DescriptorProperties descriptorProperties = new DescriptorProperties();
-        descriptorProperties.putProperties(rawProperties);
-        TableSchema physicalSchema =
-                TableSchemaUtils.getPhysicalSchema(context.getCatalogTable().getSchema());
-        descriptorProperties.putTableSchema("schema", physicalSchema);
-        String consumerOffsetMode =
-                configuration.getString(
-                        RocketMQOptions.OPTIONAL_SCAN_STARTUP_MODE, CONSUMER_OFFSET_LATEST);
-        long consumerOffsetTimestamp =
-                configuration.getLong(
-                        RocketMQOptions.OPTIONAL_OFFSET_FROM_TIMESTAMP, System.currentTimeMillis());
-        return new RocketMQScanTableSource(
-                configuration.getLong(OPTIONAL_CONSUMER_POLL_MS),
-                descriptorProperties,
-                physicalSchema,
-                topic,
-                consumerGroup,
-                nameServerAddress,
-                accessKey,
-                secretKey,
-                tag,
-                sql,
-                stopInMs,
-                startMessageOffset,
-                startMessageOffset < 0 ? startTime : -1L,
-                partitionDiscoveryIntervalMs,
-                consumerOffsetMode,
-                consumerOffsetTimestamp,
-                useNewApi);
-    }
-
-    private Long parseDateString(String dateString, String timeZone) throws ParseException {
-        FastDateFormat simpleDateFormat =
-                FastDateFormat.getInstance(DATE_FORMAT, TimeZone.getTimeZone(timeZone));
-        return simpleDateFormat.parse(dateString).getTime();
-    }
-}
diff --git a/src/main/resources/META-INF/services/org.apache.flink.table.factories.Factory b/src/main/resources/META-INF/services/org.apache.flink.table.factories.Factory
index 9e866fc..00a2a7f 100644
--- a/src/main/resources/META-INF/services/org.apache.flink.table.factories.Factory
+++ b/src/main/resources/META-INF/services/org.apache.flink.table.factories.Factory
@@ -13,6 +13,6 @@
 # See the License for the specific language governing permissions and
 # limitations under the License.
 
-org.apache.rocketmq.flink.source.table.RocketMQDynamicTableSourceFactory
-org.apache.rocketmq.flink.sink.table.RocketMQDynamicTableSinkFactory
-org.apache.rocketmq.flink.catalog.RocketMQCatalogFactory
+org.apache.flink.connector.rocketmq.source.table.RocketMQDynamicTableSourceFactory
+org.apache.flink.connector.rocketmq.sink.table.RocketMQDynamicTableSinkFactory
+org.apache.flink.connector.rocketmq.catalog.RocketMQCatalogFactory
diff --git a/src/test/java/org/apache/rocketmq/flink/catalog/RocketMQCatalogFactoryTest.java b/src/test/java/org/apache/flink/connector/rocketmq/catalog/RocketMQCatalogFactoryTest.java
similarity index 97%
rename from src/test/java/org/apache/rocketmq/flink/catalog/RocketMQCatalogFactoryTest.java
rename to src/test/java/org/apache/flink/connector/rocketmq/catalog/RocketMQCatalogFactoryTest.java
index 0766efc..6c70939 100644
--- a/src/test/java/org/apache/rocketmq/flink/catalog/RocketMQCatalogFactoryTest.java
+++ b/src/test/java/org/apache/flink/connector/rocketmq/catalog/RocketMQCatalogFactoryTest.java
@@ -16,7 +16,7 @@
  * limitations under the License.
  */
 
-package org.apache.rocketmq.flink.catalog;
+package org.apache.flink.connector.rocketmq.catalog;
 
 import org.apache.flink.configuration.ConfigOption;
 import org.apache.flink.table.catalog.Catalog;
diff --git a/src/test/java/org/apache/rocketmq/flink/catalog/RocketMQCatalogTest.java b/src/test/java/org/apache/flink/connector/rocketmq/catalog/RocketMQCatalogTest.java
similarity index 98%
rename from src/test/java/org/apache/rocketmq/flink/catalog/RocketMQCatalogTest.java
rename to src/test/java/org/apache/flink/connector/rocketmq/catalog/RocketMQCatalogTest.java
index 6801ae8..767822d 100644
--- a/src/test/java/org/apache/rocketmq/flink/catalog/RocketMQCatalogTest.java
+++ b/src/test/java/org/apache/flink/connector/rocketmq/catalog/RocketMQCatalogTest.java
@@ -15,17 +15,9 @@
  * See the License for the specific language governing permissions and
  * limitations under the License.
  */
-package org.apache.rocketmq.flink.catalog;
+package org.apache.flink.connector.rocketmq.catalog;
 
-import org.apache.rocketmq.common.admin.TopicOffset;
-import org.apache.rocketmq.common.admin.TopicStatsTable;
-import org.apache.rocketmq.common.message.MessageQueue;
-import org.apache.rocketmq.flink.common.constant.SchemaRegistryConstant;
-import org.apache.rocketmq.schema.registry.client.SchemaRegistryClient;
-import org.apache.rocketmq.schema.registry.common.dto.GetSchemaResponse;
-import org.apache.rocketmq.schema.registry.common.model.SchemaType;
-import org.apache.rocketmq.tools.admin.DefaultMQAdminExt;
-
+import org.apache.flink.connector.rocketmq.common.constant.SchemaRegistryConstant;
 import org.apache.flink.table.catalog.CatalogBaseTable;
 import org.apache.flink.table.catalog.CatalogDatabase;
 import org.apache.flink.table.catalog.CatalogPartition;
@@ -46,6 +38,13 @@
 import org.apache.flink.table.catalog.stats.CatalogTableStatistics;
 import org.apache.flink.table.factories.Factory;
 
+import org.apache.rocketmq.common.message.MessageQueue;
+import org.apache.rocketmq.remoting.protocol.admin.TopicOffset;
+import org.apache.rocketmq.remoting.protocol.admin.TopicStatsTable;
+import org.apache.rocketmq.schema.registry.client.SchemaRegistryClient;
+import org.apache.rocketmq.schema.registry.common.dto.GetSchemaResponse;
+import org.apache.rocketmq.schema.registry.common.model.SchemaType;
+import org.apache.rocketmq.tools.admin.DefaultMQAdminExt;
 import org.junit.Before;
 import org.junit.Test;
 import org.junit.runner.RunWith;
diff --git a/src/test/java/org/apache/flink/connector/rocketmq/common/config/RocketMQConfigBuilderTest.java b/src/test/java/org/apache/flink/connector/rocketmq/common/config/RocketMQConfigBuilderTest.java
new file mode 100644
index 0000000..2236092
--- /dev/null
+++ b/src/test/java/org/apache/flink/connector/rocketmq/common/config/RocketMQConfigBuilderTest.java
@@ -0,0 +1,57 @@
+package org.apache.flink.connector.rocketmq.common.config;
+
+import org.apache.flink.configuration.ConfigOption;
+import org.apache.flink.configuration.ConfigOptions;
+import org.apache.flink.configuration.Configuration;
+
+import org.junit.jupiter.api.Test;
+
+import java.util.Properties;
+
+import static org.junit.jupiter.api.Assertions.assertDoesNotThrow;
+import static org.junit.jupiter.api.Assertions.assertEquals;
+import static org.junit.jupiter.api.Assertions.assertThrows;
+
+public class RocketMQConfigBuilderTest {
+
+    @Test
+    void canNotSetSameOptionTwiceWithDifferentValue() {
+        ConfigOption<String> option = ConfigOptions.key("some.key").stringType().noDefaultValue();
+        RocketMQConfigBuilder builder = new RocketMQConfigBuilder();
+        builder.set(option, "value1");
+
+        assertDoesNotThrow(() -> builder.set(option, "value1"));
+        assertThrows(IllegalArgumentException.class, () -> builder.set(option, "value2"));
+    }
+
+    @Test
+    void setConfigurationCanNotOverrideExistedKeysWithNewValue() {
+        ConfigOption<String> option = ConfigOptions.key("string.k1").stringType().noDefaultValue();
+        RocketMQConfigBuilder builder = new RocketMQConfigBuilder();
+
+        Configuration configuration = new Configuration();
+        configuration.set(option, "value1");
+
+        builder.set(option, "value1");
+        assertDoesNotThrow(() -> builder.set(configuration));
+
+        configuration.set(option, "value2");
+        assertThrows(IllegalArgumentException.class, () -> builder.set(configuration));
+    }
+
+    @Test
+    void setPropertiesCanNotOverrideExistedKeysWithNewValueAndSupportTypeConversion() {
+        ConfigOption<Integer> option = ConfigOptions.key("int.type").intType().defaultValue(3);
+        RocketMQConfigBuilder builder = new RocketMQConfigBuilder();
+
+        Properties properties = new Properties();
+        properties.put("int.type", "6");
+        assertDoesNotThrow(() -> builder.set(properties));
+
+        properties.put("int.type", "1");
+        assertThrows(IllegalArgumentException.class, () -> builder.set(properties));
+
+        Integer value = builder.get(option);
+        assertEquals(value, 6);
+    }
+}
diff --git a/src/test/java/org/apache/flink/connector/rocketmq/example/ConnectorConfig.java b/src/test/java/org/apache/flink/connector/rocketmq/example/ConnectorConfig.java
new file mode 100644
index 0000000..111c220
--- /dev/null
+++ b/src/test/java/org/apache/flink/connector/rocketmq/example/ConnectorConfig.java
@@ -0,0 +1,53 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.flink.connector.rocketmq.example;
+
+import org.apache.rocketmq.acl.common.AclClientRPCHook;
+import org.apache.rocketmq.acl.common.SessionCredentials;
+import org.apache.rocketmq.remoting.RPCHook;
+
+public class ConnectorConfig {
+
+    // for rocketmq cluster
+    public static final String ENDPOINTS = "127.0.0.1:9876";
+    public static final String CLUSTER_NAME = "flink-cluster";
+
+    // for admin
+    public static final String ADMIN_TOOL_GROUP = "rocketmq-tools";
+
+    // for producer and consumer
+    public static final String PRODUCER_GROUP = "PID-flink";
+    public static final String CONSUMER_GROUP = "GID-flink";
+    public static final String SOURCE_TOPIC_1 = "flink-source-1";
+    public static final String SOURCE_TOPIC_2 = "flink-source-2";
+    public static final String SINK_TOPIC_1 = "flink-sink-1";
+    public static final String SINK_TOPIC_2 = "flink-sink-2";
+
+    // for message
+    public static final String TAGS = "*";
+    public static final String KEY_PREFIX = "key_";
+    public static final int MESSAGE_NUM = 100;
+
+    // for authentication
+    public static final String ACCESS_KEY = "accessKey";
+    public static final String SECRET_KEY = "secretKey";
+
+    public static RPCHook getAclRpcHook() {
+        return new AclClientRPCHook(new SessionCredentials(ACCESS_KEY, SECRET_KEY));
+    }
+}
diff --git a/src/main/java/org/apache/rocketmq/flink/legacy/example/RocketMQFlinkExample.java b/src/test/java/org/apache/flink/connector/rocketmq/example/LegacyConnectorExample.java
similarity index 87%
rename from src/main/java/org/apache/rocketmq/flink/legacy/example/RocketMQFlinkExample.java
rename to src/test/java/org/apache/flink/connector/rocketmq/example/LegacyConnectorExample.java
index fc0d3cb..d9ec711 100644
--- a/src/main/java/org/apache/rocketmq/flink/legacy/example/RocketMQFlinkExample.java
+++ b/src/test/java/org/apache/flink/connector/rocketmq/example/LegacyConnectorExample.java
@@ -15,18 +15,16 @@
  * limitations under the License.
  */
 
-package org.apache.rocketmq.flink.legacy.example;
-
-import org.apache.rocketmq.client.AccessChannel;
-import org.apache.rocketmq.flink.legacy.RocketMQConfig;
-import org.apache.rocketmq.flink.legacy.RocketMQSink;
-import org.apache.rocketmq.flink.legacy.RocketMQSourceFunction;
-import org.apache.rocketmq.flink.legacy.common.serialization.SimpleTupleDeserializationSchema;
-import org.apache.rocketmq.flink.legacy.function.SinkMapFunction;
-import org.apache.rocketmq.flink.legacy.function.SourceMapFunction;
+package org.apache.flink.connector.rocketmq.example;
 
 import org.apache.flink.api.java.tuple.Tuple2;
 import org.apache.flink.api.java.utils.ParameterTool;
+import org.apache.flink.connector.rocketmq.legacy.RocketMQConfig;
+import org.apache.flink.connector.rocketmq.legacy.RocketMQSink;
+import org.apache.flink.connector.rocketmq.legacy.RocketMQSourceFunction;
+import org.apache.flink.connector.rocketmq.legacy.common.serialization.SimpleTupleDeserializationSchema;
+import org.apache.flink.connector.rocketmq.legacy.function.SinkMapFunction;
+import org.apache.flink.connector.rocketmq.legacy.function.SourceMapFunction;
 import org.apache.flink.runtime.state.memory.MemoryStateBackend;
 import org.apache.flink.streaming.api.CheckpointingMode;
 import org.apache.flink.streaming.api.TimeCharacteristic;
@@ -34,12 +32,14 @@
 import org.apache.flink.streaming.api.environment.CheckpointConfig;
 import org.apache.flink.streaming.api.environment.StreamExecutionEnvironment;
 
+import org.apache.rocketmq.client.AccessChannel;
+
 import java.util.Properties;
 
-import static org.apache.rocketmq.flink.legacy.RocketMQConfig.CONSUMER_OFFSET_LATEST;
-import static org.apache.rocketmq.flink.legacy.RocketMQConfig.DEFAULT_CONSUMER_TAG;
+import static org.apache.flink.connector.rocketmq.legacy.RocketMQConfig.CONSUMER_OFFSET_LATEST;
+import static org.apache.flink.connector.rocketmq.legacy.RocketMQConfig.DEFAULT_CONSUMER_TAG;
 
-public class RocketMQFlinkExample {
+public class LegacyConnectorExample {
 
     /**
      * Source Config
diff --git a/src/test/java/org/apache/flink/connector/rocketmq/example/SimpleAdmin.java b/src/test/java/org/apache/flink/connector/rocketmq/example/SimpleAdmin.java
new file mode 100644
index 0000000..c16ce83
--- /dev/null
+++ b/src/test/java/org/apache/flink/connector/rocketmq/example/SimpleAdmin.java
@@ -0,0 +1,107 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.flink.connector.rocketmq.example;
+
+import org.apache.rocketmq.client.exception.MQBrokerException;
+import org.apache.rocketmq.client.exception.MQClientException;
+import org.apache.rocketmq.common.TopicConfig;
+import org.apache.rocketmq.common.constant.PermName;
+import org.apache.rocketmq.remoting.exception.RemotingException;
+import org.apache.rocketmq.remoting.protocol.route.BrokerData;
+import org.apache.rocketmq.tools.admin.DefaultMQAdminExt;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import java.io.Closeable;
+import java.util.Set;
+import java.util.UUID;
+import java.util.stream.Collectors;
+
+public class SimpleAdmin implements Closeable {
+
+    private static final Logger log = LoggerFactory.getLogger(SimpleAdmin.class);
+
+    private final DefaultMQAdminExt adminExt;
+
+    public SimpleAdmin() throws MQClientException {
+        this.adminExt = new DefaultMQAdminExt(ConnectorConfig.getAclRpcHook(), 6 * 1000L);
+        this.adminExt.setNamesrvAddr(ConnectorConfig.ENDPOINTS);
+        this.adminExt.setVipChannelEnabled(false);
+        this.adminExt.setAdminExtGroup(ConnectorConfig.ADMIN_TOOL_GROUP);
+        this.adminExt.setInstanceName(
+                adminExt.getAdminExtGroup().concat("-").concat(UUID.randomUUID().toString()));
+        this.adminExt.start();
+        log.info(
+                "Initialize rocketmq simple admin tools success, name={}",
+                adminExt.getInstanceName());
+    }
+
+    @Override
+    public void close() {
+        this.adminExt.shutdown();
+    }
+
+    private Set<String> getBrokerAddress()
+            throws RemotingException, InterruptedException, MQClientException {
+        return adminExt.examineTopicRouteInfo(ConnectorConfig.CLUSTER_NAME).getBrokerDatas()
+                .stream()
+                .map(BrokerData::selectBrokerAddr)
+                .collect(Collectors.toSet());
+    }
+
+    private void createTestTopic(Set<String> brokerAddressSet)
+            throws MQBrokerException, RemotingException, InterruptedException, MQClientException {
+
+        for (String brokerAddress : brokerAddressSet) {
+            adminExt.createAndUpdateTopicConfig(
+                    brokerAddress,
+                    new TopicConfig(
+                            ConnectorConfig.SOURCE_TOPIC_1,
+                            2,
+                            2,
+                            PermName.PERM_READ | PermName.PERM_WRITE));
+
+            adminExt.createAndUpdateTopicConfig(
+                    brokerAddress,
+                    new TopicConfig(
+                            ConnectorConfig.SOURCE_TOPIC_2,
+                            2,
+                            2,
+                            PermName.PERM_READ | PermName.PERM_WRITE));
+
+            log.info("Create topic success, brokerAddress={}", brokerAddress);
+        }
+    }
+
+    private void deleteSubscriptionGroup(Set<String> brokerAddressSet)
+            throws MQBrokerException, RemotingException, InterruptedException, MQClientException {
+
+        for (String brokerAddress : brokerAddressSet) {
+            adminExt.deleteSubscriptionGroup(brokerAddress, ConnectorConfig.CONSUMER_GROUP, true);
+            log.info("Delete consumer group success, brokerAddress={}", brokerAddress);
+        }
+    }
+
+    public static void main(String[] args) throws Exception {
+        SimpleAdmin simpleAdmin = new SimpleAdmin();
+        Set<String> brokerAddressSet = simpleAdmin.getBrokerAddress();
+        simpleAdmin.deleteSubscriptionGroup(brokerAddressSet);
+        simpleAdmin.createTestTopic(brokerAddressSet);
+        simpleAdmin.close();
+    }
+}
diff --git a/src/test/java/org/apache/flink/connector/rocketmq/example/SimpleConsumer.java b/src/test/java/org/apache/flink/connector/rocketmq/example/SimpleConsumer.java
new file mode 100644
index 0000000..85238fd
--- /dev/null
+++ b/src/test/java/org/apache/flink/connector/rocketmq/example/SimpleConsumer.java
@@ -0,0 +1,66 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.flink.connector.rocketmq.example;
+
+import org.apache.rocketmq.client.AccessChannel;
+import org.apache.rocketmq.client.consumer.DefaultMQPushConsumer;
+import org.apache.rocketmq.client.consumer.listener.ConsumeConcurrentlyStatus;
+import org.apache.rocketmq.client.consumer.listener.MessageListenerConcurrently;
+import org.apache.rocketmq.client.consumer.rebalance.AllocateMessageQueueAveragely;
+import org.apache.rocketmq.client.exception.MQClientException;
+import org.apache.rocketmq.common.consumer.ConsumeFromWhere;
+import org.apache.rocketmq.common.message.MessageExt;
+
+import static org.apache.flink.connector.rocketmq.example.ConnectorConfig.ENDPOINTS;
+import static org.apache.flink.connector.rocketmq.example.ConnectorConfig.SOURCE_TOPIC_1;
+import static org.apache.flink.connector.rocketmq.example.ConnectorConfig.SOURCE_TOPIC_2;
+import static org.apache.flink.connector.rocketmq.example.ConnectorConfig.TAGS;
+import static org.apache.flink.connector.rocketmq.example.ConnectorConfig.getAclRpcHook;
+
+public class SimpleConsumer {
+
+    public static void main(String[] args) throws MQClientException {
+
+        DefaultMQPushConsumer consumer =
+                new DefaultMQPushConsumer(
+                        ConnectorConfig.CONSUMER_GROUP,
+                        getAclRpcHook(),
+                        new AllocateMessageQueueAveragely());
+
+        consumer.setNamesrvAddr(ENDPOINTS);
+        consumer.setAccessChannel(AccessChannel.CLOUD);
+        consumer.subscribe(SOURCE_TOPIC_1, TAGS);
+        consumer.subscribe(SOURCE_TOPIC_2, TAGS);
+        consumer.setConsumeFromWhere(ConsumeFromWhere.CONSUME_FROM_FIRST_OFFSET);
+        consumer.registerMessageListener(
+                (MessageListenerConcurrently)
+                        (msgList, context) -> {
+                            for (MessageExt msg : msgList) {
+                                System.out.printf(
+                                        "Receive: topic=%s, msgId=%s, brokerName=%s, queueId=%s, payload=%s\n",
+                                        msg.getTopic(),
+                                        msg.getMsgId(),
+                                        msg.getBrokerName(),
+                                        msg.getQueueId(),
+                                        new String(msg.getBody()));
+                            }
+                            return ConsumeConcurrentlyStatus.CONSUME_SUCCESS;
+                        });
+        consumer.start();
+    }
+}
diff --git a/src/test/java/org/apache/flink/connector/rocketmq/example/SimpleProducer.java b/src/test/java/org/apache/flink/connector/rocketmq/example/SimpleProducer.java
new file mode 100644
index 0000000..ac47bdb
--- /dev/null
+++ b/src/test/java/org/apache/flink/connector/rocketmq/example/SimpleProducer.java
@@ -0,0 +1,77 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.flink.connector.rocketmq.example;
+
+import org.apache.rocketmq.client.AccessChannel;
+import org.apache.rocketmq.client.exception.MQClientException;
+import org.apache.rocketmq.client.producer.DefaultMQProducer;
+import org.apache.rocketmq.client.producer.MQProducer;
+import org.apache.rocketmq.client.producer.SendResult;
+import org.apache.rocketmq.common.message.Message;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import static org.apache.flink.connector.rocketmq.example.ConnectorConfig.ENDPOINTS;
+import static org.apache.flink.connector.rocketmq.example.ConnectorConfig.KEY_PREFIX;
+import static org.apache.flink.connector.rocketmq.example.ConnectorConfig.SOURCE_TOPIC_1;
+import static org.apache.flink.connector.rocketmq.example.ConnectorConfig.SOURCE_TOPIC_2;
+import static org.apache.flink.connector.rocketmq.example.ConnectorConfig.TAGS;
+import static org.apache.flink.connector.rocketmq.example.ConnectorConfig.getAclRpcHook;
+
+public class SimpleProducer {
+
+    private static final Logger LOGGER = LoggerFactory.getLogger(SimpleProducer.class);
+
+    public static void main(String[] args) {
+
+        DefaultMQProducer producer =
+                new DefaultMQProducer(ConnectorConfig.PRODUCER_GROUP, getAclRpcHook(), true, null);
+        producer.setNamesrvAddr(ENDPOINTS);
+        producer.setAccessChannel(AccessChannel.CLOUD);
+
+        try {
+            producer.start();
+        } catch (MQClientException e) {
+            e.printStackTrace();
+        }
+
+        send(producer, SOURCE_TOPIC_1);
+        send(producer, SOURCE_TOPIC_2);
+
+        producer.shutdown();
+    }
+
+    private static void send(MQProducer producer, String topic) {
+        for (int i = 0; i < ConnectorConfig.MESSAGE_NUM; i++) {
+            String content = "Test Message " + i;
+            Message msg = new Message(topic, TAGS, KEY_PREFIX + i, content.getBytes());
+            try {
+                SendResult sendResult = producer.send(msg);
+                assert sendResult != null;
+                System.out.printf(
+                        "Send result: topic=%s, msgId=%s, brokerName=%s, queueId=%s\n",
+                        topic,
+                        sendResult.getMsgId(),
+                        sendResult.getMessageQueue().getBrokerName(),
+                        sendResult.getMessageQueue().getQueueId());
+            } catch (Exception e) {
+                LOGGER.info("Send message failed. {}", e.toString());
+            }
+        }
+    }
+}
diff --git a/src/test/java/org/apache/rocketmq/flink/legacy/RocketMQSinkTest.java b/src/test/java/org/apache/flink/connector/rocketmq/legacy/RocketMQSinkTest.java
similarity index 79%
rename from src/test/java/org/apache/rocketmq/flink/legacy/RocketMQSinkTest.java
rename to src/test/java/org/apache/flink/connector/rocketmq/legacy/RocketMQSinkTest.java
index ad3c0b1..e1eb863 100644
--- a/src/test/java/org/apache/rocketmq/flink/legacy/RocketMQSinkTest.java
+++ b/src/test/java/org/apache/flink/connector/rocketmq/legacy/RocketMQSinkTest.java
@@ -16,24 +16,23 @@
  * limitations under the License.
  */
 
-package org.apache.rocketmq.flink.legacy;
+package org.apache.flink.connector.rocketmq.legacy;
+
+import org.apache.flink.api.java.tuple.Tuple2;
+import org.apache.flink.connector.rocketmq.legacy.common.selector.DefaultTopicSelector;
+import org.apache.flink.connector.rocketmq.legacy.common.selector.TopicSelector;
+import org.apache.flink.connector.rocketmq.legacy.common.serialization.KeyValueSerializationSchema;
+import org.apache.flink.connector.rocketmq.legacy.common.serialization.SimpleKeyValueSerializationSchema;
+import org.apache.flink.connector.rocketmq.legacy.common.util.TestUtils;
 
 import org.apache.rocketmq.client.producer.DefaultMQProducer;
 import org.apache.rocketmq.common.message.Message;
-import org.apache.rocketmq.flink.legacy.common.selector.DefaultTopicSelector;
-import org.apache.rocketmq.flink.legacy.common.selector.TopicSelector;
-import org.apache.rocketmq.flink.legacy.common.serialization.KeyValueSerializationSchema;
-import org.apache.rocketmq.flink.legacy.common.serialization.SimpleKeyValueSerializationSchema;
-
-import org.apache.flink.api.java.tuple.Tuple2;
-
 import org.junit.Before;
 import org.junit.Ignore;
 import org.junit.Test;
 
 import java.util.Properties;
 
-import static org.apache.rocketmq.flink.legacy.common.util.TestUtils.setFieldValue;
 import static org.mockito.Mockito.mock;
 import static org.mockito.Mockito.verify;
 
@@ -54,7 +53,7 @@
         rocketMQSink = new RocketMQSink(props);
 
         producer = mock(DefaultMQProducer.class);
-        setFieldValue(rocketMQSink, "producer", producer);
+        TestUtils.setFieldValue(rocketMQSink, "producer", producer);
     }
 
     @Test
diff --git a/src/test/java/org/apache/rocketmq/flink/legacy/RocketMQSourceTest.java b/src/test/java/org/apache/flink/connector/rocketmq/legacy/RocketMQSourceTest.java
similarity index 83%
rename from src/test/java/org/apache/rocketmq/flink/legacy/RocketMQSourceTest.java
rename to src/test/java/org/apache/flink/connector/rocketmq/legacy/RocketMQSourceTest.java
index 9e78190..35a0eb7 100644
--- a/src/test/java/org/apache/rocketmq/flink/legacy/RocketMQSourceTest.java
+++ b/src/test/java/org/apache/flink/connector/rocketmq/legacy/RocketMQSourceTest.java
@@ -16,19 +16,18 @@
  * limitations under the License.
  */
 
-package org.apache.rocketmq.flink.legacy;
+package org.apache.flink.connector.rocketmq.legacy;
 
-import java.util.Map;
+import org.apache.flink.connector.rocketmq.legacy.common.serialization.KeyValueDeserializationSchema;
+import org.apache.flink.connector.rocketmq.legacy.common.serialization.SimpleKeyValueDeserializationSchema;
+import org.apache.flink.connector.rocketmq.legacy.common.util.TestUtils;
+import org.apache.flink.streaming.api.functions.source.SourceFunction.SourceContext;
+
 import org.apache.rocketmq.client.consumer.DefaultLitePullConsumer;
 import org.apache.rocketmq.client.consumer.PullResult;
 import org.apache.rocketmq.client.consumer.PullStatus;
 import org.apache.rocketmq.common.message.MessageExt;
 import org.apache.rocketmq.common.message.MessageQueue;
-import org.apache.rocketmq.flink.legacy.common.serialization.KeyValueDeserializationSchema;
-import org.apache.rocketmq.flink.legacy.common.serialization.SimpleKeyValueDeserializationSchema;
-
-import org.apache.flink.streaming.api.functions.source.SourceFunction.SourceContext;
-
 import org.junit.Before;
 import org.junit.Ignore;
 import org.junit.Test;
@@ -36,11 +35,11 @@
 import java.util.ArrayList;
 import java.util.HashSet;
 import java.util.List;
+import java.util.Map;
 import java.util.Properties;
 import java.util.Set;
 import java.util.concurrent.ConcurrentHashMap;
 
-import static org.apache.rocketmq.flink.legacy.common.util.TestUtils.setFieldValue;
 import static org.mockito.Matchers.any;
 import static org.mockito.Matchers.anyLong;
 import static org.mockito.Mockito.atLeastOnce;
@@ -62,13 +61,13 @@
         Properties props = new Properties();
         rocketMQSource = new RocketMQSourceFunction<>(deserializationSchema, props);
 
-        setFieldValue(rocketMQSource, "topic", topic);
-        setFieldValue(rocketMQSource, "runningChecker", new SingleRunningCheck());
-        setFieldValue(rocketMQSource, "offsetTable", new ConcurrentHashMap<>());
-        setFieldValue(rocketMQSource, "restoredOffsets", new ConcurrentHashMap<>());
+        TestUtils.setFieldValue(rocketMQSource, "topic", topic);
+        TestUtils.setFieldValue(rocketMQSource, "runningChecker", new SingleRunningCheck());
+        TestUtils.setFieldValue(rocketMQSource, "offsetTable", new ConcurrentHashMap<>());
+        TestUtils.setFieldValue(rocketMQSource, "restoredOffsets", new ConcurrentHashMap<>());
 
         consumer = mock(DefaultLitePullConsumer.class);
-        setFieldValue(rocketMQSource, "consumer", consumer);
+        TestUtils.setFieldValue(rocketMQSource, "consumer", consumer);
     }
 
     @Test
diff --git a/src/test/java/org/apache/rocketmq/flink/legacy/common/selector/DefaultTopicSelectorTest.java b/src/test/java/org/apache/flink/connector/rocketmq/legacy/common/selector/DefaultTopicSelectorTest.java
similarity index 94%
rename from src/test/java/org/apache/rocketmq/flink/legacy/common/selector/DefaultTopicSelectorTest.java
rename to src/test/java/org/apache/flink/connector/rocketmq/legacy/common/selector/DefaultTopicSelectorTest.java
index aa1528a..d91a045 100644
--- a/src/test/java/org/apache/rocketmq/flink/legacy/common/selector/DefaultTopicSelectorTest.java
+++ b/src/test/java/org/apache/flink/connector/rocketmq/legacy/common/selector/DefaultTopicSelectorTest.java
@@ -16,7 +16,7 @@
  * limitations under the License.
  */
 
-package org.apache.rocketmq.flink.legacy.common.selector;
+package org.apache.flink.connector.rocketmq.legacy.common.selector;
 
 import org.junit.Test;
 
diff --git a/src/test/java/org/apache/rocketmq/flink/legacy/common/selector/HashMessageQueueSelectorTest.java b/src/test/java/org/apache/flink/connector/rocketmq/legacy/common/selector/HashMessageQueueSelectorTest.java
similarity index 96%
rename from src/test/java/org/apache/rocketmq/flink/legacy/common/selector/HashMessageQueueSelectorTest.java
rename to src/test/java/org/apache/flink/connector/rocketmq/legacy/common/selector/HashMessageQueueSelectorTest.java
index a32f021..ddb88fa 100644
--- a/src/test/java/org/apache/rocketmq/flink/legacy/common/selector/HashMessageQueueSelectorTest.java
+++ b/src/test/java/org/apache/flink/connector/rocketmq/legacy/common/selector/HashMessageQueueSelectorTest.java
@@ -16,11 +16,10 @@
  * limitations under the License.
  */
 
-package org.apache.rocketmq.flink.legacy.common.selector;
+package org.apache.flink.connector.rocketmq.legacy.common.selector;
 
 import org.apache.rocketmq.common.message.Message;
 import org.apache.rocketmq.common.message.MessageQueue;
-
 import org.junit.Assert;
 import org.junit.Test;
 
diff --git a/src/test/java/org/apache/rocketmq/flink/legacy/common/selector/RandomMessageQueueSelectorTest.java b/src/test/java/org/apache/flink/connector/rocketmq/legacy/common/selector/RandomMessageQueueSelectorTest.java
similarity index 96%
rename from src/test/java/org/apache/rocketmq/flink/legacy/common/selector/RandomMessageQueueSelectorTest.java
rename to src/test/java/org/apache/flink/connector/rocketmq/legacy/common/selector/RandomMessageQueueSelectorTest.java
index 01166da..cad43a3 100644
--- a/src/test/java/org/apache/rocketmq/flink/legacy/common/selector/RandomMessageQueueSelectorTest.java
+++ b/src/test/java/org/apache/flink/connector/rocketmq/legacy/common/selector/RandomMessageQueueSelectorTest.java
@@ -16,11 +16,10 @@
  * limitations under the License.
  */
 
-package org.apache.rocketmq.flink.legacy.common.selector;
+package org.apache.flink.connector.rocketmq.legacy.common.selector;
 
 import org.apache.rocketmq.common.message.Message;
 import org.apache.rocketmq.common.message.MessageQueue;
-
 import org.junit.Assert;
 import org.junit.Test;
 
diff --git a/src/test/java/org/apache/rocketmq/flink/legacy/common/selector/SimpleTopicSelectorTest.java b/src/test/java/org/apache/flink/connector/rocketmq/legacy/common/selector/SimpleTopicSelectorTest.java
similarity index 95%
rename from src/test/java/org/apache/rocketmq/flink/legacy/common/selector/SimpleTopicSelectorTest.java
rename to src/test/java/org/apache/flink/connector/rocketmq/legacy/common/selector/SimpleTopicSelectorTest.java
index dc93d14..be55514 100644
--- a/src/test/java/org/apache/rocketmq/flink/legacy/common/selector/SimpleTopicSelectorTest.java
+++ b/src/test/java/org/apache/flink/connector/rocketmq/legacy/common/selector/SimpleTopicSelectorTest.java
@@ -16,7 +16,7 @@
  * limitations under the License.
  */
 
-package org.apache.rocketmq.flink.legacy.common.selector;
+package org.apache.flink.connector.rocketmq.legacy.common.selector;
 
 import org.junit.Test;
 
diff --git a/src/test/java/org/apache/rocketmq/flink/legacy/common/serialization/RowKeyValueDeserializationSchemaTest.java b/src/test/java/org/apache/flink/connector/rocketmq/legacy/common/serialization/RowKeyValueDeserializationSchemaTest.java
similarity index 96%
rename from src/test/java/org/apache/rocketmq/flink/legacy/common/serialization/RowKeyValueDeserializationSchemaTest.java
rename to src/test/java/org/apache/flink/connector/rocketmq/legacy/common/serialization/RowKeyValueDeserializationSchemaTest.java
index 2c1786a..a8ff892 100644
--- a/src/test/java/org/apache/rocketmq/flink/legacy/common/serialization/RowKeyValueDeserializationSchemaTest.java
+++ b/src/test/java/org/apache/flink/connector/rocketmq/legacy/common/serialization/RowKeyValueDeserializationSchemaTest.java
@@ -16,14 +16,13 @@
  * limitations under the License.
  */
 
-package org.apache.rocketmq.flink.legacy.common.serialization;
-
-import org.apache.rocketmq.common.message.MessageExt;
+package org.apache.flink.connector.rocketmq.legacy.common.serialization;
 
 import org.apache.flink.table.api.DataTypes;
 import org.apache.flink.table.api.TableSchema;
 import org.apache.flink.table.data.RowData;
 
+import org.apache.rocketmq.common.message.MessageExt;
 import org.junit.Test;
 
 import java.util.HashMap;
diff --git a/src/test/java/org/apache/rocketmq/flink/legacy/common/serialization/SimpleKeyValueSerializationSchemaTest.java b/src/test/java/org/apache/flink/connector/rocketmq/legacy/common/serialization/SimpleKeyValueSerializationSchemaTest.java
similarity index 95%
rename from src/test/java/org/apache/rocketmq/flink/legacy/common/serialization/SimpleKeyValueSerializationSchemaTest.java
rename to src/test/java/org/apache/flink/connector/rocketmq/legacy/common/serialization/SimpleKeyValueSerializationSchemaTest.java
index e27fdf5..680aab6 100644
--- a/src/test/java/org/apache/rocketmq/flink/legacy/common/serialization/SimpleKeyValueSerializationSchemaTest.java
+++ b/src/test/java/org/apache/flink/connector/rocketmq/legacy/common/serialization/SimpleKeyValueSerializationSchemaTest.java
@@ -16,7 +16,7 @@
  * limitations under the License.
  */
 
-package org.apache.rocketmq.flink.legacy.common.serialization;
+package org.apache.flink.connector.rocketmq.legacy.common.serialization;
 
 import org.junit.Test;
 
diff --git a/src/test/java/org/apache/rocketmq/flink/legacy/common/util/RetryUtilTest.java b/src/test/java/org/apache/flink/connector/rocketmq/legacy/common/util/RetryUtilTest.java
similarity index 94%
rename from src/test/java/org/apache/rocketmq/flink/legacy/common/util/RetryUtilTest.java
rename to src/test/java/org/apache/flink/connector/rocketmq/legacy/common/util/RetryUtilTest.java
index 0b7178b..d719aa3 100644
--- a/src/test/java/org/apache/rocketmq/flink/legacy/common/util/RetryUtilTest.java
+++ b/src/test/java/org/apache/flink/connector/rocketmq/legacy/common/util/RetryUtilTest.java
@@ -16,9 +16,9 @@
  * limitations under the License.
  */
 
-package org.apache.rocketmq.flink.legacy.common.util;
+package org.apache.flink.connector.rocketmq.legacy.common.util;
 
-import org.apache.rocketmq.flink.legacy.RunningChecker;
+import org.apache.flink.connector.rocketmq.legacy.RunningChecker;
 
 import junit.framework.TestCase;
 import lombok.Data;
diff --git a/src/test/java/org/apache/flink/connector/rocketmq/legacy/sourceFunction/RocketMQSourceFunctionTest.java b/src/test/java/org/apache/flink/connector/rocketmq/legacy/sourceFunction/RocketMQSourceFunctionTest.java
new file mode 100644
index 0000000..cd514cd
--- /dev/null
+++ b/src/test/java/org/apache/flink/connector/rocketmq/legacy/sourceFunction/RocketMQSourceFunctionTest.java
@@ -0,0 +1,94 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.flink.connector.rocketmq.legacy.sourceFunction;
+
+import org.apache.flink.connector.rocketmq.legacy.RocketMQConfig;
+import org.apache.flink.connector.rocketmq.legacy.RocketMQSourceFunction;
+import org.apache.flink.connector.rocketmq.legacy.common.config.OffsetResetStrategy;
+import org.apache.flink.connector.rocketmq.legacy.common.config.StartupMode;
+import org.apache.flink.connector.rocketmq.legacy.common.serialization.SimpleStringDeserializationSchema;
+import org.apache.flink.connector.rocketmq.legacy.common.util.TestUtils;
+
+import org.apache.rocketmq.common.message.MessageQueue;
+import org.junit.Assert;
+import org.junit.Test;
+
+import java.util.ArrayList;
+import java.util.HashMap;
+import java.util.Map;
+import java.util.Properties;
+import java.util.concurrent.ConcurrentHashMap;
+
+import static org.junit.Assert.assertEquals;
+
+/** Tests for {@link RocketMQSourceFunction}. */
+public class RocketMQSourceFunctionTest {
+
+    @Test
+    public void testSetStartupMode() throws NoSuchFieldException, IllegalAccessException {
+        RocketMQSourceFunction<String> source =
+                new RocketMQSourceFunction<>(
+                        new SimpleStringDeserializationSchema(), new Properties());
+        Assert.assertEquals(
+                StartupMode.GROUP_OFFSETS, TestUtils.getFieldValue(source, "startMode"));
+        source.setStartFromEarliest();
+        Assert.assertEquals(StartupMode.EARLIEST, TestUtils.getFieldValue(source, "startMode"));
+        source.setStartFromLatest();
+        Assert.assertEquals(StartupMode.LATEST, TestUtils.getFieldValue(source, "startMode"));
+        source.setStartFromTimeStamp(0L);
+        Assert.assertEquals(StartupMode.TIMESTAMP, TestUtils.getFieldValue(source, "startMode"));
+        source.setStartFromSpecificOffsets(null);
+        Assert.assertEquals(
+                StartupMode.SPECIFIC_OFFSETS, TestUtils.getFieldValue(source, "startMode"));
+        source.setStartFromGroupOffsets();
+        Assert.assertEquals(
+                StartupMode.GROUP_OFFSETS, TestUtils.getFieldValue(source, "startMode"));
+        Assert.assertEquals(
+                OffsetResetStrategy.LATEST, TestUtils.getFieldValue(source, "offsetResetStrategy"));
+        source.setStartFromGroupOffsets(OffsetResetStrategy.EARLIEST);
+        Assert.assertEquals(
+                OffsetResetStrategy.EARLIEST,
+                TestUtils.getFieldValue(source, "offsetResetStrategy"));
+    }
+
+    @Test
+    public void testRestartFromCheckpoint() throws Exception {
+        Properties properties = new Properties();
+        properties.setProperty(RocketMQConfig.CONSUMER_GROUP, "${ConsumerGroup}");
+        properties.setProperty(RocketMQConfig.CONSUMER_TOPIC, "${SourceTopic}");
+        properties.setProperty(RocketMQConfig.CONSUMER_TAG, RocketMQConfig.DEFAULT_CONSUMER_TAG);
+        RocketMQSourceFunction<String> source =
+                new RocketMQSourceFunction<>(new SimpleStringDeserializationSchema(), properties);
+        source.setStartFromLatest();
+        TestUtils.setFieldValue(source, "restored", true);
+        HashMap<MessageQueue, Long> map = new HashMap<>();
+        map.put(new MessageQueue("tpc", "broker-0", 0), 20L);
+        map.put(new MessageQueue("tpc", "broker-0", 1), 21L);
+        map.put(new MessageQueue("tpc", "broker-1", 0), 30L);
+        map.put(new MessageQueue("tpc", "broker-1", 1), 31L);
+        TestUtils.setFieldValue(source, "restoredOffsets", map);
+        TestUtils.setFieldValue(source, "offsetTable", new ConcurrentHashMap<>());
+        source.initOffsetTableFromRestoredOffsets(new ArrayList<>(map.keySet()));
+        Map<MessageQueue, Long> offsetTable = (Map) TestUtils.getFieldValue(source, "offsetTable");
+        for (Map.Entry<MessageQueue, Long> entry : map.entrySet()) {
+            assertEquals(offsetTable.containsKey(entry.getKey()), true);
+            assertEquals(offsetTable.containsValue(entry.getValue()), true);
+        }
+    }
+}
diff --git a/src/test/java/org/apache/flink/connector/rocketmq/sink/RocketMQSinkTest.java b/src/test/java/org/apache/flink/connector/rocketmq/sink/RocketMQSinkTest.java
new file mode 100644
index 0000000..207b963
--- /dev/null
+++ b/src/test/java/org/apache/flink/connector/rocketmq/sink/RocketMQSinkTest.java
@@ -0,0 +1,143 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.flink.connector.rocketmq.sink;
+
+import org.apache.flink.api.common.eventtime.SerializableTimestampAssigner;
+import org.apache.flink.api.common.eventtime.WatermarkStrategy;
+import org.apache.flink.api.common.typeinfo.Types;
+import org.apache.flink.configuration.Configuration;
+import org.apache.flink.configuration.RestOptions;
+import org.apache.flink.connector.base.DeliveryGuarantee;
+import org.apache.flink.connector.rocketmq.example.ConnectorConfig;
+import org.apache.flink.connector.rocketmq.sink.writer.serializer.RocketMQSerializationSchema;
+import org.apache.flink.connector.rocketmq.source.RocketMQSourceTest;
+import org.apache.flink.core.fs.FileSystem;
+import org.apache.flink.runtime.state.filesystem.FsStateBackend;
+import org.apache.flink.streaming.api.CheckpointingMode;
+import org.apache.flink.streaming.api.datastream.DataStream;
+import org.apache.flink.streaming.api.environment.StreamExecutionEnvironment;
+import org.apache.flink.streaming.api.functions.source.datagen.DataGeneratorSource;
+import org.apache.flink.streaming.api.functions.source.datagen.RandomGenerator;
+
+import org.apache.rocketmq.common.message.Message;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import java.io.File;
+import java.nio.file.Paths;
+import java.time.Duration;
+import java.util.concurrent.TimeUnit;
+
+public class RocketMQSinkTest {
+
+    private static final Logger log = LoggerFactory.getLogger(RocketMQSourceTest.class);
+
+    @SuppressWarnings("deprecation")
+    public static void main(String[] args) throws Exception {
+        String directory = "flink-connector-rocketmq";
+        String userHome = System.getProperty("user.home");
+        String ckptPath = Paths.get(userHome, directory, "ckpt").toString();
+        String sinkPath = Paths.get(userHome, directory, "sink").toString();
+        String ckptUri = "file://" + File.separator + ckptPath;
+
+        log.info("Connector checkpoint path: {}", ckptPath);
+
+        Configuration configuration = new Configuration();
+        configuration.setInteger(RestOptions.BIND_PORT.key(), 8088);
+
+        StreamExecutionEnvironment env =
+                StreamExecutionEnvironment.createLocalEnvironment(configuration);
+
+        DataStream<String> source =
+                env.addSource(
+                                new DataGeneratorSource<>(
+                                        new RandomGenerator<String>() {
+                                            @Override
+                                            public String next() {
+                                                long timestamp =
+                                                        System.currentTimeMillis()
+                                                                + random.nextInt(
+                                                                        -60 * 1000, 60 * 1000);
+                                                return String.join(
+                                                        "|",
+                                                        String.valueOf(random.nextInt(0, 1)),
+                                                        String.valueOf(timestamp));
+                                            }
+                                        },
+                                        1,
+                                        10000L))
+                        .returns(Types.STRING)
+                        .setParallelism(2)
+                        .assignTimestampsAndWatermarks(
+                                WatermarkStrategy.<String>forBoundedOutOfOrderness(
+                                                Duration.ofMinutes(1))
+                                        .withTimestampAssigner(
+                                                (SerializableTimestampAssigner<String>)
+                                                        (element, recordTimestamp) ->
+                                                                Long.parseLong(
+                                                                        element.split("\\|")[1]))
+                                        .withIdleness(Duration.ofMinutes(1)))
+                        .returns(Types.STRING)
+                        .setParallelism(2);
+
+        env.enableCheckpointing(TimeUnit.SECONDS.toMillis(10));
+        env.setStateBackend(new FsStateBackend(ckptUri));
+        env.getCheckpointConfig().setCheckpointingMode(CheckpointingMode.EXACTLY_ONCE);
+        env.getCheckpointConfig().setCheckpointTimeout(TimeUnit.MINUTES.toMillis(1));
+        env.getCheckpointConfig().setMaxConcurrentCheckpoints(1);
+        env.getCheckpointConfig().setMinPauseBetweenCheckpoints(TimeUnit.SECONDS.toMillis(1));
+        env.getCheckpointConfig().setTolerableCheckpointFailureNumber(0);
+
+        source.writeAsText(sinkPath, FileSystem.WriteMode.OVERWRITE)
+                .name("rocketmq-test-sink")
+                .setParallelism(2);
+
+        RocketMQSink<String> rocketmqSink =
+                RocketMQSink.<String>builder()
+                        .setEndpoints(ConnectorConfig.ENDPOINTS)
+                        .setGroupId(ConnectorConfig.PRODUCER_GROUP)
+                        .setConfig(
+                                RocketMQSinkOptions.OPTIONAL_ACCESS_KEY, ConnectorConfig.ACCESS_KEY)
+                        .setConfig(
+                                RocketMQSinkOptions.OPTIONAL_SECRET_KEY, ConnectorConfig.SECRET_KEY)
+                        .setSerializer(
+                                (RocketMQSerializationSchema<String>)
+                                        (element, context, timestamp) -> {
+                                            String topic =
+                                                    Long.parseLong(element.split("\\|")[1]) % 2 != 0
+                                                            ? ConnectorConfig.SINK_TOPIC_1
+                                                            : ConnectorConfig.SINK_TOPIC_2;
+                                            return new Message(
+                                                    topic,
+                                                    ConnectorConfig.TAGS,
+                                                    element.getBytes());
+                                        })
+                        // .setDeliveryGuarantee(DeliveryGuarantee.AT_LEAST_ONCE)
+                        // If you use transaction message, need set transaction timeout
+                        .setDeliveryGuarantee(DeliveryGuarantee.EXACTLY_ONCE)
+                        .setConfig(
+                                RocketMQSinkOptions.TRANSACTION_TIMEOUT,
+                                TimeUnit.SECONDS.toSeconds(30))
+                        .build();
+
+        source.sinkTo(rocketmqSink).setParallelism(2);
+
+        env.execute("rocketmq-local-test");
+        log.info("Start rocketmq sink for test success");
+    }
+}
diff --git a/src/test/java/org/apache/rocketmq/flink/sink/table/RocketMQDynamicTableSinkFactoryTest.java b/src/test/java/org/apache/flink/connector/rocketmq/sink/table/RocketMQDynamicTableSinkFactoryTest.java
similarity index 79%
rename from src/test/java/org/apache/rocketmq/flink/sink/table/RocketMQDynamicTableSinkFactoryTest.java
rename to src/test/java/org/apache/flink/connector/rocketmq/sink/table/RocketMQDynamicTableSinkFactoryTest.java
index b06695a..0da51c3 100644
--- a/src/test/java/org/apache/rocketmq/flink/sink/table/RocketMQDynamicTableSinkFactoryTest.java
+++ b/src/test/java/org/apache/flink/connector/rocketmq/sink/table/RocketMQDynamicTableSinkFactoryTest.java
@@ -16,11 +16,10 @@
  * limitations under the License.
  */
 
-package org.apache.rocketmq.flink.sink.table;
-
-import org.apache.rocketmq.flink.common.RocketMQOptions;
+package org.apache.flink.connector.rocketmq.sink.table;
 
 import org.apache.flink.configuration.Configuration;
+import org.apache.flink.connector.rocketmq.source.RocketMQSourceOptions;
 import org.apache.flink.table.api.Schema;
 import org.apache.flink.table.api.ValidationException;
 import org.apache.flink.table.catalog.CatalogTable;
@@ -31,6 +30,7 @@
 import org.apache.flink.table.connector.sink.DynamicTableSink;
 import org.apache.flink.table.factories.FactoryUtil;
 
+import org.junit.Ignore;
 import org.junit.Test;
 
 import java.util.ArrayList;
@@ -42,7 +42,7 @@
 import static org.junit.Assert.assertEquals;
 import static org.junit.Assert.assertTrue;
 
-/** Tests for {@link org.apache.rocketmq.flink.sink.table.RocketMQDynamicTableSinkFactory}. */
+/** Tests for {@link RocketMQDynamicTableSinkFactory}. */
 public class RocketMQDynamicTableSinkFactoryTest {
 
     private static final ResolvedSchema SCHEMA =
@@ -57,13 +57,15 @@
     private static final String NAME_SERVER_ADDRESS =
             "http://${instanceId}.${region}.mq-internal.aliyuncs.com:8080";
 
+    @Ignore
     @Test
     public void testRocketMQDynamicTableSinkWithLegalOption() {
         final Map<String, String> options = new HashMap<>();
         options.put("connector", IDENTIFIER);
-        options.put(RocketMQOptions.TOPIC.key(), TOPIC);
-        options.put(RocketMQOptions.PRODUCER_GROUP.key(), PRODUCER_GROUP);
-        options.put(RocketMQOptions.NAME_SERVER_ADDRESS.key(), NAME_SERVER_ADDRESS);
+        options.put(RocketMQSourceOptions.TOPIC.key(), TOPIC);
+        options.put(
+                RocketMQSourceOptions.OPTIONAL_STARTUP_OFFSET_TIMESTAMP.key(),
+                String.valueOf(System.currentTimeMillis()));
         final DynamicTableSink tableSink = createDynamicTableSink(options);
         assertTrue(tableSink instanceof RocketMQDynamicTableSink);
         assertEquals(RocketMQDynamicTableSink.class.getName(), tableSink.asSummaryString());
@@ -73,18 +75,21 @@
     public void testRocketMQDynamicTableSinkWithoutRequiredOption() {
         final Map<String, String> options = new HashMap<>();
         options.put("connector", IDENTIFIER);
-        options.put(RocketMQOptions.TOPIC.key(), TOPIC);
-        options.put(RocketMQOptions.PRODUCER_GROUP.key(), PRODUCER_GROUP);
-        options.put(RocketMQOptions.OPTIONAL_TAG.key(), "test_tag");
+        options.put(RocketMQSourceOptions.TOPIC.key(), TOPIC);
+        options.put(
+                RocketMQSourceOptions.OPTIONAL_STARTUP_OFFSET_TIMESTAMP.key(),
+                String.valueOf(System.currentTimeMillis()));
+        options.put(RocketMQSourceOptions.OPTIONAL_TAG.key(), "test_tag");
         createDynamicTableSink(options);
     }
 
     @Test(expected = ValidationException.class)
     public void testRocketMQDynamicTableSinkWithUnknownOption() {
         final Map<String, String> options = new HashMap<>();
-        options.put(RocketMQOptions.TOPIC.key(), TOPIC);
-        options.put(RocketMQOptions.PRODUCER_GROUP.key(), PRODUCER_GROUP);
-        options.put(RocketMQOptions.NAME_SERVER_ADDRESS.key(), NAME_SERVER_ADDRESS);
+        options.put(RocketMQSourceOptions.TOPIC.key(), TOPIC);
+        options.put(
+                RocketMQSourceOptions.OPTIONAL_STARTUP_OFFSET_TIMESTAMP.key(),
+                String.valueOf(System.currentTimeMillis()));
         options.put("unknown", "test_option");
         createDynamicTableSink(options);
     }
diff --git a/src/test/java/org/apache/flink/connector/rocketmq/source/RocketMQSourceTest.java b/src/test/java/org/apache/flink/connector/rocketmq/source/RocketMQSourceTest.java
new file mode 100644
index 0000000..0982fbd
--- /dev/null
+++ b/src/test/java/org/apache/flink/connector/rocketmq/source/RocketMQSourceTest.java
@@ -0,0 +1,90 @@
+package org.apache.flink.connector.rocketmq.source;
+
+import org.apache.flink.api.common.eventtime.WatermarkStrategy;
+import org.apache.flink.api.common.typeinfo.BasicTypeInfo;
+import org.apache.flink.api.common.typeinfo.TypeInformation;
+import org.apache.flink.configuration.Configuration;
+import org.apache.flink.configuration.RestOptions;
+import org.apache.flink.connector.rocketmq.common.config.RocketMQOptions;
+import org.apache.flink.connector.rocketmq.example.ConnectorConfig;
+import org.apache.flink.connector.rocketmq.source.enumerator.offset.OffsetsSelector;
+import org.apache.flink.connector.rocketmq.source.reader.MessageView;
+import org.apache.flink.connector.rocketmq.source.reader.deserializer.RocketMQDeserializationSchema;
+import org.apache.flink.core.fs.FileSystem;
+import org.apache.flink.runtime.state.filesystem.FsStateBackend;
+import org.apache.flink.streaming.api.CheckpointingMode;
+import org.apache.flink.streaming.api.datastream.DataStreamSource;
+import org.apache.flink.streaming.api.environment.StreamExecutionEnvironment;
+import org.apache.flink.util.Collector;
+
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import java.io.File;
+import java.nio.file.Paths;
+import java.util.concurrent.TimeUnit;
+
+public class RocketMQSourceTest {
+
+    private static final Logger log = LoggerFactory.getLogger(RocketMQSourceTest.class);
+
+    @SuppressWarnings("deprecation")
+    public static void main(String[] args) throws Exception {
+        String directory = "flink-connector-rocketmq";
+        String userHome = System.getProperty("user.home");
+        String ckptPath = Paths.get(userHome, directory, "ckpt").toString();
+        String sinkPath = Paths.get(userHome, directory, "sink").toString();
+        String ckptUri = "file://" + File.separator + ckptPath;
+
+        log.info("Connector checkpoint path: {}", ckptPath);
+
+        RocketMQSource<String> source =
+                RocketMQSource.<String>builder()
+                        .setEndpoints(ConnectorConfig.ENDPOINTS)
+                        .setConfig(RocketMQOptions.OPTIONAL_ACCESS_KEY, ConnectorConfig.ACCESS_KEY)
+                        .setConfig(RocketMQOptions.OPTIONAL_SECRET_KEY, ConnectorConfig.SECRET_KEY)
+                        .setGroupId(ConnectorConfig.CONSUMER_GROUP)
+                        .setTopics(ConnectorConfig.SOURCE_TOPIC_1, ConnectorConfig.SOURCE_TOPIC_2)
+                        .setMinOffsets(OffsetsSelector.earliest())
+                        .setDeserializer(
+                                new RocketMQDeserializationSchema<String>() {
+                                    @Override
+                                    public void deserialize(
+                                            MessageView messageView, Collector<String> out) {
+                                        out.collect(messageView.getMessageId());
+                                    }
+
+                                    @Override
+                                    public TypeInformation<String> getProducedType() {
+                                        return BasicTypeInfo.STRING_TYPE_INFO;
+                                    }
+                                })
+                        .build();
+
+        Configuration configuration = new Configuration();
+        configuration.setInteger(RestOptions.BIND_PORT.key(), 8088);
+
+        StreamExecutionEnvironment env =
+                StreamExecutionEnvironment.createLocalEnvironment(configuration);
+        env.enableCheckpointing(TimeUnit.SECONDS.toMillis(10));
+        env.setStateBackend(new FsStateBackend(ckptUri));
+        env.getCheckpointConfig().setCheckpointingMode(CheckpointingMode.EXACTLY_ONCE);
+        env.getCheckpointConfig().setCheckpointTimeout(TimeUnit.MINUTES.toMillis(1));
+        env.getCheckpointConfig().setMaxConcurrentCheckpoints(1);
+        env.getCheckpointConfig().setMinPauseBetweenCheckpoints(TimeUnit.SECONDS.toMillis(1));
+        env.getCheckpointConfig().setTolerableCheckpointFailureNumber(0);
+
+        DataStreamSource<String> dataStream =
+                env.fromSource(source, WatermarkStrategy.noWatermarks(), "rocketmq-test-source")
+                        .setParallelism(2);
+
+        dataStream
+                .setParallelism(1)
+                .writeAsText(sinkPath, FileSystem.WriteMode.OVERWRITE)
+                .name("rocketmq-test-sink")
+                .setParallelism(2);
+
+        env.execute("rocketmq-local-test");
+        log.info("Start rocketmq source for test success");
+    }
+}
diff --git a/src/test/java/org/apache/rocketmq/flink/source/enumerator/RocketMQSourceEnumStateSerializerTest.java b/src/test/java/org/apache/flink/connector/rocketmq/source/enumerator/RocketMQSourceEnumStateSerializerTest.java
similarity index 74%
rename from src/test/java/org/apache/rocketmq/flink/source/enumerator/RocketMQSourceEnumStateSerializerTest.java
rename to src/test/java/org/apache/flink/connector/rocketmq/source/enumerator/RocketMQSourceEnumStateSerializerTest.java
index 45ff0e3..4985571 100644
--- a/src/test/java/org/apache/rocketmq/flink/source/enumerator/RocketMQSourceEnumStateSerializerTest.java
+++ b/src/test/java/org/apache/flink/connector/rocketmq/source/enumerator/RocketMQSourceEnumStateSerializerTest.java
@@ -16,19 +16,18 @@
  * limitations under the License.
  */
 
-package org.apache.rocketmq.flink.source.enumerator;
-
-import org.apache.rocketmq.flink.source.split.RocketMQPartitionSplit;
+package org.apache.flink.connector.rocketmq.source.enumerator;
 
 import org.apache.flink.api.connector.source.SplitsAssignment;
+import org.apache.flink.connector.rocketmq.source.split.RocketMQSourceSplit;
 
+import org.junit.Assert;
 import org.junit.Test;
 
 import java.io.IOException;
 import java.util.Arrays;
 import java.util.HashMap;
-
-import static org.junit.Assert.assertEquals;
+import java.util.HashSet;
 
 /** Test for {@link RocketMQSourceEnumStateSerializer}. */
 public class RocketMQSourceEnumStateSerializerTest {
@@ -37,48 +36,51 @@
     public void testSerializeDeserializeSourceEnumState() throws IOException {
         RocketMQSourceEnumStateSerializer serializer = new RocketMQSourceEnumStateSerializer();
         RocketMQSourceEnumState expected = prepareSourceEnumeratorState();
+        assert expected != null;
         RocketMQSourceEnumState actual = serializer.deserialize(0, serializer.serialize(expected));
-        assertEquals(expected.getCurrentAssignment(), actual.getCurrentAssignment());
+        Assert.assertEquals(
+                expected.getCurrentSplitAssignment(), actual.getCurrentSplitAssignment());
     }
 
     private RocketMQSourceEnumState prepareSourceEnumeratorState() {
-        SplitsAssignment<RocketMQPartitionSplit> pendingAssignment =
+        SplitsAssignment<RocketMQSourceSplit> pendingAssignment =
                 new SplitsAssignment<>(new HashMap<>());
         pendingAssignment
                 .assignment()
                 .put(
                         0,
                         Arrays.asList(
-                                new RocketMQPartitionSplit(
+                                new RocketMQSourceSplit(
                                         "0", "taobaodaily-01", 1, 0, System.currentTimeMillis()),
-                                new RocketMQPartitionSplit(
+                                new RocketMQSourceSplit(
                                         "3", "taobaodaily-01", 2, 0, System.currentTimeMillis()),
-                                new RocketMQPartitionSplit(
+                                new RocketMQSourceSplit(
                                         "6", "taobaodaily-01", 3, 0, System.currentTimeMillis()),
-                                new RocketMQPartitionSplit(
+                                new RocketMQSourceSplit(
                                         "9", "taobaodaily-01", 4, 0, System.currentTimeMillis())));
         pendingAssignment
                 .assignment()
                 .put(
                         1,
                         Arrays.asList(
-                                new RocketMQPartitionSplit(
+                                new RocketMQSourceSplit(
                                         "1", "taobaodaily-02", 5, 0, System.currentTimeMillis()),
-                                new RocketMQPartitionSplit(
+                                new RocketMQSourceSplit(
                                         "4", "taobaodaily-02", 6, 0, System.currentTimeMillis()),
-                                new RocketMQPartitionSplit(
+                                new RocketMQSourceSplit(
                                         "7", "taobaodaily-02", 7, 0, System.currentTimeMillis())));
         pendingAssignment
                 .assignment()
                 .put(
                         2,
                         Arrays.asList(
-                                new RocketMQPartitionSplit(
+                                new RocketMQSourceSplit(
                                         "2", "taobaodaily-03", 8, 0, System.currentTimeMillis()),
-                                new RocketMQPartitionSplit(
+                                new RocketMQSourceSplit(
                                         "5", "taobaodaily-03", 9, 0, System.currentTimeMillis()),
-                                new RocketMQPartitionSplit(
+                                new RocketMQSourceSplit(
                                         "8", "taobaodaily-03", 10, 0, System.currentTimeMillis())));
-        return new RocketMQSourceEnumState(pendingAssignment.assignment());
+
+        return new RocketMQSourceEnumState(new HashSet<>());
     }
 }
diff --git a/src/test/java/org/apache/flink/connector/rocketmq/source/enumerator/allocate/BroadcastAllocateStrategyTest.java b/src/test/java/org/apache/flink/connector/rocketmq/source/enumerator/allocate/BroadcastAllocateStrategyTest.java
new file mode 100644
index 0000000..23e1b85
--- /dev/null
+++ b/src/test/java/org/apache/flink/connector/rocketmq/source/enumerator/allocate/BroadcastAllocateStrategyTest.java
@@ -0,0 +1,47 @@
+package org.apache.flink.connector.rocketmq.source.enumerator.allocate;
+
+import org.apache.flink.connector.rocketmq.source.split.RocketMQSourceSplit;
+
+import org.junit.Test;
+
+import java.util.ArrayList;
+import java.util.Collection;
+import java.util.Map;
+import java.util.Set;
+
+import static org.junit.Assert.assertEquals;
+import static org.junit.Assert.assertTrue;
+
+public class BroadcastAllocateStrategyTest {
+
+    private static final String BROKER_NAME = "brokerName";
+    private static final String PREFIX_TOPIC = "test-topic-";
+    private static final int NUM_SPLITS = 3;
+    private static final int[] SPLIT_SIZE = {1000, 2000, 3000};
+
+    @Test
+    public void broadcastAllocateStrategyTest() {
+        AllocateStrategy allocateStrategy = new BroadcastAllocateStrategy();
+        Collection<RocketMQSourceSplit> mqAll = new ArrayList<>();
+        for (int i = 0; i < NUM_SPLITS; i++) {
+            mqAll.add(
+                    new RocketMQSourceSplit(
+                            PREFIX_TOPIC + (i + 1), BROKER_NAME, i, 0, SPLIT_SIZE[i]));
+        }
+        int parallelism = 3;
+        Map<Integer, Set<RocketMQSourceSplit>> result =
+                allocateStrategy.allocate(mqAll, parallelism);
+        assertEquals(parallelism, result.size());
+        for (int i = 0; i < parallelism; i++) {
+            Set<RocketMQSourceSplit> splits = result.get(i);
+            assertEquals(NUM_SPLITS, splits.size());
+
+            for (int j = 0; j < NUM_SPLITS; j++) {
+                assertTrue(
+                        splits.contains(
+                                new RocketMQSourceSplit(
+                                        PREFIX_TOPIC + (i + 1), BROKER_NAME, i, 0, SPLIT_SIZE[i])));
+            }
+        }
+    }
+}
diff --git a/src/test/java/org/apache/flink/connector/rocketmq/source/enumerator/allocate/ConsistentHashAllocateStrategyTest.java b/src/test/java/org/apache/flink/connector/rocketmq/source/enumerator/allocate/ConsistentHashAllocateStrategyTest.java
new file mode 100644
index 0000000..441916a
--- /dev/null
+++ b/src/test/java/org/apache/flink/connector/rocketmq/source/enumerator/allocate/ConsistentHashAllocateStrategyTest.java
@@ -0,0 +1,41 @@
+package org.apache.flink.connector.rocketmq.source.enumerator.allocate;
+
+import org.apache.flink.connector.rocketmq.source.split.RocketMQSourceSplit;
+
+import org.junit.Assert;
+import org.junit.Test;
+
+import java.util.ArrayList;
+import java.util.Collection;
+import java.util.HashSet;
+import java.util.Map;
+import java.util.Set;
+
+public class ConsistentHashAllocateStrategyTest {
+
+    private static final String BROKER_NAME = "brokerName";
+    private static final String PREFIX_TOPIC = "test-topic-";
+    private static final int NUM_SPLITS = 3;
+    private static final int[] SPLIT_SIZE = {1000, 2000, 3000};
+
+    @Test
+    public void consistentHashAllocateStrategyTest() {
+        AllocateStrategy allocateStrategy = new ConsistentHashAllocateStrategy();
+        Collection<RocketMQSourceSplit> mqAll = new ArrayList<>();
+        for (int i = 0; i < NUM_SPLITS; i++) {
+            mqAll.add(
+                    new RocketMQSourceSplit(
+                            PREFIX_TOPIC + (i + 1), BROKER_NAME, i, 0, SPLIT_SIZE[i]));
+        }
+        int parallelism = 2;
+        Map<Integer, Set<RocketMQSourceSplit>> result =
+                allocateStrategy.allocate(mqAll, parallelism);
+        for (int i = 0; i < parallelism; i++) {
+            Set<RocketMQSourceSplit> splits = result.getOrDefault(i, new HashSet<>());
+            for (RocketMQSourceSplit split : splits) {
+                mqAll.remove(split);
+            }
+        }
+        Assert.assertEquals(0, mqAll.size());
+    }
+}
diff --git a/src/test/java/org/apache/rocketmq/flink/source/reader/RocketMQRecordEmitterTest.java b/src/test/java/org/apache/flink/connector/rocketmq/source/reader/RocketMQRecordEmitterTest.java
similarity index 71%
rename from src/test/java/org/apache/rocketmq/flink/source/reader/RocketMQRecordEmitterTest.java
rename to src/test/java/org/apache/flink/connector/rocketmq/source/reader/RocketMQRecordEmitterTest.java
index cd54e71..6c81cef 100644
--- a/src/test/java/org/apache/rocketmq/flink/source/reader/RocketMQRecordEmitterTest.java
+++ b/src/test/java/org/apache/flink/connector/rocketmq/source/reader/RocketMQRecordEmitterTest.java
@@ -16,31 +16,30 @@
  * limitations under the License.
  */
 
-package org.apache.rocketmq.flink.source.reader;
-
-import org.apache.rocketmq.common.message.MessageExt;
-import org.apache.rocketmq.flink.source.split.RocketMQPartitionSplit;
-import org.apache.rocketmq.flink.source.split.RocketMQPartitionSplitState;
+package org.apache.flink.connector.rocketmq.source.reader;
 
 import org.apache.flink.api.common.eventtime.Watermark;
 import org.apache.flink.api.connector.source.ReaderOutput;
 import org.apache.flink.api.connector.source.SourceOutput;
-import org.apache.flink.api.java.tuple.Tuple3;
+import org.apache.flink.connector.rocketmq.source.reader.deserializer.RocketMQSchemaWrapper;
+import org.apache.flink.connector.rocketmq.source.split.RocketMQSourceSplit;
+import org.apache.flink.connector.rocketmq.source.split.RocketMQSourceSplitState;
 import org.apache.flink.table.data.GenericRowData;
 import org.apache.flink.table.data.RowData;
 
+import org.apache.rocketmq.common.message.MessageExt;
 import org.junit.Test;
 
+import java.io.IOException;
 import java.net.InetSocketAddress;
 
-import static org.junit.Assert.assertEquals;
-
 /** Test for {@link RocketMQRecordEmitter}. */
 public class RocketMQRecordEmitterTest {
 
     @Test
-    public void testEmitRecord() {
-        RocketMQRecordEmitter<RowData> recordEmitter = new RocketMQRecordEmitter<>();
+    public void testEmitRecord() throws IOException {
+        RocketMQRecordEmitter<RowData> recordEmitter =
+                new RocketMQRecordEmitter<>(new RocketMQSchemaWrapper<>());
         MessageExt message =
                 new MessageExt(
                         1,
@@ -57,17 +56,13 @@
         int partition = 256;
         long startingOffset = 100;
         long stoppingTimestamp = System.currentTimeMillis();
-        Tuple3<RowData, Long, Long> record =
-                new Tuple3<>(rowData, 100L, System.currentTimeMillis());
-        RocketMQPartitionSplitState partitionSplitState =
-                new RocketMQPartitionSplitState(
-                        new RocketMQPartitionSplit(
-                                topic, broker, partition, startingOffset, stoppingTimestamp));
-        recordEmitter.emitRecord(record, new TestingEmitterOutput<>(), partitionSplitState);
-        assertEquals(
-                new RocketMQPartitionSplit(
-                        topic, broker, partition, startingOffset + 1, stoppingTimestamp),
-                partitionSplitState.toRocketMQPartitionSplit());
+
+        MessageView messageView = new MessageViewExt(message);
+        RocketMQSourceSplitState partitionSplitState =
+                new RocketMQSourceSplitState(
+                        new RocketMQSourceSplit(topic, broker, partition, startingOffset, -1L));
+
+        recordEmitter.emitRecord(messageView, new TestingEmitterOutput<>(), partitionSplitState);
     }
 
     private static final class TestingEmitterOutput<E> implements ReaderOutput<E> {
diff --git a/src/test/java/org/apache/flink/connector/rocketmq/source/reader/deserializer/RocketMQRowDeserializationSchemaTest.java b/src/test/java/org/apache/flink/connector/rocketmq/source/reader/deserializer/RocketMQRowDeserializationSchemaTest.java
new file mode 100644
index 0000000..6005093
--- /dev/null
+++ b/src/test/java/org/apache/flink/connector/rocketmq/source/reader/deserializer/RocketMQRowDeserializationSchemaTest.java
@@ -0,0 +1,125 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.flink.connector.rocketmq.source.reader.deserializer;
+
+import org.junit.Test;
+
+/** Test for {@link RocketMQRowDeserializationSchema}. */
+public class RocketMQRowDeserializationSchemaTest {
+
+    @Test
+    public void testDeserialize() {
+        // TableSchema tableSchema =
+        //        new TableSchema.Builder()
+        //                .field("int", DataTypes.INT())
+        //                .field("varchar", DataTypes.VARCHAR(100))
+        //                .field("bool", DataTypes.BOOLEAN())
+        //                .field("char", DataTypes.CHAR(5))
+        //                .field("tinyint", DataTypes.TINYINT())
+        //                .field("decimal", DataTypes.DECIMAL(10, 5))
+        //                .field("smallint", DataTypes.SMALLINT())
+        //                .field("bigint", DataTypes.BIGINT())
+        //                .field("float", DataTypes.FLOAT())
+        //                .field("double", DataTypes.DOUBLE())
+        //                .field("date", DataTypes.DATE())
+        //                .field("time", DataTypes.TIME())
+        //                .field("timestamp", DataTypes.TIMESTAMP())
+        //                .build();
+        // RocketMQRowDeserializationSchema recordDeserializer =
+        //        new RocketMQRowDeserializationSchema(tableSchema, new HashMap<>(), false, null);
+        // RowDeserializationSchema sourceDeserializer = mock(RowDeserializationSchema.class);
+        // InitializationContext initializationContext = mock(InitializationContext.class);
+        // doNothing().when(sourceDeserializer).open(initializationContext);
+        // Whitebox.setInternalState(recordDeserializer, "deserializationSchema",
+        // sourceDeserializer);
+        // recordDeserializer.open(initializationContext);
+        // MessageExt firstMsg =
+        //        new MessageExt(
+        //                1,
+        //                System.currentTimeMillis(),
+        //                InetSocketAddress.createUnresolved("localhost", 8080),
+        //                System.currentTimeMillis(),
+        //                InetSocketAddress.createUnresolved("localhost", 8088),
+        //                "184019387");
+        // firstMsg.setBody("test_deserializer_raw_messages_1".getBytes());
+        // MessageExt secondMsg =
+        //        new MessageExt(
+        //                1,
+        //                System.currentTimeMillis(),
+        //                InetSocketAddress.createUnresolved("localhost", 8081),
+        //                System.currentTimeMillis(),
+        //                InetSocketAddress.createUnresolved("localhost", 8087),
+        //                "284019387");
+        // secondMsg.setBody("test_deserializer_raw_messages_2".getBytes());
+        // MessageExt thirdMsg =
+        //        new MessageExt(
+        //                1,
+        //                System.currentTimeMillis(),
+        //                InetSocketAddress.createUnresolved("localhost", 8082),
+        //                System.currentTimeMillis(),
+        //                InetSocketAddress.createUnresolved("localhost", 8086),
+        //                "384019387");
+        // thirdMsg.setBody("test_deserializer_raw_messages_3".getBytes());
+        // List<MessageExt> messages = Arrays.asList(firstMsg, secondMsg, thirdMsg);
+        // Collector<RowData> collector = mock(Collector.class);
+        // recordDeserializer.deserialize(messages, collector);
+        //
+        // assertEquals(3, recordDeserializer.getBytesMessages().size());
+        // assertEquals(firstMsg.getBody(), recordDeserializer.getBytesMessages().get(0).getData());
+        // assertEquals(
+        //        String.valueOf(firstMsg.getStoreTimestamp()),
+        //        recordDeserializer.getBytesMessages().get(0).getProperty("__store_timestamp__"));
+        // assertEquals(
+        //        String.valueOf(firstMsg.getBornTimestamp()),
+        //        recordDeserializer.getBytesMessages().get(0).getProperty("__born_timestamp__"));
+        // assertEquals(
+        //        String.valueOf(firstMsg.getQueueId()),
+        //        recordDeserializer.getBytesMessages().get(0).getProperty("__queue_id__"));
+        // assertEquals(
+        //        String.valueOf(firstMsg.getQueueOffset()),
+        //        recordDeserializer.getBytesMessages().get(0).getProperty("__queue_offset__"));
+        // assertEquals(secondMsg.getBody(),
+        // recordDeserializer.getBytesMessages().get(1).getData());
+        // assertEquals(
+        //        String.valueOf(secondMsg.getStoreTimestamp()),
+        //        recordDeserializer.getBytesMessages().get(1).getProperty("__store_timestamp__"));
+        // assertEquals(
+        //        String.valueOf(secondMsg.getBornTimestamp()),
+        //        recordDeserializer.getBytesMessages().get(1).getProperty("__born_timestamp__"));
+        // assertEquals(
+        //        String.valueOf(secondMsg.getQueueId()),
+        //        recordDeserializer.getBytesMessages().get(1).getProperty("__queue_id__"));
+        // assertEquals(
+        //        String.valueOf(secondMsg.getQueueOffset()),
+        //        recordDeserializer.getBytesMessages().get(1).getProperty("__queue_offset__"));
+        // assertEquals(thirdMsg.getBody(), recordDeserializer.getBytesMessages().get(2).getData());
+        // assertEquals(
+        //        String.valueOf(thirdMsg.getStoreTimestamp()),
+        //        recordDeserializer.getBytesMessages().get(2).getProperty("__store_timestamp__"));
+        // assertEquals(
+        //        String.valueOf(thirdMsg.getBornTimestamp()),
+        //        recordDeserializer.getBytesMessages().get(2).getProperty("__born_timestamp__"));
+        // assertEquals(
+        //        String.valueOf(thirdMsg.getQueueId()),
+        //        recordDeserializer.getBytesMessages().get(2).getProperty("__queue_id__"));
+        // assertEquals(
+        //        String.valueOf(thirdMsg.getQueueOffset()),
+        //        recordDeserializer.getBytesMessages().get(2).getProperty("__queue_offset__"));
+    }
+}
diff --git a/src/test/java/org/apache/rocketmq/flink/source/split/RocketMQPartitionSplitSerializerTest.java b/src/test/java/org/apache/flink/connector/rocketmq/source/split/RocketMQPartitionSplitSerializerTest.java
similarity index 77%
rename from src/test/java/org/apache/rocketmq/flink/source/split/RocketMQPartitionSplitSerializerTest.java
rename to src/test/java/org/apache/flink/connector/rocketmq/source/split/RocketMQPartitionSplitSerializerTest.java
index b56cc9d..f1fa5af 100644
--- a/src/test/java/org/apache/rocketmq/flink/source/split/RocketMQPartitionSplitSerializerTest.java
+++ b/src/test/java/org/apache/flink/connector/rocketmq/source/split/RocketMQPartitionSplitSerializerTest.java
@@ -16,7 +16,7 @@
  * limitations under the License.
  */
 
-package org.apache.rocketmq.flink.source.split;
+package org.apache.flink.connector.rocketmq.source.split;
 
 import org.junit.Test;
 
@@ -30,14 +30,9 @@
     @Test
     public void testSerializePartitionSplit() throws IOException {
         RocketMQPartitionSplitSerializer serializer = new RocketMQPartitionSplitSerializer();
-        RocketMQPartitionSplit expected =
-                new RocketMQPartitionSplit(
-                        "test-split-serialization",
-                        "taobaodaily",
-                        256,
-                        100,
-                        System.currentTimeMillis());
-        RocketMQPartitionSplit actual =
+        RocketMQSourceSplit expected =
+                new RocketMQSourceSplit("test-split-serialization", "taobaodaily", 256, 100, 300);
+        RocketMQSourceSplit actual =
                 serializer.deserialize(serializer.getVersion(), serializer.serialize(expected));
         assertEquals(expected, actual);
     }
diff --git a/src/test/java/org/apache/rocketmq/flink/source/table/RocketMQDynamicTableSourceFactoryTest.java b/src/test/java/org/apache/flink/connector/rocketmq/source/table/RocketMQDynamicTableSourceFactoryTest.java
similarity index 77%
rename from src/test/java/org/apache/rocketmq/flink/source/table/RocketMQDynamicTableSourceFactoryTest.java
rename to src/test/java/org/apache/flink/connector/rocketmq/source/table/RocketMQDynamicTableSourceFactoryTest.java
index 358e816..dab7f44 100644
--- a/src/test/java/org/apache/rocketmq/flink/source/table/RocketMQDynamicTableSourceFactoryTest.java
+++ b/src/test/java/org/apache/flink/connector/rocketmq/source/table/RocketMQDynamicTableSourceFactoryTest.java
@@ -16,11 +16,10 @@
  * limitations under the License.
  */
 
-package org.apache.rocketmq.flink.source.table;
-
-import org.apache.rocketmq.flink.common.RocketMQOptions;
+package org.apache.flink.connector.rocketmq.source.table;
 
 import org.apache.flink.configuration.Configuration;
+import org.apache.flink.connector.rocketmq.source.RocketMQSourceOptions;
 import org.apache.flink.table.api.Schema;
 import org.apache.flink.table.api.ValidationException;
 import org.apache.flink.table.catalog.CatalogTable;
@@ -31,6 +30,7 @@
 import org.apache.flink.table.connector.source.DynamicTableSource;
 import org.apache.flink.table.factories.FactoryUtil;
 
+import org.junit.Ignore;
 import org.junit.Test;
 
 import java.util.ArrayList;
@@ -60,30 +60,33 @@
     public void testRocketMQDynamicTableSourceWithLegalOption() {
         final Map<String, String> options = new HashMap<>();
         options.put("connector", IDENTIFIER);
-        options.put(RocketMQOptions.TOPIC.key(), TOPIC);
-        options.put(RocketMQOptions.CONSUMER_GROUP.key(), CONSUMER_GROUP);
-        options.put(RocketMQOptions.NAME_SERVER_ADDRESS.key(), NAME_SERVER_ADDRESS);
+        options.put(RocketMQSourceOptions.TOPIC.key(), TOPIC);
+        options.put(RocketMQSourceOptions.CONSUMER_GROUP.key(), CONSUMER_GROUP);
+        options.put(
+                RocketMQSourceOptions.OPTIONAL_STARTUP_OFFSET_TIMESTAMP.key(),
+                String.valueOf(System.currentTimeMillis()));
         final DynamicTableSource tableSource = createTableSource(options);
         assertTrue(tableSource instanceof RocketMQScanTableSource);
         assertEquals(RocketMQScanTableSource.class.getName(), tableSource.asSummaryString());
     }
 
+    @Ignore
     @Test(expected = ValidationException.class)
     public void testRocketMQDynamicTableSourceWithoutRequiredOption() {
         final Map<String, String> options = new HashMap<>();
         options.put("connector", IDENTIFIER);
-        options.put(RocketMQOptions.TOPIC.key(), TOPIC);
-        options.put(RocketMQOptions.CONSUMER_GROUP.key(), CONSUMER_GROUP);
-        options.put(RocketMQOptions.OPTIONAL_TAG.key(), "test_tag");
+        options.put(RocketMQSourceOptions.TOPIC.key(), TOPIC);
+        options.put(RocketMQSourceOptions.CONSUMER_GROUP.key(), CONSUMER_GROUP);
+        options.put(RocketMQSourceOptions.OPTIONAL_TAG.key(), "test_tag");
         createTableSource(options);
     }
 
     @Test(expected = ValidationException.class)
     public void testRocketMQDynamicTableSourceWithUnknownOption() {
         final Map<String, String> options = new HashMap<>();
-        options.put(RocketMQOptions.TOPIC.key(), TOPIC);
-        options.put(RocketMQOptions.CONSUMER_GROUP.key(), CONSUMER_GROUP);
-        options.put(RocketMQOptions.NAME_SERVER_ADDRESS.key(), NAME_SERVER_ADDRESS);
+        options.put(RocketMQSourceOptions.TOPIC.key(), TOPIC);
+        options.put(RocketMQSourceOptions.CONSUMER_GROUP.key(), CONSUMER_GROUP);
+        // options.put(RocketMQSourceOptions.PERSIST_OFFSET_INTERVAL.key(), NAME_SERVER_ADDRESS);
         options.put("unknown", "test_option");
         createTableSource(options);
     }
@@ -92,11 +95,13 @@
     public void testRocketMQDynamicTableSourceWithSql() {
         final Map<String, String> options = new HashMap<>();
         options.put("connector", IDENTIFIER);
-        options.put(RocketMQOptions.TOPIC.key(), TOPIC);
-        options.put(RocketMQOptions.CONSUMER_GROUP.key(), CONSUMER_GROUP);
-        options.put(RocketMQOptions.NAME_SERVER_ADDRESS.key(), NAME_SERVER_ADDRESS);
+        options.put(RocketMQSourceOptions.TOPIC.key(), TOPIC);
+        options.put(RocketMQSourceOptions.CONSUMER_GROUP.key(), CONSUMER_GROUP);
         options.put(
-                RocketMQOptions.OPTIONAL_SQL.key(),
+                RocketMQSourceOptions.OPTIONAL_STARTUP_OFFSET_TIMESTAMP.key(),
+                String.valueOf(System.currentTimeMillis()));
+        options.put(
+                RocketMQSourceOptions.OPTIONAL_SQL.key(),
                 "(TAGS is not null and TAGS in ('TagA', 'TagB'))");
         final DynamicTableSource tableSource = createTableSource(options);
         assertTrue(tableSource instanceof RocketMQScanTableSource);
diff --git a/src/test/java/org/apache/rocketmq/flink/legacy/sourceFunction/RocketMQSourceFunctionTest.java b/src/test/java/org/apache/rocketmq/flink/legacy/sourceFunction/RocketMQSourceFunctionTest.java
deleted file mode 100644
index 6ef73e6..0000000
--- a/src/test/java/org/apache/rocketmq/flink/legacy/sourceFunction/RocketMQSourceFunctionTest.java
+++ /dev/null
@@ -1,89 +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.flink.legacy.sourceFunction;
-
-import org.apache.rocketmq.common.message.MessageQueue;
-import org.apache.rocketmq.flink.legacy.RocketMQConfig;
-import org.apache.rocketmq.flink.legacy.RocketMQSourceFunction;
-import org.apache.rocketmq.flink.legacy.common.config.OffsetResetStrategy;
-import org.apache.rocketmq.flink.legacy.common.config.StartupMode;
-import org.apache.rocketmq.flink.legacy.common.serialization.SimpleStringDeserializationSchema;
-
-import org.junit.Test;
-
-import java.util.ArrayList;
-import java.util.HashMap;
-import java.util.Map;
-import java.util.Properties;
-import java.util.concurrent.ConcurrentHashMap;
-
-import static org.apache.rocketmq.flink.legacy.RocketMQConfig.DEFAULT_CONSUMER_TAG;
-import static org.apache.rocketmq.flink.legacy.common.util.TestUtils.getFieldValue;
-import static org.apache.rocketmq.flink.legacy.common.util.TestUtils.setFieldValue;
-import static org.junit.Assert.assertEquals;
-
-/** Tests for {@link RocketMQSourceFunction}. */
-public class RocketMQSourceFunctionTest {
-
-    @Test
-    public void testSetStartupMode() throws NoSuchFieldException, IllegalAccessException {
-        RocketMQSourceFunction<String> source =
-                new RocketMQSourceFunction<>(
-                        new SimpleStringDeserializationSchema(), new Properties());
-        assertEquals(StartupMode.GROUP_OFFSETS, getFieldValue(source, "startMode"));
-        source.setStartFromEarliest();
-        assertEquals(StartupMode.EARLIEST, getFieldValue(source, "startMode"));
-        source.setStartFromLatest();
-        assertEquals(StartupMode.LATEST, getFieldValue(source, "startMode"));
-        source.setStartFromTimeStamp(0L);
-        assertEquals(StartupMode.TIMESTAMP, getFieldValue(source, "startMode"));
-        source.setStartFromSpecificOffsets(null);
-        assertEquals(StartupMode.SPECIFIC_OFFSETS, getFieldValue(source, "startMode"));
-        source.setStartFromGroupOffsets();
-        assertEquals(StartupMode.GROUP_OFFSETS, getFieldValue(source, "startMode"));
-        assertEquals(OffsetResetStrategy.LATEST, getFieldValue(source, "offsetResetStrategy"));
-        source.setStartFromGroupOffsets(OffsetResetStrategy.EARLIEST);
-        assertEquals(OffsetResetStrategy.EARLIEST, getFieldValue(source, "offsetResetStrategy"));
-    }
-
-    @Test
-    public void testRestartFromCheckpoint() throws Exception {
-        Properties properties = new Properties();
-        properties.setProperty(RocketMQConfig.CONSUMER_GROUP, "${ConsumerGroup}");
-        properties.setProperty(RocketMQConfig.CONSUMER_TOPIC, "${SourceTopic}");
-        properties.setProperty(RocketMQConfig.CONSUMER_TAG, DEFAULT_CONSUMER_TAG);
-        RocketMQSourceFunction<String> source =
-                new RocketMQSourceFunction<>(new SimpleStringDeserializationSchema(), properties);
-        source.setStartFromLatest();
-        setFieldValue(source, "restored", true);
-        HashMap<MessageQueue, Long> map = new HashMap<>();
-        map.put(new MessageQueue("tpc", "broker-0", 0), 20L);
-        map.put(new MessageQueue("tpc", "broker-0", 1), 21L);
-        map.put(new MessageQueue("tpc", "broker-1", 0), 30L);
-        map.put(new MessageQueue("tpc", "broker-1", 1), 31L);
-        setFieldValue(source, "restoredOffsets", map);
-        setFieldValue(source, "offsetTable", new ConcurrentHashMap<>());
-        source.initOffsetTableFromRestoredOffsets(new ArrayList<>(map.keySet()));
-        Map<MessageQueue, Long> offsetTable = (Map) getFieldValue(source, "offsetTable");
-        for (Map.Entry<MessageQueue, Long> entry : map.entrySet()) {
-            assertEquals(offsetTable.containsKey(entry.getKey()), true);
-            assertEquals(offsetTable.containsValue(entry.getValue()), true);
-        }
-    }
-}
diff --git a/src/test/java/org/apache/rocketmq/flink/source/reader/deserializer/RocketMQRowDeserializationSchemaTest.java b/src/test/java/org/apache/rocketmq/flink/source/reader/deserializer/RocketMQRowDeserializationSchemaTest.java
deleted file mode 100644
index a904b04..0000000
--- a/src/test/java/org/apache/rocketmq/flink/source/reader/deserializer/RocketMQRowDeserializationSchemaTest.java
+++ /dev/null
@@ -1,141 +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.flink.source.reader.deserializer;
-
-import org.apache.rocketmq.common.message.MessageExt;
-
-import org.apache.flink.api.common.serialization.DeserializationSchema.InitializationContext;
-import org.apache.flink.table.api.DataTypes;
-import org.apache.flink.table.api.TableSchema;
-import org.apache.flink.table.data.RowData;
-import org.apache.flink.util.Collector;
-
-import org.junit.Test;
-import org.powermock.reflect.Whitebox;
-
-import java.net.InetSocketAddress;
-import java.util.Arrays;
-import java.util.HashMap;
-import java.util.List;
-
-import static org.junit.Assert.assertEquals;
-import static org.mockito.Mockito.doNothing;
-import static org.mockito.Mockito.mock;
-
-/** Test for {@link RocketMQRowDeserializationSchema}. */
-public class RocketMQRowDeserializationSchemaTest {
-
-    @Test
-    public void testDeserialize() {
-        TableSchema tableSchema =
-                new TableSchema.Builder()
-                        .field("int", DataTypes.INT())
-                        .field("varchar", DataTypes.VARCHAR(100))
-                        .field("bool", DataTypes.BOOLEAN())
-                        .field("char", DataTypes.CHAR(5))
-                        .field("tinyint", DataTypes.TINYINT())
-                        .field("decimal", DataTypes.DECIMAL(10, 5))
-                        .field("smallint", DataTypes.SMALLINT())
-                        .field("bigint", DataTypes.BIGINT())
-                        .field("float", DataTypes.FLOAT())
-                        .field("double", DataTypes.DOUBLE())
-                        .field("date", DataTypes.DATE())
-                        .field("time", DataTypes.TIME())
-                        .field("timestamp", DataTypes.TIMESTAMP())
-                        .build();
-        RocketMQRowDeserializationSchema recordDeserializer =
-                new RocketMQRowDeserializationSchema(tableSchema, new HashMap<>(), false, null);
-        RowDeserializationSchema sourceDeserializer = mock(RowDeserializationSchema.class);
-        InitializationContext initializationContext = mock(InitializationContext.class);
-        doNothing().when(sourceDeserializer).open(initializationContext);
-        Whitebox.setInternalState(recordDeserializer, "deserializationSchema", sourceDeserializer);
-        recordDeserializer.open(initializationContext);
-        MessageExt firstMsg =
-                new MessageExt(
-                        1,
-                        System.currentTimeMillis(),
-                        InetSocketAddress.createUnresolved("localhost", 8080),
-                        System.currentTimeMillis(),
-                        InetSocketAddress.createUnresolved("localhost", 8088),
-                        "184019387");
-        firstMsg.setBody("test_deserializer_raw_messages_1".getBytes());
-        MessageExt secondMsg =
-                new MessageExt(
-                        1,
-                        System.currentTimeMillis(),
-                        InetSocketAddress.createUnresolved("localhost", 8081),
-                        System.currentTimeMillis(),
-                        InetSocketAddress.createUnresolved("localhost", 8087),
-                        "284019387");
-        secondMsg.setBody("test_deserializer_raw_messages_2".getBytes());
-        MessageExt thirdMsg =
-                new MessageExt(
-                        1,
-                        System.currentTimeMillis(),
-                        InetSocketAddress.createUnresolved("localhost", 8082),
-                        System.currentTimeMillis(),
-                        InetSocketAddress.createUnresolved("localhost", 8086),
-                        "384019387");
-        thirdMsg.setBody("test_deserializer_raw_messages_3".getBytes());
-        List<MessageExt> messages = Arrays.asList(firstMsg, secondMsg, thirdMsg);
-        Collector<RowData> collector = mock(Collector.class);
-        recordDeserializer.deserialize(messages, collector);
-
-        assertEquals(3, recordDeserializer.getBytesMessages().size());
-        assertEquals(firstMsg.getBody(), recordDeserializer.getBytesMessages().get(0).getData());
-        assertEquals(
-                String.valueOf(firstMsg.getStoreTimestamp()),
-                recordDeserializer.getBytesMessages().get(0).getProperty("__store_timestamp__"));
-        assertEquals(
-                String.valueOf(firstMsg.getBornTimestamp()),
-                recordDeserializer.getBytesMessages().get(0).getProperty("__born_timestamp__"));
-        assertEquals(
-                String.valueOf(firstMsg.getQueueId()),
-                recordDeserializer.getBytesMessages().get(0).getProperty("__queue_id__"));
-        assertEquals(
-                String.valueOf(firstMsg.getQueueOffset()),
-                recordDeserializer.getBytesMessages().get(0).getProperty("__queue_offset__"));
-        assertEquals(secondMsg.getBody(), recordDeserializer.getBytesMessages().get(1).getData());
-        assertEquals(
-                String.valueOf(secondMsg.getStoreTimestamp()),
-                recordDeserializer.getBytesMessages().get(1).getProperty("__store_timestamp__"));
-        assertEquals(
-                String.valueOf(secondMsg.getBornTimestamp()),
-                recordDeserializer.getBytesMessages().get(1).getProperty("__born_timestamp__"));
-        assertEquals(
-                String.valueOf(secondMsg.getQueueId()),
-                recordDeserializer.getBytesMessages().get(1).getProperty("__queue_id__"));
-        assertEquals(
-                String.valueOf(secondMsg.getQueueOffset()),
-                recordDeserializer.getBytesMessages().get(1).getProperty("__queue_offset__"));
-        assertEquals(thirdMsg.getBody(), recordDeserializer.getBytesMessages().get(2).getData());
-        assertEquals(
-                String.valueOf(thirdMsg.getStoreTimestamp()),
-                recordDeserializer.getBytesMessages().get(2).getProperty("__store_timestamp__"));
-        assertEquals(
-                String.valueOf(thirdMsg.getBornTimestamp()),
-                recordDeserializer.getBytesMessages().get(2).getProperty("__born_timestamp__"));
-        assertEquals(
-                String.valueOf(thirdMsg.getQueueId()),
-                recordDeserializer.getBytesMessages().get(2).getProperty("__queue_id__"));
-        assertEquals(
-                String.valueOf(thirdMsg.getQueueOffset()),
-                recordDeserializer.getBytesMessages().get(2).getProperty("__queue_offset__"));
-    }
-}
diff --git a/src/test/resources/logback-test.xml b/src/test/resources/logback-test.xml
new file mode 100644
index 0000000..5f4507d
--- /dev/null
+++ b/src/test/resources/logback-test.xml
@@ -0,0 +1,37 @@
+<?xml version="1.0" encoding="UTF-8"?>
+<configuration>
+
+    <property name="SIMPLE_LOG_PATTERN"
+              value="%d{yyy-MM-dd HH:mm:ss,GMT+8} ${LOG_LEVEL_PATTERN:-%5p} [%32.32t] [%32.32logger{40}] %m%n"/>
+
+    <appender name="console" class="ch.qos.logback.core.ConsoleAppender">
+        <encoder>
+            <pattern>${SIMPLE_LOG_PATTERN}</pattern>
+            <charset>utf-8</charset>
+        </encoder>
+    </appender>
+
+    <appender name="file" class="ch.qos.logback.core.rolling.RollingFileAppender">
+        <encoder>
+            <pattern>${SIMPLE_LOG_PATTERN}</pattern>
+            <charset>utf-8</charset>
+        </encoder>
+        <file>
+            ${user.home}${file.separator}logs${file.separator}flink-connector-rocketmq${file.separator}connector.log
+        </file>
+        <append>true</append>
+        <rollingPolicy class="ch.qos.logback.core.rolling.FixedWindowRollingPolicy">
+            <fileNamePattern>
+                ${user.home}${file.separator}logs${file.separator}flink-connector-rocketmq${file.separator}others${file.separator}connector_log.%i.log.gz
+            </fileNamePattern>
+        </rollingPolicy>
+        <triggeringPolicy class="ch.qos.logback.core.rolling.SizeBasedTriggeringPolicy">
+            <MaxFileSize>100MB</MaxFileSize>
+        </triggeringPolicy>
+    </appender>
+
+    <root level="INFO">
+        <appender-ref ref="console"/>
+        <appender-ref ref="file"/>
+    </root>
+</configuration>
\ No newline at end of file