You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@rocketmq.apache.org by du...@apache.org on 2021/08/24 11:18:13 UTC

[rocketmq-flink] branch main updated (ba029e7 -> aafb8e4)

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

duhengforever pushed a change to branch main
in repository https://gitbox.apache.org/repos/asf/rocketmq-flink.git.


 discard ba029e7  first commit
     new c05c018  Create directory for beam,flink,spark,storm,mysql,redis,mongodb
     new 12f0d0c  [ROCKETMQ-82] RocketMQ-Flink Integration (#45)
     new 11e2e6d  ISSUE-61: Added javadocs for RocketMQ-Flink (#63)
     new 624994d  Fix NPE  when getting offset(#153)
     new 1c36e71  Remove python and go repositories
     new 452b57f  Add batch size param for flink sink (#198)
     new b5c05e0  flink-rocketmq-sink , producer send message set delay level (optional… (#237)
     new 58d07fb  Fix getting wrong offset bug when the source restart (#190)
     new c9564d6  fix(module): fix load wrong offset from savepoint (#288)
     new 42a5d6c  Verify flink prs
     new fc79313  create client instance for each sink/source
     new 0be1ed7  code format
     new 65e3029  update consumer offset after checkpoint completed
     new f74cc27  Optimizing update offset code logic
     new 8089fb1  clean up some dirty code
     new 8531921  fix concurrent checkpoint bug
     new c6cb79d  Merge pull request #229 from Jennifer-sarah/master
     new 0698d86  Producer failed to shutdown when exception happened (#388)
     new 7059330  [ISSUE #385]Add acl feature support for rocketmq-flink (#384)
     new a04b0bf  No need to call this function twice (#329)
     new 92d4cc3  Fix Checkstyle issue (#532) (#533)
     new a78d575  feat(rocketmq-flink)upgrade rocketmq client to 4.7.1 (#589)
     new 635a72d  [ISSUE #337] throw exception when send message to broker fail (#339)
     new fe936a5  [ISSUE #656] Update flink connector rocketmq, support flink metrics (#657)
     new da92a64  [rocketmq-connector-flink] rebalance cause offset rollback to long time ago (#672)
     new f882acf  [#705] Support the implementation of new Source interface (#706)
     new 340ea48  [#715] Support the RocketMQ TableSource based on the new Source interface (#716)
     new 465c60a  Build pass jdk9+
     new 4edc4da  [#715] Support the RocketMQ TableSource based on the legacy Source implementation (#779)
     new bbd7c6d  [#780] Support the RocketMQ TableSink based on the legacy Sink implementation (#781)
     new bf27790  [#786] RocketMQSourceFunction supports the close of ExecutorService and ScheduledExecutorService (#790)
     new aae7134  RocketMQSource improves the message consume of RocketMQPartitionSplitReader (#791)
     new aafb8e4  introduce RocketMQ SQL connector document (#793)

This update added new revisions after undoing existing revisions.
That is to say, some revisions that were in the old version of the
branch are not in the new version.  This situation occurs
when a user --force pushes a change and generates a repository
containing something like this:

 * -- * -- B -- O -- O -- O   (ba029e7)
            \
             N -- N -- N   refs/heads/main (aafb8e4)

You should already have received notification emails for all of the O
revisions, and so the following emails describe only the N revisions
from the common base, B.

Any revisions marked "omit" are not gone; other references still
refer to them.  Any revisions marked "discard" are gone forever.

The 33 revisions listed above as "new" are entirely new to this
repository and will be described in separate emails.  The revisions
listed as "add" were already present in the repository and have only
been added to this reference.


Summary of changes:
 LICENSE                                            | 201 +++++++
 NOTICE                                             |   5 +
 README.md                                          | 204 ++++++-
 pom.xml                                            | 289 ++++++++++
 .../rocketmq/flink/common/RocketMQOptions.java     |  98 ++++
 .../rocketmq/flink/legacy/RocketMQConfig.java      | 183 +++++++
 .../apache/rocketmq/flink/legacy/RocketMQSink.java | 209 +++++++
 .../flink/legacy/RocketMQSourceFunction.java       | 544 ++++++++++++++++++
 .../rocketmq/flink/legacy/RunningChecker.java      |  31 ++
 .../common/selector/DefaultTopicSelector.java      |  41 ++
 .../common/selector/SimpleTopicSelector.java       |  77 +++
 .../legacy/common/selector/TopicSelector.java      |  26 +
 .../ForwardMessageExtDeserialization.java          |  37 ++
 .../KeyValueDeserializationSchema.java             |  25 +
 .../serialization/KeyValueSerializationSchema.java |  26 +
 .../MessageExtDeserializationScheme.java           |  39 ++
 .../RowKeyValueDeserializationSchema.java          | 407 ++++++++++++++
 .../SimpleKeyValueDeserializationSchema.java       |  65 +++
 .../SimpleKeyValueSerializationSchema.java         |  61 +++
 .../SimpleTupleDeserializationSchema.java          |  23 +
 .../flink/legacy/common/util/MetricUtils.java      |  87 +++
 .../flink/legacy/common/util/RetryUtil.java        |  58 ++
 .../flink/legacy/common/util/RocketMQUtils.java    |  81 +++
 .../flink/legacy/common/util/TestUtils.java        |  31 ++
 .../watermark/BoundedOutOfOrdernessGenerator.java  |  59 ++
 .../BoundedOutOfOrdernessGeneratorPerQueue.java    |  70 +++
 .../common/watermark/PunctuatedAssigner.java       |  48 ++
 .../watermark/TimeLagWatermarkGenerator.java       |  53 ++
 .../legacy/common/watermark/WaterMarkForAll.java   |  41 ++
 .../legacy/common/watermark/WaterMarkPerQueue.java |  64 +++
 .../flink/legacy/example/RocketMQFlinkExample.java | 132 +++++
 .../flink/legacy/example/SimpleConsumer.java       |  88 +++
 .../flink/legacy/example/SimpleProducer.java       |  81 +++
 .../flink/legacy/function/SinkMapFunction.java     |  50 ++
 .../flink/legacy/function/SourceMapFunction.java   |  33 ++
 .../flink/sink/table/RocketMQDynamicTableSink.java | 257 +++++++++
 .../table/RocketMQDynamicTableSinkFactory.java     | 174 ++++++
 .../flink/sink/table/RocketMQRowDataConverter.java | 217 ++++++++
 .../flink/sink/table/RocketMQRowDataSink.java      |  61 +++
 .../rocketmq/flink/source/RocketMQSource.java      | 181 ++++++
 .../source/enumerator/RocketMQSourceEnumState.java |  38 ++
 .../RocketMQSourceEnumStateSerializer.java         |  64 +++
 .../enumerator/RocketMQSourceEnumerator.java       | 351 ++++++++++++
 .../reader/RocketMQPartitionSplitReader.java       | 386 +++++++++++++
 .../flink/source/reader/RocketMQRecordEmitter.java |  39 ++
 .../flink/source/reader/RocketMQSourceReader.java  |  64 +++
 .../source/reader/deserializer/BytesMessage.java   |  53 ++
 .../reader/deserializer/DeserializationSchema.java |  39 ++
 .../reader/deserializer/DirtyDataStrategy.java     |  28 +
 .../RocketMQDeserializationSchema.java             |  41 ++
 .../RocketMQRowDeserializationSchema.java          | 104 ++++
 .../deserializer/RowDeserializationSchema.java     | 606 +++++++++++++++++++++
 .../flink/source/split/RocketMQPartitionSplit.java | 101 ++++
 .../split/RocketMQPartitionSplitSerializer.java    |  66 +++
 .../source/split/RocketMQPartitionSplitState.java  |  57 ++
 .../table/RocketMQDynamicTableSourceFactory.java   | 215 ++++++++
 .../source/table/RocketMQScanTableSource.java      | 228 ++++++++
 .../rocketmq/flink/source/util/ByteSerializer.java | 156 ++++++
 .../rocketmq/flink/source/util/ByteUtils.java      | 219 ++++++++
 .../flink/source/util/StringSerializer.java        | 154 ++++++
 .../org.apache.flink.table.factories.Factory       |  17 +
 .../rocketmq/flink/legacy/RocketMQSinkTest.java    |  74 +++
 .../rocketmq/flink/legacy/RocketMQSourceTest.java  | 125 +++++
 .../common/selector/DefaultTopicSelectorTest.java  |  36 ++
 .../common/selector/SimpleTopicSelectorTest.java   |  48 ++
 .../RowKeyValueDeserializationSchemaTest.java      |  50 ++
 .../SimpleKeyValueSerializationSchemaTest.java     |  46 ++
 .../table/RocketMQDynamicTableSinkFactoryTest.java | 107 ++++
 .../RocketMQSourceEnumStateSerializerTest.java     |  84 +++
 .../source/reader/RocketMQRecordEmitterTest.java   |  97 ++++
 .../RocketMQRowDeserializationSchemaTest.java      | 141 +++++
 .../RocketMQPartitionSplitSerializerTest.java      |  44 ++
 .../RocketMQDynamicTableSourceFactoryTest.java     | 111 ++++
 style/copyright/Apache.xml                         |  24 +
 style/copyright/profiles_settings.xml              |  64 +++
 style/rmq_checkstyle.xml                           | 142 +++++
 style/rmq_codeStyle.xml                            | 157 ++++++
 77 files changed, 8832 insertions(+), 1 deletion(-)
 create mode 100644 LICENSE
 create mode 100644 NOTICE
 create mode 100644 pom.xml
 create mode 100644 src/main/java/org/apache/rocketmq/flink/common/RocketMQOptions.java
 create mode 100644 src/main/java/org/apache/rocketmq/flink/legacy/RocketMQConfig.java
 create mode 100644 src/main/java/org/apache/rocketmq/flink/legacy/RocketMQSink.java
 create mode 100644 src/main/java/org/apache/rocketmq/flink/legacy/RocketMQSourceFunction.java
 create mode 100644 src/main/java/org/apache/rocketmq/flink/legacy/RunningChecker.java
 create mode 100644 src/main/java/org/apache/rocketmq/flink/legacy/common/selector/DefaultTopicSelector.java
 create mode 100644 src/main/java/org/apache/rocketmq/flink/legacy/common/selector/SimpleTopicSelector.java
 create mode 100644 src/main/java/org/apache/rocketmq/flink/legacy/common/selector/TopicSelector.java
 create mode 100644 src/main/java/org/apache/rocketmq/flink/legacy/common/serialization/ForwardMessageExtDeserialization.java
 create mode 100644 src/main/java/org/apache/rocketmq/flink/legacy/common/serialization/KeyValueDeserializationSchema.java
 create mode 100644 src/main/java/org/apache/rocketmq/flink/legacy/common/serialization/KeyValueSerializationSchema.java
 create mode 100644 src/main/java/org/apache/rocketmq/flink/legacy/common/serialization/MessageExtDeserializationScheme.java
 create mode 100644 src/main/java/org/apache/rocketmq/flink/legacy/common/serialization/RowKeyValueDeserializationSchema.java
 create mode 100644 src/main/java/org/apache/rocketmq/flink/legacy/common/serialization/SimpleKeyValueDeserializationSchema.java
 create mode 100644 src/main/java/org/apache/rocketmq/flink/legacy/common/serialization/SimpleKeyValueSerializationSchema.java
 create mode 100644 src/main/java/org/apache/rocketmq/flink/legacy/common/serialization/SimpleTupleDeserializationSchema.java
 create mode 100644 src/main/java/org/apache/rocketmq/flink/legacy/common/util/MetricUtils.java
 create mode 100644 src/main/java/org/apache/rocketmq/flink/legacy/common/util/RetryUtil.java
 create mode 100644 src/main/java/org/apache/rocketmq/flink/legacy/common/util/RocketMQUtils.java
 create mode 100644 src/main/java/org/apache/rocketmq/flink/legacy/common/util/TestUtils.java
 create mode 100644 src/main/java/org/apache/rocketmq/flink/legacy/common/watermark/BoundedOutOfOrdernessGenerator.java
 create mode 100644 src/main/java/org/apache/rocketmq/flink/legacy/common/watermark/BoundedOutOfOrdernessGeneratorPerQueue.java
 create mode 100644 src/main/java/org/apache/rocketmq/flink/legacy/common/watermark/PunctuatedAssigner.java
 create mode 100644 src/main/java/org/apache/rocketmq/flink/legacy/common/watermark/TimeLagWatermarkGenerator.java
 create mode 100644 src/main/java/org/apache/rocketmq/flink/legacy/common/watermark/WaterMarkForAll.java
 create mode 100644 src/main/java/org/apache/rocketmq/flink/legacy/common/watermark/WaterMarkPerQueue.java
 create mode 100644 src/main/java/org/apache/rocketmq/flink/legacy/example/RocketMQFlinkExample.java
 create mode 100644 src/main/java/org/apache/rocketmq/flink/legacy/example/SimpleConsumer.java
 create mode 100644 src/main/java/org/apache/rocketmq/flink/legacy/example/SimpleProducer.java
 create mode 100644 src/main/java/org/apache/rocketmq/flink/legacy/function/SinkMapFunction.java
 create mode 100644 src/main/java/org/apache/rocketmq/flink/legacy/function/SourceMapFunction.java
 create mode 100644 src/main/java/org/apache/rocketmq/flink/sink/table/RocketMQDynamicTableSink.java
 create mode 100644 src/main/java/org/apache/rocketmq/flink/sink/table/RocketMQDynamicTableSinkFactory.java
 create mode 100644 src/main/java/org/apache/rocketmq/flink/sink/table/RocketMQRowDataConverter.java
 create mode 100644 src/main/java/org/apache/rocketmq/flink/sink/table/RocketMQRowDataSink.java
 create mode 100644 src/main/java/org/apache/rocketmq/flink/source/RocketMQSource.java
 create mode 100644 src/main/java/org/apache/rocketmq/flink/source/enumerator/RocketMQSourceEnumState.java
 create mode 100644 src/main/java/org/apache/rocketmq/flink/source/enumerator/RocketMQSourceEnumStateSerializer.java
 create mode 100644 src/main/java/org/apache/rocketmq/flink/source/enumerator/RocketMQSourceEnumerator.java
 create mode 100644 src/main/java/org/apache/rocketmq/flink/source/reader/RocketMQPartitionSplitReader.java
 create mode 100644 src/main/java/org/apache/rocketmq/flink/source/reader/RocketMQRecordEmitter.java
 create mode 100644 src/main/java/org/apache/rocketmq/flink/source/reader/RocketMQSourceReader.java
 create mode 100644 src/main/java/org/apache/rocketmq/flink/source/reader/deserializer/BytesMessage.java
 create mode 100644 src/main/java/org/apache/rocketmq/flink/source/reader/deserializer/DeserializationSchema.java
 create mode 100644 src/main/java/org/apache/rocketmq/flink/source/reader/deserializer/DirtyDataStrategy.java
 create mode 100644 src/main/java/org/apache/rocketmq/flink/source/reader/deserializer/RocketMQDeserializationSchema.java
 create mode 100644 src/main/java/org/apache/rocketmq/flink/source/reader/deserializer/RocketMQRowDeserializationSchema.java
 create mode 100644 src/main/java/org/apache/rocketmq/flink/source/reader/deserializer/RowDeserializationSchema.java
 create mode 100644 src/main/java/org/apache/rocketmq/flink/source/split/RocketMQPartitionSplit.java
 create mode 100644 src/main/java/org/apache/rocketmq/flink/source/split/RocketMQPartitionSplitSerializer.java
 create mode 100644 src/main/java/org/apache/rocketmq/flink/source/split/RocketMQPartitionSplitState.java
 create mode 100644 src/main/java/org/apache/rocketmq/flink/source/table/RocketMQDynamicTableSourceFactory.java
 create mode 100644 src/main/java/org/apache/rocketmq/flink/source/table/RocketMQScanTableSource.java
 create mode 100644 src/main/java/org/apache/rocketmq/flink/source/util/ByteSerializer.java
 create mode 100644 src/main/java/org/apache/rocketmq/flink/source/util/ByteUtils.java
 create mode 100644 src/main/java/org/apache/rocketmq/flink/source/util/StringSerializer.java
 create mode 100644 src/main/resources/META-INF/services/org.apache.flink.table.factories.Factory
 create mode 100644 src/test/java/org/apache/rocketmq/flink/legacy/RocketMQSinkTest.java
 create mode 100644 src/test/java/org/apache/rocketmq/flink/legacy/RocketMQSourceTest.java
 create mode 100644 src/test/java/org/apache/rocketmq/flink/legacy/common/selector/DefaultTopicSelectorTest.java
 create mode 100644 src/test/java/org/apache/rocketmq/flink/legacy/common/selector/SimpleTopicSelectorTest.java
 create mode 100644 src/test/java/org/apache/rocketmq/flink/legacy/common/serialization/RowKeyValueDeserializationSchemaTest.java
 create mode 100644 src/test/java/org/apache/rocketmq/flink/legacy/common/serialization/SimpleKeyValueSerializationSchemaTest.java
 create mode 100644 src/test/java/org/apache/rocketmq/flink/sink/table/RocketMQDynamicTableSinkFactoryTest.java
 create mode 100644 src/test/java/org/apache/rocketmq/flink/source/enumerator/RocketMQSourceEnumStateSerializerTest.java
 create mode 100644 src/test/java/org/apache/rocketmq/flink/source/reader/RocketMQRecordEmitterTest.java
 create mode 100644 src/test/java/org/apache/rocketmq/flink/source/reader/deserializer/RocketMQRowDeserializationSchemaTest.java
 create mode 100644 src/test/java/org/apache/rocketmq/flink/source/split/RocketMQPartitionSplitSerializerTest.java
 create mode 100644 src/test/java/org/apache/rocketmq/flink/source/table/RocketMQDynamicTableSourceFactoryTest.java
 create mode 100644 style/copyright/Apache.xml
 create mode 100644 style/copyright/profiles_settings.xml
 create mode 100644 style/rmq_checkstyle.xml
 create mode 100644 style/rmq_codeStyle.xml

[rocketmq-flink] 05/33: Remove python and go repositories

Posted by du...@apache.org.
This is an automated email from the ASF dual-hosted git repository.

duhengforever pushed a commit to branch main
in repository https://gitbox.apache.org/repos/asf/rocketmq-flink.git

commit 1c36e71b71b99f803616d95a874b056f7dfde7b8
Author: vongosling <vo...@apache.org>
AuthorDate: Wed Jan 9 18:46:28 2019 +0800

    Remove python and go repositories
---
 pom.xml | 2 +-
 1 file changed, 1 insertion(+), 1 deletion(-)

diff --git a/pom.xml b/pom.xml
index de824bd..c715f7f 100644
--- a/pom.xml
+++ b/pom.xml
@@ -34,7 +34,7 @@
         <maven.compiler.source>1.8</maven.compiler.source>
         <maven.compiler.target>1.8</maven.compiler.target>
         <rocketmq.version>4.2.0</rocketmq.version>
-        <flink.version>1.4.0</flink.version>
+        <flink.version>1.7.0</flink.version>
         <commons-lang.version>2.5</commons-lang.version>
         <scala.binary.version>2.11</scala.binary.version>
     </properties>

[rocketmq-flink] 32/33: RocketMQSource improves the message consume of RocketMQPartitionSplitReader (#791)

Posted by du...@apache.org.
This is an automated email from the ASF dual-hosted git repository.

duhengforever pushed a commit to branch main
in repository https://gitbox.apache.org/repos/asf/rocketmq-flink.git

commit aae7134f56c38963711488331595ac1888d28558
Author: SteNicholas <pr...@163.com>
AuthorDate: Tue Aug 24 15:48:20 2021 +0800

    RocketMQSource improves the message consume of RocketMQPartitionSplitReader (#791)
---
 .../flink/source/reader/RocketMQPartitionSplitReader.java      | 10 +++++++---
 .../rocketmq/flink/source/split/RocketMQPartitionSplit.java    |  7 ++++---
 2 files changed, 11 insertions(+), 6 deletions(-)

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
index 41fbbea..1846114 100644
--- a/src/main/java/org/apache/rocketmq/flink/source/reader/RocketMQPartitionSplitReader.java
+++ b/src/main/java/org/apache/rocketmq/flink/source/reader/RocketMQPartitionSplitReader.java
@@ -133,7 +133,7 @@ public class RocketMQPartitionSplitReader<T>
                                         ? consumer.searchOffset(messageQueue, startTime)
                                         : startOffset;
                     } catch (MQClientException e) {
-                        LOG.error(
+                        LOG.warn(
                                 String.format(
                                         "Search RocketMQ message offset of topic[%s] broker[%s] queue[%d] exception.",
                                         messageQueue.getTopic(),
@@ -159,13 +159,13 @@ public class RocketMQPartitionSplitReader<T>
                         return recordsBySplits;
                     }
                     pullResult =
-                            consumer.pullBlockIfNotFound(
+                            consumer.pull(
                                     messageQueue, tag, messageOffset, MAX_MESSAGE_NUMBER_PER_BLOCK);
                 } catch (MQClientException
                         | RemotingException
                         | MQBrokerException
                         | InterruptedException e) {
-                    LOG.error(
+                    LOG.warn(
                             String.format(
                                     "Pull RocketMQ messages of topic[%s] broker[%s] queue[%d] tag[%s] from offset[%d] exception.",
                                     messageQueue.getTopic(),
@@ -222,6 +222,10 @@ public class RocketMQPartitionSplitReader<T>
             }
         }
         recordsBySplits.prepareForRead();
+        LOG.debug(
+                String.format(
+                        "Fetch record splits for MetaQ subscribe message queues of topic[%s].",
+                        topic));
         return recordsBySplits;
     }
 
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
index 9bda60f..5717767 100644
--- a/src/main/java/org/apache/rocketmq/flink/source/split/RocketMQPartitionSplit.java
+++ b/src/main/java/org/apache/rocketmq/flink/source/split/RocketMQPartitionSplit.java
@@ -73,13 +73,13 @@ public class RocketMQPartitionSplit implements SourceSplit {
     @Override
     public String toString() {
         return String.format(
-                "[Topic: %s, Partition: %s, StartingOffset: %d, StoppingTimestamp: %d]",
-                topic, partition, startingOffset, stoppingTimestamp);
+                "[Topic: %s, Broker: %s, Partition: %s, StartingOffset: %d, StoppingTimestamp: %d]",
+                topic, broker, partition, startingOffset, stoppingTimestamp);
     }
 
     @Override
     public int hashCode() {
-        return Objects.hash(topic, partition, startingOffset, stoppingTimestamp);
+        return Objects.hash(topic, broker, partition, startingOffset, stoppingTimestamp);
     }
 
     @Override
@@ -89,6 +89,7 @@ public class RocketMQPartitionSplit implements SourceSplit {
         }
         RocketMQPartitionSplit other = (RocketMQPartitionSplit) obj;
         return topic.equals(other.topic)
+                && broker.equals(other.broker)
                 && partition == other.partition
                 && startingOffset == other.startingOffset
                 && stoppingTimestamp == other.stoppingTimestamp;

[rocketmq-flink] 11/33: flink-rocketmq-sink , producer send message set delay level (optional… (#237)

Posted by du...@apache.org.
This is an automated email from the ASF dual-hosted git repository.

duhengforever pushed a commit to branch main
in repository https://gitbox.apache.org/repos/asf/rocketmq-flink.git

commit b5c05e0f5ed71ba1f4aa9eae55136995df724f9f
Author: xlzl <75...@qq.com>
AuthorDate: Mon Jun 3 11:05:31 2019 +0800

    flink-rocketmq-sink , producer send message set delay level (optional… (#237)
---
 .../org/apache/rocketmq/flink/RocketMQConfig.java  | 27 +++++++++++++++++++---
 .../org/apache/rocketmq/flink/RocketMQSink.java    | 20 ++++++++++++----
 .../apache/rocketmq/flink/RocketMQSinkTest.java    |  1 +
 .../flink/example/RocketMQFlinkExample.java        |  6 ++++-
 4 files changed, 46 insertions(+), 8 deletions(-)

diff --git a/src/main/java/org/apache/rocketmq/flink/RocketMQConfig.java b/src/main/java/org/apache/rocketmq/flink/RocketMQConfig.java
index 8ec760b..5b43b31 100644
--- a/src/main/java/org/apache/rocketmq/flink/RocketMQConfig.java
+++ b/src/main/java/org/apache/rocketmq/flink/RocketMQConfig.java
@@ -34,7 +34,7 @@ import static org.apache.rocketmq.flink.RocketMQUtils.getInteger;
  * RocketMQConfig for Consumer/Producer.
  */
 public class RocketMQConfig {
-    // common
+    // Server Config
     public static final String NAME_SERVER_ADDR = "nameserver.address"; // Required
 
     public static final String NAME_SERVER_POLL_INTERVAL = "nameserver.poll.interval";
@@ -44,7 +44,7 @@ public class RocketMQConfig {
     public static final int DEFAULT_BROKER_HEART_BEAT_INTERVAL = 30000; // 30 seconds
 
 
-    // producer
+    // Producer related config
     public static final String PRODUCER_GROUP = "producer.group";
 
     public static final String PRODUCER_RETRY_TIMES = "producer.retry.times";
@@ -54,7 +54,7 @@ public class RocketMQConfig {
     public static final int DEFAULT_PRODUCER_TIMEOUT = 3000; // 3 seconds
 
 
-    // consumer
+    // Consumer related config
     public static final String CONSUMER_GROUP = "consumer.group"; // Required
 
     public static final String CONSUMER_TOPIC = "consumer.topic"; // Required
@@ -80,6 +80,27 @@ public class RocketMQConfig {
     public static final String CONSUMER_DELAY_WHEN_MESSAGE_NOT_FOUND = "consumer.delay.when.message.not.found";
     public static final int DEFAULT_CONSUMER_DELAY_WHEN_MESSAGE_NOT_FOUND = 10;
 
+    public static final String MSG_DELAY_LEVEL = "msg.delay.level";
+    public static final int MSG_DELAY_LEVEL00 = 0; // no delay
+    public static final int MSG_DELAY_LEVEL01 = 1; // 1s
+    public static final int MSG_DELAY_LEVEL02 = 2; // 5s
+    public static final int MSG_DELAY_LEVEL03 = 3; // 10s
+    public static final int MSG_DELAY_LEVEL04 = 4; // 30s
+    public static final int MSG_DELAY_LEVEL05 = 5; // 1min
+    public static final int MSG_DELAY_LEVEL06 = 6; // 2min
+    public static final int MSG_DELAY_LEVEL07 = 7; // 3min
+    public static final int MSG_DELAY_LEVEL08 = 8; // 4min
+    public static final int MSG_DELAY_LEVEL09 = 9; // 5min
+    public static final int MSG_DELAY_LEVEL10 = 10; // 6min
+    public static final int MSG_DELAY_LEVEL11 = 11; // 7min
+    public static final int MSG_DELAY_LEVEL12 = 12; // 8min
+    public static final int MSG_DELAY_LEVEL13 = 13; // 9min
+    public static final int MSG_DELAY_LEVEL14 = 14; // 10min
+    public static final int MSG_DELAY_LEVEL15 = 15; // 20min
+    public static final int MSG_DELAY_LEVEL16 = 16; // 30min
+    public static final int MSG_DELAY_LEVEL17 = 17; // 1h
+    public static final int MSG_DELAY_LEVEL18 = 18; // 2h
+
     /**
      * Build Producer Configs.
      * @param props Properties
diff --git a/src/main/java/org/apache/rocketmq/flink/RocketMQSink.java b/src/main/java/org/apache/rocketmq/flink/RocketMQSink.java
index 65274af..41bbcbe 100644
--- a/src/main/java/org/apache/rocketmq/flink/RocketMQSink.java
+++ b/src/main/java/org/apache/rocketmq/flink/RocketMQSink.java
@@ -62,10 +62,22 @@ public class RocketMQSink<IN> extends RichSinkFunction<IN> implements Checkpoint
     private int batchSize = 1000;
     private List<Message> batchList;
 
+    private int messageDeliveryDelayLevel = RocketMQConfig.MSG_DELAY_LEVEL00;
+
     public RocketMQSink(KeyValueSerializationSchema<IN> schema, TopicSelector<IN> topicSelector, Properties props) {
         this.serializationSchema = schema;
         this.topicSelector = topicSelector;
         this.props = props;
+
+        if (this.props != null) {
+            this.messageDeliveryDelayLevel  = RocketMQUtils.getInteger(this.props, RocketMQConfig.MSG_DELAY_LEVEL,
+                    RocketMQConfig.MSG_DELAY_LEVEL00);
+            if (this.messageDeliveryDelayLevel  < RocketMQConfig.MSG_DELAY_LEVEL00) {
+                this.messageDeliveryDelayLevel  = RocketMQConfig.MSG_DELAY_LEVEL00;
+            } else if (this.messageDeliveryDelayLevel  > RocketMQConfig.MSG_DELAY_LEVEL18) {
+                this.messageDeliveryDelayLevel  = RocketMQConfig.MSG_DELAY_LEVEL18;
+            }
+        }
     }
 
     @Override
@@ -105,7 +117,6 @@ public class RocketMQSink<IN> extends RichSinkFunction<IN> implements Checkpoint
         }
 
         if (async) {
-            // async sending
             try {
                 producer.send(msg, new SendCallback() {
                     @Override
@@ -124,7 +135,6 @@ public class RocketMQSink<IN> extends RichSinkFunction<IN> implements Checkpoint
                 LOG.error("Async send message failure!", e);
             }
         } else {
-            // sync sending, will return a SendResult
             try {
                 SendResult result = producer.send(msg);
                 LOG.debug("Sync send message result: {}", result);
@@ -134,7 +144,6 @@ public class RocketMQSink<IN> extends RichSinkFunction<IN> implements Checkpoint
         }
     }
 
-    // Mapping: from storm tuple -> rocketmq Message
     private Message prepareMessage(IN input) {
         String topic = topicSelector.getTopic(input);
         String tag = topicSelector.getTag(input) != null ? topicSelector.getTag(input) : "";
@@ -147,6 +156,9 @@ public class RocketMQSink<IN> extends RichSinkFunction<IN> implements Checkpoint
         Validate.notNull(value, "the message body is null");
 
         Message msg = new Message(topic, tag, key, value);
+        if (this.messageDeliveryDelayLevel > RocketMQConfig.MSG_DELAY_LEVEL00) {
+            msg.setDelayTimeLevel(this.messageDeliveryDelayLevel);
+        }
         return msg;
     }
 
@@ -191,6 +203,6 @@ public class RocketMQSink<IN> extends RichSinkFunction<IN> implements Checkpoint
 
     @Override
     public void initializeState(FunctionInitializationContext context) throws Exception {
-        // nothing to do
+        // Nothing to do
     }
 }
diff --git a/src/test/java/org/apache/rocketmq/flink/RocketMQSinkTest.java b/src/test/java/org/apache/rocketmq/flink/RocketMQSinkTest.java
index ec844f2..74a10b0 100644
--- a/src/test/java/org/apache/rocketmq/flink/RocketMQSinkTest.java
+++ b/src/test/java/org/apache/rocketmq/flink/RocketMQSinkTest.java
@@ -46,6 +46,7 @@ public class RocketMQSinkTest {
         KeyValueSerializationSchema serializationSchema = new SimpleKeyValueSerializationSchema("id", "name");
         TopicSelector topicSelector = new DefaultTopicSelector("tpc");
         Properties props = new Properties();
+        props.setProperty(RocketMQConfig.MSG_DELAY_LEVEL,String.valueOf(RocketMQConfig.MSG_DELAY_LEVEL04));
         rocketMQSink = new RocketMQSink(serializationSchema, topicSelector, props);
 
         producer = mock(DefaultMQProducer.class);
diff --git a/src/test/java/org/apache/rocketmq/flink/example/RocketMQFlinkExample.java b/src/test/java/org/apache/rocketmq/flink/example/RocketMQFlinkExample.java
index b2a4034..f4f654e 100644
--- a/src/test/java/org/apache/rocketmq/flink/example/RocketMQFlinkExample.java
+++ b/src/test/java/org/apache/rocketmq/flink/example/RocketMQFlinkExample.java
@@ -46,6 +46,10 @@ public class RocketMQFlinkExample {
 
         Properties producerProps = new Properties();
         producerProps.setProperty(RocketMQConfig.NAME_SERVER_ADDR, "localhost:9876");
+        int msgDelayLevel = RocketMQConfig.MSG_DELAY_LEVEL05;
+        producerProps.setProperty(RocketMQConfig.MSG_DELAY_LEVEL,String.valueOf(msgDelayLevel));
+        // TimeDelayLevel is not supported for batching
+        boolean batchFlag = msgDelayLevel <= 0;
 
         env.addSource(new RocketMQSource(new SimpleKeyValueDeserializationSchema("id", "address"), consumerProps))
             .name("rocketmq-source")
@@ -63,7 +67,7 @@ public class RocketMQFlinkExample {
             .name("upper-processor")
             .setParallelism(2)
             .addSink(new RocketMQSink(new SimpleKeyValueSerializationSchema("id", "province"),
-                new DefaultTopicSelector("flink-sink2"), producerProps).withBatchFlushOnCheckpoint(true))
+                new DefaultTopicSelector("flink-sink2"), producerProps).withBatchFlushOnCheckpoint(batchFlag))
             .name("rocketmq-sink")
             .setParallelism(2);
 

[rocketmq-flink] 03/33: ISSUE-61: Added javadocs for RocketMQ-Flink (#63)

Posted by du...@apache.org.
This is an automated email from the ASF dual-hosted git repository.

duhengforever pushed a commit to branch main
in repository https://gitbox.apache.org/repos/asf/rocketmq-flink.git

commit 11e2e6d10145050c9703e14d64c0693bb1cb1735
Author: Xin Wang <be...@163.com>
AuthorDate: Thu Jun 7 15:40:10 2018 +0800

    ISSUE-61: Added javadocs for RocketMQ-Flink (#63)
    
    * ISSUE-61: Added javadocs for RocketMQ-Flink and remove the generated javadoc files
---
 README.md |  1 -
 pom.xml   | 10 ++++++++++
 2 files changed, 10 insertions(+), 1 deletion(-)

diff --git a/README.md b/README.md
index ab1d456..600c57a 100644
--- a/README.md
+++ b/README.md
@@ -2,7 +2,6 @@
 
 RocketMQ integration for [Apache Flink](https://flink.apache.org/). This module includes the RocketMQ source and sink that allows a flink job to either write messages into a topic or read from topics in a flink job.
 
-
 ## RocketMQSource
 To use the `RocketMQSource`,  you construct an instance of it by specifying a KeyValueDeserializationSchema instance and a Properties instance which including rocketmq configs.
 `RocketMQSource(KeyValueDeserializationSchema<OUT> schema, Properties props)`
diff --git a/pom.xml b/pom.xml
index ecb3436..de824bd 100644
--- a/pom.xml
+++ b/pom.xml
@@ -169,6 +169,16 @@
                     </execution>
                 </executions>
             </plugin>
+            <plugin>
+                <groupId>org.apache.maven.plugins</groupId>
+                <artifactId>maven-javadoc-plugin</artifactId>
+                <version>2.10.4</version>
+                <configuration>
+                    <aggregate>true</aggregate>
+                    <reportOutputDirectory>javadocs</reportOutputDirectory>
+                    <locale>en</locale>
+                </configuration>
+            </plugin>
         </plugins>
     </build>
 </project>
\ No newline at end of file

[rocketmq-flink] 08/33: Optimizing update offset code logic

Posted by du...@apache.org.
This is an automated email from the ASF dual-hosted git repository.

duhengforever pushed a commit to branch main
in repository https://gitbox.apache.org/repos/asf/rocketmq-flink.git

commit f74cc2709208efa82588701554fb30410fd33469
Author: Jennifer-sarah <42...@users.noreply.github.com>
AuthorDate: Fri Mar 22 00:48:05 2019 +0800

    Optimizing update offset code logic
    
    Optimizing update offset code logic
---
 .../org/apache/rocketmq/flink/RocketMQSource.java     | 19 +++++++++++++------
 1 file changed, 13 insertions(+), 6 deletions(-)

diff --git a/src/main/java/org/apache/rocketmq/flink/RocketMQSource.java b/src/main/java/org/apache/rocketmq/flink/RocketMQSource.java
index 14b8042..f610efe 100644
--- a/src/main/java/org/apache/rocketmq/flink/RocketMQSource.java
+++ b/src/main/java/org/apache/rocketmq/flink/RocketMQSource.java
@@ -37,6 +37,7 @@ import org.apache.flink.runtime.state.FunctionInitializationContext;
 import org.apache.flink.runtime.state.FunctionSnapshotContext;
 import org.apache.flink.streaming.api.checkpoint.CheckpointedFunction;
 import org.apache.flink.streaming.api.functions.source.RichParallelSourceFunction;
+import org.apache.flink.streaming.api.operators.StreamingRuntimeContext;
 import org.apache.rocketmq.client.consumer.DefaultMQPullConsumer;
 import org.apache.rocketmq.client.consumer.MQPullConsumerScheduleService;
 import org.apache.rocketmq.client.consumer.PullResult;
@@ -85,6 +86,7 @@ public class RocketMQSource<OUT> extends RichParallelSourceFunction<OUT>
     private static final String OFFSETS_STATE_NAME = "topic-partition-offset-states";
 
     private transient volatile boolean restored;
+    private transient boolean enableCheckpoint;
 
     public RocketMQSource(KeyValueDeserializationSchema<OUT> schema, Properties props) {
         this.schema = schema;
@@ -103,6 +105,8 @@ public class RocketMQSource<OUT> extends RichParallelSourceFunction<OUT>
         Validate.notEmpty(topic, "Consumer topic can not be empty");
         Validate.notEmpty(group, "Consumer group can not be empty");
 
+        this.enableCheckpoint = ((StreamingRuntimeContext) getRuntimeContext()).isCheckpointingEnabled();
+
         if (offsetTable == null) {
             offsetTable = new ConcurrentHashMap<>();
         }
@@ -243,7 +247,9 @@ public class RocketMQSource<OUT> extends RichParallelSourceFunction<OUT>
 
     private void putMessageQueueOffset(MessageQueue mq, long offset) throws MQClientException {
         offsetTable.put(mq, offset);
-        consumer.updateConsumeOffset(mq, offset);
+        if (!enableCheckpoint) {
+            consumer.updateConsumeOffset(mq, offset);
+        }
     }
 
     @Override
@@ -285,14 +291,14 @@ public class RocketMQSource<OUT> extends RichParallelSourceFunction<OUT>
 
         unionOffsetStates.clear();
 
+        for (Map.Entry<MessageQueue, Long> entry : offsetTable.entrySet()) {
+            unionOffsetStates.add(Tuple2.of(entry.getKey(), entry.getValue()));
+        }
+
         if (LOG.isDebugEnabled()) {
             LOG.debug("Snapshotted state, last processed offsets: {}, checkpoint id: {}, timestamp: {}",
                     offsetTable, context.getCheckpointId(), context.getCheckpointTimestamp());
         }
-
-        for (Map.Entry<MessageQueue, Long> entry : offsetTable.entrySet()) {
-            unionOffsetStates.add(Tuple2.of(entry.getKey(), entry.getValue()));
-        }
     }
 
     @Override
@@ -330,7 +336,8 @@ public class RocketMQSource<OUT> extends RichParallelSourceFunction<OUT>
     }
 
     @Override
-    public void notifyCheckpointComplete(long l) throws Exception {
+    public void notifyCheckpointComplete(long checkpointId) throws Exception {
+        // consumer.c
         if (!runningChecker.isRunning()) {
             LOG.debug("notifyCheckpointComplete() called on closed source; returning null.");
             return;

[rocketmq-flink] 28/33: Build pass jdk9+

Posted by du...@apache.org.
This is an automated email from the ASF dual-hosted git repository.

duhengforever pushed a commit to branch main
in repository https://gitbox.apache.org/repos/asf/rocketmq-flink.git

commit 465c60acdc38db00c1fa732c2fa38736a0d92e2b
Author: vongosling <vo...@apache.org>
AuthorDate: Tue Aug 17 10:37:14 2021 +0800

    Build pass jdk9+
---
 .../org/apache/rocketmq/flink/source/util/StringSerializer.java    | 7 +++----
 1 file changed, 3 insertions(+), 4 deletions(-)

diff --git a/src/main/java/org/apache/rocketmq/flink/source/util/StringSerializer.java b/src/main/java/org/apache/rocketmq/flink/source/util/StringSerializer.java
index b468ac9..c0a67b8 100644
--- a/src/main/java/org/apache/rocketmq/flink/source/util/StringSerializer.java
+++ b/src/main/java/org/apache/rocketmq/flink/source/util/StringSerializer.java
@@ -17,6 +17,7 @@
 
 package org.apache.rocketmq.flink.source.util;
 
+import java.util.Base64;
 import org.apache.flink.table.data.DecimalData;
 import org.apache.flink.table.data.binary.BinaryStringData;
 import org.apache.flink.table.data.util.DataFormatConverters;
@@ -24,8 +25,6 @@ import org.apache.flink.table.data.util.DataFormatConverters.TimestampConverter;
 import org.apache.flink.table.types.DataType;
 import org.apache.flink.table.types.logical.DecimalType;
 
-import sun.misc.BASE64Decoder;
-
 import java.math.BigDecimal;
 import java.math.BigInteger;
 import java.sql.Date;
@@ -37,7 +36,7 @@ import java.util.Set;
 public class StringSerializer {
 
     public static TimestampConverter timestampConverter = new TimestampConverter(3);
-    private static final BASE64Decoder decoder = new BASE64Decoder();
+    private static final Base64.Decoder decoder = Base64.getDecoder();
 
     public static Object deserialize(
             String value,
@@ -61,7 +60,7 @@ public class StringSerializer {
                 if (isRGData) {
                     byte[] bytes = null;
                     try {
-                        bytes = decoder.decodeBuffer(value);
+                        bytes = decoder.decode(value);
                     } catch (Exception e) {
                         //
                     }

[rocketmq-flink] 07/33: update consumer offset after checkpoint completed

Posted by du...@apache.org.
This is an automated email from the ASF dual-hosted git repository.

duhengforever pushed a commit to branch main
in repository https://gitbox.apache.org/repos/asf/rocketmq-flink.git

commit 65e30294aaaa25992752853a04fdfa69cd3c0e86
Author: Jennifer-sarah <42...@users.noreply.github.com>
AuthorDate: Thu Mar 21 13:43:12 2019 +0800

    update consumer offset after checkpoint completed
    
    update consumer offset after checkpoint completed
---
 .../org/apache/rocketmq/flink/RocketMQSource.java  | 27 ++++++++++++++++------
 1 file changed, 20 insertions(+), 7 deletions(-)

diff --git a/src/main/java/org/apache/rocketmq/flink/RocketMQSource.java b/src/main/java/org/apache/rocketmq/flink/RocketMQSource.java
index 8e8e57b..14b8042 100644
--- a/src/main/java/org/apache/rocketmq/flink/RocketMQSource.java
+++ b/src/main/java/org/apache/rocketmq/flink/RocketMQSource.java
@@ -32,6 +32,7 @@ import org.apache.flink.api.common.typeinfo.TypeInformation;
 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.runtime.state.CheckpointListener;
 import org.apache.flink.runtime.state.FunctionInitializationContext;
 import org.apache.flink.runtime.state.FunctionSnapshotContext;
 import org.apache.flink.streaming.api.checkpoint.CheckpointedFunction;
@@ -60,7 +61,7 @@ import static org.apache.rocketmq.flink.RocketMQUtils.getLong;
  * Otherwise, the source doesn't provide any reliability guarantees.
  */
 public class RocketMQSource<OUT> extends RichParallelSourceFunction<OUT>
-    implements CheckpointedFunction, ResultTypeQueryable<OUT> {
+        implements CheckpointedFunction, CheckpointListener, ResultTypeQueryable<OUT> {
 
     private static final long serialVersionUID = 1L;
 
@@ -126,15 +127,15 @@ public class RocketMQSource<OUT> extends RichParallelSourceFunction<OUT>
         final Object lock = context.getCheckpointLock();
 
         int delayWhenMessageNotFound = getInteger(props, RocketMQConfig.CONSUMER_DELAY_WHEN_MESSAGE_NOT_FOUND,
-            RocketMQConfig.DEFAULT_CONSUMER_DELAY_WHEN_MESSAGE_NOT_FOUND);
+                RocketMQConfig.DEFAULT_CONSUMER_DELAY_WHEN_MESSAGE_NOT_FOUND);
 
         String tag = props.getProperty(RocketMQConfig.CONSUMER_TAG, RocketMQConfig.DEFAULT_CONSUMER_TAG);
 
         int pullPoolSize = getInteger(props, RocketMQConfig.CONSUMER_PULL_POOL_SIZE,
-            RocketMQConfig.DEFAULT_CONSUMER_PULL_POOL_SIZE);
+                RocketMQConfig.DEFAULT_CONSUMER_PULL_POOL_SIZE);
 
         int pullBatchSize = getInteger(props, RocketMQConfig.CONSUMER_BATCH_SIZE,
-            RocketMQConfig.DEFAULT_CONSUMER_BATCH_SIZE);
+                RocketMQConfig.DEFAULT_CONSUMER_BATCH_SIZE);
 
         pullConsumerScheduleService.setPullThreadNums(pullPoolSize);
         pullConsumerScheduleService.registerPullTaskCallback(topic, new PullTaskCallback() {
@@ -229,7 +230,7 @@ public class RocketMQSource<OUT> extends RichParallelSourceFunction<OUT>
                         break;
                     case CONSUMER_OFFSET_TIMESTAMP:
                         offset = consumer.searchOffset(mq, getLong(props,
-                            RocketMQConfig.CONSUMER_OFFSET_FROM_TIMESTAMP, System.currentTimeMillis()));
+                                RocketMQConfig.CONSUMER_OFFSET_FROM_TIMESTAMP, System.currentTimeMillis()));
                         break;
                     default:
                         throw new IllegalArgumentException("Unknown value for CONSUMER_OFFSET_RESET_TO.");
@@ -286,7 +287,7 @@ public class RocketMQSource<OUT> extends RichParallelSourceFunction<OUT>
 
         if (LOG.isDebugEnabled()) {
             LOG.debug("Snapshotted state, last processed offsets: {}, checkpoint id: {}, timestamp: {}",
-                offsetTable, context.getCheckpointId(), context.getCheckpointTimestamp());
+                    offsetTable, context.getCheckpointId(), context.getCheckpointTimestamp());
         }
 
         for (Map.Entry<MessageQueue, Long> entry : offsetTable.entrySet()) {
@@ -304,7 +305,7 @@ public class RocketMQSource<OUT> extends RichParallelSourceFunction<OUT>
         LOG.debug("initialize State ...");
 
         this.unionOffsetStates = context.getOperatorStateStore().getUnionListState(new ListStateDescriptor<>(
-            OFFSETS_STATE_NAME, TypeInformation.of(new TypeHint<Tuple2<MessageQueue, Long>>() { })));
+                OFFSETS_STATE_NAME, TypeInformation.of(new TypeHint<Tuple2<MessageQueue, Long>>() { })));
 
         this.restored = context.isRestored();
 
@@ -327,4 +328,16 @@ public class RocketMQSource<OUT> extends RichParallelSourceFunction<OUT>
     public TypeInformation<OUT> getProducedType() {
         return schema.getProducedType();
     }
+
+    @Override
+    public void notifyCheckpointComplete(long l) throws Exception {
+        if (!runningChecker.isRunning()) {
+            LOG.debug("notifyCheckpointComplete() called on closed source; returning null.");
+            return;
+        }
+
+        for (Map.Entry<MessageQueue, Long> entry : offsetTable.entrySet()) {
+            consumer.updateConsumeOffset(entry.getKey(), entry.getValue());
+        }
+    }
 }

[rocketmq-flink] 02/33: [ROCKETMQ-82] RocketMQ-Flink Integration (#45)

Posted by du...@apache.org.
This is an automated email from the ASF dual-hosted git repository.

duhengforever pushed a commit to branch main
in repository https://gitbox.apache.org/repos/asf/rocketmq-flink.git

commit 12f0d0c710ea91b0767b84f95245f3eb44c29ce8
Author: Xin Wang <be...@163.com>
AuthorDate: Fri Mar 23 21:47:33 2018 +0800

    [ROCKETMQ-82] RocketMQ-Flink Integration (#45)
---
 README.md                                          | 147 +++++++++
 pom.xml                                            | 174 +++++++++++
 .../org/apache/rocketmq/flink/RocketMQConfig.java  | 134 +++++++++
 .../org/apache/rocketmq/flink/RocketMQSink.java    | 187 ++++++++++++
 .../org/apache/rocketmq/flink/RocketMQSource.java  | 331 +++++++++++++++++++++
 .../org/apache/rocketmq/flink/RocketMQUtils.java   |  36 +++
 .../org/apache/rocketmq/flink/RunningChecker.java  |  33 ++
 .../common/selector/DefaultTopicSelector.java      |  43 +++
 .../flink/common/selector/SimpleTopicSelector.java |  73 +++++
 .../flink/common/selector/TopicSelector.java       |  28 ++
 .../KeyValueDeserializationSchema.java             |  27 ++
 .../serialization/KeyValueSerializationSchema.java |  28 ++
 .../SimpleKeyValueDeserializationSchema.java       |  66 ++++
 .../SimpleKeyValueSerializationSchema.java         |  63 ++++
 .../apache/rocketmq/flink/RocketMQSinkTest.java    |  75 +++++
 .../apache/rocketmq/flink/RocketMQSourceTest.java  | 121 ++++++++
 .../java/org/apache/rocketmq/flink/TestUtils.java  |  33 ++
 .../common/selector/DefaultTopicSelectorTest.java  |  37 +++
 .../common/selector/SimpleTopicSelectorTest.java   |  49 +++
 .../SimpleKeyValueSerializationSchemaTest.java     |  42 +++
 .../rocketmq/flink/example/ConsumerTest.java       |  54 ++++
 .../rocketmq/flink/example/ProducerTest.java       |  57 ++++
 .../flink/example/RocketMQFlinkExample.java        |  76 +++++
 style/copyright/Apache.xml                         |  24 ++
 style/copyright/profiles_settings.xml              |  64 ++++
 style/rmq_checkstyle.xml                           | 135 +++++++++
 style/rmq_codeStyle.xml                            | 157 ++++++++++
 27 files changed, 2294 insertions(+)

diff --git a/README.md b/README.md
new file mode 100644
index 0000000..ab1d456
--- /dev/null
+++ b/README.md
@@ -0,0 +1,147 @@
+# RocketMQ-Flink
+
+RocketMQ integration for [Apache Flink](https://flink.apache.org/). This module includes the RocketMQ source and sink that allows a flink job to either write messages into a topic or read from topics in a flink job.
+
+
+## RocketMQSource
+To use the `RocketMQSource`,  you construct an instance of it by specifying a KeyValueDeserializationSchema instance and a Properties instance which including rocketmq configs.
+`RocketMQSource(KeyValueDeserializationSchema<OUT> schema, Properties props)`
+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 guarantees.
+
+### KeyValueDeserializationSchema
+The main API for deserializing topic and tags is the `org.apache.rocketmq.flink.common.serialization.KeyValueDeserializationSchema` interface.
+`rocketmq-flink` includes general purpose `KeyValueDeserializationSchema` implementations called `SimpleKeyValueDeserializationSchema`.
+
+```java
+public interface KeyValueDeserializationSchema<T> extends ResultTypeQueryable<T>, Serializable {
+    T deserializeKeyAndValue(byte[] key, byte[] value);
+}
+```
+
+## RocketMQSink
+To use the `RocketMQSink`,  you construct an instance of it by specifying KeyValueSerializationSchema & TopicSelector instances and a Properties instance which including rocketmq configs.
+`RocketMQSink(KeyValueSerializationSchema<IN> schema, TopicSelector<IN> topicSelector, Properties props)`
+The RocketMQSink provides at-least-once reliability guarantees when checkpoints are enabled and `withBatchFlushOnCheckpoint(true)` is set.
+Otherwise, the sink reliability guarantees depends on rocketmq producer's retry policy, for this case, the messages sending way is sync by default,
+but you can change it by invoking `withAsync(true)`. 
+
+### KeyValueSerializationSchema
+The main API for serializing topic and tags is the `org.apache.rocketmq.flink.common.serialization.KeyValueSerializationSchema` interface.
+`rocketmq-flink` includes general purpose `KeyValueSerializationSchema` implementations called `SimpleKeyValueSerializationSchema`.
+
+```java
+public interface KeyValueSerializationSchema<T> extends Serializable {
+
+    byte[] serializeKey(T tuple);
+
+    byte[] serializeValue(T tuple);
+}
+```
+
+### TopicSelector
+The main API for selecting topic and tags is the `org.apache.rocketmq.flink.common.selector.TopicSelector` interface.
+`rocketmq-flink` includes general purpose `TopicSelector` implementations called `DefaultTopicSelector` and `SimpleTopicSelector`.
+
+```java
+public interface TopicSelector<T> extends Serializable {
+
+    String getTopic(T tuple);
+
+    String getTag(T tuple);
+}
+```
+
+## Examples
+The following is an example which receive messages from RocketMQ brokers and send messages to broker after processing.
+
+ ```java
+StreamExecutionEnvironment env = StreamExecutionEnvironment.getExecutionEnvironment();
+
+        // enable checkpoint
+        env.enableCheckpointing(3000);
+
+        Properties consumerProps = new Properties();
+        consumerProps.setProperty(RocketMqConfig.NAME_SERVER_ADDR, "localhost:9876");
+        consumerProps.setProperty(RocketMqConfig.CONSUMER_GROUP, "c002");
+        consumerProps.setProperty(RocketMqConfig.CONSUMER_TOPIC, "flink-source2");
+
+        Properties producerProps = new Properties();
+        producerProps.setProperty(RocketMqConfig.NAME_SERVER_ADDR, "localhost:9876");
+
+        env.addSource(new RocketMQSource(new SimpleKeyValueDeserializationSchema("id", "address"), consumerProps))
+            .name("rocketmq-source")
+            .setParallelism(2)
+            .process(new ProcessFunction<Map, Map>() {
+                @Override
+                public void processElement(Map in, Context ctx, Collector<Map> out) throws Exception {
+                    HashMap result = new HashMap();
+                    result.put("id", in.get("id"));
+                    String[] arr = in.get("address").toString().split("\\s+");
+                    result.put("province", arr[arr.length-1]);
+                    out.collect(result);
+                }
+            })
+            .name("upper-processor")
+            .setParallelism(2)
+            .addSink(new RocketMQSink(new SimpleKeyValueSerializationSchema("id", "province"),
+                new DefaultTopicSelector("flink-sink2"), producerProps).withBatchFlushOnCheckpoint(true))
+            .name("rocketmq-sink")
+            .setParallelism(2);
+
+        try {
+            env.execute("rocketmq-flink-example");
+        } catch (Exception e) {
+            e.printStackTrace();
+        }
+ ```
+
+## Configurations
+The following configurations are all from the class `org.apache.rocketmq.flink.RocketMQConfig`.
+
+### Producer Configurations
+| NAME        | DESCRIPTION           | DEFAULT  |
+| ------------- |:-------------:|:------:|
+| nameserver.address      | name server address *Required* | null |
+| nameserver.poll.interval      | name server poll topic info interval     |   30000 |
+| brokerserver.heartbeat.interval | broker server heartbeat interval      |    30000 |
+| producer.group | producer group      |    `UUID.randomUUID().toString()` |
+| producer.retry.times | producer send messages retry times      |    3 |
+| producer.timeout | producer send messages timeout      |    3000 |
+
+
+### Consumer Configurations
+| NAME        | DESCRIPTION           | DEFAULT  |
+| ------------- |:-------------:|:------:|
+| nameserver.address      | name server address *Required* | null |
+| nameserver.poll.interval      | name server poll topic info interval     |   30000 |
+| brokerserver.heartbeat.interval | broker server heartbeat interval      |    30000 |
+| consumer.group | consumer group *Required*     |    null |
+| consumer.topic | consumer topic *Required*       |    null |
+| consumer.tag | consumer topic tag      |    * |
+| consumer.offset.reset.to | what to do when there is no initial offset on the server      |   latest/earliest/timestamp |
+| consumer.offset.from.timestamp | the timestamp when `consumer.offset.reset.to=timestamp` was set   |   `System.currentTimeMillis()` |
+| consumer.offset.persist.interval | auto commit offset interval      |    5000 |
+| consumer.pull.thread.pool.size | consumer pull thread pool size      |    20 |
+| consumer.batch.size | consumer messages batch size      |    32 |
+| consumer.delay.when.message.not.found | the delay time when messages were not found      |    10 |
+
+
+## License
+
+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.
diff --git a/pom.xml b/pom.xml
new file mode 100644
index 0000000..ecb3436
--- /dev/null
+++ b/pom.xml
@@ -0,0 +1,174 @@
+<?xml version="1.0" encoding="UTF-8"?>
+<!--
+ Licensed to the Apache Software Foundation (ASF) under one or more
+ contributor license agreements.  See the NOTICE file distributed with
+ this work for additional information regarding copyright ownership.
+ The ASF licenses this file to You under the Apache License, Version 2.0
+ (the "License"); you may not use this file except in compliance with
+ the License.  You may obtain a copy of the License at
+
+     http://www.apache.org/licenses/LICENSE-2.0
+
+ Unless required by applicable law or agreed to in writing, software
+ distributed under the License is distributed on an "AS IS" BASIS,
+ WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ See the License for the specific language governing permissions and
+ limitations under the License.
+-->
+<project xmlns="http://maven.apache.org/POM/4.0.0"
+         xmlns:xsi="http://www.w3.org/2001/XMLSchema-instance"
+         xsi:schemaLocation="http://maven.apache.org/POM/4.0.0 http://maven.apache.org/xsd/maven-4.0.0.xsd">
+    <modelVersion>4.0.0</modelVersion>
+
+    <groupId>org.apache.rocketmq</groupId>
+    <artifactId>rocketmq-flink</artifactId>
+    <version>0.0.1-SNAPSHOT</version>
+    <packaging>jar</packaging>
+
+    <properties>
+        <project.build.sourceEncoding>UTF-8</project.build.sourceEncoding>
+        <!--maven properties -->
+        <maven.test.skip>false</maven.test.skip>
+        <maven.javadoc.skip>false</maven.javadoc.skip>
+        <!-- compiler settings properties -->
+        <maven.compiler.source>1.8</maven.compiler.source>
+        <maven.compiler.target>1.8</maven.compiler.target>
+        <rocketmq.version>4.2.0</rocketmq.version>
+        <flink.version>1.4.0</flink.version>
+        <commons-lang.version>2.5</commons-lang.version>
+        <scala.binary.version>2.11</scala.binary.version>
+    </properties>
+
+    <dependencies>
+        <dependency>
+            <groupId>org.apache.flink</groupId>
+            <artifactId>flink-java</artifactId>
+            <version>${flink.version}</version>
+        </dependency>
+        <dependency>
+            <groupId>org.apache.flink</groupId>
+            <artifactId>flink-streaming-java_${scala.binary.version}</artifactId>
+            <version>${flink.version}</version>
+        </dependency>
+        <dependency>
+            <groupId>org.apache.flink</groupId>
+            <artifactId>flink-clients_${scala.binary.version}</artifactId>
+            <version>${flink.version}</version>
+        </dependency>
+        <dependency>
+            <groupId>org.apache.flink</groupId>
+            <artifactId>flink-queryable-state-runtime_${scala.binary.version}</artifactId>
+            <version>${flink.version}</version>
+        </dependency>
+        <dependency>
+            <groupId>org.apache.rocketmq</groupId>
+            <artifactId>rocketmq-client</artifactId>
+            <version>${rocketmq.version}</version>
+        </dependency>
+        <dependency>
+            <groupId>org.apache.rocketmq</groupId>
+            <artifactId>rocketmq-common</artifactId>
+            <version>${rocketmq.version}</version>
+            <exclusions>
+                <exclusion>
+                    <groupId>io.netty</groupId>
+                    <artifactId>netty-tcnative</artifactId>
+                </exclusion>
+            </exclusions>
+        </dependency>
+
+        <dependency>
+            <groupId>commons-lang</groupId>
+            <artifactId>commons-lang</artifactId>
+            <version>${commons-lang.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>
+            <scope>test</scope>
+        </dependency>
+        <dependency>
+            <groupId>org.apache.rocketmq</groupId>
+            <artifactId>rocketmq-namesrv</artifactId>
+            <version>${rocketmq.version}</version>
+            <scope>test</scope>
+        </dependency>
+        <dependency>
+            <groupId>org.apache.rocketmq</groupId>
+            <artifactId>rocketmq-broker</artifactId>
+            <version>${rocketmq.version}</version>
+            <scope>test</scope>
+        </dependency>
+    </dependencies>
+
+    <build>
+        <plugins>
+            <plugin>
+                <artifactId>maven-compiler-plugin</artifactId>
+                <version>3.5.1</version>
+                <configuration>
+                    <source>${maven.compiler.source}</source>
+                    <target>${maven.compiler.target}</target>
+                    <encoding>UTF-8</encoding>
+                    <compilerVersion>${maven.compiler.source}</compilerVersion>
+                    <showDeprecation>true</showDeprecation>
+                    <showWarnings>true</showWarnings>
+                </configuration>
+            </plugin>
+            <plugin>
+                <groupId>org.apache.maven.plugins</groupId>
+                <artifactId>maven-surefire-plugin</artifactId>
+                <version>2.12.4</version>
+                <configuration>
+                    <skipTests>${maven.test.skip}</skipTests>
+                </configuration>
+            </plugin>
+            <plugin>
+                <groupId>org.apache.rat</groupId>
+                <artifactId>apache-rat-plugin</artifactId>
+                <version>0.12</version>
+                <configuration>
+                    <excludes>
+                        <exclude>README.md</exclude>
+                    </excludes>
+                </configuration>
+            </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>
+        </plugins>
+    </build>
+</project>
\ No newline at end of file
diff --git a/src/main/java/org/apache/rocketmq/flink/RocketMQConfig.java b/src/main/java/org/apache/rocketmq/flink/RocketMQConfig.java
new file mode 100644
index 0000000..8ec760b
--- /dev/null
+++ b/src/main/java/org/apache/rocketmq/flink/RocketMQConfig.java
@@ -0,0 +1,134 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.rocketmq.flink;
+
+import java.util.Properties;
+import java.util.UUID;
+
+import org.apache.commons.lang.StringUtils;
+import org.apache.commons.lang.Validate;
+import org.apache.rocketmq.client.ClientConfig;
+import org.apache.rocketmq.client.consumer.DefaultMQPullConsumer;
+import org.apache.rocketmq.client.producer.DefaultMQProducer;
+import org.apache.rocketmq.common.protocol.heartbeat.MessageModel;
+
+import static org.apache.rocketmq.flink.RocketMQUtils.getInteger;
+
+/**
+ * RocketMQConfig for Consumer/Producer.
+ */
+public class RocketMQConfig {
+    // common
+    public static final String NAME_SERVER_ADDR = "nameserver.address"; // Required
+
+    public static final String NAME_SERVER_POLL_INTERVAL = "nameserver.poll.interval";
+    public static final int DEFAULT_NAME_SERVER_POLL_INTERVAL = 30000; // 30 seconds
+
+    public static final String BROKER_HEART_BEAT_INTERVAL = "brokerserver.heartbeat.interval";
+    public static final int DEFAULT_BROKER_HEART_BEAT_INTERVAL = 30000; // 30 seconds
+
+
+    // producer
+    public static final String PRODUCER_GROUP = "producer.group";
+
+    public static final String PRODUCER_RETRY_TIMES = "producer.retry.times";
+    public static final int DEFAULT_PRODUCER_RETRY_TIMES = 3;
+
+    public static final String PRODUCER_TIMEOUT = "producer.timeout";
+    public static final int DEFAULT_PRODUCER_TIMEOUT = 3000; // 3 seconds
+
+
+    // consumer
+    public static final String CONSUMER_GROUP = "consumer.group"; // Required
+
+    public static final String CONSUMER_TOPIC = "consumer.topic"; // Required
+
+    public static final String CONSUMER_TAG = "consumer.tag";
+    public static final String DEFAULT_CONSUMER_TAG = "*";
+
+    public static final String CONSUMER_OFFSET_RESET_TO = "consumer.offset.reset.to";
+    public static final String CONSUMER_OFFSET_LATEST = "latest";
+    public static final String CONSUMER_OFFSET_EARLIEST = "earliest";
+    public static final String CONSUMER_OFFSET_TIMESTAMP = "timestamp";
+    public static final String CONSUMER_OFFSET_FROM_TIMESTAMP = "consumer.offset.from.timestamp";
+
+    public static final String CONSUMER_OFFSET_PERSIST_INTERVAL = "consumer.offset.persist.interval";
+    public static final int DEFAULT_CONSUMER_OFFSET_PERSIST_INTERVAL = 5000; // 5 seconds
+
+    public static final String CONSUMER_PULL_POOL_SIZE = "consumer.pull.thread.pool.size";
+    public static final int DEFAULT_CONSUMER_PULL_POOL_SIZE = 20;
+
+    public static final String CONSUMER_BATCH_SIZE = "consumer.batch.size";
+    public static final int DEFAULT_CONSUMER_BATCH_SIZE = 32;
+
+    public static final String CONSUMER_DELAY_WHEN_MESSAGE_NOT_FOUND = "consumer.delay.when.message.not.found";
+    public static final int DEFAULT_CONSUMER_DELAY_WHEN_MESSAGE_NOT_FOUND = 10;
+
+    /**
+     * Build Producer Configs.
+     * @param props Properties
+     * @param producer DefaultMQProducer
+     */
+    public static void buildProducerConfigs(Properties props, DefaultMQProducer producer) {
+        buildCommonConfigs(props, producer);
+
+        String group = props.getProperty(PRODUCER_GROUP);
+        if (StringUtils.isEmpty(group)) {
+            group = UUID.randomUUID().toString();
+        }
+        producer.setProducerGroup(props.getProperty(PRODUCER_GROUP, group));
+
+        producer.setRetryTimesWhenSendFailed(getInteger(props,
+            PRODUCER_RETRY_TIMES, DEFAULT_PRODUCER_RETRY_TIMES));
+        producer.setRetryTimesWhenSendAsyncFailed(getInteger(props,
+            PRODUCER_RETRY_TIMES, DEFAULT_PRODUCER_RETRY_TIMES));
+        producer.setSendMsgTimeout(getInteger(props,
+            PRODUCER_TIMEOUT, DEFAULT_PRODUCER_TIMEOUT));
+    }
+
+    /**
+     * Build Consumer Configs.
+     * @param props Properties
+     * @param consumer DefaultMQPushConsumer
+     */
+    public static void buildConsumerConfigs(Properties props, DefaultMQPullConsumer consumer) {
+        buildCommonConfigs(props, consumer);
+
+        consumer.setMessageModel(MessageModel.CLUSTERING);
+
+        consumer.setPersistConsumerOffsetInterval(getInteger(props,
+            CONSUMER_OFFSET_PERSIST_INTERVAL, DEFAULT_CONSUMER_OFFSET_PERSIST_INTERVAL));
+    }
+
+    /**
+     * Build Common Configs.
+     * @param props Properties
+     * @param client ClientConfig
+     */
+    public static void buildCommonConfigs(Properties props, ClientConfig client) {
+        String nameServers = props.getProperty(NAME_SERVER_ADDR);
+        Validate.notEmpty(nameServers);
+        client.setNamesrvAddr(nameServers);
+
+        client.setPollNameServerInterval(getInteger(props,
+            NAME_SERVER_POLL_INTERVAL, DEFAULT_NAME_SERVER_POLL_INTERVAL));
+        client.setHeartbeatBrokerInterval(getInteger(props,
+            BROKER_HEART_BEAT_INTERVAL, DEFAULT_BROKER_HEART_BEAT_INTERVAL));
+    }
+}
diff --git a/src/main/java/org/apache/rocketmq/flink/RocketMQSink.java b/src/main/java/org/apache/rocketmq/flink/RocketMQSink.java
new file mode 100644
index 0000000..e79d1b4
--- /dev/null
+++ b/src/main/java/org/apache/rocketmq/flink/RocketMQSink.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.rocketmq.flink;
+
+import java.nio.charset.StandardCharsets;
+import java.util.LinkedList;
+import java.util.List;
+import java.util.Properties;
+
+import org.apache.commons.lang.Validate;
+import org.apache.flink.configuration.Configuration;
+import org.apache.flink.runtime.state.FunctionInitializationContext;
+import org.apache.flink.runtime.state.FunctionSnapshotContext;
+import org.apache.flink.streaming.api.checkpoint.CheckpointedFunction;
+import org.apache.flink.streaming.api.functions.sink.RichSinkFunction;
+import org.apache.flink.streaming.api.operators.StreamingRuntimeContext;
+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.common.message.Message;
+import org.apache.rocketmq.flink.common.selector.TopicSelector;
+import org.apache.rocketmq.flink.common.serialization.KeyValueSerializationSchema;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+/**
+ * The RocketMQSink provides at-least-once reliability guarantees when
+ * checkpoints are enabled and batchFlushOnCheckpoint(true) is set.
+ * Otherwise, the sink reliability guarantees depends on rocketmq producer's retry policy.
+ */
+public class RocketMQSink<IN> extends RichSinkFunction<IN> implements CheckpointedFunction {
+
+    private static final long serialVersionUID = 1L;
+
+    private static final Logger LOG = LoggerFactory.getLogger(RocketMQSink.class);
+
+    private transient DefaultMQProducer producer;
+    private boolean async; // false by default
+
+    private Properties props;
+    private TopicSelector<IN> topicSelector;
+    private KeyValueSerializationSchema<IN> serializationSchema;
+
+    private boolean batchFlushOnCheckpoint; // false by default
+    private List<Message> batchList;
+
+    public RocketMQSink(KeyValueSerializationSchema<IN> schema, TopicSelector<IN> topicSelector, Properties props) {
+        this.serializationSchema = schema;
+        this.topicSelector = topicSelector;
+        this.props = props;
+    }
+
+    @Override
+    public void open(Configuration parameters) throws Exception {
+        Validate.notEmpty(props, "Producer properties can not be empty");
+        Validate.notNull(topicSelector, "TopicSelector can not be null");
+        Validate.notNull(serializationSchema, "KeyValueSerializationSchema can not be null");
+
+        producer = new DefaultMQProducer();
+        producer.setInstanceName(String.valueOf(getRuntimeContext().getIndexOfThisSubtask()));
+        RocketMQConfig.buildProducerConfigs(props, producer);
+
+        batchList = new LinkedList<>();
+
+        if (batchFlushOnCheckpoint && !((StreamingRuntimeContext) getRuntimeContext()).isCheckpointingEnabled()) {
+            LOG.warn("Flushing on checkpoint is enabled, but checkpointing is not enabled. Disabling flushing.");
+            batchFlushOnCheckpoint = false;
+        }
+
+        try {
+            producer.start();
+        } catch (MQClientException e) {
+            throw new RuntimeException(e);
+        }
+    }
+
+    @Override
+    public void invoke(IN input, Context context) throws Exception {
+        Message msg = prepareMessage(input);
+
+        if (batchFlushOnCheckpoint) {
+            batchList.add(msg);
+            return;
+        }
+
+        if (async) {
+            // async sending
+            try {
+                producer.send(msg, new SendCallback() {
+                    @Override
+                    public void onSuccess(SendResult sendResult) {
+                        LOG.debug("Async send message success! result: {}", sendResult);
+                    }
+
+                    @Override
+                    public void onException(Throwable throwable) {
+                        if (throwable != null) {
+                            LOG.error("Async send message failure!", throwable);
+                        }
+                    }
+                });
+            } catch (Exception e) {
+                LOG.error("Async send message failure!", e);
+            }
+        } else {
+            // sync sending, will return a SendResult
+            try {
+                SendResult result = producer.send(msg);
+                LOG.debug("Sync send message result: {}", result);
+            } catch (Exception e) {
+                LOG.error("Sync send message failure!", e);
+            }
+        }
+    }
+
+    // Mapping: from storm tuple -> rocketmq Message
+    private Message prepareMessage(IN input) {
+        String topic = topicSelector.getTopic(input);
+        String tag = topicSelector.getTag(input) != null ? topicSelector.getTag(input) : "";
+
+        byte[] k = serializationSchema.serializeKey(input);
+        String key = k != null ? new String(k, StandardCharsets.UTF_8) : "";
+        byte[] value = serializationSchema.serializeValue(input);
+
+        Validate.notNull(topic, "the message topic is null");
+        Validate.notNull(value, "the message body is null");
+
+        Message msg = new Message(topic, tag, key, value);
+        return msg;
+    }
+
+    public RocketMQSink<IN> withAsync(boolean async) {
+        this.async = async;
+        return this;
+    }
+
+    public RocketMQSink<IN> withBatchFlushOnCheckpoint(boolean batchFlushOnCheckpoint) {
+        this.batchFlushOnCheckpoint = batchFlushOnCheckpoint;
+        return this;
+    }
+
+    @Override
+    public void close() throws Exception {
+        if (producer != null) {
+            flushSync();
+            producer.shutdown();
+        }
+    }
+
+    private void flushSync() throws Exception {
+        if (batchFlushOnCheckpoint) {
+            synchronized (batchList) {
+                if (batchList.size() > 0) {
+                    producer.send(batchList);
+                    batchList.clear();
+                }
+            }
+        }
+    }
+
+    @Override
+    public void snapshotState(FunctionSnapshotContext context) throws Exception {
+        flushSync();
+    }
+
+    @Override
+    public void initializeState(FunctionInitializationContext context) throws Exception {
+        // nothing to do
+    }
+}
diff --git a/src/main/java/org/apache/rocketmq/flink/RocketMQSource.java b/src/main/java/org/apache/rocketmq/flink/RocketMQSource.java
new file mode 100644
index 0000000..2dc8fd5
--- /dev/null
+++ b/src/main/java/org/apache/rocketmq/flink/RocketMQSource.java
@@ -0,0 +1,331 @@
+/**
+ * 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;
+
+import java.nio.charset.StandardCharsets;
+import java.util.List;
+import java.util.Map;
+import java.util.Properties;
+import java.util.concurrent.ConcurrentHashMap;
+
+import org.apache.commons.lang.Validate;
+import org.apache.flink.api.common.state.ListState;
+import org.apache.flink.api.common.state.ListStateDescriptor;
+import org.apache.flink.api.common.typeinfo.TypeHint;
+import org.apache.flink.api.common.typeinfo.TypeInformation;
+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.runtime.state.FunctionInitializationContext;
+import org.apache.flink.runtime.state.FunctionSnapshotContext;
+import org.apache.flink.streaming.api.checkpoint.CheckpointedFunction;
+import org.apache.flink.streaming.api.functions.source.RichParallelSourceFunction;
+import org.apache.rocketmq.client.consumer.DefaultMQPullConsumer;
+import org.apache.rocketmq.client.consumer.MQPullConsumerScheduleService;
+import org.apache.rocketmq.client.consumer.PullResult;
+import org.apache.rocketmq.client.consumer.PullTaskCallback;
+import org.apache.rocketmq.client.consumer.PullTaskContext;
+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.common.serialization.KeyValueDeserializationSchema;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import static org.apache.rocketmq.flink.RocketMQConfig.CONSUMER_OFFSET_EARLIEST;
+import static org.apache.rocketmq.flink.RocketMQConfig.CONSUMER_OFFSET_LATEST;
+import static org.apache.rocketmq.flink.RocketMQConfig.CONSUMER_OFFSET_TIMESTAMP;
+import static org.apache.rocketmq.flink.RocketMQUtils.getInteger;
+import static org.apache.rocketmq.flink.RocketMQUtils.getLong;
+
+/**
+ * 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 guarantees.
+ */
+public class RocketMQSource<OUT> extends RichParallelSourceFunction<OUT>
+    implements CheckpointedFunction, ResultTypeQueryable<OUT> {
+
+    private static final long serialVersionUID = 1L;
+
+    private static final Logger LOG = LoggerFactory.getLogger(RocketMQSource.class);
+
+    private transient MQPullConsumerScheduleService pullConsumerScheduleService;
+    private DefaultMQPullConsumer consumer;
+
+    private KeyValueDeserializationSchema<OUT> schema;
+
+    private RunningChecker runningChecker;
+
+    private transient ListState<Tuple2<MessageQueue, Long>> unionOffsetStates;
+    private Map<MessageQueue, Long> offsetTable;
+    private Map<MessageQueue, Long> restoredOffsets;
+
+    private Properties props;
+    private String topic;
+    private String group;
+
+    private static final String OFFSETS_STATE_NAME = "topic-partition-offset-states";
+
+    private transient volatile boolean restored;
+
+    public RocketMQSource(KeyValueDeserializationSchema<OUT> schema, Properties props) {
+        this.schema = schema;
+        this.props = props;
+    }
+
+    @Override
+    public void open(Configuration parameters) throws Exception {
+        LOG.debug("source open....");
+        Validate.notEmpty(props, "Consumer properties can not be empty");
+        Validate.notNull(schema, "KeyValueDeserializationSchema can not be null");
+
+        this.topic = props.getProperty(RocketMQConfig.CONSUMER_TOPIC);
+        this.group = props.getProperty(RocketMQConfig.CONSUMER_GROUP);
+
+        Validate.notEmpty(topic, "Consumer topic can not be empty");
+        Validate.notEmpty(group, "Consumer group can not be empty");
+
+        if (offsetTable == null) {
+            offsetTable = new ConcurrentHashMap<>();
+        }
+        if (restoredOffsets == null) {
+            restoredOffsets = new ConcurrentHashMap<>();
+        }
+
+        runningChecker = new RunningChecker();
+
+        pullConsumerScheduleService = new MQPullConsumerScheduleService(group);
+        consumer = pullConsumerScheduleService.getDefaultMQPullConsumer();
+
+        consumer.setInstanceName(String.valueOf(getRuntimeContext().getIndexOfThisSubtask()));
+        RocketMQConfig.buildConsumerConfigs(props, consumer);
+    }
+
+    @Override
+    public void run(SourceContext context) throws Exception {
+        LOG.debug("source run....");
+        // The lock that guarantees that record emission and state updates are atomic,
+        // from the view of taking a checkpoint.
+        final Object lock = context.getCheckpointLock();
+
+        int delayWhenMessageNotFound = getInteger(props, RocketMQConfig.CONSUMER_DELAY_WHEN_MESSAGE_NOT_FOUND,
+            RocketMQConfig.DEFAULT_CONSUMER_DELAY_WHEN_MESSAGE_NOT_FOUND);
+
+        String tag = props.getProperty(RocketMQConfig.CONSUMER_TAG, RocketMQConfig.DEFAULT_CONSUMER_TAG);
+
+        int pullPoolSize = getInteger(props, RocketMQConfig.CONSUMER_PULL_POOL_SIZE,
+            RocketMQConfig.DEFAULT_CONSUMER_PULL_POOL_SIZE);
+
+        int pullBatchSize = getInteger(props, RocketMQConfig.CONSUMER_BATCH_SIZE,
+            RocketMQConfig.DEFAULT_CONSUMER_BATCH_SIZE);
+
+        pullConsumerScheduleService.setPullThreadNums(pullPoolSize);
+        pullConsumerScheduleService.registerPullTaskCallback(topic, new PullTaskCallback() {
+
+            @Override
+            public void doPullTask(MessageQueue mq, PullTaskContext pullTaskContext) {
+                try {
+                    long offset = getMessageQueueOffset(mq);
+                    if (offset < 0) {
+                        return;
+                    }
+
+                    PullResult pullResult = consumer.pull(mq, tag, offset, pullBatchSize);
+                    boolean found = false;
+                    switch (pullResult.getPullStatus()) {
+                        case FOUND:
+                            List<MessageExt> messages = pullResult.getMsgFoundList();
+                            for (MessageExt msg : messages) {
+                                byte[] key = msg.getKeys() != null ? msg.getKeys().getBytes(StandardCharsets.UTF_8) : null;
+                                byte[] value = msg.getBody();
+                                OUT data = schema.deserializeKeyAndValue(key, value);
+
+                                // output and state update are atomic
+                                synchronized (lock) {
+                                    context.collectWithTimestamp(data, msg.getBornTimestamp());
+                                }
+                            }
+                            found = true;
+                            break;
+                        case NO_MATCHED_MSG:
+                            LOG.debug("No matched message after offset {} for queue {}", offset, mq);
+                            break;
+                        case NO_NEW_MSG:
+                            break;
+                        case OFFSET_ILLEGAL:
+                            LOG.warn("Offset {} is illegal for queue {}", offset, mq);
+                            break;
+                        default:
+                            break;
+                    }
+
+                    synchronized (lock) {
+                        putMessageQueueOffset(mq, pullResult.getNextBeginOffset());
+                    }
+
+                    if (found) {
+                        pullTaskContext.setPullNextDelayTimeMillis(0); // no delay when messages were found
+                    } else {
+                        pullTaskContext.setPullNextDelayTimeMillis(delayWhenMessageNotFound);
+                    }
+                } catch (Exception e) {
+                    throw new RuntimeException(e);
+                }
+            }
+        });
+
+        try {
+            pullConsumerScheduleService.start();
+        } catch (MQClientException e) {
+            throw new RuntimeException(e);
+        }
+
+        runningChecker.setRunning(true);
+
+        awaitTermination();
+
+    }
+
+    private void awaitTermination() throws InterruptedException {
+        while (runningChecker.isRunning()) {
+            Thread.sleep(50);
+        }
+    }
+
+    private long getMessageQueueOffset(MessageQueue mq) throws MQClientException {
+        Long offset = offsetTable.get(mq);
+        // restoredOffsets(unionOffsetStates) is the restored global union state;
+        // should only snapshot mqs that actually belong to us
+        if (restored && offset == null) {
+            offset = restoredOffsets.get(mq);
+        }
+        if (offset == null) {
+            offset = consumer.fetchConsumeOffset(mq, false);
+            if (offset < 0) {
+                String initialOffset = props.getProperty(RocketMQConfig.CONSUMER_OFFSET_RESET_TO, CONSUMER_OFFSET_LATEST);
+                switch (initialOffset) {
+                    case CONSUMER_OFFSET_EARLIEST:
+                        offset = consumer.minOffset(mq);
+                        break;
+                    case CONSUMER_OFFSET_LATEST:
+                        offset = consumer.maxOffset(mq);
+                        break;
+                    case CONSUMER_OFFSET_TIMESTAMP:
+                        offset = consumer.searchOffset(mq, getLong(props,
+                            RocketMQConfig.CONSUMER_OFFSET_FROM_TIMESTAMP, System.currentTimeMillis()));
+                        break;
+                    default:
+                        throw new IllegalArgumentException("Unknown value for CONSUMER_OFFSET_RESET_TO.");
+                }
+            }
+            offsetTable.put(mq, offset);
+        }
+
+        return offsetTable.get(mq);
+    }
+
+    private void putMessageQueueOffset(MessageQueue mq, long offset) throws MQClientException {
+        offsetTable.put(mq, offset);
+        consumer.updateConsumeOffset(mq, offset);
+    }
+
+    @Override
+    public void cancel() {
+        LOG.debug("cancel ...");
+        runningChecker.setRunning(false);
+
+        if (pullConsumerScheduleService != null) {
+            pullConsumerScheduleService.shutdown();
+        }
+
+        offsetTable.clear();
+        restoredOffsets.clear();
+    }
+
+    @Override
+    public void close() throws Exception {
+        LOG.debug("close ...");
+        // pretty much the same logic as cancelling
+        try {
+            cancel();
+        } finally {
+            super.close();
+        }
+    }
+
+    @Override
+    public void snapshotState(FunctionSnapshotContext context) throws Exception {
+        // called when a snapshot for a checkpoint is requested
+
+        if (!runningChecker.isRunning()) {
+            LOG.debug("snapshotState() called on closed source; returning null.");
+            return;
+        }
+
+        if (LOG.isDebugEnabled()) {
+            LOG.debug("Snapshotting state {} ...", context.getCheckpointId());
+        }
+
+        unionOffsetStates.clear();
+
+        if (LOG.isDebugEnabled()) {
+            LOG.debug("Snapshotted state, last processed offsets: {}, checkpoint id: {}, timestamp: {}",
+                offsetTable, context.getCheckpointId(), context.getCheckpointTimestamp());
+        }
+
+        for (Map.Entry<MessageQueue, Long> entry : offsetTable.entrySet()) {
+            unionOffsetStates.add(Tuple2.of(entry.getKey(), entry.getValue()));
+        }
+    }
+
+    @Override
+    public void initializeState(FunctionInitializationContext context) throws Exception {
+        // called every time the user-defined function is initialized,
+        // be that when the function is first initialized or be that
+        // when the function is actually recovering from an earlier checkpoint.
+        // Given this, initializeState() is not only the place where different types of state are initialized,
+        // but also where state recovery logic is included.
+        LOG.debug("initialize State ...");
+
+        this.unionOffsetStates = context.getOperatorStateStore().getUnionListState(new ListStateDescriptor<>(
+            OFFSETS_STATE_NAME, TypeInformation.of(new TypeHint<Tuple2<MessageQueue, Long>>() { })));
+
+        this.restored = context.isRestored();
+
+        if (restored) {
+            if (restoredOffsets == null) {
+                restoredOffsets = new ConcurrentHashMap<>();
+            }
+            for (Tuple2<MessageQueue, Long> mqOffsets : unionOffsetStates.get()) {
+                // unionOffsetStates is the restored global union state;
+                // should only snapshot mqs that actually belong to us
+                restoredOffsets.put(mqOffsets.f0, mqOffsets.f1);
+            }
+            LOG.info("Setting restore state in the consumer. Using the following offsets: {}", restoredOffsets);
+        } else {
+            LOG.info("No restore state for the consumer.");
+        }
+    }
+
+    @Override
+    public TypeInformation<OUT> getProducedType() {
+        return schema.getProducedType();
+    }
+}
diff --git a/src/main/java/org/apache/rocketmq/flink/RocketMQUtils.java b/src/main/java/org/apache/rocketmq/flink/RocketMQUtils.java
new file mode 100644
index 0000000..9ca1de2
--- /dev/null
+++ b/src/main/java/org/apache/rocketmq/flink/RocketMQUtils.java
@@ -0,0 +1,36 @@
+/**
+ * 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;
+
+import java.util.Properties;
+
+public final class RocketMQUtils {
+
+    public static int getInteger(Properties props, String key, int defaultValue) {
+        return Integer.parseInt(props.getProperty(key, String.valueOf(defaultValue)));
+    }
+
+    public static long getLong(Properties props, String key, long defaultValue) {
+        return Long.parseLong(props.getProperty(key, String.valueOf(defaultValue)));
+    }
+
+    public static boolean getBoolean(Properties props, String key, boolean defaultValue) {
+        return Boolean.parseBoolean(props.getProperty(key, String.valueOf(defaultValue)));
+    }
+}
diff --git a/src/main/java/org/apache/rocketmq/flink/RunningChecker.java b/src/main/java/org/apache/rocketmq/flink/RunningChecker.java
new file mode 100644
index 0000000..b7bc2b9
--- /dev/null
+++ b/src/main/java/org/apache/rocketmq/flink/RunningChecker.java
@@ -0,0 +1,33 @@
+/**
+ * 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;
+
+import java.io.Serializable;
+
+public class RunningChecker implements Serializable {
+    private volatile boolean isRunning = false;
+
+    public boolean isRunning() {
+        return isRunning;
+    }
+
+    public void setRunning(boolean running) {
+        isRunning = running;
+    }
+}
diff --git a/src/main/java/org/apache/rocketmq/flink/common/selector/DefaultTopicSelector.java b/src/main/java/org/apache/rocketmq/flink/common/selector/DefaultTopicSelector.java
new file mode 100644
index 0000000..264d211
--- /dev/null
+++ b/src/main/java/org/apache/rocketmq/flink/common/selector/DefaultTopicSelector.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.rocketmq.flink.common.selector;
+
+public class DefaultTopicSelector<T> implements TopicSelector<T> {
+    private final String topicName;
+    private final String tagName;
+
+    public DefaultTopicSelector(final String topicName, final String tagName) {
+        this.topicName = topicName;
+        this.tagName = tagName;
+    }
+
+    public DefaultTopicSelector(final String topicName) {
+        this(topicName, "");
+    }
+
+    @Override
+    public String getTopic(T tuple) {
+        return topicName;
+    }
+
+    @Override
+    public String getTag(T tuple) {
+        return tagName;
+    }
+}
diff --git a/src/main/java/org/apache/rocketmq/flink/common/selector/SimpleTopicSelector.java b/src/main/java/org/apache/rocketmq/flink/common/selector/SimpleTopicSelector.java
new file mode 100644
index 0000000..3ad8a03
--- /dev/null
+++ b/src/main/java/org/apache/rocketmq/flink/common/selector/SimpleTopicSelector.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.rocketmq.flink.common.selector;
+
+import java.util.Map;
+
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+/**
+ * Uses field name to select topic and tag name from tuple.
+ */
+public class SimpleTopicSelector implements TopicSelector<Map> {
+    private static final Logger LOG = LoggerFactory.getLogger(SimpleTopicSelector.class);
+
+    private final String topicFieldName;
+    private final String defaultTopicName;
+
+    private final String tagFieldName;
+    private final String defaultTagName;
+
+    /**
+     * SimpleTopicSelector Constructor.
+     * @param topicFieldName field name used for selecting topic
+     * @param defaultTopicName default field name used for selecting topic
+     * @param tagFieldName field name used for selecting tag
+     * @param defaultTagName default field name used for selecting tag
+     */
+    public SimpleTopicSelector(String topicFieldName, String defaultTopicName, String tagFieldName, String defaultTagName) {
+        this.topicFieldName = topicFieldName;
+        this.defaultTopicName = defaultTopicName;
+        this.tagFieldName = tagFieldName;
+        this.defaultTagName = defaultTagName;
+    }
+
+    @Override
+    public String getTopic(Map tuple) {
+        if (tuple.containsKey(topicFieldName)) {
+            Object topic =  tuple.get(topicFieldName);
+            return topic != null ? topic.toString() : defaultTopicName;
+        } else {
+            LOG.warn("Field {} Not Found. Returning default topic {}", topicFieldName, defaultTopicName);
+            return defaultTopicName;
+        }
+    }
+
+    @Override
+    public String getTag(Map tuple) {
+        if (tuple.containsKey(tagFieldName)) {
+            Object tag = tuple.get(tagFieldName);
+            return tag != null ? tag.toString() : defaultTagName;
+        } else {
+            LOG.warn("Field {} Not Found. Returning default tag {}", tagFieldName, defaultTagName);
+            return defaultTagName;
+        }
+    }
+}
diff --git a/src/main/java/org/apache/rocketmq/flink/common/selector/TopicSelector.java b/src/main/java/org/apache/rocketmq/flink/common/selector/TopicSelector.java
new file mode 100644
index 0000000..2a347db
--- /dev/null
+++ b/src/main/java/org/apache/rocketmq/flink/common/selector/TopicSelector.java
@@ -0,0 +1,28 @@
+/**
+ * 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.selector;
+
+import java.io.Serializable;
+
+public interface TopicSelector<T> extends Serializable {
+
+    String getTopic(T tuple);
+
+    String getTag(T tuple);
+}
diff --git a/src/main/java/org/apache/rocketmq/flink/common/serialization/KeyValueDeserializationSchema.java b/src/main/java/org/apache/rocketmq/flink/common/serialization/KeyValueDeserializationSchema.java
new file mode 100644
index 0000000..d8759f9
--- /dev/null
+++ b/src/main/java/org/apache/rocketmq/flink/common/serialization/KeyValueDeserializationSchema.java
@@ -0,0 +1,27 @@
+/**
+ * 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.serialization;
+
+import java.io.Serializable;
+
+import org.apache.flink.api.java.typeutils.ResultTypeQueryable;
+
+public interface KeyValueDeserializationSchema<T> extends ResultTypeQueryable<T>, Serializable {
+    T deserializeKeyAndValue(byte[] key, byte[] value);
+}
diff --git a/src/main/java/org/apache/rocketmq/flink/common/serialization/KeyValueSerializationSchema.java b/src/main/java/org/apache/rocketmq/flink/common/serialization/KeyValueSerializationSchema.java
new file mode 100644
index 0000000..d847e8a
--- /dev/null
+++ b/src/main/java/org/apache/rocketmq/flink/common/serialization/KeyValueSerializationSchema.java
@@ -0,0 +1,28 @@
+/**
+ * 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.serialization;
+
+import java.io.Serializable;
+
+public interface KeyValueSerializationSchema<T> extends Serializable {
+
+    byte[] serializeKey(T tuple);
+
+    byte[] serializeValue(T tuple);
+}
diff --git a/src/main/java/org/apache/rocketmq/flink/common/serialization/SimpleKeyValueDeserializationSchema.java b/src/main/java/org/apache/rocketmq/flink/common/serialization/SimpleKeyValueDeserializationSchema.java
new file mode 100644
index 0000000..df6390b
--- /dev/null
+++ b/src/main/java/org/apache/rocketmq/flink/common/serialization/SimpleKeyValueDeserializationSchema.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.rocketmq.flink.common.serialization;
+
+import java.nio.charset.StandardCharsets;
+import java.util.HashMap;
+import java.util.Map;
+
+import org.apache.flink.api.common.typeinfo.TypeInformation;
+
+public class SimpleKeyValueDeserializationSchema implements KeyValueDeserializationSchema<Map> {
+    public static final String DEFAULT_KEY_FIELD = "key";
+    public static final String DEFAULT_VALUE_FIELD = "value";
+
+    public String keyField;
+    public String valueField;
+
+    public SimpleKeyValueDeserializationSchema() {
+        this(DEFAULT_KEY_FIELD, DEFAULT_VALUE_FIELD);
+    }
+
+    /**
+     * SimpleKeyValueDeserializationSchema Constructor.
+     * @param keyField tuple field for selecting the key
+     * @param valueField  tuple field for selecting the value
+     */
+    public SimpleKeyValueDeserializationSchema(String keyField, String valueField) {
+        this.keyField = keyField;
+        this.valueField = valueField;
+    }
+
+    @Override
+    public Map deserializeKeyAndValue(byte[] key, byte[] value) {
+        HashMap map = new HashMap(2);
+        if (keyField != null) {
+            String k = key != null ? new String(key, StandardCharsets.UTF_8) : null;
+            map.put(keyField, k);
+        }
+        if (valueField != null) {
+            String v = value != null ? new String(value, StandardCharsets.UTF_8) : null;
+            map.put(valueField, v);
+        }
+        return map;
+    }
+
+    @Override
+    public TypeInformation<Map> getProducedType() {
+        return TypeInformation.of(Map.class);
+    }
+}
diff --git a/src/main/java/org/apache/rocketmq/flink/common/serialization/SimpleKeyValueSerializationSchema.java b/src/main/java/org/apache/rocketmq/flink/common/serialization/SimpleKeyValueSerializationSchema.java
new file mode 100644
index 0000000..bbd6da3
--- /dev/null
+++ b/src/main/java/org/apache/rocketmq/flink/common/serialization/SimpleKeyValueSerializationSchema.java
@@ -0,0 +1,63 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.rocketmq.flink.common.serialization;
+
+import java.nio.charset.StandardCharsets;
+import java.util.Map;
+
+public class SimpleKeyValueSerializationSchema implements KeyValueSerializationSchema<Map> {
+    public static final String DEFAULT_KEY_FIELD = "key";
+    public static final String DEFAULT_VALUE_FIELD = "value";
+
+    public String keyField;
+    public String valueField;
+
+    public SimpleKeyValueSerializationSchema() {
+        this(DEFAULT_KEY_FIELD, DEFAULT_VALUE_FIELD);
+    }
+
+    /**
+     * SimpleKeyValueSerializationSchema Constructor.
+     * @param keyField tuple field for selecting the key
+     * @param valueField  tuple field for selecting the value
+     */
+    public SimpleKeyValueSerializationSchema(String keyField, String valueField) {
+        this.keyField = keyField;
+        this.valueField = valueField;
+    }
+
+    @Override
+    public byte[] serializeKey(Map tuple) {
+        if (tuple == null || keyField == null) {
+            return null;
+        }
+        Object key = tuple.get(keyField);
+        return key != null ? key.toString().getBytes(StandardCharsets.UTF_8) : null;
+    }
+
+    @Override
+    public byte[] serializeValue(Map tuple) {
+        if (tuple == null || valueField == null) {
+            return null;
+        }
+        Object value = tuple.get(valueField);
+        return value != null ? value.toString().getBytes(StandardCharsets.UTF_8) : null;
+    }
+
+}
diff --git a/src/test/java/org/apache/rocketmq/flink/RocketMQSinkTest.java b/src/test/java/org/apache/rocketmq/flink/RocketMQSinkTest.java
new file mode 100644
index 0000000..ec844f2
--- /dev/null
+++ b/src/test/java/org/apache/rocketmq/flink/RocketMQSinkTest.java
@@ -0,0 +1,75 @@
+/**
+ * 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;
+
+import java.util.HashMap;
+import java.util.Map;
+import java.util.Properties;
+
+import org.apache.rocketmq.client.producer.DefaultMQProducer;
+import org.apache.rocketmq.common.message.Message;
+import org.apache.rocketmq.flink.common.selector.DefaultTopicSelector;
+import org.apache.rocketmq.flink.common.selector.TopicSelector;
+import org.apache.rocketmq.flink.common.serialization.KeyValueSerializationSchema;
+import org.apache.rocketmq.flink.common.serialization.SimpleKeyValueSerializationSchema;
+import org.junit.Before;
+import org.junit.Test;
+
+import static org.apache.rocketmq.flink.TestUtils.setFieldValue;
+import static org.mockito.Matchers.any;
+import static org.mockito.Mockito.mock;
+import static org.mockito.Mockito.verify;
+
+public class RocketMQSinkTest {
+
+    private RocketMQSink rocketMQSink;
+    private DefaultMQProducer producer;
+
+    @Before
+    public void setUp() throws Exception {
+        KeyValueSerializationSchema serializationSchema = new SimpleKeyValueSerializationSchema("id", "name");
+        TopicSelector topicSelector = new DefaultTopicSelector("tpc");
+        Properties props = new Properties();
+        rocketMQSink = new RocketMQSink(serializationSchema, topicSelector, props);
+
+        producer = mock(DefaultMQProducer.class);
+        setFieldValue(rocketMQSink, "producer", producer);
+    }
+
+    @Test
+    public void testSink() throws Exception {
+        Map tuple = new HashMap();
+        tuple.put("id", "x001");
+        tuple.put("name", "vesense");
+        tuple.put("tpc", "tpc1");
+
+        rocketMQSink.invoke(tuple, null);
+
+        verify(producer).send(any(Message.class));
+
+    }
+
+    @Test
+    public void close() throws Exception {
+        rocketMQSink.close();
+
+        verify(producer).shutdown();
+    }
+
+}
\ No newline at end of file
diff --git a/src/test/java/org/apache/rocketmq/flink/RocketMQSourceTest.java b/src/test/java/org/apache/rocketmq/flink/RocketMQSourceTest.java
new file mode 100644
index 0000000..b7aaee0
--- /dev/null
+++ b/src/test/java/org/apache/rocketmq/flink/RocketMQSourceTest.java
@@ -0,0 +1,121 @@
+/**
+ * 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;
+
+import java.util.ArrayList;
+import java.util.HashSet;
+import java.util.List;
+import java.util.Properties;
+import java.util.Set;
+import java.util.concurrent.ConcurrentHashMap;
+
+import org.apache.flink.streaming.api.functions.source.SourceFunction.SourceContext;
+import org.apache.rocketmq.client.consumer.DefaultMQPullConsumer;
+import org.apache.rocketmq.client.consumer.MQPullConsumerScheduleService;
+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.common.serialization.KeyValueDeserializationSchema;
+import org.apache.rocketmq.flink.common.serialization.SimpleKeyValueDeserializationSchema;
+import org.junit.Before;
+import org.junit.Test;
+
+import static org.apache.rocketmq.flink.TestUtils.setFieldValue;
+import static org.mockito.Matchers.any;
+import static org.mockito.Matchers.anyBoolean;
+import static org.mockito.Matchers.anyInt;
+import static org.mockito.Matchers.anyLong;
+import static org.mockito.Matchers.anyString;
+import static org.mockito.Mockito.atLeastOnce;
+import static org.mockito.Mockito.mock;
+import static org.mockito.Mockito.verify;
+import static org.mockito.Mockito.when;
+
+public class RocketMQSourceTest {
+
+    private RocketMQSource rocketMQSource;
+    private MQPullConsumerScheduleService pullConsumerScheduleService;
+    private DefaultMQPullConsumer consumer;
+    private KeyValueDeserializationSchema deserializationSchema;
+    private String topic = "tpc";
+
+    @Before
+    public void setUp() throws Exception {
+        deserializationSchema = new SimpleKeyValueDeserializationSchema();
+        Properties props = new Properties();
+        rocketMQSource = new RocketMQSource(deserializationSchema, props);
+
+        setFieldValue(rocketMQSource, "topic", topic);
+        setFieldValue(rocketMQSource, "runningChecker", new SingleRunningCheck());
+        setFieldValue(rocketMQSource, "offsetTable", new ConcurrentHashMap<>());
+        setFieldValue(rocketMQSource, "restoredOffsets", new ConcurrentHashMap<>());
+
+        pullConsumerScheduleService = new MQPullConsumerScheduleService("g");
+
+        consumer = mock(DefaultMQPullConsumer.class);
+        pullConsumerScheduleService.setDefaultMQPullConsumer(consumer);
+        setFieldValue(rocketMQSource, "consumer", consumer);
+        setFieldValue(rocketMQSource, "pullConsumerScheduleService", pullConsumerScheduleService);
+    }
+
+    @Test
+    public void testSource() throws Exception {
+        List<MessageExt> msgFoundList = new ArrayList<>();
+        MessageExt messageExt = new MessageExt();
+        messageExt.setKeys("keys");
+        messageExt.setBody("body data".getBytes());
+        messageExt.setBornTimestamp(System.currentTimeMillis());
+        msgFoundList.add(messageExt);
+        PullResult pullResult = new PullResult(PullStatus.FOUND, 3, 1, 5, msgFoundList);
+
+        when(consumer.fetchConsumeOffset(any(MessageQueue.class), anyBoolean())).thenReturn(2L);
+        when(consumer.pull(any(MessageQueue.class), anyString(), anyLong(), anyInt())).thenReturn(pullResult);
+
+        SourceContext context = mock(SourceContext.class);
+        when(context.getCheckpointLock()).thenReturn(new Object());
+
+        rocketMQSource.run(context);
+
+        // schedule the pull task
+        Set<MessageQueue> set = new HashSet();
+        set.add(new MessageQueue(topic, "brk", 1));
+        pullConsumerScheduleService.putTask(topic, set);
+
+        MessageExt msg = pullResult.getMsgFoundList().get(0);
+
+        // atLeastOnce: re-pulling immediately when messages found before
+        verify(context, atLeastOnce()).collectWithTimestamp(deserializationSchema.deserializeKeyAndValue(msg.getKeys().getBytes(),
+            msg.getBody()), msg.getBornTimestamp());
+    }
+
+    @Test
+    public void close() throws Exception {
+        rocketMQSource.close();
+
+        verify(consumer).shutdown();
+    }
+
+    class SingleRunningCheck extends RunningChecker {
+        @Override
+        public boolean isRunning() {
+            return false;
+        }
+    }
+}
\ No newline at end of file
diff --git a/src/test/java/org/apache/rocketmq/flink/TestUtils.java b/src/test/java/org/apache/rocketmq/flink/TestUtils.java
new file mode 100644
index 0000000..d0a9450
--- /dev/null
+++ b/src/test/java/org/apache/rocketmq/flink/TestUtils.java
@@ -0,0 +1,33 @@
+/**
+ * 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;
+
+import java.lang.reflect.Field;
+
+public class TestUtils {
+    public static void setFieldValue(Object obj, String fieldName, Object value) {
+        try {
+            Field field = obj.getClass().getDeclaredField(fieldName);
+            field.setAccessible(true);
+            field.set(obj, value);
+        } catch (Exception e) {
+            e.printStackTrace();
+        }
+    }
+}
diff --git a/src/test/java/org/apache/rocketmq/flink/common/selector/DefaultTopicSelectorTest.java b/src/test/java/org/apache/rocketmq/flink/common/selector/DefaultTopicSelectorTest.java
new file mode 100644
index 0000000..2f4685c
--- /dev/null
+++ b/src/test/java/org/apache/rocketmq/flink/common/selector/DefaultTopicSelectorTest.java
@@ -0,0 +1,37 @@
+/**
+ * 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.selector;
+
+import org.junit.Test;
+
+import static org.junit.Assert.*;
+
+public class DefaultTopicSelectorTest {
+    @Test
+    public void getTopic() throws Exception {
+        DefaultTopicSelector selector = new DefaultTopicSelector("rocket");
+        assertEquals("rocket", selector.getTopic(null));
+        assertEquals("", selector.getTag(null));
+
+        selector = new DefaultTopicSelector("rocket", "tg");
+        assertEquals("rocket", selector.getTopic(null));
+        assertEquals("tg", selector.getTag(null));
+    }
+
+}
\ No newline at end of file
diff --git a/src/test/java/org/apache/rocketmq/flink/common/selector/SimpleTopicSelectorTest.java b/src/test/java/org/apache/rocketmq/flink/common/selector/SimpleTopicSelectorTest.java
new file mode 100644
index 0000000..6ac1a57
--- /dev/null
+++ b/src/test/java/org/apache/rocketmq/flink/common/selector/SimpleTopicSelectorTest.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.rocketmq.flink.common.selector;
+
+import java.util.HashMap;
+import java.util.Map;
+
+import org.junit.Test;
+
+import static org.junit.Assert.*;
+
+public class SimpleTopicSelectorTest {
+    @Test
+    public void getTopic() throws Exception {
+        SimpleTopicSelector selector = new SimpleTopicSelector("tpc", "dtpc", "tg", "dtg");
+        Map tuple = new HashMap();
+        tuple.put("id", "x001");
+        tuple.put("name", "vesense");
+        tuple.put("tpc", "tpc1");
+        tuple.put("tg", "tg1");
+
+        assertEquals("tpc1", selector.getTopic(tuple));
+        assertEquals("tg1", selector.getTag(tuple));
+
+        tuple = new HashMap();
+        tuple.put("id", "x001");
+        tuple.put("name", "vesense");
+
+        assertEquals("dtpc", selector.getTopic(tuple));
+        assertEquals("dtg", selector.getTag(tuple));
+    }
+
+}
\ No newline at end of file
diff --git a/src/test/java/org/apache/rocketmq/flink/common/serialization/SimpleKeyValueSerializationSchemaTest.java b/src/test/java/org/apache/rocketmq/flink/common/serialization/SimpleKeyValueSerializationSchemaTest.java
new file mode 100644
index 0000000..98aa793
--- /dev/null
+++ b/src/test/java/org/apache/rocketmq/flink/common/serialization/SimpleKeyValueSerializationSchemaTest.java
@@ -0,0 +1,42 @@
+/**
+ * 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.serialization;
+
+import java.util.HashMap;
+import java.util.Map;
+
+import org.junit.Test;
+
+import static org.junit.Assert.*;
+
+public class SimpleKeyValueSerializationSchemaTest {
+    @Test
+    public void serializeKeyAndValue() throws Exception {
+        SimpleKeyValueSerializationSchema serializationSchema = new SimpleKeyValueSerializationSchema("id", "name");
+        SimpleKeyValueDeserializationSchema deserializationSchema = new SimpleKeyValueDeserializationSchema("id", "name");
+
+        Map tuple = new HashMap();
+        tuple.put("id", "x001");
+        tuple.put("name", "vesense");
+
+        assertEquals(tuple, deserializationSchema.deserializeKeyAndValue(serializationSchema.serializeKey(tuple),
+            serializationSchema.serializeValue(tuple)));
+    }
+
+}
\ No newline at end of file
diff --git a/src/test/java/org/apache/rocketmq/flink/example/ConsumerTest.java b/src/test/java/org/apache/rocketmq/flink/example/ConsumerTest.java
new file mode 100644
index 0000000..1b07b8d
--- /dev/null
+++ b/src/test/java/org/apache/rocketmq/flink/example/ConsumerTest.java
@@ -0,0 +1,54 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.rocketmq.flink.example;
+
+import java.util.List;
+
+import org.apache.rocketmq.client.consumer.DefaultMQPushConsumer;
+import org.apache.rocketmq.client.consumer.listener.ConsumeConcurrentlyContext;
+import org.apache.rocketmq.client.consumer.listener.ConsumeConcurrentlyStatus;
+import org.apache.rocketmq.client.consumer.listener.MessageListenerConcurrently;
+import org.apache.rocketmq.client.exception.MQClientException;
+import org.apache.rocketmq.common.message.MessageExt;
+
+public class ConsumerTest {
+    public static void main(String[] args) {
+        DefaultMQPushConsumer consumer = new DefaultMQPushConsumer("g00003");
+        consumer.setNamesrvAddr("localhost:9876");
+        try {
+            consumer.subscribe("flink-sink2", "*");
+        } catch (MQClientException e) {
+            e.printStackTrace();
+        }
+        consumer.registerMessageListener(new MessageListenerConcurrently() {
+            @Override
+            public ConsumeConcurrentlyStatus consumeMessage(List<MessageExt> msgs, ConsumeConcurrentlyContext context) {
+                for (MessageExt msg : msgs) {
+                    System.out.println(msg.getKeys() + ":" + new String(msg.getBody()));
+                }
+                return ConsumeConcurrentlyStatus.CONSUME_SUCCESS;
+            }
+        });
+        try {
+            consumer.start();
+        } catch (MQClientException e) {
+            e.printStackTrace();
+        }
+    }
+}
diff --git a/src/test/java/org/apache/rocketmq/flink/example/ProducerTest.java b/src/test/java/org/apache/rocketmq/flink/example/ProducerTest.java
new file mode 100644
index 0000000..c04ca74
--- /dev/null
+++ b/src/test/java/org/apache/rocketmq/flink/example/ProducerTest.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.rocketmq.flink.example;
+
+import org.apache.rocketmq.client.exception.MQBrokerException;
+import org.apache.rocketmq.client.exception.MQClientException;
+import org.apache.rocketmq.client.producer.DefaultMQProducer;
+import org.apache.rocketmq.common.message.Message;
+import org.apache.rocketmq.remoting.exception.RemotingException;
+
+public class ProducerTest {
+    public static void main(String[] args) {
+        DefaultMQProducer producer = new DefaultMQProducer("p001");
+        producer.setNamesrvAddr("localhost:9876");
+        try {
+            producer.start();
+        } catch (MQClientException e) {
+            e.printStackTrace();
+        }
+        for (int i = 0; i < 10000; i++) {
+            Message msg = new Message("flink-source2" , "", "id_"+i, ("country_X province_" + i).getBytes());
+            try {
+                producer.send(msg);
+            } catch (MQClientException e) {
+                e.printStackTrace();
+            } catch (RemotingException e) {
+                e.printStackTrace();
+            } catch (MQBrokerException e) {
+                e.printStackTrace();
+            } catch (InterruptedException e) {
+                e.printStackTrace();
+            }
+            System.out.println("send " + i);
+            try {
+                Thread.sleep(10);
+            } catch (InterruptedException e) {
+                e.printStackTrace();
+            }
+        }
+    }
+}
diff --git a/src/test/java/org/apache/rocketmq/flink/example/RocketMQFlinkExample.java b/src/test/java/org/apache/rocketmq/flink/example/RocketMQFlinkExample.java
new file mode 100644
index 0000000..b2a4034
--- /dev/null
+++ b/src/test/java/org/apache/rocketmq/flink/example/RocketMQFlinkExample.java
@@ -0,0 +1,76 @@
+/**
+ * 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.example;
+
+import java.util.HashMap;
+import java.util.Map;
+import java.util.Properties;
+
+import org.apache.flink.streaming.api.environment.StreamExecutionEnvironment;
+import org.apache.flink.streaming.api.functions.ProcessFunction;
+import org.apache.flink.util.Collector;
+import org.apache.rocketmq.flink.RocketMQConfig;
+import org.apache.rocketmq.flink.RocketMQSink;
+import org.apache.rocketmq.flink.RocketMQSource;
+import org.apache.rocketmq.flink.common.selector.DefaultTopicSelector;
+import org.apache.rocketmq.flink.common.serialization.SimpleKeyValueDeserializationSchema;
+import org.apache.rocketmq.flink.common.serialization.SimpleKeyValueSerializationSchema;
+
+public class RocketMQFlinkExample {
+    public static void main(String[] args) {
+        StreamExecutionEnvironment env = StreamExecutionEnvironment.getExecutionEnvironment();
+
+        // enable checkpoint
+        env.enableCheckpointing(3000);
+
+        Properties consumerProps = new Properties();
+        consumerProps.setProperty(RocketMQConfig.NAME_SERVER_ADDR, "localhost:9876");
+        consumerProps.setProperty(RocketMQConfig.CONSUMER_GROUP, "c002");
+        consumerProps.setProperty(RocketMQConfig.CONSUMER_TOPIC, "flink-source2");
+
+        Properties producerProps = new Properties();
+        producerProps.setProperty(RocketMQConfig.NAME_SERVER_ADDR, "localhost:9876");
+
+        env.addSource(new RocketMQSource(new SimpleKeyValueDeserializationSchema("id", "address"), consumerProps))
+            .name("rocketmq-source")
+            .setParallelism(2)
+            .process(new ProcessFunction<Map, Map>() {
+                @Override
+                public void processElement(Map in, Context ctx, Collector<Map> out) throws Exception {
+                    HashMap result = new HashMap();
+                    result.put("id", in.get("id"));
+                    String[] arr = in.get("address").toString().split("\\s+");
+                    result.put("province", arr[arr.length-1]);
+                    out.collect(result);
+                }
+            })
+            .name("upper-processor")
+            .setParallelism(2)
+            .addSink(new RocketMQSink(new SimpleKeyValueSerializationSchema("id", "province"),
+                new DefaultTopicSelector("flink-sink2"), producerProps).withBatchFlushOnCheckpoint(true))
+            .name("rocketmq-sink")
+            .setParallelism(2);
+
+        try {
+            env.execute("rocketmq-flink-example");
+        } catch (Exception e) {
+            e.printStackTrace();
+        }
+    }
+}
diff --git a/style/copyright/Apache.xml b/style/copyright/Apache.xml
new file mode 100644
index 0000000..2db86d0
--- /dev/null
+++ b/style/copyright/Apache.xml
@@ -0,0 +1,24 @@
+<!--
+  ~ 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.
+  -->
+
+<component name="CopyrightManager">
+    <copyright>
+        <option name="myName" value="Apache"/>
+        <option name="notice"
+                value="Licensed to the Apache Software Foundation (ASF) under one or more&#10;contributor license agreements.  See the NOTICE file distributed with&#10;this work for additional information regarding copyright ownership.&#10;The ASF licenses this file to You under the Apache License, Version 2.0&#10;(the &quot;License&quot;); you may not use this file except in compliance with&#10;the License.  You may obtain a copy of the License at&#10;&#10;    http://www.apache.org/lice [...]
+    </copyright>
+</component>
\ No newline at end of file
diff --git a/style/copyright/profiles_settings.xml b/style/copyright/profiles_settings.xml
new file mode 100644
index 0000000..4c0e521
--- /dev/null
+++ b/style/copyright/profiles_settings.xml
@@ -0,0 +1,64 @@
+<!--
+  ~ 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.
+  -->
+
+<component name="CopyrightManager">
+    <settings default="Apache">
+        <module2copyright>
+            <element module="All" copyright="Apache"/>
+        </module2copyright>
+        <LanguageOptions name="GSP">
+            <option name="fileTypeOverride" value="3"/>
+            <option name="prefixLines" value="false"/>
+        </LanguageOptions>
+        <LanguageOptions name="HTML">
+            <option name="fileTypeOverride" value="3"/>
+            <option name="prefixLines" value="false"/>
+        </LanguageOptions>
+        <LanguageOptions name="JAVA">
+            <option name="fileTypeOverride" value="3"/>
+            <option name="addBlankAfter" value="false"/>
+        </LanguageOptions>
+        <LanguageOptions name="JSP">
+            <option name="fileTypeOverride" value="3"/>
+            <option name="prefixLines" value="false"/>
+        </LanguageOptions>
+        <LanguageOptions name="JSPX">
+            <option name="fileTypeOverride" value="3"/>
+            <option name="prefixLines" value="false"/>
+        </LanguageOptions>
+        <LanguageOptions name="MXML">
+            <option name="fileTypeOverride" value="3"/>
+            <option name="prefixLines" value="false"/>
+        </LanguageOptions>
+        <LanguageOptions name="Properties">
+            <option name="fileTypeOverride" value="3"/>
+            <option name="block" value="false"/>
+        </LanguageOptions>
+        <LanguageOptions name="SPI">
+            <option name="fileTypeOverride" value="3"/>
+            <option name="block" value="false"/>
+        </LanguageOptions>
+        <LanguageOptions name="XML">
+            <option name="fileTypeOverride" value="3"/>
+            <option name="prefixLines" value="false"/>
+        </LanguageOptions>
+        <LanguageOptions name="__TEMPLATE__">
+            <option name="separateBefore" value="true"/>
+            <option name="lenBefore" value="1"/>
+        </LanguageOptions>
+    </settings>
+</component>
\ No newline at end of file
diff --git a/style/rmq_checkstyle.xml b/style/rmq_checkstyle.xml
new file mode 100644
index 0000000..e3155cc
--- /dev/null
+++ b/style/rmq_checkstyle.xml
@@ -0,0 +1,135 @@
+<?xml version="1.0" encoding="UTF-8"?>
+<!--
+  ~ Licensed to the Apache Software Foundation (ASF) under one or more
+  ~ contributor license agreements.  See the NOTICE file distributed with
+  ~ this work for additional information regarding copyright ownership.
+  ~ The ASF licenses this file to You under the Apache License, Version 2.0
+  ~ (the "License"); you may not use this file except in compliance with
+  ~ the License.  You may obtain a copy of the License at
+  ~
+  ~     http://www.apache.org/licenses/LICENSE-2.0
+  ~
+  ~  Unless required by applicable law or agreed to in writing, software
+  ~  distributed under the License is distributed on an "AS IS" BASIS,
+  ~  WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+  ~  See the License for the specific language governing permissions and
+  ~  limitations under the License.
+  -->
+
+<!DOCTYPE module PUBLIC
+    "-//Puppy Crawl//DTD Check Configuration 1.3//EN"
+    "http://www.puppycrawl.com/dtds/configuration_1_3.dtd">
+<!--Refer http://checkstyle.sourceforge.net/reports/google-java-style.html#s2.2-file-encoding -->
+<module name="Checker">
+
+    <property name="localeLanguage" value="en"/>
+
+    <!--To configure the check to report on the first instance in each file-->
+    <module name="FileTabCharacter"/>
+
+    <!-- header -->
+    <module name="RegexpHeader">
+        <property name="header" value="/\*\nLicensed to the Apache Software Foundation*"/>
+    </module>
+
+    <module name="RegexpSingleline">
+        <property name="format" value="System\.out\.println"/>
+        <property name="message" value="Prohibit invoking System.out.println in source code !"/>
+    </module>
+
+    <module name="RegexpSingleline">
+        <property name="format" value="//FIXME"/>
+        <property name="message" value="Recommended fix FIXME task !"/>
+    </module>
+
+    <module name="RegexpSingleline">
+        <property name="format" value="//TODO"/>
+        <property name="message" value="Recommended fix TODO task !"/>
+    </module>
+
+    <module name="RegexpSingleline">
+        <property name="format" value="@alibaba"/>
+        <property name="message" value="Recommended remove @alibaba keyword!"/>
+    </module>
+    <module name="RegexpSingleline">
+        <property name="format" value="@taobao"/>
+        <property name="message" value="Recommended remove @taobao keyword!"/>
+    </module>
+    <module name="RegexpSingleline">
+        <property name="format" value="@author"/>
+        <property name="message" value="Recommended remove @author tag in javadoc!"/>
+    </module>
+
+    <module name="RegexpSingleline">
+        <property name="format"
+                  value=".*[\u3400-\u4DB5\u4E00-\u9FA5\u9FA6-\u9FBB\uF900-\uFA2D\uFA30-\uFA6A\uFA70-\uFAD9\uFF00-\uFFEF\u2E80-\u2EFF\u3000-\u303F\u31C0-\u31EF]+.*"/>
+        <property name="message" value="Not allow chinese character !"/>
+    </module>
+
+    <module name="FileLength">
+        <property name="max" value="3000"/>
+    </module>
+
+    <module name="TreeWalker">
+
+        <module name="UnusedImports">
+            <property name="processJavadoc" value="true"/>
+        </module>
+        <module name="RedundantImport"/>
+
+        <!--<module name="IllegalImport" />-->
+
+        <!--Checks that classes that override equals() also override hashCode()-->
+        <module name="EqualsHashCode"/>
+        <!--Checks for over-complicated boolean expressions. Currently finds code like if (topic == true), topic || true, !false, etc.-->
+        <module name="SimplifyBooleanExpression"/>
+        <module name="OneStatementPerLine"/>
+        <module name="UnnecessaryParentheses"/>
+        <!--Checks for over-complicated boolean return statements. For example the following code-->
+        <module name="SimplifyBooleanReturn"/>
+
+        <!--Check that the default is after all the cases in producerGroup switch statement-->
+        <module name="DefaultComesLast"/>
+        <!--Detects empty statements (standalone ";" semicolon)-->
+        <module name="EmptyStatement"/>
+        <!--Checks that long constants are defined with an upper ell-->
+        <module name="UpperEll"/>
+        <module name="ConstantName">
+            <property name="format" value="(^[A-Z][A-Z0-9]*(_[A-Z0-9]+)*$)|(^log$)"/>
+        </module>
+        <!--Checks that local, non-final variable names conform to producerGroup format specified by the format property-->
+        <module name="LocalVariableName"/>
+        <!--Validates identifiers for local, final variables, including catch parameters-->
+        <module name="LocalFinalVariableName"/>
+        <!--Validates identifiers for non-static fields-->
+        <module name="MemberName"/>
+        <!--Validates identifiers for class type parameters-->
+        <module name="ClassTypeParameterName">
+            <property name="format" value="^[A-Z0-9]*$"/>
+        </module>
+        <!--Validates identifiers for method type parameters-->
+        <module name="MethodTypeParameterName">
+            <property name="format" value="^[A-Z0-9]*$"/>
+        </module>
+        <module name="PackageName"/>
+        <module name="ParameterName"/>
+        <module name="StaticVariableName"/>
+        <module name="TypeName"/>
+        <!--Checks that there are no import statements that use the * notation-->
+        <module name="AvoidStarImport"/>
+
+        <!--whitespace-->
+        <module name="GenericWhitespace"/>
+        <module name="NoWhitespaceBefore"/>
+        <module name="WhitespaceAfter"/>
+        <module name="NoWhitespaceAfter"/>
+        <module name="WhitespaceAround">
+            <property name="allowEmptyConstructors" value="true"/>
+            <property name="allowEmptyMethods" value="true"/>
+        </module>
+        <module name="Indentation"/>
+        <module name="MethodParamPad"/>
+        <module name="ParenPad"/>
+        <module name="TypecastParenPad"/>
+    </module>
+</module>
diff --git a/style/rmq_codeStyle.xml b/style/rmq_codeStyle.xml
new file mode 100644
index 0000000..cd95ee6
--- /dev/null
+++ b/style/rmq_codeStyle.xml
@@ -0,0 +1,157 @@
+<!--
+  ~ 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.
+  -->
+
+<code_scheme name="rocketmq">
+    <option name="USE_SAME_INDENTS" value="true"/>
+    <option name="IGNORE_SAME_INDENTS_FOR_LANGUAGES" value="true"/>
+    <option name="OTHER_INDENT_OPTIONS">
+        <value>
+            <option name="INDENT_SIZE" value="4"/>
+            <option name="CONTINUATION_INDENT_SIZE" value="4"/>
+            <option name="TAB_SIZE" value="4"/>
+            <option name="USE_TAB_CHARACTER" value="false"/>
+            <option name="SMART_TABS" value="false"/>
+            <option name="LABEL_INDENT_SIZE" value="0"/>
+            <option name="LABEL_INDENT_ABSOLUTE" value="false"/>
+            <option name="USE_RELATIVE_INDENTS" value="false"/>
+        </value>
+    </option>
+    <option name="PREFER_LONGER_NAMES" value="false"/>
+    <option name="CLASS_COUNT_TO_USE_IMPORT_ON_DEMAND" value="1000"/>
+    <option name="NAMES_COUNT_TO_USE_IMPORT_ON_DEMAND" value="1000"/>
+    <option name="PACKAGES_TO_USE_IMPORT_ON_DEMAND">
+        <value/>
+    </option>
+    <option name="IMPORT_LAYOUT_TABLE">
+        <value>
+            <package name="" withSubpackages="true" static="false"/>
+            <emptyLine/>
+            <package name="" withSubpackages="true" static="true"/>
+        </value>
+    </option>
+    <option name="JD_ALIGN_PARAM_COMMENTS" value="false"/>
+    <option name="JD_ALIGN_EXCEPTION_COMMENTS" value="false"/>
+    <option name="JD_P_AT_EMPTY_LINES" value="false"/>
+    <option name="JD_KEEP_INVALID_TAGS" value="false"/>
+    <option name="JD_DO_NOT_WRAP_ONE_LINE_COMMENTS" value="true"/>
+    <option name="KEEP_CONTROL_STATEMENT_IN_ONE_LINE" value="false"/>
+    <option name="KEEP_BLANK_LINES_IN_DECLARATIONS" value="1"/>
+    <option name="KEEP_BLANK_LINES_IN_CODE" value="1"/>
+    <option name="KEEP_BLANK_LINES_BEFORE_RBRACE" value="1"/>
+    <option name="ELSE_ON_NEW_LINE" value="true"/>
+    <option name="WHILE_ON_NEW_LINE" value="true"/>
+    <option name="CATCH_ON_NEW_LINE" value="true"/>
+    <option name="FINALLY_ON_NEW_LINE" value="true"/>
+    <option name="ALIGN_MULTILINE_PARAMETERS" value="false"/>
+    <option name="ALIGN_MULTILINE_FOR" value="false"/>
+    <option name="SPACE_AFTER_TYPE_CAST" value="false"/>
+    <option name="SPACE_BEFORE_ARRAY_INITIALIZER_LBRACE" value="true"/>
+    <option name="METHOD_PARAMETERS_WRAP" value="1"/>
+    <option name="ARRAY_INITIALIZER_LBRACE_ON_NEXT_LINE" value="true"/>
+    <option name="LABELED_STATEMENT_WRAP" value="1"/>
+    <option name="WRAP_COMMENTS" value="true"/>
+    <option name="METHOD_ANNOTATION_WRAP" value="1"/>
+    <option name="CLASS_ANNOTATION_WRAP" value="1"/>
+    <option name="FIELD_ANNOTATION_WRAP" value="1"/>
+    <JavaCodeStyleSettings>
+        <option name="CLASS_NAMES_IN_JAVADOC" value="3"/>
+    </JavaCodeStyleSettings>
+    <XML>
+        <option name="XML_LEGACY_SETTINGS_IMPORTED" value="true"/>
+    </XML>
+    <ADDITIONAL_INDENT_OPTIONS fileType="haml">
+        <option name="INDENT_SIZE" value="2"/>
+    </ADDITIONAL_INDENT_OPTIONS>
+    <codeStyleSettings language="Groovy">
+        <option name="KEEP_CONTROL_STATEMENT_IN_ONE_LINE" value="false"/>
+        <option name="KEEP_BLANK_LINES_IN_DECLARATIONS" value="1"/>
+        <option name="KEEP_BLANK_LINES_IN_CODE" value="1"/>
+        <option name="KEEP_BLANK_LINES_BEFORE_RBRACE" value="1"/>
+        <option name="ELSE_ON_NEW_LINE" value="true"/>
+        <option name="CATCH_ON_NEW_LINE" value="true"/>
+        <option name="FINALLY_ON_NEW_LINE" value="true"/>
+        <option name="ALIGN_MULTILINE_PARAMETERS" value="false"/>
+        <option name="ALIGN_MULTILINE_FOR" value="false"/>
+        <option name="SPACE_AFTER_TYPE_CAST" value="false"/>
+        <option name="METHOD_PARAMETERS_WRAP" value="1"/>
+        <option name="METHOD_ANNOTATION_WRAP" value="1"/>
+        <option name="CLASS_ANNOTATION_WRAP" value="1"/>
+        <option name="FIELD_ANNOTATION_WRAP" value="1"/>
+        <option name="PARENT_SETTINGS_INSTALLED" value="true"/>
+        <indentOptions>
+            <option name="CONTINUATION_INDENT_SIZE" value="4"/>
+        </indentOptions>
+    </codeStyleSettings>
+    <codeStyleSettings language="HOCON">
+        <option name="KEEP_BLANK_LINES_BEFORE_RBRACE" value="1"/>
+        <option name="PARENT_SETTINGS_INSTALLED" value="true"/>
+    </codeStyleSettings>
+    <codeStyleSettings language="JAVA">
+        <option name="KEEP_CONTROL_STATEMENT_IN_ONE_LINE" value="false"/>
+        <option name="KEEP_BLANK_LINES_IN_DECLARATIONS" value="1"/>
+        <option name="KEEP_BLANK_LINES_IN_CODE" value="1"/>
+        <option name="KEEP_BLANK_LINES_BEFORE_RBRACE" value="1"/>
+        <option name="ELSE_ON_NEW_LINE" value="true"/>
+        <option name="WHILE_ON_NEW_LINE" value="true"/>
+        <option name="CATCH_ON_NEW_LINE" value="true"/>
+        <option name="FINALLY_ON_NEW_LINE" value="true"/>
+        <option name="ALIGN_MULTILINE_PARAMETERS" value="false"/>
+        <option name="ALIGN_MULTILINE_FOR" value="false"/>
+        <option name="SPACE_AFTER_TYPE_CAST" value="false"/>
+        <option name="SPACE_BEFORE_ARRAY_INITIALIZER_LBRACE" value="true"/>
+        <option name="METHOD_PARAMETERS_WRAP" value="1"/>
+        <option name="ARRAY_INITIALIZER_LBRACE_ON_NEXT_LINE" value="true"/>
+        <option name="LABELED_STATEMENT_WRAP" value="1"/>
+        <option name="METHOD_ANNOTATION_WRAP" value="1"/>
+        <option name="CLASS_ANNOTATION_WRAP" value="1"/>
+        <option name="FIELD_ANNOTATION_WRAP" value="1"/>
+        <option name="PARENT_SETTINGS_INSTALLED" value="true"/>
+        <indentOptions>
+            <option name="CONTINUATION_INDENT_SIZE" value="4"/>
+        </indentOptions>
+    </codeStyleSettings>
+    <codeStyleSettings language="JSON">
+        <option name="KEEP_BLANK_LINES_IN_CODE" value="1"/>
+        <option name="PARENT_SETTINGS_INSTALLED" value="true"/>
+    </codeStyleSettings>
+    <codeStyleSettings language="Scala">
+        <option name="KEEP_BLANK_LINES_IN_DECLARATIONS" value="1"/>
+        <option name="KEEP_BLANK_LINES_IN_CODE" value="1"/>
+        <option name="KEEP_BLANK_LINES_BEFORE_RBRACE" value="1"/>
+        <option name="ELSE_ON_NEW_LINE" value="true"/>
+        <option name="WHILE_ON_NEW_LINE" value="true"/>
+        <option name="CATCH_ON_NEW_LINE" value="true"/>
+        <option name="FINALLY_ON_NEW_LINE" value="true"/>
+        <option name="ALIGN_MULTILINE_PARAMETERS" value="false"/>
+        <option name="ALIGN_MULTILINE_FOR" value="false"/>
+        <option name="METHOD_PARAMETERS_WRAP" value="1"/>
+        <option name="METHOD_ANNOTATION_WRAP" value="1"/>
+        <option name="CLASS_ANNOTATION_WRAP" value="1"/>
+        <option name="FIELD_ANNOTATION_WRAP" value="1"/>
+        <option name="PARENT_SETTINGS_INSTALLED" value="true"/>
+        <indentOptions>
+            <option name="INDENT_SIZE" value="4"/>
+            <option name="CONTINUATION_INDENT_SIZE" value="4"/>
+            <option name="TAB_SIZE" value="4"/>
+        </indentOptions>
+    </codeStyleSettings>
+    <codeStyleSettings language="XML">
+        <indentOptions>
+            <option name="CONTINUATION_INDENT_SIZE" value="4"/>
+        </indentOptions>
+    </codeStyleSettings>
+</code_scheme>
\ No newline at end of file

[rocketmq-flink] 17/33: Merge pull request #229 from Jennifer-sarah/master

Posted by du...@apache.org.
This is an automated email from the ASF dual-hosted git repository.

duhengforever pushed a commit to branch main
in repository https://gitbox.apache.org/repos/asf/rocketmq-flink.git

commit c6cb79d2536991367456086228a83a5518c19e02
Merge: 0be1ed7 8531921
Author: Xin Wang <xi...@apache.org>
AuthorDate: Sun Jul 7 19:37:50 2019 +0800

    Merge pull request #229 from Jennifer-sarah/master
    
    update consumer offset after checkpoint completed

 .../org/apache/rocketmq/flink/RocketMQSource.java  | 65 ++++++++++++++++++++--
 1 file changed, 59 insertions(+), 6 deletions(-)

diff --cc src/main/java/org/apache/rocketmq/flink/RocketMQSource.java
index 18277e0,b6e68f8..06eecfb
--- a/src/main/java/org/apache/rocketmq/flink/RocketMQSource.java
+++ b/src/main/java/org/apache/rocketmq/flink/RocketMQSource.java
@@@ -22,10 -23,9 +23,11 @@@ import java.util.HashMap
  import java.util.List;
  import java.util.Map;
  import java.util.Properties;
 +import java.util.Set;
 +import java.util.UUID;
  import java.util.concurrent.ConcurrentHashMap;
  
+ import org.apache.commons.collections.map.LinkedMap;
  import org.apache.commons.lang.Validate;
  import org.apache.flink.api.common.state.ListState;
  import org.apache.flink.api.common.state.ListStateDescriptor;
@@@ -286,17 -300,18 +302,22 @@@ public class RocketMQSource<OUT> extend
  
          unionOffsetStates.clear();
  
-         if (LOG.isDebugEnabled()) {
-             LOG.debug("Snapshotted state, last processed offsets: {}, checkpoint id: {}, timestamp: {}",
-                 offsetTable, context.getCheckpointId(), context.getCheckpointTimestamp());
-         }
+         HashMap<MessageQueue, Long> currentOffsets = new HashMap<>(offsetTable.size());
  
 +        // remove the unassigned queues in order to avoid read the wrong offset when the source restart
 +        Set<MessageQueue> assignedQueues = consumer.fetchMessageQueuesInBalance(topic);
 +        offsetTable.entrySet().removeIf(item -> !assignedQueues.contains(item.getKey()));
 +
          for (Map.Entry<MessageQueue, Long> entry : offsetTable.entrySet()) {
              unionOffsetStates.add(Tuple2.of(entry.getKey(), entry.getValue()));
+             currentOffsets.put(entry.getKey(), entry.getValue());
+         }
+ 
+         pendingOffsetsToCommit.put(context.getCheckpointId(), currentOffsets);
+ 
+         if (LOG.isDebugEnabled()) {
+             LOG.debug("Snapshotted state, last processed offsets: {}, checkpoint id: {}, timestamp: {}",
+                 offsetTable, context.getCheckpointId(), context.getCheckpointTimestamp());
          }
      }
  

[rocketmq-flink] 20/33: No need to call this function twice (#329)

Posted by du...@apache.org.
This is an automated email from the ASF dual-hosted git repository.

duhengforever pushed a commit to branch main
in repository https://gitbox.apache.org/repos/asf/rocketmq-flink.git

commit a04b0bf82e14b6400a311a49702fc72bc7fcd392
Author: 胖梁 <41...@qq.com>
AuthorDate: Fri Aug 23 09:37:31 2019 +0800

    No need to call this function twice (#329)
---
 src/main/java/org/apache/rocketmq/flink/RocketMQSink.java | 2 +-
 1 file changed, 1 insertion(+), 1 deletion(-)

diff --git a/src/main/java/org/apache/rocketmq/flink/RocketMQSink.java b/src/main/java/org/apache/rocketmq/flink/RocketMQSink.java
index f3e200d..eecb72e 100644
--- a/src/main/java/org/apache/rocketmq/flink/RocketMQSink.java
+++ b/src/main/java/org/apache/rocketmq/flink/RocketMQSink.java
@@ -147,7 +147,7 @@ public class RocketMQSink<IN> extends RichSinkFunction<IN> implements Checkpoint
 
     private Message prepareMessage(IN input) {
         String topic = topicSelector.getTopic(input);
-        String tag = topicSelector.getTag(input) != null ? topicSelector.getTag(input) : "";
+        String tag = (tag = topicSelector.getTag(input)) != null ? tag : "";
 
         byte[] k = serializationSchema.serializeKey(input);
         String key = k != null ? new String(k, StandardCharsets.UTF_8) : "";

[rocketmq-flink] 06/33: Add batch size param for flink sink (#198)

Posted by du...@apache.org.
This is an automated email from the ASF dual-hosted git repository.

duhengforever pushed a commit to branch main
in repository https://gitbox.apache.org/repos/asf/rocketmq-flink.git

commit 452b57fcd480dd707ecd8b73d898e5e87c8f6679
Author: shangan <ch...@163.com>
AuthorDate: Fri Feb 1 14:37:05 2019 +0800

    Add batch size param for flink sink (#198)
    
    * Add batch size param for flink sink
    
    * Improvement. code style correction
---
 src/main/java/org/apache/rocketmq/flink/RocketMQSink.java | 9 +++++++++
 1 file changed, 9 insertions(+)

diff --git a/src/main/java/org/apache/rocketmq/flink/RocketMQSink.java b/src/main/java/org/apache/rocketmq/flink/RocketMQSink.java
index e79d1b4..65274af 100644
--- a/src/main/java/org/apache/rocketmq/flink/RocketMQSink.java
+++ b/src/main/java/org/apache/rocketmq/flink/RocketMQSink.java
@@ -59,6 +59,7 @@ public class RocketMQSink<IN> extends RichSinkFunction<IN> implements Checkpoint
     private KeyValueSerializationSchema<IN> serializationSchema;
 
     private boolean batchFlushOnCheckpoint; // false by default
+    private int batchSize = 1000;
     private List<Message> batchList;
 
     public RocketMQSink(KeyValueSerializationSchema<IN> schema, TopicSelector<IN> topicSelector, Properties props) {
@@ -97,6 +98,9 @@ public class RocketMQSink<IN> extends RichSinkFunction<IN> implements Checkpoint
 
         if (batchFlushOnCheckpoint) {
             batchList.add(msg);
+            if (batchList.size() >= batchSize) {
+                flushSync();
+            }
             return;
         }
 
@@ -156,6 +160,11 @@ public class RocketMQSink<IN> extends RichSinkFunction<IN> implements Checkpoint
         return this;
     }
 
+    public RocketMQSink<IN> withBatchSize(int batchSize) {
+        this.batchSize = batchSize;
+        return this;
+    }
+
     @Override
     public void close() throws Exception {
         if (producer != null) {

[rocketmq-flink] 13/33: fix(module): fix load wrong offset from savepoint (#288)

Posted by du...@apache.org.
This is an automated email from the ASF dual-hosted git repository.

duhengforever pushed a commit to branch main
in repository https://gitbox.apache.org/repos/asf/rocketmq-flink.git

commit c9564d6d2b122345889d33b02e2d4ae5098a675c
Author: MeYJ <Me...@users.noreply.github.com>
AuthorDate: Mon Jun 3 11:33:38 2019 +0800

    fix(module): fix load wrong offset from savepoint (#288)
---
 src/main/java/org/apache/rocketmq/flink/RocketMQSource.java | 4 +++-
 1 file changed, 3 insertions(+), 1 deletion(-)

diff --git a/src/main/java/org/apache/rocketmq/flink/RocketMQSource.java b/src/main/java/org/apache/rocketmq/flink/RocketMQSource.java
index 9940e8e..ccd6bb4 100644
--- a/src/main/java/org/apache/rocketmq/flink/RocketMQSource.java
+++ b/src/main/java/org/apache/rocketmq/flink/RocketMQSource.java
@@ -320,7 +320,9 @@ public class RocketMQSource<OUT> extends RichParallelSourceFunction<OUT>
             for (Tuple2<MessageQueue, Long> mqOffsets : unionOffsetStates.get()) {
                 // unionOffsetStates is the restored global union state;
                 // should only snapshot mqs that actually belong to us
-                restoredOffsets.put(mqOffsets.f0, mqOffsets.f1);
+                if (!restoredOffsets.containsKey(mqOffsets.f0) || restoredOffsets.get(mqOffsets.f0) < mqOffsets.f1) {
+                    restoredOffsets.put(mqOffsets.f0, mqOffsets.f1);
+                }
             }
             LOG.info("Setting restore state in the consumer. Using the following offsets: {}", restoredOffsets);
         } else {

[rocketmq-flink] 10/33: fix concurrent checkpoint bug

Posted by du...@apache.org.
This is an automated email from the ASF dual-hosted git repository.

duhengforever pushed a commit to branch main
in repository https://gitbox.apache.org/repos/asf/rocketmq-flink.git

commit 8531921af25bd94a32f6fa149f3147fe8ad370be
Author: Jennifer-sarah <42...@users.noreply.github.com>
AuthorDate: Fri Mar 22 08:23:08 2019 +0800

    fix concurrent checkpoint bug
    
    fix concurrent checkpoint bug
---
 .../org/apache/rocketmq/flink/RocketMQSource.java  | 39 ++++++++++++++++++++--
 1 file changed, 36 insertions(+), 3 deletions(-)

diff --git a/src/main/java/org/apache/rocketmq/flink/RocketMQSource.java b/src/main/java/org/apache/rocketmq/flink/RocketMQSource.java
index 5b76e54..b6e68f8 100644
--- a/src/main/java/org/apache/rocketmq/flink/RocketMQSource.java
+++ b/src/main/java/org/apache/rocketmq/flink/RocketMQSource.java
@@ -19,11 +19,13 @@
 package org.apache.rocketmq.flink;
 
 import java.nio.charset.StandardCharsets;
+import java.util.HashMap;
 import java.util.List;
 import java.util.Map;
 import java.util.Properties;
 import java.util.concurrent.ConcurrentHashMap;
 
+import org.apache.commons.collections.map.LinkedMap;
 import org.apache.commons.lang.Validate;
 import org.apache.flink.api.common.state.ListState;
 import org.apache.flink.api.common.state.ListStateDescriptor;
@@ -43,6 +45,7 @@ import org.apache.rocketmq.client.consumer.MQPullConsumerScheduleService;
 import org.apache.rocketmq.client.consumer.PullResult;
 import org.apache.rocketmq.client.consumer.PullTaskCallback;
 import org.apache.rocketmq.client.consumer.PullTaskContext;
+import org.apache.rocketmq.client.consumer.store.OffsetStore;
 import org.apache.rocketmq.client.exception.MQClientException;
 import org.apache.rocketmq.common.message.MessageExt;
 import org.apache.rocketmq.common.message.MessageQueue;
@@ -78,6 +81,8 @@ public class RocketMQSource<OUT> extends RichParallelSourceFunction<OUT>
     private transient ListState<Tuple2<MessageQueue, Long>> unionOffsetStates;
     private Map<MessageQueue, Long> offsetTable;
     private Map<MessageQueue, Long> restoredOffsets;
+    /** Data for pending but uncommitted offsets. */
+    private LinkedMap pendingOffsetsToCommit;
 
     private Properties props;
     private String topic;
@@ -113,6 +118,9 @@ public class RocketMQSource<OUT> extends RichParallelSourceFunction<OUT>
         if (restoredOffsets == null) {
             restoredOffsets = new ConcurrentHashMap<>();
         }
+        if (pendingOffsetsToCommit == null) {
+            pendingOffsetsToCommit = new LinkedMap();
+        }
 
         runningChecker = new RunningChecker();
 
@@ -263,6 +271,7 @@ public class RocketMQSource<OUT> extends RichParallelSourceFunction<OUT>
 
         offsetTable.clear();
         restoredOffsets.clear();
+        pendingOffsetsToCommit.clear();
     }
 
     @Override
@@ -291,13 +300,18 @@ public class RocketMQSource<OUT> extends RichParallelSourceFunction<OUT>
 
         unionOffsetStates.clear();
 
+        HashMap<MessageQueue, Long> currentOffsets = new HashMap<>(offsetTable.size());
+
         for (Map.Entry<MessageQueue, Long> entry : offsetTable.entrySet()) {
             unionOffsetStates.add(Tuple2.of(entry.getKey(), entry.getValue()));
+            currentOffsets.put(entry.getKey(), entry.getValue());
         }
 
+        pendingOffsetsToCommit.put(context.getCheckpointId(), currentOffsets);
+
         if (LOG.isDebugEnabled()) {
             LOG.debug("Snapshotted state, last processed offsets: {}, checkpoint id: {}, timestamp: {}",
-                    offsetTable, context.getCheckpointId(), context.getCheckpointTimestamp());
+                offsetTable, context.getCheckpointId(), context.getCheckpointTimestamp());
         }
     }
 
@@ -337,14 +351,33 @@ public class RocketMQSource<OUT> extends RichParallelSourceFunction<OUT>
 
     @Override
     public void notifyCheckpointComplete(long checkpointId) throws Exception {
-        // callback when checkpoint complete 
+        // callback when checkpoint complete
         if (!runningChecker.isRunning()) {
             LOG.debug("notifyCheckpointComplete() called on closed source; returning null.");
             return;
         }
 
-        for (Map.Entry<MessageQueue, Long> entry : offsetTable.entrySet()) {
+        final int posInMap = pendingOffsetsToCommit.indexOf(checkpointId);
+        if (posInMap == -1) {
+            LOG.warn("Received confirmation for unknown checkpoint id {}", checkpointId);
+            return;
+        }
+
+        Map<MessageQueue, Long> offsets = (Map<MessageQueue, Long>)pendingOffsetsToCommit.remove(posInMap);
+
+        // remove older checkpoints in map
+        for (int i = 0; i < posInMap; i++) {
+            pendingOffsetsToCommit.remove(0);
+        }
+
+        if (offsets == null || offsets.size() == 0) {
+            LOG.debug("Checkpoint state was empty.");
+            return;
+        }
+
+        for (Map.Entry<MessageQueue, Long> entry : offsets.entrySet()) {
             consumer.updateConsumeOffset(entry.getKey(), entry.getValue());
         }
+
     }
 }

[rocketmq-flink] 04/33: Fix NPE when getting offset(#153)

Posted by du...@apache.org.
This is an automated email from the ASF dual-hosted git repository.

duhengforever pushed a commit to branch main
in repository https://gitbox.apache.org/repos/asf/rocketmq-flink.git

commit 624994d402106d7858960c1e1b5bd9c05d66f823
Author: SuXingLee <91...@qq.com>
AuthorDate: Fri Nov 23 10:06:20 2018 +0800

    Fix NPE  when getting offset(#153)
    
    rockemq-flink: RocketMQSource will throw a NullPointerException
---
 src/main/java/org/apache/rocketmq/flink/RocketMQSource.java | 3 +--
 1 file changed, 1 insertion(+), 2 deletions(-)

diff --git a/src/main/java/org/apache/rocketmq/flink/RocketMQSource.java b/src/main/java/org/apache/rocketmq/flink/RocketMQSource.java
index 2dc8fd5..8e8e57b 100644
--- a/src/main/java/org/apache/rocketmq/flink/RocketMQSource.java
+++ b/src/main/java/org/apache/rocketmq/flink/RocketMQSource.java
@@ -235,9 +235,8 @@ public class RocketMQSource<OUT> extends RichParallelSourceFunction<OUT>
                         throw new IllegalArgumentException("Unknown value for CONSUMER_OFFSET_RESET_TO.");
                 }
             }
-            offsetTable.put(mq, offset);
         }
-
+        offsetTable.put(mq, offset);
         return offsetTable.get(mq);
     }
 

[rocketmq-flink] 12/33: Fix getting wrong offset bug when the source restart (#190)

Posted by du...@apache.org.
This is an automated email from the ASF dual-hosted git repository.

duhengforever pushed a commit to branch main
in repository https://gitbox.apache.org/repos/asf/rocketmq-flink.git

commit 58d07fbe52968e43e7eb041045bbcc194bd904b7
Author: tangyoupeng <to...@juicedata.io>
AuthorDate: Mon Jun 3 11:10:17 2019 +0800

    Fix getting wrong offset bug when the source restart (#190)
---
 src/main/java/org/apache/rocketmq/flink/RocketMQSource.java | 5 +++++
 1 file changed, 5 insertions(+)

diff --git a/src/main/java/org/apache/rocketmq/flink/RocketMQSource.java b/src/main/java/org/apache/rocketmq/flink/RocketMQSource.java
index 8e8e57b..9940e8e 100644
--- a/src/main/java/org/apache/rocketmq/flink/RocketMQSource.java
+++ b/src/main/java/org/apache/rocketmq/flink/RocketMQSource.java
@@ -22,6 +22,7 @@ import java.nio.charset.StandardCharsets;
 import java.util.List;
 import java.util.Map;
 import java.util.Properties;
+import java.util.Set;
 import java.util.concurrent.ConcurrentHashMap;
 
 import org.apache.commons.lang.Validate;
@@ -289,6 +290,10 @@ public class RocketMQSource<OUT> extends RichParallelSourceFunction<OUT>
                 offsetTable, context.getCheckpointId(), context.getCheckpointTimestamp());
         }
 
+        // remove the unassigned queues in order to avoid read the wrong offset when the source restart
+        Set<MessageQueue> assignedQueues = consumer.fetchMessageQueuesInBalance(topic);
+        offsetTable.entrySet().removeIf(item -> !assignedQueues.contains(item.getKey()));
+
         for (Map.Entry<MessageQueue, Long> entry : offsetTable.entrySet()) {
             unionOffsetStates.add(Tuple2.of(entry.getKey(), entry.getValue()));
         }

[rocketmq-flink] 26/33: [#705] Support the implementation of new Source interface (#706)

Posted by du...@apache.org.
This is an automated email from the ASF dual-hosted git repository.

duhengforever pushed a commit to branch main
in repository https://gitbox.apache.org/repos/asf/rocketmq-flink.git

commit f882acfc1b6336ed24564bd992e15b87e01e66fd
Author: SteNicholas <pr...@163.com>
AuthorDate: Mon Apr 19 14:01:18 2021 +0800

    [#705] Support the implementation of new Source interface (#706)
---
 README.md                                          |   8 +-
 pom.xml                                            |  57 ++-
 .../org/apache/rocketmq/flink/RocketMQSource.java  | 433 -----------------
 .../org/apache/rocketmq/flink/RunningChecker.java  |  33 --
 .../flink/common/selector/TopicSelector.java       |  28 --
 .../KeyValueDeserializationSchema.java             |  27 --
 .../serialization/KeyValueSerializationSchema.java |  28 --
 .../rocketmq/flink/common/util/MetricUtils.java    |  80 ----
 .../rocketmq/flink/common/util/RocketMQUtils.java  |  73 ---
 .../rocketmq/flink/common/util/TestUtils.java      |  33 --
 .../flink/{ => legacy}/RocketMQConfig.java         |  70 +--
 .../rocketmq/flink/{ => legacy}/RocketMQSink.java  |  98 ++--
 .../rocketmq/flink/legacy/RocketMQSource.java      | 528 +++++++++++++++++++++
 .../rocketmq/flink/legacy/RunningChecker.java      |  29 ++
 .../common/selector/DefaultTopicSelector.java      |  39 ++
 .../common/selector/SimpleTopicSelector.java       |  46 +-
 .../legacy/common/selector/TopicSelector.java      |  24 +
 .../ForwardMessageExtDeserialization.java          |  14 +-
 .../KeyValueDeserializationSchema.java             |  23 +
 .../serialization/KeyValueSerializationSchema.java |  24 +
 .../MessageExtDeserializationScheme.java           |   8 +-
 .../SimpleKeyValueDeserializationSchema.java       |  33 +-
 .../SimpleKeyValueSerializationSchema.java         |  28 +-
 .../SimpleTupleDeserializationSchema.java          |   7 +-
 .../flink/legacy/common/util/MetricUtils.java      |  85 ++++
 .../flink/{ => legacy}/common/util/RetryUtil.java  |  27 +-
 .../flink/legacy/common/util/RocketMQUtils.java    |  79 +++
 .../flink/legacy/common/util/TestUtils.java        |  29 ++
 .../watermark/BoundedOutOfOrdernessGenerator.java  |  18 +-
 .../BoundedOutOfOrdernessGeneratorPerQueue.java    |  27 +-
 .../common/watermark/PunctuatedAssigner.java       |  23 +-
 .../watermark/TimeLagWatermarkGenerator.java       |  19 +-
 .../common/watermark/WaterMarkForAll.java          |  10 +-
 .../common/watermark/WaterMarkPerQueue.java        |  18 +-
 .../{ => legacy}/example/RocketMQFlinkExample.java |  46 +-
 .../flink/{ => legacy}/example/SimpleConsumer.java |  31 +-
 .../flink/{ => legacy}/example/SimpleProducer.java |  14 +-
 .../{ => legacy}/function/SinkMapFunction.java     |  14 +-
 .../{ => legacy}/function/SourceMapFunction.java   |   9 +-
 .../rocketmq/flink/source/RocketMQSource.java      | 175 +++++++
 .../enumerator/RocketMQSourceEnumState.java}       |  33 +-
 .../RocketMQSourceEnumStateSerializer.java         |  64 +++
 .../enumerator/RocketMQSourceEnumerator.java       | 337 +++++++++++++
 .../reader/RocketMQPartitionSplitReader.java       | 373 +++++++++++++++
 .../flink/source/reader/RocketMQRecordEmitter.java |  39 ++
 .../flink/source/reader/RocketMQSourceReader.java  |  64 +++
 .../RocketMQRecordDeserializationSchema.java       |  43 ++
 .../flink/source/split/RocketMQPartitionSplit.java | 100 ++++
 .../split/RocketMQPartitionSplitSerializer.java    |  66 +++
 .../source/split/RocketMQPartitionSplitState.java  |  57 +++
 .../apache/rocketmq/flink/RocketMQSinkTest.java    |  70 ---
 .../common/selector/DefaultTopicSelectorTest.java  |  37 --
 .../common/selector/SimpleTopicSelectorTest.java   |  49 --
 .../SimpleKeyValueSerializationSchemaTest.java     |  42 --
 .../rocketmq/flink/legacy/RocketMQSinkTest.java    |  70 +++
 .../flink/{ => legacy}/RocketMQSourceTest.java     |  52 +-
 .../common/selector/DefaultTopicSelectorTest.java  |  32 ++
 .../common/selector/SimpleTopicSelectorTest.java   |  44 ++
 .../SimpleKeyValueSerializationSchemaTest.java     |  42 ++
 .../RocketMQSourceEnumStateSerializerTest.java     |  84 ++++
 .../source/reader/RocketMQRecordEmitterTest.java   |  97 ++++
 .../RocketMQPartitionSplitSerializerTest.java      |  44 ++
 62 files changed, 2979 insertions(+), 1255 deletions(-)

diff --git a/README.md b/README.md
index 600c57a..97cb1f9 100644
--- a/README.md
+++ b/README.md
@@ -9,7 +9,7 @@ The RocketMQSource is based on RocketMQ pull consumer mode, and provides exactly
 Otherwise, the source doesn't provide any reliability guarantees.
 
 ### KeyValueDeserializationSchema
-The main API for deserializing topic and tags is the `org.apache.rocketmq.flink.common.serialization.KeyValueDeserializationSchema` interface.
+The main API for deserializing topic and tags is the `org.apache.rocketmq.flink.legacy.common.serialization.KeyValueDeserializationSchema` interface.
 `rocketmq-flink` includes general purpose `KeyValueDeserializationSchema` implementations called `SimpleKeyValueDeserializationSchema`.
 
 ```java
@@ -26,7 +26,7 @@ Otherwise, the sink reliability guarantees depends on rocketmq producer's retry
 but you can change it by invoking `withAsync(true)`. 
 
 ### KeyValueSerializationSchema
-The main API for serializing topic and tags is the `org.apache.rocketmq.flink.common.serialization.KeyValueSerializationSchema` interface.
+The main API for serializing topic and tags is the `org.apache.rocketmq.flink.legacy.common.serialization.KeyValueSerializationSchema` interface.
 `rocketmq-flink` includes general purpose `KeyValueSerializationSchema` implementations called `SimpleKeyValueSerializationSchema`.
 
 ```java
@@ -39,7 +39,7 @@ public interface KeyValueSerializationSchema<T> extends Serializable {
 ```
 
 ### TopicSelector
-The main API for selecting topic and tags is the `org.apache.rocketmq.flink.common.selector.TopicSelector` interface.
+The main API for selecting topic and tags is the `org.apache.rocketmq.flink.legacy.common.selector.TopicSelector` interface.
 `rocketmq-flink` includes general purpose `TopicSelector` implementations called `DefaultTopicSelector` and `SimpleTopicSelector`.
 
 ```java
@@ -96,7 +96,7 @@ StreamExecutionEnvironment env = StreamExecutionEnvironment.getExecutionEnvironm
  ```
 
 ## Configurations
-The following configurations are all from the class `org.apache.rocketmq.flink.RocketMQConfig`.
+The following configurations are all from the class `org.apache.rocketmq.flink.legacy.RocketMQConfig`.
 
 ### Producer Configurations
 | NAME        | DESCRIPTION           | DEFAULT  |
diff --git a/pom.xml b/pom.xml
index 2e19ce5..d5fc49f 100644
--- a/pom.xml
+++ b/pom.xml
@@ -34,9 +34,10 @@
         <maven.compiler.source>1.8</maven.compiler.source>
         <maven.compiler.target>1.8</maven.compiler.target>
         <rocketmq.version>4.7.1</rocketmq.version>
-        <flink.version>1.10.1</flink.version>
+        <flink.version>1.12.2</flink.version>
         <commons-lang.version>2.5</commons-lang.version>
         <scala.binary.version>2.11</scala.binary.version>
+        <spotless.version>2.4.2</spotless.version>
     </properties>
 
     <dependencies>
@@ -57,6 +58,26 @@
         </dependency>
         <dependency>
             <groupId>org.apache.flink</groupId>
+            <artifactId>flink-connector-base</artifactId>
+            <version>${flink.version}</version>
+        </dependency>
+        <dependency>
+            <groupId>org.apache.flink</groupId>
+            <artifactId>flink-table-common</artifactId>
+            <version>${flink.version}</version>
+        </dependency>
+        <dependency>
+            <groupId>org.apache.flink</groupId>
+            <artifactId>flink-table-api-java-bridge_${scala.binary.version}</artifactId>
+            <version>${flink.version}</version>
+        </dependency>
+        <dependency>
+            <groupId>org.apache.flink</groupId>
+            <artifactId>flink-table-runtime-blink_2.11</artifactId>
+            <version>${flink.version}</version>
+        </dependency>
+        <dependency>
+            <groupId>org.apache.flink</groupId>
             <artifactId>flink-queryable-state-runtime_${scala.binary.version}</artifactId>
             <version>${flink.version}</version>
         </dependency>
@@ -149,7 +170,7 @@
                                     <resource>reference.conf</resource>
                                 </transformer>
                                 <transformer implementation="org.apache.maven.plugins.shade.resource.ManifestResourceTransformer">
-                                    <mainClass>org.apache.rocketmq.flink.example.RocketMQFlinkExample</mainClass>
+                                    <mainClass>org.apache.rocketmq.flink.legacy.example.RocketMQFlinkExample</mainClass>
                                 </transformer>
                             </transformers>
                         </configuration>
@@ -218,6 +239,38 @@
                     <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" -->
+            <plugin>
+                <groupId>com.diffplug.spotless</groupId>
+                <artifactId>spotless-maven-plugin</artifactId>
+                <version>${spotless.version}</version>
+                <configuration>
+                    <java>
+                        <googleJavaFormat>
+                            <version>1.7</version>
+                            <style>AOSP</style>
+                        </googleJavaFormat>
+
+                        <!-- \# refers to the static imports -->
+                        <importOrder>
+                            <order>org.apache.rocketmq,org.apache.flink,org.apache.flink.shaded,,javax,java,scala,\#</order>
+                        </importOrder>
+
+                        <removeUnusedImports />
+                    </java>
+                </configuration>
+                <executions>
+                    <execution>
+                        <id>spotless-check</id>
+                        <phase>validate</phase>
+                        <goals>
+                            <goal>check</goal>
+                        </goals>
+                    </execution>
+                </executions>
+            </plugin>
         </plugins>
     </build>
 </project>
\ No newline at end of file
diff --git a/src/main/java/org/apache/rocketmq/flink/RocketMQSource.java b/src/main/java/org/apache/rocketmq/flink/RocketMQSource.java
deleted file mode 100644
index 72783a8..0000000
--- a/src/main/java/org/apache/rocketmq/flink/RocketMQSource.java
+++ /dev/null
@@ -1,433 +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
- * <p>
- * http://www.apache.org/licenses/LICENSE-2.0
- * <p>
- * Unless required by applicable law or agreed to in writing, software distributed under the License is distributed on
- * an "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. See the License for the
- * specific language governing permissions and limitations under the License.
- */
-
-package org.apache.rocketmq.flink;
-
-import org.apache.commons.collections.map.LinkedMap;
-import org.apache.commons.lang.Validate;
-import org.apache.flink.api.common.functions.RuntimeContext;
-import org.apache.flink.api.common.state.ListState;
-import org.apache.flink.api.common.state.ListStateDescriptor;
-import org.apache.flink.api.common.typeinfo.TypeHint;
-import org.apache.flink.api.common.typeinfo.TypeInformation;
-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.metrics.Counter;
-import org.apache.flink.util.Preconditions;
-import org.apache.flink.metrics.Meter;
-import org.apache.flink.metrics.MeterView;
-import org.apache.flink.metrics.SimpleCounter;
-import org.apache.flink.runtime.state.CheckpointListener;
-import org.apache.flink.runtime.state.FunctionInitializationContext;
-import org.apache.flink.runtime.state.FunctionSnapshotContext;
-import org.apache.flink.shaded.curator.org.apache.curator.shaded.com.google.common.util.concurrent.ThreadFactoryBuilder;
-import org.apache.flink.streaming.api.checkpoint.CheckpointedFunction;
-import org.apache.flink.streaming.api.functions.source.RichParallelSourceFunction;
-import org.apache.flink.streaming.api.operators.StreamingRuntimeContext;
-import org.apache.rocketmq.client.consumer.DefaultMQPullConsumer;
-import org.apache.rocketmq.client.consumer.PullResult;
-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.common.serialization.KeyValueDeserializationSchema;
-import org.apache.rocketmq.flink.common.util.MetricUtils;
-import org.apache.rocketmq.flink.common.util.RetryUtil;
-import org.apache.rocketmq.flink.common.util.RocketMQUtils;
-import org.apache.rocketmq.flink.common.watermark.WaterMarkForAll;
-import org.apache.rocketmq.flink.common.watermark.WaterMarkPerQueue;
-import org.slf4j.Logger;
-import org.slf4j.LoggerFactory;
-
-import java.lang.management.ManagementFactory;
-import java.nio.charset.StandardCharsets;
-import java.util.*;
-import java.util.concurrent.*;
-import java.util.concurrent.locks.ReentrantLock;
-
-import static org.apache.rocketmq.flink.RocketMQConfig.*;
-import static org.apache.rocketmq.flink.common.util.RocketMQUtils.getInteger;
-import static org.apache.rocketmq.flink.common.util.RocketMQUtils.getLong;
-
-/**
- * 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 guarantees.
- */
-public class RocketMQSource<OUT> extends RichParallelSourceFunction<OUT>
-        implements CheckpointedFunction, CheckpointListener, ResultTypeQueryable<OUT> {
-
-    private static final long serialVersionUID = 1L;
-
-    private static final Logger log = LoggerFactory.getLogger(RocketMQSource.class);
-    private static final String OFFSETS_STATE_NAME = "topic-partition-offset-states";
-    private RunningChecker runningChecker;
-    private transient DefaultMQPullConsumer consumer;
-    private KeyValueDeserializationSchema<OUT> schema;
-    private transient ListState<Tuple2<MessageQueue, Long>> unionOffsetStates;
-    private Map<MessageQueue, Long> offsetTable;
-    private Map<MessageQueue, Long> restoredOffsets;
-    private List<MessageQueue> messageQueues;
-    private ExecutorService executor;
-
-    // watermark in source
-    private WaterMarkPerQueue waterMarkPerQueue;
-    private WaterMarkForAll waterMarkForAll;
-
-    private ScheduledExecutorService timer;
-    /**
-     * Data for pending but uncommitted offsets.
-     */
-    private LinkedMap pendingOffsetsToCommit;
-    private Properties props;
-    private String topic;
-    private String group;
-    private transient volatile boolean restored;
-    private transient boolean enableCheckpoint;
-    private volatile Object checkPointLock;
-
-    private Meter tpsMetric;
-
-    public RocketMQSource(KeyValueDeserializationSchema<OUT> schema, Properties props) {
-        this.schema = schema;
-        this.props = props;
-    }
-
-    @Override
-    public void open(Configuration parameters) throws Exception {
-        log.debug("source open....");
-        Validate.notEmpty(props, "Consumer properties can not be empty");
-
-        this.topic = props.getProperty(RocketMQConfig.CONSUMER_TOPIC);
-        this.group = props.getProperty(RocketMQConfig.CONSUMER_GROUP);
-
-        Validate.notEmpty(topic, "Consumer topic can not be empty");
-        Validate.notEmpty(group, "Consumer group can not be empty");
-
-        this.enableCheckpoint = ((StreamingRuntimeContext) getRuntimeContext()).isCheckpointingEnabled();
-
-        if (offsetTable == null) {
-            offsetTable = new ConcurrentHashMap<>();
-        }
-        if (restoredOffsets == null) {
-            restoredOffsets = new ConcurrentHashMap<>();
-        }
-
-        //use restoredOffsets to init offset table.
-        initOffsetTableFromRestoredOffsets();
-
-        if (pendingOffsetsToCommit == null) {
-            pendingOffsetsToCommit = new LinkedMap();
-        }
-        if (checkPointLock == null) {
-            checkPointLock = new ReentrantLock();
-        }
-        if (waterMarkPerQueue == null) {
-            waterMarkPerQueue = new WaterMarkPerQueue(5000);
-        }
-        if (waterMarkForAll == null) {
-            waterMarkForAll = new WaterMarkForAll(5000);
-        }
-        if (timer == null) {
-            timer = Executors.newSingleThreadScheduledExecutor();
-        }
-
-        runningChecker = new RunningChecker();
-        runningChecker.setRunning(true);
-
-        final ThreadFactory threadFactory = new ThreadFactoryBuilder()
-                .setDaemon(true).setNameFormat("rmq-pull-thread-%d").build();
-        executor = Executors.newCachedThreadPool(threadFactory);
-
-        int indexOfThisSubTask = getRuntimeContext().getIndexOfThisSubtask();
-        consumer = new DefaultMQPullConsumer(group, RocketMQConfig.buildAclRPCHook(props));
-        RocketMQConfig.buildConsumerConfigs(props, consumer);
-
-        // set unique instance name, avoid exception: https://help.aliyun.com/document_detail/29646.html
-        String runtimeName = ManagementFactory.getRuntimeMXBean().getName();
-        String instanceName = RocketMQUtils.getInstanceName(runtimeName, topic, group,
-                String.valueOf(indexOfThisSubTask), String.valueOf(System.nanoTime()));
-        consumer.setInstanceName(instanceName);
-        consumer.start();
-
-        Counter outputCounter = getRuntimeContext().getMetricGroup()
-                .counter(MetricUtils.METRICS_TPS + "_counter", new SimpleCounter());
-        tpsMetric = getRuntimeContext().getMetricGroup()
-                .meter(MetricUtils.METRICS_TPS, new MeterView(outputCounter, 60));
-    }
-
-    @Override
-    public void run(SourceContext context) throws Exception {
-        String tag = props.getProperty(RocketMQConfig.CONSUMER_TAG, RocketMQConfig.DEFAULT_CONSUMER_TAG);
-        int pullBatchSize = getInteger(props, CONSUMER_BATCH_SIZE, DEFAULT_CONSUMER_BATCH_SIZE);
-
-        final RuntimeContext ctx = getRuntimeContext();
-        // The lock that guarantees that record emission and state updates are atomic,
-        // from the view of taking a checkpoint.
-        int taskNumber = ctx.getNumberOfParallelSubtasks();
-        int taskIndex = ctx.getIndexOfThisSubtask();
-        log.info("Source run, NumberOfTotalTask={}, IndexOfThisSubTask={}", taskNumber, taskIndex);
-
-
-        timer.scheduleAtFixedRate(() -> {
-            // context.emitWatermark(waterMarkPerQueue.getCurrentWatermark());
-            context.emitWatermark(waterMarkForAll.getCurrentWatermark());
-        }, 5, 5, TimeUnit.SECONDS);
-
-        Collection<MessageQueue> totalQueues = consumer.fetchSubscribeMessageQueues(topic);
-        messageQueues = RocketMQUtils.allocate(totalQueues, taskNumber, ctx.getIndexOfThisSubtask());
-        for (MessageQueue mq : messageQueues) {
-            this.executor.execute(() -> {
-                RetryUtil.call(() -> {
-                    while (runningChecker.isRunning()) {
-                        try {
-                            long offset = getMessageQueueOffset(mq);
-                            PullResult pullResult = consumer.pullBlockIfNotFound(mq, tag, offset, pullBatchSize);
-
-                            boolean found = false;
-                            switch (pullResult.getPullStatus()) {
-                                case FOUND:
-                                    List<MessageExt> messages = pullResult.getMsgFoundList();
-                                    for (MessageExt msg : messages) {
-                                        byte[] key = msg.getKeys() != null ? msg.getKeys().getBytes(StandardCharsets.UTF_8) : null;
-                                        byte[] value = msg.getBody();
-                                        OUT data = schema.deserializeKeyAndValue(key, value);
-
-                                        // output and state update are atomic
-                                        synchronized (checkPointLock) {
-                                            log.debug(msg.getMsgId() + "_" + msg.getBrokerName() + " " + msg.getQueueId() + " " + msg.getQueueOffset());
-                                            context.collectWithTimestamp(data, msg.getBornTimestamp());
-
-                                            // update max eventTime per queue
-                                            // waterMarkPerQueue.extractTimestamp(mq, msg.getBornTimestamp());
-                                            waterMarkForAll.extractTimestamp(msg.getBornTimestamp());
-                                            tpsMetric.markEvent();
-                                        }
-                                    }
-                                    found = true;
-                                    break;
-                                case NO_MATCHED_MSG:
-                                    log.debug("No matched message after offset {} for queue {}", offset, mq);
-                                    break;
-                                case NO_NEW_MSG:
-                                    log.debug("No new message after offset {} for queue {}", offset, mq);
-                                    break;
-                                case OFFSET_ILLEGAL:
-                                    log.warn("Offset {} is illegal for queue {}", offset, mq);
-                                    break;
-                                default:
-                                    break;
-                            }
-
-                            synchronized (checkPointLock) {
-                                updateMessageQueueOffset(mq, pullResult.getNextBeginOffset());
-                            }
-
-                            if (!found) {
-                                RetryUtil.waitForMs(RocketMQConfig.DEFAULT_CONSUMER_DELAY_WHEN_MESSAGE_NOT_FOUND);
-                            }
-                        } catch (Exception e) {
-                            throw new RuntimeException(e);
-                        }
-                    }
-                    return true;
-                }, "RuntimeException");
-            });
-        }
-
-        awaitTermination();
-    }
-
-    private void awaitTermination() throws InterruptedException {
-        while (runningChecker.isRunning()) {
-            Thread.sleep(50);
-        }
-    }
-
-    private long getMessageQueueOffset(MessageQueue mq) throws MQClientException {
-        Long offset = offsetTable.get(mq);
-        // restoredOffsets(unionOffsetStates) is the restored global union state;
-        // should only snapshot mqs that actually belong to us
-        if (offset == null) {
-            // fetchConsumeOffset from broker
-            offset = consumer.fetchConsumeOffset(mq, false);
-            if (!restored || offset < 0) {
-                String initialOffset = props.getProperty(RocketMQConfig.CONSUMER_OFFSET_RESET_TO, CONSUMER_OFFSET_LATEST);
-                switch (initialOffset) {
-                    case CONSUMER_OFFSET_EARLIEST:
-                        offset = consumer.minOffset(mq);
-                        break;
-                    case CONSUMER_OFFSET_LATEST:
-                        offset = consumer.maxOffset(mq);
-                        break;
-                    case CONSUMER_OFFSET_TIMESTAMP:
-                        offset = consumer.searchOffset(mq, getLong(props,
-                                RocketMQConfig.CONSUMER_OFFSET_FROM_TIMESTAMP, System.currentTimeMillis()));
-                        break;
-                    default:
-                        throw new IllegalArgumentException("Unknown value for CONSUMER_OFFSET_RESET_TO.");
-                }
-            }
-        }
-        offsetTable.put(mq, offset);
-        return offsetTable.get(mq);
-    }
-
-    private void updateMessageQueueOffset(MessageQueue mq, long offset) throws MQClientException {
-        offsetTable.put(mq, offset);
-        if (!enableCheckpoint) {
-            consumer.updateConsumeOffset(mq, offset);
-        }
-    }
-
-    @Override
-    public void cancel() {
-        log.debug("cancel ...");
-        runningChecker.setRunning(false);
-
-        if (consumer != null) {
-            consumer.shutdown();
-        }
-
-        if (offsetTable != null) {
-            offsetTable.clear();
-        }
-        if (restoredOffsets != null) {
-            restoredOffsets.clear();
-        }
-        if (pendingOffsetsToCommit != null) {
-            pendingOffsetsToCommit.clear();
-        }
-    }
-
-    @Override
-    public void close() throws Exception {
-        log.debug("close ...");
-        // pretty much the same logic as cancelling
-        try {
-            cancel();
-        } finally {
-            super.close();
-        }
-    }
-
-    public void initOffsetTableFromRestoredOffsets() {
-        Preconditions.checkNotNull(restoredOffsets, "restoredOffsets can't be null");
-        restoredOffsets.forEach((mq, offset) -> {
-            if (!offsetTable.containsKey(mq) || offsetTable.get(mq) < offset) {
-                offsetTable.put(mq, offset);
-            }
-        });
-        log.info("init offset table from restoredOffsets successful.", offsetTable);
-    }
-
-    @Override
-    public void snapshotState(FunctionSnapshotContext context) throws Exception {
-        // called when a snapshot for a checkpoint is requested
-        log.info("Snapshotting state {} ...", context.getCheckpointId());
-        if (!runningChecker.isRunning()) {
-            log.info("snapshotState() called on closed source; returning null.");
-            return;
-        }
-
-        // Discovery topic Route change when snapshot
-        RetryUtil.call(() -> {
-            Collection<MessageQueue> totalQueues = consumer.fetchSubscribeMessageQueues(topic);
-            int taskNumber = getRuntimeContext().getNumberOfParallelSubtasks();
-            int taskIndex = getRuntimeContext().getIndexOfThisSubtask();
-            List<MessageQueue> newQueues = RocketMQUtils.allocate(totalQueues, taskNumber, taskIndex);
-            Collections.sort(newQueues);
-            log.debug(taskIndex + " Topic route is same.");
-            if (!messageQueues.equals(newQueues)) {
-                throw new RuntimeException();
-            }
-            return true;
-        }, "RuntimeException due to topic route changed");
-
-        unionOffsetStates.clear();
-        HashMap<MessageQueue, Long> currentOffsets = new HashMap<>(offsetTable.size());
-        for (Map.Entry<MessageQueue, Long> entry : offsetTable.entrySet()) {
-            unionOffsetStates.add(Tuple2.of(entry.getKey(), entry.getValue()));
-            currentOffsets.put(entry.getKey(), entry.getValue());
-        }
-        pendingOffsetsToCommit.put(context.getCheckpointId(), currentOffsets);
-        log.info("Snapshotted state, last processed offsets: {}, checkpoint id: {}, timestamp: {}",
-                offsetTable, context.getCheckpointId(), context.getCheckpointTimestamp());
-    }
-
-    /**
-     * called every time the user-defined function is initialized,
-     * be that when the function is first initialized or be that
-     * when the function is actually recovering from an earlier checkpoint.
-     * Given this, initializeState() is not only the place where different types of state are initialized,
-     * but also where state recovery logic is included.
-     */
-    @Override
-    public void initializeState(FunctionInitializationContext context) throws Exception {
-        log.info("initialize State ...");
-
-        this.unionOffsetStates = context.getOperatorStateStore().getUnionListState(new ListStateDescriptor<>(
-                OFFSETS_STATE_NAME, TypeInformation.of(new TypeHint<Tuple2<MessageQueue, Long>>() {
-        })));
-        this.restored = context.isRestored();
-
-        if (restored) {
-            if (restoredOffsets == null) {
-                restoredOffsets = new ConcurrentHashMap<>();
-            }
-            for (Tuple2<MessageQueue, Long> mqOffsets : unionOffsetStates.get()) {
-                if (!restoredOffsets.containsKey(mqOffsets.f0) || restoredOffsets.get(mqOffsets.f0) < mqOffsets.f1) {
-                    restoredOffsets.put(mqOffsets.f0, mqOffsets.f1);
-                }
-            }
-            log.info("Setting restore state in the consumer. Using the following offsets: {}", restoredOffsets);
-        } else {
-            log.info("No restore state for the consumer.");
-        }
-    }
-
-    @Override
-    public TypeInformation getProducedType() {
-        return schema.getProducedType();
-    }
-
-    @Override
-    public void notifyCheckpointComplete(long checkpointId) throws Exception {
-        // callback when checkpoint complete
-        if (!runningChecker.isRunning()) {
-            log.info("notifyCheckpointComplete() called on closed source; returning null.");
-            return;
-        }
-
-        final int posInMap = pendingOffsetsToCommit.indexOf(checkpointId);
-        if (posInMap == -1) {
-            log.warn("Received confirmation for unknown checkpoint id {}", checkpointId);
-            return;
-        }
-
-        Map<MessageQueue, Long> offsets = (Map<MessageQueue, Long>) pendingOffsetsToCommit.remove(posInMap);
-
-        // remove older checkpoints in map
-        for (int i = 0; i < posInMap; i++) {
-            pendingOffsetsToCommit.remove(0);
-        }
-
-        if (offsets == null || offsets.size() == 0) {
-            log.debug("Checkpoint state was empty.");
-            return;
-        }
-
-        for (Map.Entry<MessageQueue, Long> entry : offsets.entrySet()) {
-            consumer.updateConsumeOffset(entry.getKey(), entry.getValue());
-        }
-    }
-}
diff --git a/src/main/java/org/apache/rocketmq/flink/RunningChecker.java b/src/main/java/org/apache/rocketmq/flink/RunningChecker.java
deleted file mode 100644
index b7bc2b9..0000000
--- a/src/main/java/org/apache/rocketmq/flink/RunningChecker.java
+++ /dev/null
@@ -1,33 +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;
-
-import java.io.Serializable;
-
-public class RunningChecker implements Serializable {
-    private volatile boolean isRunning = false;
-
-    public boolean isRunning() {
-        return isRunning;
-    }
-
-    public void setRunning(boolean running) {
-        isRunning = running;
-    }
-}
diff --git a/src/main/java/org/apache/rocketmq/flink/common/selector/TopicSelector.java b/src/main/java/org/apache/rocketmq/flink/common/selector/TopicSelector.java
deleted file mode 100644
index 2a347db..0000000
--- a/src/main/java/org/apache/rocketmq/flink/common/selector/TopicSelector.java
+++ /dev/null
@@ -1,28 +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.selector;
-
-import java.io.Serializable;
-
-public interface TopicSelector<T> extends Serializable {
-
-    String getTopic(T tuple);
-
-    String getTag(T tuple);
-}
diff --git a/src/main/java/org/apache/rocketmq/flink/common/serialization/KeyValueDeserializationSchema.java b/src/main/java/org/apache/rocketmq/flink/common/serialization/KeyValueDeserializationSchema.java
deleted file mode 100644
index d8759f9..0000000
--- a/src/main/java/org/apache/rocketmq/flink/common/serialization/KeyValueDeserializationSchema.java
+++ /dev/null
@@ -1,27 +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.serialization;
-
-import java.io.Serializable;
-
-import org.apache.flink.api.java.typeutils.ResultTypeQueryable;
-
-public interface KeyValueDeserializationSchema<T> extends ResultTypeQueryable<T>, Serializable {
-    T deserializeKeyAndValue(byte[] key, byte[] value);
-}
diff --git a/src/main/java/org/apache/rocketmq/flink/common/serialization/KeyValueSerializationSchema.java b/src/main/java/org/apache/rocketmq/flink/common/serialization/KeyValueSerializationSchema.java
deleted file mode 100644
index d847e8a..0000000
--- a/src/main/java/org/apache/rocketmq/flink/common/serialization/KeyValueSerializationSchema.java
+++ /dev/null
@@ -1,28 +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.serialization;
-
-import java.io.Serializable;
-
-public interface KeyValueSerializationSchema<T> extends Serializable {
-
-    byte[] serializeKey(T tuple);
-
-    byte[] serializeValue(T tuple);
-}
diff --git a/src/main/java/org/apache/rocketmq/flink/common/util/MetricUtils.java b/src/main/java/org/apache/rocketmq/flink/common/util/MetricUtils.java
deleted file mode 100644
index 764d01f..0000000
--- a/src/main/java/org/apache/rocketmq/flink/common/util/MetricUtils.java
+++ /dev/null
@@ -1,80 +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.util;
-
-import org.apache.flink.api.common.functions.RuntimeContext;
-import org.apache.flink.metrics.Counter;
-import org.apache.flink.metrics.Gauge;
-import org.apache.flink.metrics.Meter;
-import org.apache.flink.metrics.MeterView;
-import org.apache.flink.metrics.SimpleCounter;
-
-/**
- * RocketMQ connector metrics.
- */
-public class MetricUtils {
-
-    public static final String METRICS_TPS = "tps";
-
-    private static final String METRIC_GROUP_SINK = "sink";
-    private static final String METRICS_SINK_IN_TPS = "inTps";
-    private static final String METRICS_SINK_OUT_TPS = "outTps";
-    private static final String METRICS_SINK_OUT_BPS = "outBps";
-    private static final String METRICS_SINK_OUT_Latency = "outLatency";
-
-    public static Meter registerSinkInTps(RuntimeContext context) {
-        Counter parserCounter = context.getMetricGroup().addGroup(METRIC_GROUP_SINK)
-            .counter(METRICS_SINK_IN_TPS + "_counter", new SimpleCounter());
-        return context.getMetricGroup().addGroup(METRIC_GROUP_SINK)
-            .meter(METRICS_SINK_IN_TPS, new MeterView(parserCounter, 60));
-    }
-
-    public static Meter registerOutTps(RuntimeContext context) {
-        Counter parserCounter = context.getMetricGroup().addGroup(METRIC_GROUP_SINK)
-            .counter(METRICS_SINK_OUT_TPS + "_counter", new SimpleCounter());
-        return context.getMetricGroup().addGroup(METRIC_GROUP_SINK)
-                .meter(METRICS_SINK_OUT_TPS, new MeterView(parserCounter, 60));
-    }
-
-    public static Meter registerOutBps(RuntimeContext context) {
-        Counter bpsCounter = context.getMetricGroup().addGroup(METRIC_GROUP_SINK)
-                .counter(METRICS_SINK_OUT_BPS + "_counter", new SimpleCounter());
-        return context.getMetricGroup().addGroup(METRIC_GROUP_SINK)
-                .meter(METRICS_SINK_OUT_BPS, new MeterView(bpsCounter, 60));
-    }
-
-    public static LatencyGauge registerOutLatency(RuntimeContext context) {
-        return context.getMetricGroup().addGroup(METRIC_GROUP_SINK).gauge(METRICS_SINK_OUT_Latency, new LatencyGauge());
-    }
-
-    public static class LatencyGauge implements Gauge<Double> {
-        private double value;
-
-        public void report(long timeDelta, long batchSize) {
-            if (batchSize != 0) {
-                this.value = (1.0 * timeDelta) / batchSize;
-            }
-        }
-
-        @Override
-        public Double getValue() {
-            return value;
-        }
-    }
-}
diff --git a/src/main/java/org/apache/rocketmq/flink/common/util/RocketMQUtils.java b/src/main/java/org/apache/rocketmq/flink/common/util/RocketMQUtils.java
deleted file mode 100644
index fc37b04..0000000
--- a/src/main/java/org/apache/rocketmq/flink/common/util/RocketMQUtils.java
+++ /dev/null
@@ -1,73 +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.util;
-
-import org.apache.rocketmq.client.AccessChannel;
-import org.apache.rocketmq.common.message.MessageQueue;
-
-import java.lang.management.ManagementFactory;
-import java.util.*;
-
-public final class RocketMQUtils {
-
-    public static int getInteger(Properties props, String key, int defaultValue) {
-        return Integer.parseInt(props.getProperty(key, String.valueOf(defaultValue)));
-    }
-
-    public static long getLong(Properties props, String key, long defaultValue) {
-        return Long.parseLong(props.getProperty(key, String.valueOf(defaultValue)));
-    }
-
-    public static boolean getBoolean(Properties props, String key, boolean defaultValue) {
-        return Boolean.parseBoolean(props.getProperty(key, String.valueOf(defaultValue)));
-    }
-
-    public static AccessChannel getAccessChannel(Properties props, String key, AccessChannel defaultValue) {
-        return AccessChannel.valueOf(props.getProperty(key, String.valueOf(defaultValue)));
-    }
-
-    public static String getInstanceName(String... args) {
-        if (null != args && args.length > 0) {
-            return String.join("_", args);
-        }
-        return ManagementFactory.getRuntimeMXBean().getName() + "_" + System.nanoTime();
-    }
-
-    /**
-     * Average Hashing queue algorithm
-     * Refer: org.apache.rocketmq.client.consumer.rebalance.AllocateMessageQueueAveragely
-     */
-    public static List<MessageQueue> allocate(Collection<MessageQueue> mqSet,
-                                              int numberOfParallelTasks,
-                                              int indexOfThisTask) {
-        ArrayList<MessageQueue> mqAll = new ArrayList<>(mqSet);
-        Collections.sort(mqAll);
-        List<MessageQueue> result = new ArrayList<>();
-        int mod = mqAll.size() % numberOfParallelTasks;
-        int averageSize = mqAll.size() <= numberOfParallelTasks ? 1 : (mod > 0 && indexOfThisTask < mod ?
-                mqAll.size() / numberOfParallelTasks + 1 : mqAll.size() / numberOfParallelTasks);
-        int startIndex = (mod > 0 && indexOfThisTask < mod) ? indexOfThisTask * averageSize :
-                indexOfThisTask * averageSize + mod;
-        int range = Math.min(averageSize, mqAll.size() - startIndex);
-        for (int i = 0; i < range; i++) {
-            result.add(mqAll.get((startIndex + i) % mqAll.size()));
-        }
-        return result;
-    }
-}
diff --git a/src/main/java/org/apache/rocketmq/flink/common/util/TestUtils.java b/src/main/java/org/apache/rocketmq/flink/common/util/TestUtils.java
deleted file mode 100644
index 71d1265..0000000
--- a/src/main/java/org/apache/rocketmq/flink/common/util/TestUtils.java
+++ /dev/null
@@ -1,33 +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.util;
-
-import java.lang.reflect.Field;
-
-public class TestUtils {
-    public static void setFieldValue(Object obj, String fieldName, Object value) {
-        try {
-            Field field = obj.getClass().getDeclaredField(fieldName);
-            field.setAccessible(true);
-            field.set(obj, value);
-        } catch (Exception e) {
-            e.printStackTrace();
-        }
-    }
-}
diff --git a/src/main/java/org/apache/rocketmq/flink/RocketMQConfig.java b/src/main/java/org/apache/rocketmq/flink/legacy/RocketMQConfig.java
similarity index 77%
rename from src/main/java/org/apache/rocketmq/flink/RocketMQConfig.java
rename to src/main/java/org/apache/rocketmq/flink/legacy/RocketMQConfig.java
index c1bad2d..5c19b7a 100644
--- a/src/main/java/org/apache/rocketmq/flink/RocketMQConfig.java
+++ b/src/main/java/org/apache/rocketmq/flink/legacy/RocketMQConfig.java
@@ -1,25 +1,19 @@
 /**
- * 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
- * <p>
- * http://www.apache.org/licenses/LICENSE-2.0
- * <p>
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
+ * 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
+ *
+ * <p>http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * <p>Unless required by applicable law or agreed to in writing, software distributed under the
+ * License is distributed on an "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either
+ * express or implied. See the License for the specific language governing permissions and
  * limitations under the License.
  */
+package org.apache.rocketmq.flink.legacy;
 
-package org.apache.rocketmq.flink;
-
-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;
@@ -28,15 +22,16 @@ import org.apache.rocketmq.client.consumer.DefaultMQPullConsumer;
 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 java.util.Properties;
 import java.util.UUID;
 
-import static org.apache.rocketmq.flink.common.util.RocketMQUtils.getAccessChannel;
-import static org.apache.rocketmq.flink.common.util.RocketMQUtils.getInteger;
+import static org.apache.rocketmq.flink.legacy.common.util.RocketMQUtils.getAccessChannel;
+import static org.apache.rocketmq.flink.legacy.common.util.RocketMQUtils.getInteger;
 
-/**
- * RocketMQConfig for Consumer/Producer.
- */
+/** RocketMQConfig for Consumer/Producer. */
 public class RocketMQConfig {
     // Server Config
     public static final String NAME_SERVER_ADDR = "nameserver.address"; // Required
@@ -77,13 +72,15 @@ public class RocketMQConfig {
     public static final String CONSUMER_OFFSET_TIMESTAMP = "timestamp";
     public static final String CONSUMER_OFFSET_FROM_TIMESTAMP = "consumer.offset.from.timestamp";
 
-    public static final String CONSUMER_OFFSET_PERSIST_INTERVAL = "consumer.offset.persist.interval";
+    public static final String CONSUMER_OFFSET_PERSIST_INTERVAL =
+            "consumer.offset.persist.interval";
     public static final int DEFAULT_CONSUMER_OFFSET_PERSIST_INTERVAL = 5000; // 5 seconds
 
     public static final String CONSUMER_BATCH_SIZE = "consumer.batch.size";
     public static final int DEFAULT_CONSUMER_BATCH_SIZE = 32;
 
-    public static final String CONSUMER_DELAY_WHEN_MESSAGE_NOT_FOUND = "consumer.delay.when.message.not.found";
+    public static final String CONSUMER_DELAY_WHEN_MESSAGE_NOT_FOUND =
+            "consumer.delay.when.message.not.found";
     public static final int DEFAULT_CONSUMER_DELAY_WHEN_MESSAGE_NOT_FOUND = 100;
 
     public static final String CONSUMER_INDEX_OF_THIS_SUB_TASK = "consumer.index";
@@ -116,6 +113,7 @@ public class RocketMQConfig {
 
     /**
      * Build Producer Configs.
+     *
      * @param props Properties
      * @param producer DefaultMQProducer
      */
@@ -126,27 +124,32 @@ public class RocketMQConfig {
             group = UUID.randomUUID().toString();
         }
         producer.setProducerGroup(props.getProperty(PRODUCER_GROUP, group));
-        producer.setRetryTimesWhenSendFailed(getInteger(props, PRODUCER_RETRY_TIMES, DEFAULT_PRODUCER_RETRY_TIMES));
-        producer.setRetryTimesWhenSendAsyncFailed(getInteger(props,
-                PRODUCER_RETRY_TIMES, DEFAULT_PRODUCER_RETRY_TIMES));
+        producer.setRetryTimesWhenSendFailed(
+                getInteger(props, PRODUCER_RETRY_TIMES, DEFAULT_PRODUCER_RETRY_TIMES));
+        producer.setRetryTimesWhenSendAsyncFailed(
+                getInteger(props, PRODUCER_RETRY_TIMES, DEFAULT_PRODUCER_RETRY_TIMES));
         producer.setSendMsgTimeout(getInteger(props, PRODUCER_TIMEOUT, DEFAULT_PRODUCER_TIMEOUT));
-
     }
 
     /**
      * Build Consumer Configs.
+     *
      * @param props Properties
      * @param consumer DefaultMQPullConsumer
      */
     public static void buildConsumerConfigs(Properties props, DefaultMQPullConsumer consumer) {
         buildCommonConfigs(props, consumer);
         consumer.setMessageModel(MessageModel.CLUSTERING);
-        consumer.setPersistConsumerOffsetInterval(getInteger(props,
-                CONSUMER_OFFSET_PERSIST_INTERVAL, DEFAULT_CONSUMER_OFFSET_PERSIST_INTERVAL));
+        consumer.setPersistConsumerOffsetInterval(
+                getInteger(
+                        props,
+                        CONSUMER_OFFSET_PERSIST_INTERVAL,
+                        DEFAULT_CONSUMER_OFFSET_PERSIST_INTERVAL));
     }
 
     /**
      * Build Common Configs.
+     *
      * @param props Properties
      * @param client ClientConfig
      */
@@ -154,8 +157,8 @@ public class RocketMQConfig {
         String nameServers = props.getProperty(NAME_SERVER_ADDR);
         Validate.notEmpty(nameServers);
         client.setNamesrvAddr(nameServers);
-        client.setHeartbeatBrokerInterval(getInteger(props,
-                BROKER_HEART_BEAT_INTERVAL, DEFAULT_BROKER_HEART_BEAT_INTERVAL));
+        client.setHeartbeatBrokerInterval(
+                getInteger(props, BROKER_HEART_BEAT_INTERVAL, DEFAULT_BROKER_HEART_BEAT_INTERVAL));
         // When using aliyun products, you need to set up channels
         client.setAccessChannel((getAccessChannel(props, ACCESS_CHANNEL, DEFAULT_ACCESS_CHANNEL)));
         client.setUnitName(props.getProperty(UNIT_NAME, null));
@@ -163,6 +166,7 @@ public class RocketMQConfig {
 
     /**
      * Build credentials for client.
+     *
      * @param props
      * @return
      */
diff --git a/src/main/java/org/apache/rocketmq/flink/RocketMQSink.java b/src/main/java/org/apache/rocketmq/flink/legacy/RocketMQSink.java
similarity index 69%
rename from src/main/java/org/apache/rocketmq/flink/RocketMQSink.java
rename to src/main/java/org/apache/rocketmq/flink/legacy/RocketMQSink.java
index 865af75..f91a684 100644
--- a/src/main/java/org/apache/rocketmq/flink/RocketMQSink.java
+++ b/src/main/java/org/apache/rocketmq/flink/legacy/RocketMQSink.java
@@ -1,24 +1,28 @@
 /**
- * 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
- * <p>
- * http://www.apache.org/licenses/LICENSE-2.0
- * <p>
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
+ * 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
+ *
+ * <p>http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * <p>Unless required by applicable law or agreed to in writing, software distributed under the
+ * License is distributed on an "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either
+ * express or implied. See the License for the specific language governing permissions and
  * limitations under the License.
  */
+package org.apache.rocketmq.flink.legacy;
 
-package org.apache.rocketmq.flink;
+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.util.MetricUtils;
+import org.apache.rocketmq.remoting.exception.RemotingException;
 
-import org.apache.commons.lang.Validate;
 import org.apache.flink.configuration.Configuration;
 import org.apache.flink.metrics.Meter;
 import org.apache.flink.runtime.state.FunctionInitializationContext;
@@ -26,15 +30,8 @@ import org.apache.flink.runtime.state.FunctionSnapshotContext;
 import org.apache.flink.streaming.api.checkpoint.CheckpointedFunction;
 import org.apache.flink.streaming.api.functions.sink.RichSinkFunction;
 import org.apache.flink.streaming.api.operators.StreamingRuntimeContext;
-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.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.common.util.MetricUtils;
-import org.apache.rocketmq.remoting.exception.RemotingException;
+
+import org.apache.commons.lang.Validate;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 
@@ -44,9 +41,9 @@ import java.util.Properties;
 import java.util.UUID;
 
 /**
- * The RocketMQSink provides at-least-once reliability guarantees when
- * checkpoints are enabled and batchFlushOnCheckpoint(true) is set.
- * Otherwise, the sink reliability guarantees depends on rocketmq producer's retry policy.
+ * The RocketMQSink provides at-least-once reliability guarantees when checkpoints are enabled and
+ * batchFlushOnCheckpoint(true) is set. Otherwise, the sink reliability guarantees depends on
+ * rocketmq producer's retry policy.
  */
 public class RocketMQSink<IN> extends RichSinkFunction<IN> implements CheckpointedFunction {
 
@@ -78,14 +75,17 @@ public class RocketMQSink<IN> extends RichSinkFunction<IN> implements Checkpoint
 
         // with authentication hook
         producer = new DefaultMQProducer(RocketMQConfig.buildAclRPCHook(props));
-        producer.setInstanceName(getRuntimeContext().getIndexOfThisSubtask() + "_" + UUID.randomUUID());
+        producer.setInstanceName(
+                getRuntimeContext().getIndexOfThisSubtask() + "_" + UUID.randomUUID());
 
         RocketMQConfig.buildProducerConfigs(props, producer);
 
         batchList = new LinkedList<>();
 
-        if (batchFlushOnCheckpoint && !((StreamingRuntimeContext) getRuntimeContext()).isCheckpointingEnabled()) {
-            LOG.info("Flushing on checkpoint is enabled, but checkpointing is not enabled. Disabling flushing.");
+        if (batchFlushOnCheckpoint
+                && !((StreamingRuntimeContext) getRuntimeContext()).isCheckpointingEnabled()) {
+            LOG.info(
+                    "Flushing on checkpoint is enabled, but checkpointing is not enabled. Disabling flushing.");
             batchFlushOnCheckpoint = false;
         }
 
@@ -117,23 +117,25 @@ public class RocketMQSink<IN> extends RichSinkFunction<IN> implements Checkpoint
         long timeStartWriting = System.currentTimeMillis();
         if (async) {
             try {
-                producer.send(msg, new SendCallback() {
-                    @Override
-                    public void onSuccess(SendResult sendResult) {
-                        LOG.debug("Async send message success! result: {}", sendResult);
-                        long end = System.currentTimeMillis();
-                        latencyGauge.report(end - timeStartWriting, 1);
-                        outTps.markEvent();
-                        outBps.markEvent(msg.getBody().length);
-                    }
-
-                    @Override
-                    public void onException(Throwable throwable) {
-                        if (throwable != null) {
-                            LOG.error("Async send message failure!", throwable);
-                        }
-                    }
-                });
+                producer.send(
+                        msg,
+                        new SendCallback() {
+                            @Override
+                            public void onSuccess(SendResult sendResult) {
+                                LOG.debug("Async send message success! result: {}", sendResult);
+                                long end = System.currentTimeMillis();
+                                latencyGauge.report(end - timeStartWriting, 1);
+                                outTps.markEvent();
+                                outBps.markEvent(msg.getBody().length);
+                            }
+
+                            @Override
+                            public void onException(Throwable throwable) {
+                                if (throwable != null) {
+                                    LOG.error("Async send message failure!", throwable);
+                                }
+                            }
+                        });
             } catch (Exception e) {
                 LOG.error("Async send message failure!", e);
             }
diff --git a/src/main/java/org/apache/rocketmq/flink/legacy/RocketMQSource.java b/src/main/java/org/apache/rocketmq/flink/legacy/RocketMQSource.java
new file mode 100644
index 0000000..84260b6
--- /dev/null
+++ b/src/main/java/org/apache/rocketmq/flink/legacy/RocketMQSource.java
@@ -0,0 +1,528 @@
+/**
+ * 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
+ *
+ * <p>http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * <p>Unless required by applicable law or agreed to in writing, software distributed under the
+ * License is distributed on an "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either
+ * express or implied. See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.rocketmq.flink.legacy;
+
+import org.apache.rocketmq.client.consumer.DefaultMQPullConsumer;
+import org.apache.rocketmq.client.consumer.PullResult;
+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.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;
+
+import org.apache.flink.api.common.functions.RuntimeContext;
+import org.apache.flink.api.common.state.ListState;
+import org.apache.flink.api.common.state.ListStateDescriptor;
+import org.apache.flink.api.common.typeinfo.TypeHint;
+import org.apache.flink.api.common.typeinfo.TypeInformation;
+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.metrics.Counter;
+import org.apache.flink.metrics.Meter;
+import org.apache.flink.metrics.MeterView;
+import org.apache.flink.metrics.SimpleCounter;
+import org.apache.flink.runtime.state.CheckpointListener;
+import org.apache.flink.runtime.state.FunctionInitializationContext;
+import org.apache.flink.runtime.state.FunctionSnapshotContext;
+import org.apache.flink.streaming.api.checkpoint.CheckpointedFunction;
+import org.apache.flink.streaming.api.functions.source.RichParallelSourceFunction;
+import org.apache.flink.streaming.api.operators.StreamingRuntimeContext;
+import org.apache.flink.util.Preconditions;
+
+import org.apache.flink.shaded.curator4.com.google.common.util.concurrent.ThreadFactoryBuilder;
+
+import org.apache.commons.collections.map.LinkedMap;
+import org.apache.commons.lang.Validate;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import java.lang.management.ManagementFactory;
+import java.nio.charset.StandardCharsets;
+import java.util.Collection;
+import java.util.Collections;
+import java.util.HashMap;
+import java.util.List;
+import java.util.Map;
+import java.util.Properties;
+import java.util.concurrent.ConcurrentHashMap;
+import java.util.concurrent.ExecutorService;
+import java.util.concurrent.Executors;
+import java.util.concurrent.ScheduledExecutorService;
+import java.util.concurrent.ThreadFactory;
+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_OFFSET_EARLIEST;
+import static org.apache.rocketmq.flink.legacy.RocketMQConfig.CONSUMER_OFFSET_LATEST;
+import static org.apache.rocketmq.flink.legacy.RocketMQConfig.CONSUMER_OFFSET_TIMESTAMP;
+import static org.apache.rocketmq.flink.legacy.RocketMQConfig.DEFAULT_CONSUMER_BATCH_SIZE;
+import static org.apache.rocketmq.flink.legacy.common.util.RocketMQUtils.getInteger;
+import static org.apache.rocketmq.flink.legacy.common.util.RocketMQUtils.getLong;
+
+/**
+ * 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
+ * guarantees.
+ */
+public class RocketMQSource<OUT> extends RichParallelSourceFunction<OUT>
+        implements CheckpointedFunction, CheckpointListener, ResultTypeQueryable<OUT> {
+
+    private static final long serialVersionUID = 1L;
+
+    private static final Logger log = LoggerFactory.getLogger(RocketMQSource.class);
+    private static final String OFFSETS_STATE_NAME = "topic-partition-offset-states";
+    private RunningChecker runningChecker;
+    private transient DefaultMQPullConsumer consumer;
+    private KeyValueDeserializationSchema<OUT> schema;
+    private transient ListState<Tuple2<MessageQueue, Long>> unionOffsetStates;
+    private Map<MessageQueue, Long> offsetTable;
+    private Map<MessageQueue, Long> restoredOffsets;
+    private List<MessageQueue> messageQueues;
+    private ExecutorService executor;
+
+    // watermark in source
+    private WaterMarkPerQueue waterMarkPerQueue;
+    private WaterMarkForAll waterMarkForAll;
+
+    private ScheduledExecutorService timer;
+    /** Data for pending but uncommitted offsets. */
+    private LinkedMap pendingOffsetsToCommit;
+
+    private Properties props;
+    private String topic;
+    private String group;
+    private transient volatile boolean restored;
+    private transient boolean enableCheckpoint;
+    private volatile Object checkPointLock;
+
+    private Meter tpsMetric;
+
+    public RocketMQSource(KeyValueDeserializationSchema<OUT> schema, Properties props) {
+        this.schema = schema;
+        this.props = props;
+    }
+
+    @Override
+    public void open(Configuration parameters) throws Exception {
+        log.debug("source open....");
+        Validate.notEmpty(props, "Consumer properties can not be empty");
+
+        this.topic = props.getProperty(RocketMQConfig.CONSUMER_TOPIC);
+        this.group = props.getProperty(RocketMQConfig.CONSUMER_GROUP);
+
+        Validate.notEmpty(topic, "Consumer topic can not be empty");
+        Validate.notEmpty(group, "Consumer group can not be empty");
+
+        this.enableCheckpoint =
+                ((StreamingRuntimeContext) getRuntimeContext()).isCheckpointingEnabled();
+
+        if (offsetTable == null) {
+            offsetTable = new ConcurrentHashMap<>();
+        }
+        if (restoredOffsets == null) {
+            restoredOffsets = new ConcurrentHashMap<>();
+        }
+
+        // use restoredOffsets to init offset table.
+        initOffsetTableFromRestoredOffsets();
+
+        if (pendingOffsetsToCommit == null) {
+            pendingOffsetsToCommit = new LinkedMap();
+        }
+        if (checkPointLock == null) {
+            checkPointLock = new ReentrantLock();
+        }
+        if (waterMarkPerQueue == null) {
+            waterMarkPerQueue = new WaterMarkPerQueue(5000);
+        }
+        if (waterMarkForAll == null) {
+            waterMarkForAll = new WaterMarkForAll(5000);
+        }
+        if (timer == null) {
+            timer = Executors.newSingleThreadScheduledExecutor();
+        }
+
+        runningChecker = new RunningChecker();
+        runningChecker.setRunning(true);
+
+        final ThreadFactory threadFactory =
+                new ThreadFactoryBuilder()
+                        .setDaemon(true)
+                        .setNameFormat("rmq-pull-thread-%d")
+                        .build();
+        executor = Executors.newCachedThreadPool(threadFactory);
+
+        int indexOfThisSubTask = getRuntimeContext().getIndexOfThisSubtask();
+        consumer = new DefaultMQPullConsumer(group, RocketMQConfig.buildAclRPCHook(props));
+        RocketMQConfig.buildConsumerConfigs(props, consumer);
+
+        // set unique instance name, avoid exception:
+        // https://help.aliyun.com/document_detail/29646.html
+        String runtimeName = ManagementFactory.getRuntimeMXBean().getName();
+        String instanceName =
+                RocketMQUtils.getInstanceName(
+                        runtimeName,
+                        topic,
+                        group,
+                        String.valueOf(indexOfThisSubTask),
+                        String.valueOf(System.nanoTime()));
+        consumer.setInstanceName(instanceName);
+        consumer.start();
+
+        Counter outputCounter =
+                getRuntimeContext()
+                        .getMetricGroup()
+                        .counter(MetricUtils.METRICS_TPS + "_counter", new SimpleCounter());
+        tpsMetric =
+                getRuntimeContext()
+                        .getMetricGroup()
+                        .meter(MetricUtils.METRICS_TPS, new MeterView(outputCounter, 60));
+    }
+
+    @Override
+    public void run(SourceContext context) throws Exception {
+        String tag =
+                props.getProperty(RocketMQConfig.CONSUMER_TAG, RocketMQConfig.DEFAULT_CONSUMER_TAG);
+        int pullBatchSize = getInteger(props, CONSUMER_BATCH_SIZE, DEFAULT_CONSUMER_BATCH_SIZE);
+
+        final RuntimeContext ctx = getRuntimeContext();
+        // The lock that guarantees that record emission and state updates are atomic,
+        // from the view of taking a checkpoint.
+        int taskNumber = ctx.getNumberOfParallelSubtasks();
+        int taskIndex = ctx.getIndexOfThisSubtask();
+        log.info("Source run, NumberOfTotalTask={}, IndexOfThisSubTask={}", taskNumber, taskIndex);
+
+        timer.scheduleAtFixedRate(
+                () -> {
+                    // context.emitWatermark(waterMarkPerQueue.getCurrentWatermark());
+                    context.emitWatermark(waterMarkForAll.getCurrentWatermark());
+                },
+                5,
+                5,
+                TimeUnit.SECONDS);
+
+        Collection<MessageQueue> totalQueues = consumer.fetchSubscribeMessageQueues(topic);
+        messageQueues =
+                RocketMQUtils.allocate(totalQueues, taskNumber, ctx.getIndexOfThisSubtask());
+        for (MessageQueue mq : messageQueues) {
+            this.executor.execute(
+                    () -> {
+                        RetryUtil.call(
+                                () -> {
+                                    while (runningChecker.isRunning()) {
+                                        try {
+                                            long offset = getMessageQueueOffset(mq);
+                                            PullResult pullResult =
+                                                    consumer.pullBlockIfNotFound(
+                                                            mq, tag, offset, pullBatchSize);
+
+                                            boolean found = false;
+                                            switch (pullResult.getPullStatus()) {
+                                                case FOUND:
+                                                    List<MessageExt> messages =
+                                                            pullResult.getMsgFoundList();
+                                                    for (MessageExt msg : messages) {
+                                                        byte[] key =
+                                                                msg.getKeys() != null
+                                                                        ? msg.getKeys()
+                                                                                .getBytes(
+                                                                                        StandardCharsets
+                                                                                                .UTF_8)
+                                                                        : null;
+                                                        byte[] value = msg.getBody();
+                                                        OUT data =
+                                                                schema.deserializeKeyAndValue(
+                                                                        key, value);
+
+                                                        // output and state update are atomic
+                                                        synchronized (checkPointLock) {
+                                                            log.debug(
+                                                                    msg.getMsgId()
+                                                                            + "_"
+                                                                            + msg.getBrokerName()
+                                                                            + " "
+                                                                            + msg.getQueueId()
+                                                                            + " "
+                                                                            + msg.getQueueOffset());
+                                                            context.collectWithTimestamp(
+                                                                    data, msg.getBornTimestamp());
+
+                                                            // update max eventTime per queue
+                                                            // waterMarkPerQueue.extractTimestamp(mq, msg.getBornTimestamp());
+                                                            waterMarkForAll.extractTimestamp(
+                                                                    msg.getBornTimestamp());
+                                                            tpsMetric.markEvent();
+                                                        }
+                                                    }
+                                                    found = true;
+                                                    break;
+                                                case NO_MATCHED_MSG:
+                                                    log.debug(
+                                                            "No matched message after offset {} for queue {}",
+                                                            offset,
+                                                            mq);
+                                                    break;
+                                                case NO_NEW_MSG:
+                                                    log.debug(
+                                                            "No new message after offset {} for queue {}",
+                                                            offset,
+                                                            mq);
+                                                    break;
+                                                case OFFSET_ILLEGAL:
+                                                    log.warn(
+                                                            "Offset {} is illegal for queue {}",
+                                                            offset,
+                                                            mq);
+                                                    break;
+                                                default:
+                                                    break;
+                                            }
+
+                                            synchronized (checkPointLock) {
+                                                updateMessageQueueOffset(
+                                                        mq, pullResult.getNextBeginOffset());
+                                            }
+
+                                            if (!found) {
+                                                RetryUtil.waitForMs(
+                                                        RocketMQConfig
+                                                                .DEFAULT_CONSUMER_DELAY_WHEN_MESSAGE_NOT_FOUND);
+                                            }
+                                        } catch (Exception e) {
+                                            throw new RuntimeException(e);
+                                        }
+                                    }
+                                    return true;
+                                },
+                                "RuntimeException");
+                    });
+        }
+
+        awaitTermination();
+    }
+
+    private void awaitTermination() throws InterruptedException {
+        while (runningChecker.isRunning()) {
+            Thread.sleep(50);
+        }
+    }
+
+    private long getMessageQueueOffset(MessageQueue mq) throws MQClientException {
+        Long offset = offsetTable.get(mq);
+        // restoredOffsets(unionOffsetStates) is the restored global union state;
+        // should only snapshot mqs that actually belong to us
+        if (offset == null) {
+            // fetchConsumeOffset from broker
+            offset = consumer.fetchConsumeOffset(mq, false);
+            if (!restored || offset < 0) {
+                String initialOffset =
+                        props.getProperty(
+                                RocketMQConfig.CONSUMER_OFFSET_RESET_TO, CONSUMER_OFFSET_LATEST);
+                switch (initialOffset) {
+                    case CONSUMER_OFFSET_EARLIEST:
+                        offset = consumer.minOffset(mq);
+                        break;
+                    case CONSUMER_OFFSET_LATEST:
+                        offset = consumer.maxOffset(mq);
+                        break;
+                    case CONSUMER_OFFSET_TIMESTAMP:
+                        offset =
+                                consumer.searchOffset(
+                                        mq,
+                                        getLong(
+                                                props,
+                                                RocketMQConfig.CONSUMER_OFFSET_FROM_TIMESTAMP,
+                                                System.currentTimeMillis()));
+                        break;
+                    default:
+                        throw new IllegalArgumentException(
+                                "Unknown value for CONSUMER_OFFSET_RESET_TO.");
+                }
+            }
+        }
+        offsetTable.put(mq, offset);
+        return offsetTable.get(mq);
+    }
+
+    private void updateMessageQueueOffset(MessageQueue mq, long offset) throws MQClientException {
+        offsetTable.put(mq, offset);
+        if (!enableCheckpoint) {
+            consumer.updateConsumeOffset(mq, offset);
+        }
+    }
+
+    @Override
+    public void cancel() {
+        log.debug("cancel ...");
+        runningChecker.setRunning(false);
+
+        if (consumer != null) {
+            consumer.shutdown();
+        }
+
+        if (offsetTable != null) {
+            offsetTable.clear();
+        }
+        if (restoredOffsets != null) {
+            restoredOffsets.clear();
+        }
+        if (pendingOffsetsToCommit != null) {
+            pendingOffsetsToCommit.clear();
+        }
+    }
+
+    @Override
+    public void close() throws Exception {
+        log.debug("close ...");
+        // pretty much the same logic as cancelling
+        try {
+            cancel();
+        } finally {
+            super.close();
+        }
+    }
+
+    public void initOffsetTableFromRestoredOffsets() {
+        Preconditions.checkNotNull(restoredOffsets, "restoredOffsets can't be null");
+        restoredOffsets.forEach(
+                (mq, offset) -> {
+                    if (!offsetTable.containsKey(mq) || offsetTable.get(mq) < offset) {
+                        offsetTable.put(mq, offset);
+                    }
+                });
+        log.info("init offset table from restoredOffsets successful.", offsetTable);
+    }
+
+    @Override
+    public void snapshotState(FunctionSnapshotContext context) throws Exception {
+        // called when a snapshot for a checkpoint is requested
+        log.info("Snapshotting state {} ...", context.getCheckpointId());
+        if (!runningChecker.isRunning()) {
+            log.info("snapshotState() called on closed source; returning null.");
+            return;
+        }
+
+        // Discovery topic Route change when snapshot
+        RetryUtil.call(
+                () -> {
+                    Collection<MessageQueue> totalQueues =
+                            consumer.fetchSubscribeMessageQueues(topic);
+                    int taskNumber = getRuntimeContext().getNumberOfParallelSubtasks();
+                    int taskIndex = getRuntimeContext().getIndexOfThisSubtask();
+                    List<MessageQueue> newQueues =
+                            RocketMQUtils.allocate(totalQueues, taskNumber, taskIndex);
+                    Collections.sort(newQueues);
+                    log.debug(taskIndex + " Topic route is same.");
+                    if (!messageQueues.equals(newQueues)) {
+                        throw new RuntimeException();
+                    }
+                    return true;
+                },
+                "RuntimeException due to topic route changed");
+
+        unionOffsetStates.clear();
+        HashMap<MessageQueue, Long> currentOffsets = new HashMap<>(offsetTable.size());
+        for (Map.Entry<MessageQueue, Long> entry : offsetTable.entrySet()) {
+            unionOffsetStates.add(Tuple2.of(entry.getKey(), entry.getValue()));
+            currentOffsets.put(entry.getKey(), entry.getValue());
+        }
+        pendingOffsetsToCommit.put(context.getCheckpointId(), currentOffsets);
+        log.info(
+                "Snapshotted state, last processed offsets: {}, checkpoint id: {}, timestamp: {}",
+                offsetTable,
+                context.getCheckpointId(),
+                context.getCheckpointTimestamp());
+    }
+
+    /**
+     * called every time the user-defined function is initialized, be that when the function is
+     * first initialized or be that when the function is actually recovering from an earlier
+     * checkpoint. Given this, initializeState() is not only the place where different types of
+     * state are initialized, but also where state recovery logic is included.
+     */
+    @Override
+    public void initializeState(FunctionInitializationContext context) throws Exception {
+        log.info("initialize State ...");
+
+        this.unionOffsetStates =
+                context.getOperatorStateStore()
+                        .getUnionListState(
+                                new ListStateDescriptor<>(
+                                        OFFSETS_STATE_NAME,
+                                        TypeInformation.of(
+                                                new TypeHint<Tuple2<MessageQueue, Long>>() {})));
+        this.restored = context.isRestored();
+
+        if (restored) {
+            if (restoredOffsets == null) {
+                restoredOffsets = new ConcurrentHashMap<>();
+            }
+            for (Tuple2<MessageQueue, Long> mqOffsets : unionOffsetStates.get()) {
+                if (!restoredOffsets.containsKey(mqOffsets.f0)
+                        || restoredOffsets.get(mqOffsets.f0) < mqOffsets.f1) {
+                    restoredOffsets.put(mqOffsets.f0, mqOffsets.f1);
+                }
+            }
+            log.info(
+                    "Setting restore state in the consumer. Using the following offsets: {}",
+                    restoredOffsets);
+        } else {
+            log.info("No restore state for the consumer.");
+        }
+    }
+
+    @Override
+    public TypeInformation getProducedType() {
+        return schema.getProducedType();
+    }
+
+    @Override
+    public void notifyCheckpointComplete(long checkpointId) throws Exception {
+        // callback when checkpoint complete
+        if (!runningChecker.isRunning()) {
+            log.info("notifyCheckpointComplete() called on closed source; returning null.");
+            return;
+        }
+
+        final int posInMap = pendingOffsetsToCommit.indexOf(checkpointId);
+        if (posInMap == -1) {
+            log.warn("Received confirmation for unknown checkpoint id {}", checkpointId);
+            return;
+        }
+
+        Map<MessageQueue, Long> offsets =
+                (Map<MessageQueue, Long>) pendingOffsetsToCommit.remove(posInMap);
+
+        // remove older checkpoints in map
+        for (int i = 0; i < posInMap; i++) {
+            pendingOffsetsToCommit.remove(0);
+        }
+
+        if (offsets == null || offsets.size() == 0) {
+            log.debug("Checkpoint state was empty.");
+            return;
+        }
+
+        for (Map.Entry<MessageQueue, Long> entry : offsets.entrySet()) {
+            consumer.updateConsumeOffset(entry.getKey(), entry.getValue());
+        }
+    }
+}
diff --git a/src/main/java/org/apache/rocketmq/flink/legacy/RunningChecker.java b/src/main/java/org/apache/rocketmq/flink/legacy/RunningChecker.java
new file mode 100644
index 0000000..c48361a
--- /dev/null
+++ b/src/main/java/org/apache/rocketmq/flink/legacy/RunningChecker.java
@@ -0,0 +1,29 @@
+/**
+ * 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
+ *
+ * <p>http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * <p>Unless required by applicable law or agreed to in writing, software distributed under the
+ * License is distributed on an "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either
+ * express or implied. See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.rocketmq.flink.legacy;
+
+import java.io.Serializable;
+
+public class RunningChecker implements Serializable {
+    private volatile boolean isRunning = false;
+
+    public boolean isRunning() {
+        return isRunning;
+    }
+
+    public void setRunning(boolean running) {
+        isRunning = running;
+    }
+}
diff --git a/src/main/java/org/apache/rocketmq/flink/legacy/common/selector/DefaultTopicSelector.java b/src/main/java/org/apache/rocketmq/flink/legacy/common/selector/DefaultTopicSelector.java
new file mode 100644
index 0000000..6be5218
--- /dev/null
+++ b/src/main/java/org/apache/rocketmq/flink/legacy/common/selector/DefaultTopicSelector.java
@@ -0,0 +1,39 @@
+/**
+ * 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
+ *
+ * <p>http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * <p>Unless required by applicable law or agreed to in writing, software distributed under the
+ * License is distributed on an "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either
+ * express or implied. See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.rocketmq.flink.legacy.common.selector;
+
+public class DefaultTopicSelector<T> implements TopicSelector<T> {
+    private final String topicName;
+    private final String tagName;
+
+    public DefaultTopicSelector(final String topicName, final String tagName) {
+        this.topicName = topicName;
+        this.tagName = tagName;
+    }
+
+    public DefaultTopicSelector(final String topicName) {
+        this(topicName, "");
+    }
+
+    @Override
+    public String getTopic(T tuple) {
+        return topicName;
+    }
+
+    @Override
+    public String getTag(T tuple) {
+        return tagName;
+    }
+}
diff --git a/src/main/java/org/apache/rocketmq/flink/common/selector/SimpleTopicSelector.java b/src/main/java/org/apache/rocketmq/flink/legacy/common/selector/SimpleTopicSelector.java
similarity index 55%
rename from src/main/java/org/apache/rocketmq/flink/common/selector/SimpleTopicSelector.java
rename to src/main/java/org/apache/rocketmq/flink/legacy/common/selector/SimpleTopicSelector.java
index 3ad8a03..674b5a0 100644
--- a/src/main/java/org/apache/rocketmq/flink/common/selector/SimpleTopicSelector.java
+++ b/src/main/java/org/apache/rocketmq/flink/legacy/common/selector/SimpleTopicSelector.java
@@ -1,31 +1,25 @@
 /**
- * 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
+ * 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
+ * <p>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
+ * <p>Unless required by applicable law or agreed to in writing, software distributed under the
+ * License is distributed on an "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either
+ * express or implied. See the License for the specific language governing permissions and
  * limitations under the License.
  */
-
-package org.apache.rocketmq.flink.common.selector;
-
-import java.util.Map;
+package org.apache.rocketmq.flink.legacy.common.selector;
 
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 
-/**
- * Uses field name to select topic and tag name from tuple.
- */
+import java.util.Map;
+
+/** Uses field name to select topic and tag name from tuple. */
 public class SimpleTopicSelector implements TopicSelector<Map> {
     private static final Logger LOG = LoggerFactory.getLogger(SimpleTopicSelector.class);
 
@@ -37,12 +31,17 @@ public class SimpleTopicSelector implements TopicSelector<Map> {
 
     /**
      * SimpleTopicSelector Constructor.
+     *
      * @param topicFieldName field name used for selecting topic
      * @param defaultTopicName default field name used for selecting topic
      * @param tagFieldName field name used for selecting tag
      * @param defaultTagName default field name used for selecting tag
      */
-    public SimpleTopicSelector(String topicFieldName, String defaultTopicName, String tagFieldName, String defaultTagName) {
+    public SimpleTopicSelector(
+            String topicFieldName,
+            String defaultTopicName,
+            String tagFieldName,
+            String defaultTagName) {
         this.topicFieldName = topicFieldName;
         this.defaultTopicName = defaultTopicName;
         this.tagFieldName = tagFieldName;
@@ -52,10 +51,13 @@ public class SimpleTopicSelector implements TopicSelector<Map> {
     @Override
     public String getTopic(Map tuple) {
         if (tuple.containsKey(topicFieldName)) {
-            Object topic =  tuple.get(topicFieldName);
+            Object topic = tuple.get(topicFieldName);
             return topic != null ? topic.toString() : defaultTopicName;
         } else {
-            LOG.warn("Field {} Not Found. Returning default topic {}", topicFieldName, defaultTopicName);
+            LOG.warn(
+                    "Field {} Not Found. Returning default topic {}",
+                    topicFieldName,
+                    defaultTopicName);
             return defaultTopicName;
         }
     }
diff --git a/src/main/java/org/apache/rocketmq/flink/legacy/common/selector/TopicSelector.java b/src/main/java/org/apache/rocketmq/flink/legacy/common/selector/TopicSelector.java
new file mode 100644
index 0000000..581dadc
--- /dev/null
+++ b/src/main/java/org/apache/rocketmq/flink/legacy/common/selector/TopicSelector.java
@@ -0,0 +1,24 @@
+/**
+ * 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
+ *
+ * <p>http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * <p>Unless required by applicable law or agreed to in writing, software distributed under the
+ * License is distributed on an "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either
+ * express or implied. See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.rocketmq.flink.legacy.common.selector;
+
+import java.io.Serializable;
+
+public interface TopicSelector<T> extends Serializable {
+
+    String getTopic(T tuple);
+
+    String getTag(T tuple);
+}
diff --git a/src/main/java/org/apache/rocketmq/flink/common/serialization/ForwardMessageExtDeserialization.java b/src/main/java/org/apache/rocketmq/flink/legacy/common/serialization/ForwardMessageExtDeserialization.java
similarity index 84%
rename from src/main/java/org/apache/rocketmq/flink/common/serialization/ForwardMessageExtDeserialization.java
rename to src/main/java/org/apache/rocketmq/flink/legacy/common/serialization/ForwardMessageExtDeserialization.java
index 20dd700..6c6fa74 100644
--- a/src/main/java/org/apache/rocketmq/flink/common/serialization/ForwardMessageExtDeserialization.java
+++ b/src/main/java/org/apache/rocketmq/flink/legacy/common/serialization/ForwardMessageExtDeserialization.java
@@ -15,15 +15,15 @@
  * limitations under the License.
  */
 
-package org.apache.rocketmq.flink.common.serialization;
+package org.apache.rocketmq.flink.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> {
+import org.apache.flink.api.common.typeinfo.TypeInformation;
+
+/** A Forward messageExt deserialization. */
+public class ForwardMessageExtDeserialization
+        implements MessageExtDeserializationScheme<MessageExt> {
 
     @Override
     public MessageExt deserializeMessageExt(MessageExt messageExt) {
@@ -34,4 +34,4 @@ public class ForwardMessageExtDeserialization implements MessageExtDeserializati
     public TypeInformation<MessageExt> getProducedType() {
         return TypeInformation.of(MessageExt.class);
     }
-}
\ No newline at end of file
+}
diff --git a/src/main/java/org/apache/rocketmq/flink/legacy/common/serialization/KeyValueDeserializationSchema.java b/src/main/java/org/apache/rocketmq/flink/legacy/common/serialization/KeyValueDeserializationSchema.java
new file mode 100644
index 0000000..4cc8c61
--- /dev/null
+++ b/src/main/java/org/apache/rocketmq/flink/legacy/common/serialization/KeyValueDeserializationSchema.java
@@ -0,0 +1,23 @@
+/**
+ * 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
+ *
+ * <p>http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * <p>Unless required by applicable law or agreed to in writing, software distributed under the
+ * License is distributed on an "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either
+ * express or implied. See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.rocketmq.flink.legacy.common.serialization;
+
+import org.apache.flink.api.java.typeutils.ResultTypeQueryable;
+
+import java.io.Serializable;
+
+public interface KeyValueDeserializationSchema<T> extends ResultTypeQueryable<T>, Serializable {
+    T deserializeKeyAndValue(byte[] key, byte[] value);
+}
diff --git a/src/main/java/org/apache/rocketmq/flink/legacy/common/serialization/KeyValueSerializationSchema.java b/src/main/java/org/apache/rocketmq/flink/legacy/common/serialization/KeyValueSerializationSchema.java
new file mode 100644
index 0000000..66b2e29
--- /dev/null
+++ b/src/main/java/org/apache/rocketmq/flink/legacy/common/serialization/KeyValueSerializationSchema.java
@@ -0,0 +1,24 @@
+/**
+ * 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
+ *
+ * <p>http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * <p>Unless required by applicable law or agreed to in writing, software distributed under the
+ * License is distributed on an "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either
+ * express or implied. See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.rocketmq.flink.legacy.common.serialization;
+
+import java.io.Serializable;
+
+public interface KeyValueSerializationSchema<T> extends Serializable {
+
+    byte[] serializeKey(T tuple);
+
+    byte[] serializeValue(T tuple);
+}
diff --git a/src/main/java/org/apache/rocketmq/flink/common/serialization/MessageExtDeserializationScheme.java b/src/main/java/org/apache/rocketmq/flink/legacy/common/serialization/MessageExtDeserializationScheme.java
similarity index 95%
rename from src/main/java/org/apache/rocketmq/flink/common/serialization/MessageExtDeserializationScheme.java
rename to src/main/java/org/apache/rocketmq/flink/legacy/common/serialization/MessageExtDeserializationScheme.java
index 4c8cf85..173823e 100644
--- a/src/main/java/org/apache/rocketmq/flink/common/serialization/MessageExtDeserializationScheme.java
+++ b/src/main/java/org/apache/rocketmq/flink/legacy/common/serialization/MessageExtDeserializationScheme.java
@@ -15,12 +15,14 @@
  * limitations under the License.
  */
 
-package org.apache.rocketmq.flink.common.serialization;
+package org.apache.rocketmq.flink.legacy.common.serialization;
 
-import java.io.Serializable;
-import org.apache.flink.api.java.typeutils.ResultTypeQueryable;
 import org.apache.rocketmq.common.message.MessageExt;
 
+import org.apache.flink.api.java.typeutils.ResultTypeQueryable;
+
+import java.io.Serializable;
+
 /**
  * The interface Message ext deserialization scheme.
  *
diff --git a/src/main/java/org/apache/rocketmq/flink/common/serialization/SimpleKeyValueDeserializationSchema.java b/src/main/java/org/apache/rocketmq/flink/legacy/common/serialization/SimpleKeyValueDeserializationSchema.java
similarity index 60%
rename from src/main/java/org/apache/rocketmq/flink/common/serialization/SimpleKeyValueDeserializationSchema.java
rename to src/main/java/org/apache/rocketmq/flink/legacy/common/serialization/SimpleKeyValueDeserializationSchema.java
index 93d5d9b..7dada93 100644
--- a/src/main/java/org/apache/rocketmq/flink/common/serialization/SimpleKeyValueDeserializationSchema.java
+++ b/src/main/java/org/apache/rocketmq/flink/legacy/common/serialization/SimpleKeyValueDeserializationSchema.java
@@ -1,31 +1,25 @@
 /**
- * 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
+ * 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
+ * <p>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
+ * <p>Unless required by applicable law or agreed to in writing, software distributed under the
+ * License is distributed on an "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either
+ * express or implied. See the License for the specific language governing permissions and
  * limitations under the License.
  */
+package org.apache.rocketmq.flink.legacy.common.serialization;
 
-package org.apache.rocketmq.flink.common.serialization;
+import org.apache.flink.api.common.typeinfo.TypeInformation;
 
 import java.nio.charset.StandardCharsets;
 import java.util.HashMap;
 import java.util.Map;
 
-import org.apache.flink.api.common.typeinfo.TypeHint;
-import org.apache.flink.api.common.typeinfo.TypeInformation;
-import org.apache.flink.api.java.tuple.Tuple2;
-
 public class SimpleKeyValueDeserializationSchema implements KeyValueDeserializationSchema<Map> {
     public static final String DEFAULT_KEY_FIELD = "key";
     public static final String DEFAULT_VALUE_FIELD = "value";
@@ -39,8 +33,9 @@ public class SimpleKeyValueDeserializationSchema implements KeyValueDeserializat
 
     /**
      * SimpleKeyValueDeserializationSchema Constructor.
+     *
      * @param keyField tuple field for selecting the key
-     * @param valueField  tuple field for selecting the value
+     * @param valueField tuple field for selecting the value
      */
     public SimpleKeyValueDeserializationSchema(String keyField, String valueField) {
         this.keyField = keyField;
@@ -65,4 +60,4 @@ public class SimpleKeyValueDeserializationSchema implements KeyValueDeserializat
     public TypeInformation<Map> getProducedType() {
         return TypeInformation.of(Map.class);
     }
-}
\ No newline at end of file
+}
diff --git a/src/main/java/org/apache/rocketmq/flink/common/serialization/SimpleKeyValueSerializationSchema.java b/src/main/java/org/apache/rocketmq/flink/legacy/common/serialization/SimpleKeyValueSerializationSchema.java
similarity index 60%
rename from src/main/java/org/apache/rocketmq/flink/common/serialization/SimpleKeyValueSerializationSchema.java
rename to src/main/java/org/apache/rocketmq/flink/legacy/common/serialization/SimpleKeyValueSerializationSchema.java
index bbd6da3..3e92ad2 100644
--- a/src/main/java/org/apache/rocketmq/flink/common/serialization/SimpleKeyValueSerializationSchema.java
+++ b/src/main/java/org/apache/rocketmq/flink/legacy/common/serialization/SimpleKeyValueSerializationSchema.java
@@ -1,22 +1,18 @@
 /**
- * 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
+ * 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
+ * <p>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
+ * <p>Unless required by applicable law or agreed to in writing, software distributed under the
+ * License is distributed on an "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either
+ * express or implied. See the License for the specific language governing permissions and
  * limitations under the License.
  */
-
-package org.apache.rocketmq.flink.common.serialization;
+package org.apache.rocketmq.flink.legacy.common.serialization;
 
 import java.nio.charset.StandardCharsets;
 import java.util.Map;
@@ -34,8 +30,9 @@ public class SimpleKeyValueSerializationSchema implements KeyValueSerializationS
 
     /**
      * SimpleKeyValueSerializationSchema Constructor.
+     *
      * @param keyField tuple field for selecting the key
-     * @param valueField  tuple field for selecting the value
+     * @param valueField tuple field for selecting the value
      */
     public SimpleKeyValueSerializationSchema(String keyField, String valueField) {
         this.keyField = keyField;
@@ -59,5 +56,4 @@ public class SimpleKeyValueSerializationSchema implements KeyValueSerializationS
         Object value = tuple.get(valueField);
         return value != null ? value.toString().getBytes(StandardCharsets.UTF_8) : null;
     }
-
 }
diff --git a/src/main/java/org/apache/rocketmq/flink/common/serialization/SimpleTupleDeserializationSchema.java b/src/main/java/org/apache/rocketmq/flink/legacy/common/serialization/SimpleTupleDeserializationSchema.java
similarity index 71%
rename from src/main/java/org/apache/rocketmq/flink/common/serialization/SimpleTupleDeserializationSchema.java
rename to src/main/java/org/apache/rocketmq/flink/legacy/common/serialization/SimpleTupleDeserializationSchema.java
index 54106ef..3bac266 100644
--- a/src/main/java/org/apache/rocketmq/flink/common/serialization/SimpleTupleDeserializationSchema.java
+++ b/src/main/java/org/apache/rocketmq/flink/legacy/common/serialization/SimpleTupleDeserializationSchema.java
@@ -1,4 +1,4 @@
-package org.apache.rocketmq.flink.common.serialization;
+package org.apache.rocketmq.flink.legacy.common.serialization;
 
 import org.apache.flink.api.common.typeinfo.TypeHint;
 import org.apache.flink.api.common.typeinfo.TypeInformation;
@@ -6,7 +6,8 @@ import org.apache.flink.api.java.tuple.Tuple2;
 
 import java.nio.charset.StandardCharsets;
 
-public class SimpleTupleDeserializationSchema implements KeyValueDeserializationSchema<Tuple2<String, String>> {
+public class SimpleTupleDeserializationSchema
+        implements KeyValueDeserializationSchema<Tuple2<String, String>> {
 
     @Override
     public Tuple2<String, String> deserializeKeyAndValue(byte[] key, byte[] value) {
@@ -17,6 +18,6 @@ public class SimpleTupleDeserializationSchema implements KeyValueDeserialization
 
     @Override
     public TypeInformation<Tuple2<String, String>> getProducedType() {
-        return TypeInformation.of(new TypeHint<Tuple2<String,String>>(){});
+        return TypeInformation.of(new TypeHint<Tuple2<String, String>>() {});
     }
 }
diff --git a/src/main/java/org/apache/rocketmq/flink/legacy/common/util/MetricUtils.java b/src/main/java/org/apache/rocketmq/flink/legacy/common/util/MetricUtils.java
new file mode 100644
index 0000000..bb3baeb
--- /dev/null
+++ b/src/main/java/org/apache/rocketmq/flink/legacy/common/util/MetricUtils.java
@@ -0,0 +1,85 @@
+/**
+ * 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
+ *
+ * <p>http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * <p>Unless required by applicable law or agreed to in writing, software distributed under the
+ * License is distributed on an "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either
+ * express or implied. See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.rocketmq.flink.legacy.common.util;
+
+import org.apache.flink.api.common.functions.RuntimeContext;
+import org.apache.flink.metrics.Counter;
+import org.apache.flink.metrics.Gauge;
+import org.apache.flink.metrics.Meter;
+import org.apache.flink.metrics.MeterView;
+import org.apache.flink.metrics.SimpleCounter;
+
+/** RocketMQ connector metrics. */
+public class MetricUtils {
+
+    public static final String METRICS_TPS = "tps";
+
+    private static final String METRIC_GROUP_SINK = "sink";
+    private static final String METRICS_SINK_IN_TPS = "inTps";
+    private static final String METRICS_SINK_OUT_TPS = "outTps";
+    private static final String METRICS_SINK_OUT_BPS = "outBps";
+    private static final String METRICS_SINK_OUT_Latency = "outLatency";
+
+    public static Meter registerSinkInTps(RuntimeContext context) {
+        Counter parserCounter =
+                context.getMetricGroup()
+                        .addGroup(METRIC_GROUP_SINK)
+                        .counter(METRICS_SINK_IN_TPS + "_counter", new SimpleCounter());
+        return context.getMetricGroup()
+                .addGroup(METRIC_GROUP_SINK)
+                .meter(METRICS_SINK_IN_TPS, new MeterView(parserCounter, 60));
+    }
+
+    public static Meter registerOutTps(RuntimeContext context) {
+        Counter parserCounter =
+                context.getMetricGroup()
+                        .addGroup(METRIC_GROUP_SINK)
+                        .counter(METRICS_SINK_OUT_TPS + "_counter", new SimpleCounter());
+        return context.getMetricGroup()
+                .addGroup(METRIC_GROUP_SINK)
+                .meter(METRICS_SINK_OUT_TPS, new MeterView(parserCounter, 60));
+    }
+
+    public static Meter registerOutBps(RuntimeContext context) {
+        Counter bpsCounter =
+                context.getMetricGroup()
+                        .addGroup(METRIC_GROUP_SINK)
+                        .counter(METRICS_SINK_OUT_BPS + "_counter", new SimpleCounter());
+        return context.getMetricGroup()
+                .addGroup(METRIC_GROUP_SINK)
+                .meter(METRICS_SINK_OUT_BPS, new MeterView(bpsCounter, 60));
+    }
+
+    public static LatencyGauge registerOutLatency(RuntimeContext context) {
+        return context.getMetricGroup()
+                .addGroup(METRIC_GROUP_SINK)
+                .gauge(METRICS_SINK_OUT_Latency, new LatencyGauge());
+    }
+
+    public static class LatencyGauge implements Gauge<Double> {
+        private double value;
+
+        public void report(long timeDelta, long batchSize) {
+            if (batchSize != 0) {
+                this.value = (1.0 * timeDelta) / batchSize;
+            }
+        }
+
+        @Override
+        public Double getValue() {
+            return value;
+        }
+    }
+}
diff --git a/src/main/java/org/apache/rocketmq/flink/common/util/RetryUtil.java b/src/main/java/org/apache/rocketmq/flink/legacy/common/util/RetryUtil.java
similarity index 59%
rename from src/main/java/org/apache/rocketmq/flink/common/util/RetryUtil.java
rename to src/main/java/org/apache/rocketmq/flink/legacy/common/util/RetryUtil.java
index 0dbd553..7ec1dca 100644
--- a/src/main/java/org/apache/rocketmq/flink/common/util/RetryUtil.java
+++ b/src/main/java/org/apache/rocketmq/flink/legacy/common/util/RetryUtil.java
@@ -1,22 +1,18 @@
 /**
- * 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
+ * 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
+ * <p>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
+ * <p>Unless required by applicable law or agreed to in writing, software distributed under the
+ * License is distributed on an "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either
+ * express or implied. See the License for the specific language governing permissions and
  * limitations under the License.
  */
-
-package org.apache.rocketmq.flink.common.util;
+package org.apache.rocketmq.flink.legacy.common.util;
 
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
@@ -30,8 +26,7 @@ public class RetryUtil {
     private static final long MAX_BACKOFF = 5000;
     private static final int MAX_ATTEMPTS = 5;
 
-    private RetryUtil() {
-    }
+    private RetryUtil() {}
 
     public static void waitForMs(long sleepMs) {
         try {
diff --git a/src/main/java/org/apache/rocketmq/flink/legacy/common/util/RocketMQUtils.java b/src/main/java/org/apache/rocketmq/flink/legacy/common/util/RocketMQUtils.java
new file mode 100644
index 0000000..94a24a1
--- /dev/null
+++ b/src/main/java/org/apache/rocketmq/flink/legacy/common/util/RocketMQUtils.java
@@ -0,0 +1,79 @@
+/**
+ * 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
+ *
+ * <p>http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * <p>Unless required by applicable law or agreed to in writing, software distributed under the
+ * License is distributed on an "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either
+ * express or implied. See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.rocketmq.flink.legacy.common.util;
+
+import org.apache.rocketmq.client.AccessChannel;
+import org.apache.rocketmq.common.message.MessageQueue;
+
+import java.lang.management.ManagementFactory;
+import java.util.ArrayList;
+import java.util.Collection;
+import java.util.Collections;
+import java.util.List;
+import java.util.Properties;
+
+public final class RocketMQUtils {
+
+    public static int getInteger(Properties props, String key, int defaultValue) {
+        return Integer.parseInt(props.getProperty(key, String.valueOf(defaultValue)));
+    }
+
+    public static long getLong(Properties props, String key, long defaultValue) {
+        return Long.parseLong(props.getProperty(key, String.valueOf(defaultValue)));
+    }
+
+    public static boolean getBoolean(Properties props, String key, boolean defaultValue) {
+        return Boolean.parseBoolean(props.getProperty(key, String.valueOf(defaultValue)));
+    }
+
+    public static AccessChannel getAccessChannel(
+            Properties props, String key, AccessChannel defaultValue) {
+        return AccessChannel.valueOf(props.getProperty(key, String.valueOf(defaultValue)));
+    }
+
+    public static String getInstanceName(String... args) {
+        if (null != args && args.length > 0) {
+            return String.join("_", args);
+        }
+        return ManagementFactory.getRuntimeMXBean().getName() + "_" + System.nanoTime();
+    }
+
+    /**
+     * Average Hashing queue algorithm Refer:
+     * org.apache.rocketmq.client.consumer.rebalance.AllocateMessageQueueAveragely
+     */
+    public static List<MessageQueue> allocate(
+            Collection<MessageQueue> mqSet, int numberOfParallelTasks, int indexOfThisTask) {
+        ArrayList<MessageQueue> mqAll = new ArrayList<>(mqSet);
+        Collections.sort(mqAll);
+        List<MessageQueue> result = new ArrayList<>();
+        int mod = mqAll.size() % numberOfParallelTasks;
+        int averageSize =
+                mqAll.size() <= numberOfParallelTasks
+                        ? 1
+                        : (mod > 0 && indexOfThisTask < mod
+                                ? mqAll.size() / numberOfParallelTasks + 1
+                                : mqAll.size() / numberOfParallelTasks);
+        int startIndex =
+                (mod > 0 && indexOfThisTask < mod)
+                        ? indexOfThisTask * averageSize
+                        : indexOfThisTask * averageSize + mod;
+        int range = Math.min(averageSize, mqAll.size() - startIndex);
+        for (int i = 0; i < range; i++) {
+            result.add(mqAll.get((startIndex + i) % mqAll.size()));
+        }
+        return result;
+    }
+}
diff --git a/src/main/java/org/apache/rocketmq/flink/legacy/common/util/TestUtils.java b/src/main/java/org/apache/rocketmq/flink/legacy/common/util/TestUtils.java
new file mode 100644
index 0000000..407aec7
--- /dev/null
+++ b/src/main/java/org/apache/rocketmq/flink/legacy/common/util/TestUtils.java
@@ -0,0 +1,29 @@
+/**
+ * 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
+ *
+ * <p>http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * <p>Unless required by applicable law or agreed to in writing, software distributed under the
+ * License is distributed on an "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either
+ * express or implied. See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.rocketmq.flink.legacy.common.util;
+
+import java.lang.reflect.Field;
+
+public class TestUtils {
+    public static void setFieldValue(Object obj, String fieldName, Object value) {
+        try {
+            Field field = obj.getClass().getDeclaredField(fieldName);
+            field.setAccessible(true);
+            field.set(obj, value);
+        } catch (Exception e) {
+            e.printStackTrace();
+        }
+    }
+}
diff --git a/src/main/java/org/apache/rocketmq/flink/common/watermark/BoundedOutOfOrdernessGenerator.java b/src/main/java/org/apache/rocketmq/flink/legacy/common/watermark/BoundedOutOfOrdernessGenerator.java
similarity index 84%
rename from src/main/java/org/apache/rocketmq/flink/common/watermark/BoundedOutOfOrdernessGenerator.java
rename to src/main/java/org/apache/rocketmq/flink/legacy/common/watermark/BoundedOutOfOrdernessGenerator.java
index 7e38f27..2b56f54 100644
--- a/src/main/java/org/apache/rocketmq/flink/common/watermark/BoundedOutOfOrdernessGenerator.java
+++ b/src/main/java/org/apache/rocketmq/flink/legacy/common/watermark/BoundedOutOfOrdernessGenerator.java
@@ -15,11 +15,12 @@
  * limitations under the License.
  */
 
-package org.apache.rocketmq.flink.common.watermark;
+package org.apache.rocketmq.flink.legacy.common.watermark;
+
+import org.apache.rocketmq.common.message.MessageExt;
 
 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> {
 
@@ -27,8 +28,7 @@ public class BoundedOutOfOrdernessGenerator implements AssignerWithPeriodicWater
 
     private long currentMaxTimestamp;
 
-    public BoundedOutOfOrdernessGenerator() {
-    }
+    public BoundedOutOfOrdernessGenerator() {}
 
     public BoundedOutOfOrdernessGenerator(long maxOutOfOrderness) {
         this.maxOutOfOrderness = maxOutOfOrderness;
@@ -49,9 +49,11 @@ public class BoundedOutOfOrdernessGenerator implements AssignerWithPeriodicWater
 
     @Override
     public String toString() {
-        return "BoundedOutOfOrdernessGenerator{" +
-            "maxOutOfOrderness=" + maxOutOfOrderness +
-            ", currentMaxTimestamp=" + currentMaxTimestamp +
-            '}';
+        return "BoundedOutOfOrdernessGenerator{"
+                + "maxOutOfOrderness="
+                + maxOutOfOrderness
+                + ", currentMaxTimestamp="
+                + currentMaxTimestamp
+                + '}';
     }
 }
diff --git a/src/main/java/org/apache/rocketmq/flink/common/watermark/BoundedOutOfOrdernessGeneratorPerQueue.java b/src/main/java/org/apache/rocketmq/flink/legacy/common/watermark/BoundedOutOfOrdernessGeneratorPerQueue.java
similarity index 79%
rename from src/main/java/org/apache/rocketmq/flink/common/watermark/BoundedOutOfOrdernessGeneratorPerQueue.java
rename to src/main/java/org/apache/rocketmq/flink/legacy/common/watermark/BoundedOutOfOrdernessGeneratorPerQueue.java
index e56b34c..ab49131 100644
--- a/src/main/java/org/apache/rocketmq/flink/common/watermark/BoundedOutOfOrdernessGeneratorPerQueue.java
+++ b/src/main/java/org/apache/rocketmq/flink/legacy/common/watermark/BoundedOutOfOrdernessGeneratorPerQueue.java
@@ -15,27 +15,24 @@
  * limitations under the License.
  */
 
-package org.apache.rocketmq.flink.common.watermark;
+package org.apache.rocketmq.flink.legacy.common.watermark;
+
+import org.apache.rocketmq.common.message.MessageExt;
 
 import org.apache.flink.streaming.api.functions.AssignerWithPeriodicWatermarks;
 import org.apache.flink.streaming.api.watermark.Watermark;
-import org.apache.rocketmq.common.message.MessageExt;
-import org.apache.rocketmq.common.message.MessageQueue;
 
-import java.util.HashMap;
 import java.util.Map;
 import java.util.concurrent.ConcurrentHashMap;
 
-/**
- * 取每条队列中的最大eventTime的最小值作为当前source的watermark
- */
-public class BoundedOutOfOrdernessGeneratorPerQueue implements AssignerWithPeriodicWatermarks<MessageExt> {
+/** 取每条队列中的最大eventTime的最小值作为当前source的watermark */
+public class BoundedOutOfOrdernessGeneratorPerQueue
+        implements AssignerWithPeriodicWatermarks<MessageExt> {
 
     private Map<String, Long> maxEventTimeTable;
     private long maxOutOfOrderness = 5000L; // 5 seconds
 
-    public BoundedOutOfOrdernessGeneratorPerQueue() {
-    }
+    public BoundedOutOfOrdernessGeneratorPerQueue() {}
 
     public BoundedOutOfOrdernessGeneratorPerQueue(long maxOutOfOrderness) {
         this.maxOutOfOrderness = maxOutOfOrderness;
@@ -63,9 +60,11 @@ public class BoundedOutOfOrdernessGeneratorPerQueue implements AssignerWithPerio
 
     @Override
     public String toString() {
-        return "BoundedOutOfOrdernessGeneratorPerQueue{" +
-                "maxEventTimeTable=" + maxEventTimeTable +
-                ", maxOutOfOrderness=" + maxOutOfOrderness +
-                '}';
+        return "BoundedOutOfOrdernessGeneratorPerQueue{"
+                + "maxEventTimeTable="
+                + maxEventTimeTable
+                + ", maxOutOfOrderness="
+                + maxOutOfOrderness
+                + '}';
     }
 }
diff --git a/src/main/java/org/apache/rocketmq/flink/common/watermark/PunctuatedAssigner.java b/src/main/java/org/apache/rocketmq/flink/legacy/common/watermark/PunctuatedAssigner.java
similarity index 64%
rename from src/main/java/org/apache/rocketmq/flink/common/watermark/PunctuatedAssigner.java
rename to src/main/java/org/apache/rocketmq/flink/legacy/common/watermark/PunctuatedAssigner.java
index 354eecc..946e873 100644
--- a/src/main/java/org/apache/rocketmq/flink/common/watermark/PunctuatedAssigner.java
+++ b/src/main/java/org/apache/rocketmq/flink/legacy/common/watermark/PunctuatedAssigner.java
@@ -15,23 +15,24 @@
  * limitations under the License.
  */
 
-package org.apache.rocketmq.flink.common.watermark;
+package org.apache.rocketmq.flink.legacy.common.watermark;
+
+import org.apache.rocketmq.common.message.MessageExt;
+import org.apache.rocketmq.flink.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;
-import org.apache.rocketmq.flink.RocketMQConfig;
 
 /**
- * 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 first call the extractTimestamp(...) method to assign the
- * element a timestamp, and then immediately call the checkAndGetNextWatermark(...) method on that element.
+ * 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
+ * first call the extractTimestamp(...) method to assign the element a timestamp, and then
+ * immediately call the checkAndGetNextWatermark(...) method on that element.
  *
- * The checkAndGetNextWatermark(...) method is passed the timestamp that was assigned in the extractTimestamp(...)
- * method, and can decide whether it wants to generate a watermark. Whenever the checkAndGetNextWatermark(...) method
- * returns a non-null watermark, and that watermark is larger than the latest previous watermark, that new watermark
- * will be emitted.
+ * <p>The checkAndGetNextWatermark(...) method is passed the timestamp that was assigned in the
+ * extractTimestamp(...) method, and can decide whether it wants to generate a watermark. Whenever
+ * the checkAndGetNextWatermark(...) method returns a non-null watermark, and that watermark is
+ * larger than the latest previous watermark, that new watermark will be emitted.
  */
 public class PunctuatedAssigner implements AssignerWithPunctuatedWatermarks<MessageExt> {
     @Override
diff --git a/src/main/java/org/apache/rocketmq/flink/common/watermark/TimeLagWatermarkGenerator.java b/src/main/java/org/apache/rocketmq/flink/legacy/common/watermark/TimeLagWatermarkGenerator.java
similarity index 82%
rename from src/main/java/org/apache/rocketmq/flink/common/watermark/TimeLagWatermarkGenerator.java
rename to src/main/java/org/apache/rocketmq/flink/legacy/common/watermark/TimeLagWatermarkGenerator.java
index beec8f3..66cb3cd 100644
--- a/src/main/java/org/apache/rocketmq/flink/common/watermark/TimeLagWatermarkGenerator.java
+++ b/src/main/java/org/apache/rocketmq/flink/legacy/common/watermark/TimeLagWatermarkGenerator.java
@@ -15,21 +15,21 @@
  * limitations under the License.
  */
 
-package org.apache.rocketmq.flink.common.watermark;
+package org.apache.rocketmq.flink.legacy.common.watermark;
+
+import org.apache.rocketmq.common.message.MessageExt;
 
 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.
+ * 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.
  */
 public class TimeLagWatermarkGenerator implements AssignerWithPeriodicWatermarks<MessageExt> {
     private long maxTimeLag = 5000; // 5 seconds
 
-    TimeLagWatermarkGenerator() {
-    }
+    TimeLagWatermarkGenerator() {}
 
     TimeLagWatermarkGenerator(long maxTimeLag) {
         this.maxTimeLag = maxTimeLag;
@@ -46,9 +46,8 @@ public class TimeLagWatermarkGenerator implements AssignerWithPeriodicWatermarks
         return new Watermark(System.currentTimeMillis() - maxTimeLag);
     }
 
-    @Override public String toString() {
-        return "TimeLagWatermarkGenerator{" +
-            "maxTimeLag=" + maxTimeLag +
-            '}';
+    @Override
+    public String toString() {
+        return "TimeLagWatermarkGenerator{" + "maxTimeLag=" + maxTimeLag + '}';
     }
 }
diff --git a/src/main/java/org/apache/rocketmq/flink/common/watermark/WaterMarkForAll.java b/src/main/java/org/apache/rocketmq/flink/legacy/common/watermark/WaterMarkForAll.java
similarity index 83%
rename from src/main/java/org/apache/rocketmq/flink/common/watermark/WaterMarkForAll.java
rename to src/main/java/org/apache/rocketmq/flink/legacy/common/watermark/WaterMarkForAll.java
index a80fb69..8fadd77 100644
--- a/src/main/java/org/apache/rocketmq/flink/common/watermark/WaterMarkForAll.java
+++ b/src/main/java/org/apache/rocketmq/flink/legacy/common/watermark/WaterMarkForAll.java
@@ -15,14 +15,9 @@
  * limitations under the License.
  */
 
-package org.apache.rocketmq.flink.common.watermark;
+package org.apache.rocketmq.flink.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;
 
 public class WaterMarkForAll {
 
@@ -30,8 +25,7 @@ public class WaterMarkForAll {
 
     private long maxTimestamp = 0L;
 
-    public WaterMarkForAll() {
-    }
+    public WaterMarkForAll() {}
 
     public WaterMarkForAll(long maxOutOfOrderness) {
         this.maxOutOfOrderness = maxOutOfOrderness;
diff --git a/src/main/java/org/apache/rocketmq/flink/common/watermark/WaterMarkPerQueue.java b/src/main/java/org/apache/rocketmq/flink/legacy/common/watermark/WaterMarkPerQueue.java
similarity index 87%
rename from src/main/java/org/apache/rocketmq/flink/common/watermark/WaterMarkPerQueue.java
rename to src/main/java/org/apache/rocketmq/flink/legacy/common/watermark/WaterMarkPerQueue.java
index 2210cfb..941dec5 100644
--- a/src/main/java/org/apache/rocketmq/flink/common/watermark/WaterMarkPerQueue.java
+++ b/src/main/java/org/apache/rocketmq/flink/legacy/common/watermark/WaterMarkPerQueue.java
@@ -15,11 +15,12 @@
  * limitations under the License.
  */
 
-package org.apache.rocketmq.flink.common.watermark;
+package org.apache.rocketmq.flink.legacy.common.watermark;
 
-import org.apache.flink.streaming.api.watermark.Watermark;
 import org.apache.rocketmq.common.message.MessageQueue;
 
+import org.apache.flink.streaming.api.watermark.Watermark;
+
 import java.util.Map;
 import java.util.concurrent.ConcurrentHashMap;
 import java.util.concurrent.ConcurrentMap;
@@ -30,8 +31,7 @@ public class WaterMarkPerQueue {
 
     private long maxOutOfOrderness = 5000L; // 5 seconds
 
-    public WaterMarkPerQueue() {
-    }
+    public WaterMarkPerQueue() {}
 
     public WaterMarkPerQueue(long maxOutOfOrderness) {
         this.maxOutOfOrderness = maxOutOfOrderness;
@@ -54,9 +54,11 @@ public class WaterMarkPerQueue {
 
     @Override
     public String toString() {
-        return "WaterMarkPerQueue{" +
-                "maxEventTimeTable=" + maxEventTimeTable +
-                ", maxOutOfOrderness=" + maxOutOfOrderness +
-                '}';
+        return "WaterMarkPerQueue{"
+                + "maxEventTimeTable="
+                + maxEventTimeTable
+                + ", maxOutOfOrderness="
+                + maxOutOfOrderness
+                + '}';
     }
 }
diff --git a/src/main/java/org/apache/rocketmq/flink/example/RocketMQFlinkExample.java b/src/main/java/org/apache/rocketmq/flink/legacy/example/RocketMQFlinkExample.java
similarity index 77%
rename from src/main/java/org/apache/rocketmq/flink/example/RocketMQFlinkExample.java
rename to src/main/java/org/apache/rocketmq/flink/legacy/example/RocketMQFlinkExample.java
index 1f24d96..b435726 100644
--- a/src/main/java/org/apache/rocketmq/flink/example/RocketMQFlinkExample.java
+++ b/src/main/java/org/apache/rocketmq/flink/legacy/example/RocketMQFlinkExample.java
@@ -15,7 +15,15 @@
  * limitations under the License.
  */
 
-package org.apache.rocketmq.flink.example;
+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.RocketMQSource;
+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;
 
 import org.apache.flink.api.java.tuple.Tuple2;
 import org.apache.flink.api.java.utils.ParameterTool;
@@ -25,29 +33,23 @@ import org.apache.flink.streaming.api.TimeCharacteristic;
 import org.apache.flink.streaming.api.datastream.DataStreamSource;
 import org.apache.flink.streaming.api.environment.CheckpointConfig;
 import org.apache.flink.streaming.api.environment.StreamExecutionEnvironment;
-import org.apache.rocketmq.client.AccessChannel;
-import org.apache.rocketmq.flink.RocketMQConfig;
-import org.apache.rocketmq.flink.RocketMQSink;
-import org.apache.rocketmq.flink.RocketMQSource;
-import org.apache.rocketmq.flink.common.serialization.SimpleKeyValueDeserializationSchema;
-import org.apache.rocketmq.flink.common.serialization.SimpleTupleDeserializationSchema;
-import org.apache.rocketmq.flink.function.SinkMapFunction;
-import org.apache.rocketmq.flink.function.SourceMapFunction;
 
 import java.util.Properties;
 
-import static org.apache.rocketmq.flink.RocketMQConfig.CONSUMER_OFFSET_LATEST;
-import static org.apache.rocketmq.flink.RocketMQConfig.DEFAULT_CONSUMER_TAG;
+import static org.apache.rocketmq.flink.legacy.RocketMQConfig.CONSUMER_OFFSET_LATEST;
+import static org.apache.rocketmq.flink.legacy.RocketMQConfig.DEFAULT_CONSUMER_TAG;
 
 public class RocketMQFlinkExample {
 
     /**
      * Source Config
+     *
      * @return properties
      */
     private static Properties getConsumerProps() {
         Properties consumerProps = new Properties();
-        consumerProps.setProperty(RocketMQConfig.NAME_SERVER_ADDR,
+        consumerProps.setProperty(
+                RocketMQConfig.NAME_SERVER_ADDR,
                 "http://${instanceId}.${region}.mq-internal.aliyuncs.com:8080");
         consumerProps.setProperty(RocketMQConfig.CONSUMER_GROUP, "${ConsumerGroup}");
         consumerProps.setProperty(RocketMQConfig.CONSUMER_TOPIC, "${SourceTopic}");
@@ -61,11 +63,13 @@ public class RocketMQFlinkExample {
 
     /**
      * Sink Config
+     *
      * @return properties
      */
     private static Properties getProducerProps() {
         Properties producerProps = new Properties();
-        producerProps.setProperty(RocketMQConfig.NAME_SERVER_ADDR,
+        producerProps.setProperty(
+                RocketMQConfig.NAME_SERVER_ADDR,
                 "http://${instanceId}.${region}.mq-internal.aliyuncs.com:8080");
         producerProps.setProperty(RocketMQConfig.PRODUCER_GROUP, "${ProducerGroup}");
         producerProps.setProperty(RocketMQConfig.ACCESS_KEY, "${AccessKey}");
@@ -100,22 +104,26 @@ public class RocketMQFlinkExample {
         // allow only one checkpoint to be in progress at the same time
         env.getCheckpointConfig().setMaxConcurrentCheckpoints(1);
         // enable externalized checkpoints which are retained after job cancellation
-        env.getCheckpointConfig().enableExternalizedCheckpoints(
-                CheckpointConfig.ExternalizedCheckpointCleanup.RETAIN_ON_CANCELLATION);
+        env.getCheckpointConfig()
+                .enableExternalizedCheckpoints(
+                        CheckpointConfig.ExternalizedCheckpointCleanup.RETAIN_ON_CANCELLATION);
 
         Properties consumerProps = getConsumerProps();
         Properties producerProps = getProducerProps();
 
         SimpleTupleDeserializationSchema schema = new SimpleTupleDeserializationSchema();
 
-        DataStreamSource<Tuple2<String, String>> source = env.addSource(
-                new RocketMQSource<>(schema, consumerProps)).setParallelism(2);
+        DataStreamSource<Tuple2<String, String>> source =
+                env.addSource(new RocketMQSource<>(schema, consumerProps)).setParallelism(2);
 
         source.print();
         source.process(new SourceMapFunction())
                 .process(new SinkMapFunction("FLINK_SINK", "*"))
-                .addSink(new RocketMQSink(producerProps).withBatchFlushOnCheckpoint(true).withBatchSize(32)
-                        .withAsync(true))
+                .addSink(
+                        new RocketMQSink(producerProps)
+                                .withBatchFlushOnCheckpoint(true)
+                                .withBatchSize(32)
+                                .withAsync(true))
                 .setParallelism(2);
 
         env.execute("rocketmq-connect-flink");
diff --git a/src/main/java/org/apache/rocketmq/flink/example/SimpleConsumer.java b/src/main/java/org/apache/rocketmq/flink/legacy/example/SimpleConsumer.java
similarity index 72%
rename from src/main/java/org/apache/rocketmq/flink/example/SimpleConsumer.java
rename to src/main/java/org/apache/rocketmq/flink/legacy/example/SimpleConsumer.java
index 601d37d..9e025bb 100644
--- a/src/main/java/org/apache/rocketmq/flink/example/SimpleConsumer.java
+++ b/src/main/java/org/apache/rocketmq/flink/legacy/example/SimpleConsumer.java
@@ -15,7 +15,7 @@
  * limitations under the License.
  */
 
-package org.apache.rocketmq.flink.example;
+package org.apache.rocketmq.flink.legacy.example;
 
 import org.apache.rocketmq.acl.common.AclClientRPCHook;
 import org.apache.rocketmq.acl.common.SessionCredentials;
@@ -28,6 +28,7 @@ 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;
 
@@ -36,7 +37,8 @@ public class SimpleConsumer {
     private static final Logger log = 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 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 = "*";
@@ -48,8 +50,9 @@ public class SimpleConsumer {
     }
 
     public static void main(String[] args) {
-        DefaultMQPushConsumer consumer = new DefaultMQPushConsumer(
-                GROUP, getAclRPCHook(), new AllocateMessageQueueAveragely());
+        DefaultMQPushConsumer consumer =
+                new DefaultMQPushConsumer(
+                        GROUP, getAclRPCHook(), new AllocateMessageQueueAveragely());
         consumer.setNamesrvAddr(NAME_SERVER_ADDR);
 
         // When using aliyun products, you need to set up channels
@@ -62,13 +65,19 @@ public class SimpleConsumer {
         }
 
         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;
-        });
+        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();
diff --git a/src/main/java/org/apache/rocketmq/flink/example/SimpleProducer.java b/src/main/java/org/apache/rocketmq/flink/legacy/example/SimpleProducer.java
similarity index 88%
rename from src/main/java/org/apache/rocketmq/flink/example/SimpleProducer.java
rename to src/main/java/org/apache/rocketmq/flink/legacy/example/SimpleProducer.java
index 9d7ba45..ea24f60 100644
--- a/src/main/java/org/apache/rocketmq/flink/example/SimpleProducer.java
+++ b/src/main/java/org/apache/rocketmq/flink/legacy/example/SimpleProducer.java
@@ -15,7 +15,7 @@
  * limitations under the License.
  */
 
-package org.apache.rocketmq.flink.example;
+package org.apache.rocketmq.flink.legacy.example;
 
 import org.apache.rocketmq.acl.common.AclClientRPCHook;
 import org.apache.rocketmq.acl.common.SessionCredentials;
@@ -24,8 +24,8 @@ 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.flink.RocketMQSource;
 import org.apache.rocketmq.remoting.RPCHook;
+
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 
@@ -36,7 +36,8 @@ public class SimpleProducer {
     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 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 = "*";
@@ -49,8 +50,8 @@ public class SimpleProducer {
     }
 
     public static void main(String[] args) {
-        DefaultMQProducer producer = new DefaultMQProducer(
-                PRODUCER_GROUP, getAclRPCHook(), true, null);
+        DefaultMQProducer producer =
+                new DefaultMQProducer(PRODUCER_GROUP, getAclRPCHook(), true, null);
         producer.setNamesrvAddr(NAME_SERVER_ADDR);
 
         // When using aliyun products, you need to set up channels
@@ -68,7 +69,8 @@ public class SimpleProducer {
             try {
                 SendResult sendResult = producer.send(msg);
                 assert sendResult != null;
-                System.out.printf("send result: %s %s\n",
+                System.out.printf(
+                        "send result: %s %s\n",
                         sendResult.getMsgId(), sendResult.getMessageQueue().toString());
                 Thread.sleep(50);
             } catch (Exception e) {
diff --git a/src/main/java/org/apache/rocketmq/flink/function/SinkMapFunction.java b/src/main/java/org/apache/rocketmq/flink/legacy/function/SinkMapFunction.java
similarity index 91%
rename from src/main/java/org/apache/rocketmq/flink/function/SinkMapFunction.java
rename to src/main/java/org/apache/rocketmq/flink/legacy/function/SinkMapFunction.java
index c3a6af5..f63b636 100644
--- a/src/main/java/org/apache/rocketmq/flink/function/SinkMapFunction.java
+++ b/src/main/java/org/apache/rocketmq/flink/legacy/function/SinkMapFunction.java
@@ -15,13 +15,15 @@
  * limitations under the License.
  */
 
-package org.apache.rocketmq.flink.function;
+package org.apache.rocketmq.flink.legacy.function;
+
+import org.apache.rocketmq.common.message.Message;
 
-import org.apache.commons.lang.Validate;
 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.rocketmq.common.message.Message;
+
+import org.apache.commons.lang.Validate;
 
 public class SinkMapFunction extends ProcessFunction<Tuple2<String, String>, Message> {
 
@@ -29,8 +31,7 @@ public class SinkMapFunction extends ProcessFunction<Tuple2<String, String>, Mes
 
     private String tag;
 
-    public SinkMapFunction() {
-    }
+    public SinkMapFunction() {}
 
     public SinkMapFunction(String topic, String tag) {
         this.topic = topic;
@@ -38,7 +39,8 @@ public class SinkMapFunction extends ProcessFunction<Tuple2<String, String>, Mes
     }
 
     @Override
-    public void processElement(Tuple2<String, String> tuple, Context ctx, Collector<Message> out) throws Exception {
+    public void processElement(Tuple2<String, String> tuple, Context ctx, Collector<Message> out)
+            throws Exception {
         Validate.notNull(topic, "the message topic is null");
         Validate.notNull(tuple.f1.getBytes(), "the message body is null");
 
diff --git a/src/main/java/org/apache/rocketmq/flink/function/SourceMapFunction.java b/src/main/java/org/apache/rocketmq/flink/legacy/function/SourceMapFunction.java
similarity index 76%
rename from src/main/java/org/apache/rocketmq/flink/function/SourceMapFunction.java
rename to src/main/java/org/apache/rocketmq/flink/legacy/function/SourceMapFunction.java
index 8dd07c6..a49df95 100644
--- a/src/main/java/org/apache/rocketmq/flink/function/SourceMapFunction.java
+++ b/src/main/java/org/apache/rocketmq/flink/legacy/function/SourceMapFunction.java
@@ -15,16 +15,19 @@
  * limitations under the License.
  */
 
-package org.apache.rocketmq.flink.function;
+package org.apache.rocketmq.flink.legacy.function;
 
 import org.apache.flink.api.java.tuple.Tuple2;
 import org.apache.flink.streaming.api.functions.ProcessFunction;
 import org.apache.flink.util.Collector;
 
-public class SourceMapFunction extends ProcessFunction<Tuple2<String, String>, Tuple2<String, String>> {
+public class SourceMapFunction
+        extends ProcessFunction<Tuple2<String, String>, Tuple2<String, String>> {
 
     @Override
-    public void processElement(Tuple2<String, String> value, Context ctx, Collector<Tuple2<String, String>> out) throws Exception {
+    public void processElement(
+            Tuple2<String, String> value, Context ctx, Collector<Tuple2<String, String>> out)
+            throws Exception {
         out.collect(new Tuple2<>(value.f0, value.f1));
     }
 }
diff --git a/src/main/java/org/apache/rocketmq/flink/source/RocketMQSource.java b/src/main/java/org/apache/rocketmq/flink/source/RocketMQSource.java
new file mode 100644
index 0000000..b899618
--- /dev/null
+++ b/src/main/java/org/apache/rocketmq/flink/source/RocketMQSource.java
@@ -0,0 +1,175 @@
+/*
+ * 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.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.RocketMQRecordDeserializationSchema;
+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 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 = -6755372893283732098L;
+
+    private final String topic;
+    private final String consumerGroup;
+    private final String tag;
+    private final long stopInMs;
+    private final long startTime;
+    private final long startOffset;
+    private final long partitionDiscoveryIntervalMs;
+
+    // Boundedness
+    private final Boundedness boundedness;
+    private final RocketMQRecordDeserializationSchema<OUT> deserializationSchema;
+
+    public RocketMQSource(
+            String topic,
+            String consumerGroup,
+            String tag,
+            long stopInMs,
+            long startTime,
+            long startOffset,
+            long partitionDiscoveryIntervalMs,
+            Boundedness boundedness,
+            RocketMQRecordDeserializationSchema<OUT> deserializationSchema) {
+        this.topic = topic;
+        this.consumerGroup = consumerGroup;
+        this.tag = tag;
+        this.stopInMs = stopInMs;
+        this.startTime = startTime;
+        this.startOffset = startOffset;
+        this.partitionDiscoveryIntervalMs = partitionDiscoveryIntervalMs;
+        this.boundedness = boundedness;
+        this.deserializationSchema = deserializationSchema;
+    }
+
+    @Override
+    public Boundedness getBoundedness() {
+        return this.boundedness;
+    }
+
+    @Override
+    public SourceReader<OUT, RocketMQPartitionSplit> createReader(SourceReaderContext readerContext)
+            throws Exception {
+        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<>(
+                                topic,
+                                consumerGroup,
+                                tag,
+                                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,
+                stopInMs,
+                startOffset,
+                partitionDiscoveryIntervalMs,
+                boundedness,
+                enumContext);
+    }
+
+    @Override
+    public SplitEnumerator<RocketMQPartitionSplit, RocketMQSourceEnumState> restoreEnumerator(
+            SplitEnumeratorContext<RocketMQPartitionSplit> enumContext,
+            RocketMQSourceEnumState checkpoint) {
+        return new RocketMQSourceEnumerator(
+                topic,
+                consumerGroup,
+                stopInMs,
+                startOffset,
+                partitionDiscoveryIntervalMs,
+                boundedness,
+                enumContext,
+                checkpoint.getCurrentAssignment());
+    }
+
+    @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/common/selector/DefaultTopicSelector.java b/src/main/java/org/apache/rocketmq/flink/source/enumerator/RocketMQSourceEnumState.java
similarity index 54%
rename from src/main/java/org/apache/rocketmq/flink/common/selector/DefaultTopicSelector.java
rename to src/main/java/org/apache/rocketmq/flink/source/enumerator/RocketMQSourceEnumState.java
index 264d211..a23139f 100644
--- a/src/main/java/org/apache/rocketmq/flink/common/selector/DefaultTopicSelector.java
+++ b/src/main/java/org/apache/rocketmq/flink/source/enumerator/RocketMQSourceEnumState.java
@@ -1,4 +1,4 @@
-/**
+/*
  * 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
@@ -7,7 +7,7 @@
  * "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
+ *     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,
@@ -16,28 +16,23 @@
  * limitations under the License.
  */
 
-package org.apache.rocketmq.flink.common.selector;
+package org.apache.rocketmq.flink.source.enumerator;
 
-public class DefaultTopicSelector<T> implements TopicSelector<T> {
-    private final String topicName;
-    private final String tagName;
+import org.apache.rocketmq.flink.source.split.RocketMQPartitionSplit;
 
-    public DefaultTopicSelector(final String topicName, final String tagName) {
-        this.topicName = topicName;
-        this.tagName = tagName;
-    }
+import java.util.List;
+import java.util.Map;
 
-    public DefaultTopicSelector(final String topicName) {
-        this(topicName, "");
-    }
+/** The state of RocketMQ source enumerator. */
+public class RocketMQSourceEnumState {
+
+    private final Map<Integer, List<RocketMQPartitionSplit>> currentAssignment;
 
-    @Override
-    public String getTopic(T tuple) {
-        return topicName;
+    RocketMQSourceEnumState(Map<Integer, List<RocketMQPartitionSplit>> currentAssignment) {
+        this.currentAssignment = currentAssignment;
     }
 
-    @Override
-    public String getTag(T tuple) {
-        return tagName;
+    public Map<Integer, List<RocketMQPartitionSplit>> getCurrentAssignment() {
+        return currentAssignment;
     }
 }
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
new file mode 100644
index 0000000..ce45b51
--- /dev/null
+++ b/src/main/java/org/apache/rocketmq/flink/source/enumerator/RocketMQSourceEnumStateSerializer.java
@@ -0,0 +1,64 @@
+/*
+ * 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
new file mode 100644
index 0000000..08290c6
--- /dev/null
+++ b/src/main/java/org/apache/rocketmq/flink/source/enumerator/RocketMQSourceEnumerator.java
@@ -0,0 +1,337 @@
+/*
+ * 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.client.consumer.DefaultMQPullConsumer;
+import org.apache.rocketmq.client.consumer.MQPullConsumer;
+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.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import javax.annotation.Nullable;
+
+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 java.util.stream.Collectors;
+
+/** The enumerator class for RocketMQ source. */
+@Internal
+public class RocketMQSourceEnumerator
+        implements SplitEnumerator<RocketMQPartitionSplit, RocketMQSourceEnumState> {
+
+    private static final Logger LOG = LoggerFactory.getLogger(RocketMQSourceEnumerator.class);
+
+    /** The topic used for this RocketMQSource. */
+    private final String topic;
+    /** The consumer group used for this RocketMQSource. */
+    private final String consumerGroup;
+    /** 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;
+
+    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 MQPullConsumer consumer;
+    private boolean noMoreNewPartitionSplits = false;
+
+    public RocketMQSourceEnumerator(
+            String topic,
+            String consumerGroup,
+            long stopInMs,
+            long startOffset,
+            long partitionDiscoveryIntervalMs,
+            Boundedness boundedness,
+            SplitEnumeratorContext<RocketMQPartitionSplit> context) {
+        this(
+                topic,
+                consumerGroup,
+                stopInMs,
+                startOffset,
+                partitionDiscoveryIntervalMs,
+                boundedness,
+                context,
+                new HashMap<>());
+    }
+
+    public RocketMQSourceEnumerator(
+            String topic,
+            String consumerGroup,
+            long stopInMs,
+            long startOffset,
+            long partitionDiscoveryIntervalMs,
+            Boundedness boundedness,
+            SplitEnumeratorContext<RocketMQPartitionSplit> context,
+            Map<Integer, List<RocketMQPartitionSplit>> currentSplitsAssignments) {
+        this.topic = topic;
+        this.consumerGroup = consumerGroup;
+        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<>();
+    }
+
+    @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() {
+        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));
+        Set<MessageQueue> messageQueues = consumer.fetchSubscribeMessageQueues(topic);
+        for (MessageQueue messageQueue : messageQueues) {
+            Tuple3<String, String, Integer> topicPartition =
+                    new Tuple3<>(
+                            messageQueue.getTopic(),
+                            messageQueue.getBrokerName(),
+                            messageQueue.getQueueId());
+            if (!removedPartitions.remove(topicPartition)) {
+                newPartitions.add(topicPartition);
+            }
+        }
+        discoveredPartitions.addAll(Collections.unmodifiableSet(newPartitions));
+        return newPartitions.stream()
+                .map(
+                        messageQueue ->
+                                new RocketMQPartitionSplit(
+                                        messageQueue.f0,
+                                        messageQueue.f1,
+                                        messageQueue.f2,
+                                        startOffset,
+                                        stopInMs))
+                .collect(Collectors.toSet());
+    }
+
+    // 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 void initialRocketMQConsumer() {
+        try {
+            consumer = new DefaultMQPullConsumer(consumerGroup);
+            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
new file mode 100644
index 0000000..3bbeec8
--- /dev/null
+++ b/src/main/java/org/apache/rocketmq/flink/source/reader/RocketMQPartitionSplitReader.java
@@ -0,0 +1,373 @@
+/*
+ * 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.client.consumer.DefaultMQPullConsumer;
+import org.apache.rocketmq.client.consumer.MQPullConsumer;
+import org.apache.rocketmq.client.consumer.PullResult;
+import org.apache.rocketmq.client.exception.MQBrokerException;
+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.RocketMQRecordDeserializationSchema;
+import org.apache.rocketmq.flink.source.split.RocketMQPartitionSplit;
+import org.apache.rocketmq.remoting.exception.RemotingException;
+
+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.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import javax.annotation.Nullable;
+
+import java.io.IOException;
+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;
+
+import static org.apache.rocketmq.client.consumer.PullStatus.FOUND;
+
+/**
+ * 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 long stopInMs;
+    private final long startTime;
+    private final long startOffset;
+
+    private final RocketMQRecordDeserializationSchema<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 MQPullConsumer consumer;
+
+    private volatile boolean wakeup = false;
+
+    private static final int MAX_MESSAGE_NUMBER_PER_BLOCK = 64;
+
+    public RocketMQPartitionSplitReader(
+            String topic,
+            String consumerGroup,
+            String tag,
+            long stopInMs,
+            long startTime,
+            long startOffset,
+            RocketMQRecordDeserializationSchema<T> deserializationSchema) {
+        this.topic = topic;
+        this.tag = tag;
+        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);
+    }
+
+    @Override
+    public RecordsWithSplitIds<Tuple3<T, Long, Long>> fetch() throws IOException {
+        RocketMQPartitionSplitRecords<Tuple3<T, Long, Long>> recordsBySplits =
+                new RocketMQPartitionSplitRecords<>();
+        Set<MessageQueue> messageQueues;
+        try {
+            messageQueues = consumer.fetchSubscribeMessageQueues(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.searchOffset(messageQueue, startTime)
+                                        : startOffset;
+                    } catch (MQClientException e) {
+                        LOG.error(
+                                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;
+                }
+                PullResult pullResult = null;
+                try {
+                    if (wakeup) {
+                        LOG.info(
+                                String.format(
+                                        "Wake up pulling messages of topic[%s] broker[%s] queue[%d] tag[%s] from offset[%d].",
+                                        messageQueue.getTopic(),
+                                        messageQueue.getBrokerName(),
+                                        messageQueue.getQueueId(),
+                                        tag,
+                                        messageOffset));
+                        wakeup = false;
+                        recordsBySplits.prepareForRead();
+                        return recordsBySplits;
+                    }
+                    pullResult =
+                            consumer.pullBlockIfNotFound(
+                                    messageQueue, tag, messageOffset, MAX_MESSAGE_NUMBER_PER_BLOCK);
+                } catch (MQClientException
+                        | RemotingException
+                        | MQBrokerException
+                        | InterruptedException e) {
+                    LOG.error(
+                            String.format(
+                                    "Pull RocketMQ messages of topic[%s] broker[%s] queue[%d] tag[%s] from offset[%d] exception.",
+                                    messageQueue.getTopic(),
+                                    messageQueue.getBrokerName(),
+                                    messageQueue.getQueueId(),
+                                    tag,
+                                    messageOffset),
+                            e);
+                }
+                startingOffsets.put(
+                        topicPartition,
+                        pullResult == null ? messageOffset : pullResult.getNextBeginOffset());
+                if (pullResult != null && pullResult.getPullStatus() == FOUND) {
+                    Collection<Tuple3<T, Long, Long>> recordsForSplit =
+                            recordsBySplits.recordsForSplit(
+                                    messageQueue.getTopic()
+                                            + "-"
+                                            + messageQueue.getBrokerName()
+                                            + "-"
+                                            + messageQueue.getQueueId());
+                    for (MessageExt messageExt : pullResult.getMsgFoundList()) {
+                        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();
+        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()));
+        }
+        // Setup 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) {
+        try {
+            consumer = new DefaultMQPullConsumer(consumerGroup);
+            consumer.start();
+        } 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
new file mode 100644
index 0000000..25270b5
--- /dev/null
+++ b/src/main/java/org/apache/rocketmq/flink/source/reader/RocketMQRecordEmitter.java
@@ -0,0 +1,39 @@
+/*
+ * 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
new file mode 100644
index 0000000..0257e34
--- /dev/null
+++ b/src/main/java/org/apache/rocketmq/flink/source/reader/RocketMQSourceReader.java
@@ -0,0 +1,64 @@
+/*
+ * 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/RocketMQRecordDeserializationSchema.java b/src/main/java/org/apache/rocketmq/flink/source/reader/deserializer/RocketMQRecordDeserializationSchema.java
new file mode 100644
index 0000000..455f8af
--- /dev/null
+++ b/src/main/java/org/apache/rocketmq/flink/source/reader/deserializer/RocketMQRecordDeserializationSchema.java
@@ -0,0 +1,43 @@
+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.api.java.typeutils.ResultTypeQueryable;
+import org.apache.flink.util.Collector;
+
+import java.io.IOException;
+import java.io.Serializable;
+import java.util.List;
+
+/** An interface for the deserialization of RocketMQ records. */
+public interface RocketMQRecordDeserializationSchema<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.
+     */
+    @PublicEvolving
+    default void open(InitializationContext context) throws Exception {}
+
+    /**
+     * 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.
+     */
+    @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
new file mode 100644
index 0000000..9bda60f
--- /dev/null
+++ b/src/main/java/org/apache/rocketmq/flink/source/split/RocketMQPartitionSplit.java
@@ -0,0 +1,100 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.rocketmq.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, Partition: %s, StartingOffset: %d, StoppingTimestamp: %d]",
+                topic, partition, startingOffset, stoppingTimestamp);
+    }
+
+    @Override
+    public int hashCode() {
+        return Objects.hash(topic, partition, startingOffset, stoppingTimestamp);
+    }
+
+    @Override
+    public boolean equals(Object obj) {
+        if (!(obj instanceof RocketMQPartitionSplit)) {
+            return false;
+        }
+        RocketMQPartitionSplit other = (RocketMQPartitionSplit) obj;
+        return topic.equals(other.topic)
+                && 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
new file mode 100644
index 0000000..2363257
--- /dev/null
+++ b/src/main/java/org/apache/rocketmq/flink/source/split/RocketMQPartitionSplitSerializer.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.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/split/RocketMQPartitionSplitState.java b/src/main/java/org/apache/rocketmq/flink/source/split/RocketMQPartitionSplitState.java
new file mode 100644
index 0000000..4fbb3da
--- /dev/null
+++ b/src/main/java/org/apache/rocketmq/flink/source/split/RocketMQPartitionSplitState.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.rocketmq.flink.source.split;
+
+/** This class extends RocketMQPartitionSplit to track a mutable current offset. */
+public class RocketMQPartitionSplitState extends RocketMQPartitionSplit {
+
+    private long currentOffset;
+
+    public RocketMQPartitionSplitState(RocketMQPartitionSplit partitionSplit) {
+        super(
+                partitionSplit.getTopic(),
+                partitionSplit.getBroker(),
+                partitionSplit.getPartition(),
+                partitionSplit.getStartingOffset(),
+                partitionSplit.getStoppingTimestamp());
+        this.currentOffset = partitionSplit.getStartingOffset();
+    }
+
+    public long getCurrentOffset() {
+        return currentOffset;
+    }
+
+    public void setCurrentOffset(long currentOffset) {
+        this.currentOffset = currentOffset;
+    }
+
+    /**
+     * Use the current offset as the starting offset to create a new RocketMQPartitionSplit.
+     *
+     * @return a new RocketMQPartitionSplit which uses the current offset as its starting offset.
+     */
+    public RocketMQPartitionSplit toRocketMQPartitionSplit() {
+        return new RocketMQPartitionSplit(
+                getTopic(),
+                getBroker(),
+                getPartition(),
+                getCurrentOffset(),
+                getStoppingTimestamp());
+    }
+}
diff --git a/src/test/java/org/apache/rocketmq/flink/RocketMQSinkTest.java b/src/test/java/org/apache/rocketmq/flink/RocketMQSinkTest.java
deleted file mode 100644
index 6738ec3..0000000
--- a/src/test/java/org/apache/rocketmq/flink/RocketMQSinkTest.java
+++ /dev/null
@@ -1,70 +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;
-
-import java.util.Properties;
-import org.apache.flink.api.java.tuple.Tuple2;
-import org.apache.rocketmq.client.producer.DefaultMQProducer;
-import org.apache.rocketmq.common.message.Message;
-import org.apache.rocketmq.flink.common.selector.DefaultTopicSelector;
-import org.apache.rocketmq.flink.common.selector.TopicSelector;
-import org.apache.rocketmq.flink.common.serialization.KeyValueSerializationSchema;
-import org.apache.rocketmq.flink.common.serialization.SimpleKeyValueSerializationSchema;
-import org.junit.Before;
-import org.junit.Ignore;
-import org.junit.Test;
-
-import static org.apache.rocketmq.flink.common.util.TestUtils.setFieldValue;
-import static org.mockito.Mockito.mock;
-import static org.mockito.Mockito.verify;
-
-@Ignore
-public class RocketMQSinkTest {
-
-    private RocketMQSink rocketMQSink;
-    private DefaultMQProducer producer;
-
-    @Before
-    public void setUp() throws Exception {
-        KeyValueSerializationSchema serializationSchema = new SimpleKeyValueSerializationSchema("id", "name");
-        TopicSelector topicSelector = new DefaultTopicSelector("tpc");
-        Properties props = new Properties();
-        props.setProperty(RocketMQConfig.MSG_DELAY_LEVEL, String.valueOf(RocketMQConfig.MSG_DELAY_LEVEL04));
-        rocketMQSink = new RocketMQSink(props);
-
-        producer = mock(DefaultMQProducer.class);
-        setFieldValue(rocketMQSink, "producer", producer);
-    }
-
-    @Test
-    public void testSink() throws Exception {
-        Tuple2<String, String> tuple = new Tuple2<>("id", "province");
-        String topic = "testTopic";
-        String tag = "testTag";
-        Message message = new Message(topic, tag, tuple.f0, tuple.f1.getBytes());
-    }
-
-    @Test
-    public void close() throws Exception {
-        rocketMQSink.close();
-
-        verify(producer).shutdown();
-    }
-
-}
\ No newline at end of file
diff --git a/src/test/java/org/apache/rocketmq/flink/common/selector/DefaultTopicSelectorTest.java b/src/test/java/org/apache/rocketmq/flink/common/selector/DefaultTopicSelectorTest.java
deleted file mode 100644
index 2f4685c..0000000
--- a/src/test/java/org/apache/rocketmq/flink/common/selector/DefaultTopicSelectorTest.java
+++ /dev/null
@@ -1,37 +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.selector;
-
-import org.junit.Test;
-
-import static org.junit.Assert.*;
-
-public class DefaultTopicSelectorTest {
-    @Test
-    public void getTopic() throws Exception {
-        DefaultTopicSelector selector = new DefaultTopicSelector("rocket");
-        assertEquals("rocket", selector.getTopic(null));
-        assertEquals("", selector.getTag(null));
-
-        selector = new DefaultTopicSelector("rocket", "tg");
-        assertEquals("rocket", selector.getTopic(null));
-        assertEquals("tg", selector.getTag(null));
-    }
-
-}
\ No newline at end of file
diff --git a/src/test/java/org/apache/rocketmq/flink/common/selector/SimpleTopicSelectorTest.java b/src/test/java/org/apache/rocketmq/flink/common/selector/SimpleTopicSelectorTest.java
deleted file mode 100644
index 6ac1a57..0000000
--- a/src/test/java/org/apache/rocketmq/flink/common/selector/SimpleTopicSelectorTest.java
+++ /dev/null
@@ -1,49 +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.selector;
-
-import java.util.HashMap;
-import java.util.Map;
-
-import org.junit.Test;
-
-import static org.junit.Assert.*;
-
-public class SimpleTopicSelectorTest {
-    @Test
-    public void getTopic() throws Exception {
-        SimpleTopicSelector selector = new SimpleTopicSelector("tpc", "dtpc", "tg", "dtg");
-        Map tuple = new HashMap();
-        tuple.put("id", "x001");
-        tuple.put("name", "vesense");
-        tuple.put("tpc", "tpc1");
-        tuple.put("tg", "tg1");
-
-        assertEquals("tpc1", selector.getTopic(tuple));
-        assertEquals("tg1", selector.getTag(tuple));
-
-        tuple = new HashMap();
-        tuple.put("id", "x001");
-        tuple.put("name", "vesense");
-
-        assertEquals("dtpc", selector.getTopic(tuple));
-        assertEquals("dtg", selector.getTag(tuple));
-    }
-
-}
\ No newline at end of file
diff --git a/src/test/java/org/apache/rocketmq/flink/common/serialization/SimpleKeyValueSerializationSchemaTest.java b/src/test/java/org/apache/rocketmq/flink/common/serialization/SimpleKeyValueSerializationSchemaTest.java
deleted file mode 100644
index 98aa793..0000000
--- a/src/test/java/org/apache/rocketmq/flink/common/serialization/SimpleKeyValueSerializationSchemaTest.java
+++ /dev/null
@@ -1,42 +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.serialization;
-
-import java.util.HashMap;
-import java.util.Map;
-
-import org.junit.Test;
-
-import static org.junit.Assert.*;
-
-public class SimpleKeyValueSerializationSchemaTest {
-    @Test
-    public void serializeKeyAndValue() throws Exception {
-        SimpleKeyValueSerializationSchema serializationSchema = new SimpleKeyValueSerializationSchema("id", "name");
-        SimpleKeyValueDeserializationSchema deserializationSchema = new SimpleKeyValueDeserializationSchema("id", "name");
-
-        Map tuple = new HashMap();
-        tuple.put("id", "x001");
-        tuple.put("name", "vesense");
-
-        assertEquals(tuple, deserializationSchema.deserializeKeyAndValue(serializationSchema.serializeKey(tuple),
-            serializationSchema.serializeValue(tuple)));
-    }
-
-}
\ No newline at end of file
diff --git a/src/test/java/org/apache/rocketmq/flink/legacy/RocketMQSinkTest.java b/src/test/java/org/apache/rocketmq/flink/legacy/RocketMQSinkTest.java
new file mode 100644
index 0000000..c45dbdf
--- /dev/null
+++ b/src/test/java/org/apache/rocketmq/flink/legacy/RocketMQSinkTest.java
@@ -0,0 +1,70 @@
+/**
+ * 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
+ *
+ * <p>http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * <p>Unless required by applicable law or agreed to in writing, software distributed under the
+ * License is distributed on an "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either
+ * express or implied. See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.rocketmq.flink.legacy;
+
+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;
+
+@Ignore
+public class RocketMQSinkTest {
+
+    private RocketMQSink rocketMQSink;
+    private DefaultMQProducer producer;
+
+    @Before
+    public void setUp() throws Exception {
+        KeyValueSerializationSchema serializationSchema =
+                new SimpleKeyValueSerializationSchema("id", "name");
+        TopicSelector topicSelector = new DefaultTopicSelector("tpc");
+        Properties props = new Properties();
+        props.setProperty(
+                RocketMQConfig.MSG_DELAY_LEVEL, String.valueOf(RocketMQConfig.MSG_DELAY_LEVEL04));
+        rocketMQSink = new RocketMQSink(props);
+
+        producer = mock(DefaultMQProducer.class);
+        setFieldValue(rocketMQSink, "producer", producer);
+    }
+
+    @Test
+    public void testSink() throws Exception {
+        Tuple2<String, String> tuple = new Tuple2<>("id", "province");
+        String topic = "testTopic";
+        String tag = "testTag";
+        Message message = new Message(topic, tag, tuple.f0, tuple.f1.getBytes());
+    }
+
+    @Test
+    public void close() throws Exception {
+        rocketMQSink.close();
+
+        verify(producer).shutdown();
+    }
+}
diff --git a/src/test/java/org/apache/rocketmq/flink/RocketMQSourceTest.java b/src/test/java/org/apache/rocketmq/flink/legacy/RocketMQSourceTest.java
similarity index 77%
rename from src/test/java/org/apache/rocketmq/flink/RocketMQSourceTest.java
rename to src/test/java/org/apache/rocketmq/flink/legacy/RocketMQSourceTest.java
index 2f16a96..a863ddd 100644
--- a/src/test/java/org/apache/rocketmq/flink/RocketMQSourceTest.java
+++ b/src/test/java/org/apache/rocketmq/flink/legacy/RocketMQSourceTest.java
@@ -1,43 +1,42 @@
 /**
- * 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
+ * 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
+ * <p>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
+ * <p>Unless required by applicable law or agreed to in writing, software distributed under the
+ * License is distributed on an "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either
+ * express or implied. See the License for the specific language governing permissions and
  * limitations under the License.
  */
+package org.apache.rocketmq.flink.legacy;
 
-package org.apache.rocketmq.flink;
-
-import java.util.ArrayList;
-import java.util.HashSet;
-import java.util.List;
-import java.util.Properties;
-import java.util.Set;
-import java.util.concurrent.ConcurrentHashMap;
-import org.apache.flink.streaming.api.functions.source.SourceFunction.SourceContext;
 import org.apache.rocketmq.client.consumer.DefaultMQPullConsumer;
 import org.apache.rocketmq.client.consumer.MQPullConsumerScheduleService;
 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.common.serialization.KeyValueDeserializationSchema;
-import org.apache.rocketmq.flink.common.serialization.SimpleKeyValueDeserializationSchema;
+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;
 
-import static org.apache.rocketmq.flink.common.util.TestUtils.setFieldValue;
+import java.util.ArrayList;
+import java.util.HashSet;
+import java.util.List;
+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.anyBoolean;
 import static org.mockito.Matchers.anyInt;
@@ -87,7 +86,8 @@ public class RocketMQSourceTest {
         PullResult pullResult = new PullResult(PullStatus.FOUND, 3, 1, 5, msgFoundList);
 
         when(consumer.fetchConsumeOffset(any(MessageQueue.class), anyBoolean())).thenReturn(2L);
-        when(consumer.pull(any(MessageQueue.class), anyString(), anyLong(), anyInt())).thenReturn(pullResult);
+        when(consumer.pull(any(MessageQueue.class), anyString(), anyLong(), anyInt()))
+                .thenReturn(pullResult);
 
         SourceContext context = mock(SourceContext.class);
         when(context.getCheckpointLock()).thenReturn(new Object());
@@ -118,4 +118,4 @@ public class RocketMQSourceTest {
             return false;
         }
     }
-}
\ No newline at end of file
+}
diff --git a/src/test/java/org/apache/rocketmq/flink/legacy/common/selector/DefaultTopicSelectorTest.java b/src/test/java/org/apache/rocketmq/flink/legacy/common/selector/DefaultTopicSelectorTest.java
new file mode 100644
index 0000000..b235c63
--- /dev/null
+++ b/src/test/java/org/apache/rocketmq/flink/legacy/common/selector/DefaultTopicSelectorTest.java
@@ -0,0 +1,32 @@
+/**
+ * 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
+ *
+ * <p>http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * <p>Unless required by applicable law or agreed to in writing, software distributed under the
+ * License is distributed on an "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either
+ * express or implied. See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.rocketmq.flink.legacy.common.selector;
+
+import org.junit.Test;
+
+import static org.junit.Assert.assertEquals;
+
+public class DefaultTopicSelectorTest {
+    @Test
+    public void getTopic() throws Exception {
+        DefaultTopicSelector selector = new DefaultTopicSelector("rocket");
+        assertEquals("rocket", selector.getTopic(null));
+        assertEquals("", selector.getTag(null));
+
+        selector = new DefaultTopicSelector("rocket", "tg");
+        assertEquals("rocket", selector.getTopic(null));
+        assertEquals("tg", selector.getTag(null));
+    }
+}
diff --git a/src/test/java/org/apache/rocketmq/flink/legacy/common/selector/SimpleTopicSelectorTest.java b/src/test/java/org/apache/rocketmq/flink/legacy/common/selector/SimpleTopicSelectorTest.java
new file mode 100644
index 0000000..5c0f755
--- /dev/null
+++ b/src/test/java/org/apache/rocketmq/flink/legacy/common/selector/SimpleTopicSelectorTest.java
@@ -0,0 +1,44 @@
+/**
+ * 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
+ *
+ * <p>http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * <p>Unless required by applicable law or agreed to in writing, software distributed under the
+ * License is distributed on an "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either
+ * express or implied. See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.rocketmq.flink.legacy.common.selector;
+
+import org.junit.Test;
+
+import java.util.HashMap;
+import java.util.Map;
+
+import static org.junit.Assert.assertEquals;
+
+public class SimpleTopicSelectorTest {
+    @Test
+    public void getTopic() throws Exception {
+        SimpleTopicSelector selector = new SimpleTopicSelector("tpc", "dtpc", "tg", "dtg");
+        Map tuple = new HashMap();
+        tuple.put("id", "x001");
+        tuple.put("name", "vesense");
+        tuple.put("tpc", "tpc1");
+        tuple.put("tg", "tg1");
+
+        assertEquals("tpc1", selector.getTopic(tuple));
+        assertEquals("tg1", selector.getTag(tuple));
+
+        tuple = new HashMap();
+        tuple.put("id", "x001");
+        tuple.put("name", "vesense");
+
+        assertEquals("dtpc", selector.getTopic(tuple));
+        assertEquals("dtg", selector.getTag(tuple));
+    }
+}
diff --git a/src/test/java/org/apache/rocketmq/flink/legacy/common/serialization/SimpleKeyValueSerializationSchemaTest.java b/src/test/java/org/apache/rocketmq/flink/legacy/common/serialization/SimpleKeyValueSerializationSchemaTest.java
new file mode 100644
index 0000000..78baf20
--- /dev/null
+++ b/src/test/java/org/apache/rocketmq/flink/legacy/common/serialization/SimpleKeyValueSerializationSchemaTest.java
@@ -0,0 +1,42 @@
+/**
+ * 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
+ *
+ * <p>http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * <p>Unless required by applicable law or agreed to in writing, software distributed under the
+ * License is distributed on an "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either
+ * express or implied. See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.rocketmq.flink.legacy.common.serialization;
+
+import org.junit.Test;
+
+import java.util.HashMap;
+import java.util.Map;
+
+import static org.junit.Assert.assertEquals;
+
+public class SimpleKeyValueSerializationSchemaTest {
+    @Test
+    public void serializeKeyAndValue() throws Exception {
+        SimpleKeyValueSerializationSchema serializationSchema =
+                new SimpleKeyValueSerializationSchema("id", "name");
+        SimpleKeyValueDeserializationSchema deserializationSchema =
+                new SimpleKeyValueDeserializationSchema("id", "name");
+
+        Map tuple = new HashMap();
+        tuple.put("id", "x001");
+        tuple.put("name", "vesense");
+
+        assertEquals(
+                tuple,
+                deserializationSchema.deserializeKeyAndValue(
+                        serializationSchema.serializeKey(tuple),
+                        serializationSchema.serializeValue(tuple)));
+    }
+}
diff --git a/src/test/java/org/apache/rocketmq/flink/source/enumerator/RocketMQSourceEnumStateSerializerTest.java b/src/test/java/org/apache/rocketmq/flink/source/enumerator/RocketMQSourceEnumStateSerializerTest.java
new file mode 100644
index 0000000..45ff0e3
--- /dev/null
+++ b/src/test/java/org/apache/rocketmq/flink/source/enumerator/RocketMQSourceEnumStateSerializerTest.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.rocketmq.flink.source.enumerator;
+
+import org.apache.rocketmq.flink.source.split.RocketMQPartitionSplit;
+
+import org.apache.flink.api.connector.source.SplitsAssignment;
+
+import org.junit.Test;
+
+import java.io.IOException;
+import java.util.Arrays;
+import java.util.HashMap;
+
+import static org.junit.Assert.assertEquals;
+
+/** Test for {@link RocketMQSourceEnumStateSerializer}. */
+public class RocketMQSourceEnumStateSerializerTest {
+
+    @Test
+    public void testSerializeDeserializeSourceEnumState() throws IOException {
+        RocketMQSourceEnumStateSerializer serializer = new RocketMQSourceEnumStateSerializer();
+        RocketMQSourceEnumState expected = prepareSourceEnumeratorState();
+        RocketMQSourceEnumState actual = serializer.deserialize(0, serializer.serialize(expected));
+        assertEquals(expected.getCurrentAssignment(), actual.getCurrentAssignment());
+    }
+
+    private RocketMQSourceEnumState prepareSourceEnumeratorState() {
+        SplitsAssignment<RocketMQPartitionSplit> pendingAssignment =
+                new SplitsAssignment<>(new HashMap<>());
+        pendingAssignment
+                .assignment()
+                .put(
+                        0,
+                        Arrays.asList(
+                                new RocketMQPartitionSplit(
+                                        "0", "taobaodaily-01", 1, 0, System.currentTimeMillis()),
+                                new RocketMQPartitionSplit(
+                                        "3", "taobaodaily-01", 2, 0, System.currentTimeMillis()),
+                                new RocketMQPartitionSplit(
+                                        "6", "taobaodaily-01", 3, 0, System.currentTimeMillis()),
+                                new RocketMQPartitionSplit(
+                                        "9", "taobaodaily-01", 4, 0, System.currentTimeMillis())));
+        pendingAssignment
+                .assignment()
+                .put(
+                        1,
+                        Arrays.asList(
+                                new RocketMQPartitionSplit(
+                                        "1", "taobaodaily-02", 5, 0, System.currentTimeMillis()),
+                                new RocketMQPartitionSplit(
+                                        "4", "taobaodaily-02", 6, 0, System.currentTimeMillis()),
+                                new RocketMQPartitionSplit(
+                                        "7", "taobaodaily-02", 7, 0, System.currentTimeMillis())));
+        pendingAssignment
+                .assignment()
+                .put(
+                        2,
+                        Arrays.asList(
+                                new RocketMQPartitionSplit(
+                                        "2", "taobaodaily-03", 8, 0, System.currentTimeMillis()),
+                                new RocketMQPartitionSplit(
+                                        "5", "taobaodaily-03", 9, 0, System.currentTimeMillis()),
+                                new RocketMQPartitionSplit(
+                                        "8", "taobaodaily-03", 10, 0, System.currentTimeMillis())));
+        return new RocketMQSourceEnumState(pendingAssignment.assignment());
+    }
+}
diff --git a/src/test/java/org/apache/rocketmq/flink/source/reader/RocketMQRecordEmitterTest.java b/src/test/java/org/apache/rocketmq/flink/source/reader/RocketMQRecordEmitterTest.java
new file mode 100644
index 0000000..83c1c4b
--- /dev/null
+++ b/src/test/java/org/apache/rocketmq/flink/source/reader/RocketMQRecordEmitterTest.java
@@ -0,0 +1,97 @@
+/*
+ * 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.common.message.MessageExt;
+import org.apache.rocketmq.flink.source.split.RocketMQPartitionSplit;
+import org.apache.rocketmq.flink.source.split.RocketMQPartitionSplitState;
+
+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.table.data.GenericRowData;
+import org.apache.flink.table.data.RowData;
+
+import org.junit.Test;
+
+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<>();
+        MessageExt message =
+                new MessageExt(
+                        1,
+                        System.currentTimeMillis(),
+                        InetSocketAddress.createUnresolved("localhost", 8080),
+                        System.currentTimeMillis(),
+                        InetSocketAddress.createUnresolved("localhost", 8088),
+                        "184019387");
+        message.setBody("test_emit_record_message".getBytes());
+        GenericRowData rowData = new GenericRowData(1);
+        rowData.setField(0, message.getBody());
+        String topic = "test-record-emitter";
+        String broker = "taobaodaily";
+        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());
+    }
+
+    private static final class TestingEmitterOutput<E> implements ReaderOutput<E> {
+
+        private TestingEmitterOutput() {}
+
+        public void collect(E record) {}
+
+        public void collect(E record, long timestamp) {
+            this.collect(record);
+        }
+
+        public void emitWatermark(Watermark watermark) {
+            throw new UnsupportedOperationException();
+        }
+
+        public void markIdle() {
+            throw new UnsupportedOperationException();
+        }
+
+        public SourceOutput<E> createOutputForSplit(String splitId) {
+            return this;
+        }
+
+        public void releaseOutputForSplit(String splitId) {}
+    }
+}
diff --git a/src/test/java/org/apache/rocketmq/flink/source/split/RocketMQPartitionSplitSerializerTest.java b/src/test/java/org/apache/rocketmq/flink/source/split/RocketMQPartitionSplitSerializerTest.java
new file mode 100644
index 0000000..b56cc9d
--- /dev/null
+++ b/src/test/java/org/apache/rocketmq/flink/source/split/RocketMQPartitionSplitSerializerTest.java
@@ -0,0 +1,44 @@
+/*
+ * 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.junit.Test;
+
+import java.io.IOException;
+
+import static org.junit.Assert.assertEquals;
+
+/** Test for {@link RocketMQPartitionSplitSerializer}. */
+public class RocketMQPartitionSplitSerializerTest {
+
+    @Test
+    public void testSerializePartitionSplit() throws IOException {
+        RocketMQPartitionSplitSerializer serializer = new RocketMQPartitionSplitSerializer();
+        RocketMQPartitionSplit expected =
+                new RocketMQPartitionSplit(
+                        "test-split-serialization",
+                        "taobaodaily",
+                        256,
+                        100,
+                        System.currentTimeMillis());
+        RocketMQPartitionSplit actual =
+                serializer.deserialize(serializer.getVersion(), serializer.serialize(expected));
+        assertEquals(expected, actual);
+    }
+}

[rocketmq-flink] 21/33: Fix Checkstyle issue (#532) (#533)

Posted by du...@apache.org.
This is an automated email from the ASF dual-hosted git repository.

duhengforever pushed a commit to branch main
in repository https://gitbox.apache.org/repos/asf/rocketmq-flink.git

commit 92d4cc396e94751a5280c78c6d439fd13b7bf921
Author: Nishadi Kirielle <nd...@gmail.com>
AuthorDate: Mon Mar 23 13:11:45 2020 +1100

    Fix Checkstyle issue (#532) (#533)
    
    * Update Checkstyle file
    
     * Fix RocketMQSource Checkstyle issue
---
 .../org/apache/rocketmq/flink/RocketMQSource.java  |  2 +-
 style/rmq_checkstyle.xml                           | 41 ++++++++++++----------
 2 files changed, 24 insertions(+), 19 deletions(-)

diff --git a/src/main/java/org/apache/rocketmq/flink/RocketMQSource.java b/src/main/java/org/apache/rocketmq/flink/RocketMQSource.java
index e289b49..b3b37dc 100644
--- a/src/main/java/org/apache/rocketmq/flink/RocketMQSource.java
+++ b/src/main/java/org/apache/rocketmq/flink/RocketMQSource.java
@@ -331,7 +331,7 @@ public class RocketMQSource<OUT> extends RichParallelSourceFunction<OUT>
         this.unionOffsetStates = context.getOperatorStateStore().getUnionListState(new ListStateDescriptor<>(
                 OFFSETS_STATE_NAME, TypeInformation.of(new TypeHint<Tuple2<MessageQueue, Long>>() {
 
-        })));
+                })));
         this.restored = context.isRestored();
 
         if (restored) {
diff --git a/style/rmq_checkstyle.xml b/style/rmq_checkstyle.xml
index d5d591d..15c267e 100644
--- a/style/rmq_checkstyle.xml
+++ b/style/rmq_checkstyle.xml
@@ -1,19 +1,19 @@
 <?xml version="1.0" encoding="UTF-8"?>
 <!--
-  ~ Licensed to the Apache Software Foundation (ASF) under one or more
-  ~ contributor license agreements.  See the NOTICE file distributed with
-  ~ this work for additional information regarding copyright ownership.
-  ~ The ASF licenses this file to You under the Apache License, Version 2.0
-  ~ (the "License"); you may not use this file except in compliance with
-  ~ the License.  You may obtain a copy of the License at
-  ~
-  ~     http://www.apache.org/licenses/LICENSE-2.0
-  ~
-  ~  Unless required by applicable law or agreed to in writing, software
-  ~  distributed under the License is distributed on an "AS IS" BASIS,
-  ~  WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
-  ~  See the License for the specific language governing permissions and
-  ~  limitations under the License.
+  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.
   -->
 
 <!DOCTYPE module PUBLIC
@@ -30,6 +30,12 @@
     <!-- header -->
     <module name="RegexpHeader">
         <property name="header" value="/\*\nLicensed to the Apache Software Foundation*"/>
+        <property name="fileExtensions" value="java"/>
+    </module>
+
+    <module name="RegexpHeader">
+        <property name="header" value="#[\s]*Licensed to the Apache Software Foundation*"/>
+        <property name="fileExtensions" value="properties"/>
     </module>
 
     <!--
@@ -122,9 +128,8 @@
 
         <!--whitespace-->
         <module name="GenericWhitespace"/>
-        <module name="NoWhitespaceBefore"/>
-        <module name="WhitespaceAfter"/>
-        <module name="NoWhitespaceAfter"/>
+        <!--<module name="NoWhitespaceBefore"/>-->
+        <!--<module name="NoWhitespaceAfter"/>-->
         <module name="WhitespaceAround">
             <property name="allowEmptyConstructors" value="true"/>
             <property name="allowEmptyMethods" value="true"/>
@@ -134,4 +139,4 @@
         <module name="ParenPad"/>
         <module name="TypecastParenPad"/>
     </module>
-</module>
+</module>
\ No newline at end of file

[rocketmq-flink] 24/33: [ISSUE #656] Update flink connector rocketmq, support flink metrics (#657)

Posted by du...@apache.org.
This is an automated email from the ASF dual-hosted git repository.

duhengforever pushed a commit to branch main
in repository https://gitbox.apache.org/repos/asf/rocketmq-flink.git

commit fe936a5c66b546462d2b84047fc6edf30c305135
Author: lizhimins <70...@qq.com>
AuthorDate: Fri Dec 4 10:35:54 2020 +0800

    [ISSUE #656] Update flink connector rocketmq, support flink metrics (#657)
    
    * [ISSUE #656] Update flink connector rocketmq, support flink metrics
    
    * [ISSUE #656] Update flink connector rocketmq, support flink metrics
    
    Co-authored-by: 斜阳 <te...@alibaba-inc.com>
---
 pom.xml                                            |  38 ++-
 .../org/apache/rocketmq/flink/RocketMQConfig.java  |  67 ++---
 .../org/apache/rocketmq/flink/RocketMQSink.java    |  90 +++---
 .../org/apache/rocketmq/flink/RocketMQSource.java  | 327 ++++++++++++---------
 .../org/apache/rocketmq/flink/RocketMQUtils.java   |  36 ---
 .../ForwardMessageExtDeserialization.java          |  37 +++
 .../MessageExtDeserializationScheme.java           |  37 +++
 .../SimpleKeyValueDeserializationSchema.java       |   4 +-
 .../SimpleTupleDeserializationSchema.java          |  22 ++
 .../rocketmq/flink/common/util/MetricUtils.java    |  80 +++++
 .../rocketmq/flink/common/util/RetryUtil.java      |  61 ++++
 .../rocketmq/flink/common/util/RocketMQUtils.java  |  73 +++++
 .../rocketmq/flink/common/util}/TestUtils.java     |   2 +-
 .../watermark/BoundedOutOfOrdernessGenerator.java  |  57 ++++
 .../BoundedOutOfOrdernessGeneratorPerQueue.java    |  71 +++++
 .../flink/common/watermark/PunctuatedAssigner.java |  47 +++
 .../watermark/TimeLagWatermarkGenerator.java       |  54 ++++
 .../flink/common/watermark/WaterMarkForAll.java    |  47 +++
 .../flink/common/watermark/WaterMarkPerQueue.java  |  62 ++++
 .../flink/example/RocketMQFlinkExample.java        | 123 ++++++++
 .../rocketmq/flink/example/SimpleConsumer.java     |  79 +++++
 .../rocketmq/flink/example/SimpleProducer.java     |  79 +++++
 .../example/example/RocketMQFlinkExample.java      |  79 -----
 .../flink/example/example/SimpleConsumer.java      |  53 ----
 .../flink/example/example/SimpleProducer.java      |  48 ---
 .../rocketmq/flink/function/SinkMapFunction.java   |  48 +++
 .../rocketmq/flink/function/SourceMapFunction.java |  30 ++
 .../apache/rocketmq/flink/RocketMQSinkTest.java    |  26 +-
 .../apache/rocketmq/flink/RocketMQSourceTest.java  |   8 +-
 29 files changed, 1313 insertions(+), 472 deletions(-)

diff --git a/pom.xml b/pom.xml
index b00d460..2e19ce5 100644
--- a/pom.xml
+++ b/pom.xml
@@ -22,7 +22,7 @@
 
     <groupId>org.apache.rocketmq</groupId>
     <artifactId>rocketmq-flink</artifactId>
-    <version>0.0.1-SNAPSHOT</version>
+    <version>1.0.0-SNAPSHOT</version>
     <packaging>jar</packaging>
 
     <properties>
@@ -34,7 +34,7 @@
         <maven.compiler.source>1.8</maven.compiler.source>
         <maven.compiler.target>1.8</maven.compiler.target>
         <rocketmq.version>4.7.1</rocketmq.version>
-        <flink.version>1.7.0</flink.version>
+        <flink.version>1.10.1</flink.version>
         <commons-lang.version>2.5</commons-lang.version>
         <scala.binary.version>2.11</scala.binary.version>
     </properties>
@@ -124,6 +124,40 @@
     <build>
         <plugins>
             <plugin>
+                <groupId>org.apache.maven.plugins</groupId>
+                <artifactId>maven-shade-plugin</artifactId>
+                <version>2.4.3</version>
+                <executions>
+                    <execution>
+                        <phase>package</phase>
+                        <goals>
+                            <goal>shade</goal>
+                        </goals>
+                        <configuration>
+                            <filters>
+                                <filter>
+                                    <artifact>*:*</artifact>
+                                    <excludes>
+                                        <exclude>META-INF/*.SF</exclude>
+                                        <exclude>META-INF/*.DSA</exclude>
+                                        <exclude>META-INF/*.RSA</exclude>
+                                    </excludes>
+                                </filter>
+                            </filters>
+                            <transformers>
+                                <transformer implementation="org.apache.maven.plugins.shade.resource.AppendingTransformer">
+                                    <resource>reference.conf</resource>
+                                </transformer>
+                                <transformer implementation="org.apache.maven.plugins.shade.resource.ManifestResourceTransformer">
+                                    <mainClass>org.apache.rocketmq.flink.example.RocketMQFlinkExample</mainClass>
+                                </transformer>
+                            </transformers>
+                        </configuration>
+                    </execution>
+                </executions>
+            </plugin>
+
+            <plugin>
                 <artifactId>maven-compiler-plugin</artifactId>
                 <version>3.5.1</version>
                 <configuration>
diff --git a/src/main/java/org/apache/rocketmq/flink/RocketMQConfig.java b/src/main/java/org/apache/rocketmq/flink/RocketMQConfig.java
index 5a0784b..c1bad2d 100644
--- a/src/main/java/org/apache/rocketmq/flink/RocketMQConfig.java
+++ b/src/main/java/org/apache/rocketmq/flink/RocketMQConfig.java
@@ -6,9 +6,9 @@
  * 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
- *
+ * <p>
  * http://www.apache.org/licenses/LICENSE-2.0
- *
+ * <p>
  * Unless required by applicable law or agreed to in writing, software
  * distributed under the License is distributed on an "AS IS" BASIS,
  * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
@@ -18,19 +18,21 @@
 
 package org.apache.rocketmq.flink;
 
-import java.util.Properties;
-import java.util.UUID;
-
 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.DefaultMQPullConsumer;
 import org.apache.rocketmq.client.producer.DefaultMQProducer;
 import org.apache.rocketmq.common.protocol.heartbeat.MessageModel;
 
-import static org.apache.rocketmq.flink.RocketMQUtils.getInteger;
+import java.util.Properties;
+import java.util.UUID;
+
+import static org.apache.rocketmq.flink.common.util.RocketMQUtils.getAccessChannel;
+import static org.apache.rocketmq.flink.common.util.RocketMQUtils.getInteger;
 
 /**
  * RocketMQConfig for Consumer/Producer.
@@ -45,8 +47,15 @@ public class RocketMQConfig {
     public static final String BROKER_HEART_BEAT_INTERVAL = "brokerserver.heartbeat.interval";
     public static final int DEFAULT_BROKER_HEART_BEAT_INTERVAL = 30000; // 30 seconds
 
+    // Access control config
+    public static final String ACCESS_KEY = "access.key";
+    public static final String SECRET_KEY = "secret.key";
+
+    public static final String ACCESS_CHANNEL = "access.channel";
+    public static final AccessChannel DEFAULT_ACCESS_CHANNEL = AccessChannel.LOCAL;
 
     // Producer related config
+    public static final String PRODUCER_TOPIC = "producer.topic";
     public static final String PRODUCER_GROUP = "producer.group";
 
     public static final String PRODUCER_RETRY_TIMES = "producer.retry.times";
@@ -55,13 +64,8 @@ public class RocketMQConfig {
     public static final String PRODUCER_TIMEOUT = "producer.timeout";
     public static final int DEFAULT_PRODUCER_TIMEOUT = 3000; // 3 seconds
 
-    public static final String ACCESS_KEY = "access.key";
-    public static final String SECRET_KEY = "secret.key";
-
-
     // Consumer related config
     public static final String CONSUMER_GROUP = "consumer.group"; // Required
-
     public static final String CONSUMER_TOPIC = "consumer.topic"; // Required
 
     public static final String CONSUMER_TAG = "consumer.tag";
@@ -76,15 +80,19 @@ public class RocketMQConfig {
     public static final String CONSUMER_OFFSET_PERSIST_INTERVAL = "consumer.offset.persist.interval";
     public static final int DEFAULT_CONSUMER_OFFSET_PERSIST_INTERVAL = 5000; // 5 seconds
 
-    public static final String CONSUMER_PULL_POOL_SIZE = "consumer.pull.thread.pool.size";
-    public static final int DEFAULT_CONSUMER_PULL_POOL_SIZE = 20;
-
     public static final String CONSUMER_BATCH_SIZE = "consumer.batch.size";
     public static final int DEFAULT_CONSUMER_BATCH_SIZE = 32;
 
     public static final String CONSUMER_DELAY_WHEN_MESSAGE_NOT_FOUND = "consumer.delay.when.message.not.found";
-    public static final int DEFAULT_CONSUMER_DELAY_WHEN_MESSAGE_NOT_FOUND = 10;
+    public static final int DEFAULT_CONSUMER_DELAY_WHEN_MESSAGE_NOT_FOUND = 100;
+
+    public static final String CONSUMER_INDEX_OF_THIS_SUB_TASK = "consumer.index";
 
+    public static final String UNIT_NAME = "unit.name";
+
+    public static final String WATERMARK = "watermark";
+
+    // Delay message related config
     public static final String MSG_DELAY_LEVEL = "msg.delay.level";
     public static final int MSG_DELAY_LEVEL00 = 0; // no delay
     public static final int MSG_DELAY_LEVEL01 = 1; // 1s
@@ -113,33 +121,28 @@ public class RocketMQConfig {
      */
     public static void buildProducerConfigs(Properties props, DefaultMQProducer producer) {
         buildCommonConfigs(props, producer);
-
         String group = props.getProperty(PRODUCER_GROUP);
         if (StringUtils.isEmpty(group)) {
             group = UUID.randomUUID().toString();
         }
         producer.setProducerGroup(props.getProperty(PRODUCER_GROUP, group));
-
-        producer.setRetryTimesWhenSendFailed(getInteger(props,
-            PRODUCER_RETRY_TIMES, DEFAULT_PRODUCER_RETRY_TIMES));
+        producer.setRetryTimesWhenSendFailed(getInteger(props, PRODUCER_RETRY_TIMES, DEFAULT_PRODUCER_RETRY_TIMES));
         producer.setRetryTimesWhenSendAsyncFailed(getInteger(props,
-            PRODUCER_RETRY_TIMES, DEFAULT_PRODUCER_RETRY_TIMES));
-        producer.setSendMsgTimeout(getInteger(props,
-            PRODUCER_TIMEOUT, DEFAULT_PRODUCER_TIMEOUT));
+                PRODUCER_RETRY_TIMES, DEFAULT_PRODUCER_RETRY_TIMES));
+        producer.setSendMsgTimeout(getInteger(props, PRODUCER_TIMEOUT, DEFAULT_PRODUCER_TIMEOUT));
+
     }
 
     /**
      * Build Consumer Configs.
      * @param props Properties
-     * @param consumer DefaultMQPushConsumer
+     * @param consumer DefaultMQPullConsumer
      */
     public static void buildConsumerConfigs(Properties props, DefaultMQPullConsumer consumer) {
         buildCommonConfigs(props, consumer);
-
         consumer.setMessageModel(MessageModel.CLUSTERING);
-
         consumer.setPersistConsumerOffsetInterval(getInteger(props,
-            CONSUMER_OFFSET_PERSIST_INTERVAL, DEFAULT_CONSUMER_OFFSET_PERSIST_INTERVAL));
+                CONSUMER_OFFSET_PERSIST_INTERVAL, DEFAULT_CONSUMER_OFFSET_PERSIST_INTERVAL));
     }
 
     /**
@@ -151,14 +154,13 @@ public class RocketMQConfig {
         String nameServers = props.getProperty(NAME_SERVER_ADDR);
         Validate.notEmpty(nameServers);
         client.setNamesrvAddr(nameServers);
-
-        client.setPollNameServerInterval(getInteger(props,
-            NAME_SERVER_POLL_INTERVAL, DEFAULT_NAME_SERVER_POLL_INTERVAL));
         client.setHeartbeatBrokerInterval(getInteger(props,
-            BROKER_HEART_BEAT_INTERVAL, DEFAULT_BROKER_HEART_BEAT_INTERVAL));
+                BROKER_HEART_BEAT_INTERVAL, DEFAULT_BROKER_HEART_BEAT_INTERVAL));
+        // When using aliyun products, you need to set up channels
+        client.setAccessChannel((getAccessChannel(props, ACCESS_CHANNEL, DEFAULT_ACCESS_CHANNEL)));
+        client.setUnitName(props.getProperty(UNIT_NAME, null));
     }
 
-
     /**
      * Build credentials for client.
      * @param props
@@ -168,8 +170,7 @@ public class RocketMQConfig {
         String accessKey = props.getProperty(ACCESS_KEY);
         String secretKey = props.getProperty(SECRET_KEY);
         if (!StringUtils.isEmpty(accessKey) && !StringUtils.isEmpty(secretKey)) {
-            AclClientRPCHook aclClientRPCHook = new AclClientRPCHook(new SessionCredentials(accessKey, secretKey));
-            return aclClientRPCHook;
+            return new AclClientRPCHook(new SessionCredentials(accessKey, secretKey));
         }
         return null;
     }
diff --git a/src/main/java/org/apache/rocketmq/flink/RocketMQSink.java b/src/main/java/org/apache/rocketmq/flink/RocketMQSink.java
index 76d6a1f..865af75 100644
--- a/src/main/java/org/apache/rocketmq/flink/RocketMQSink.java
+++ b/src/main/java/org/apache/rocketmq/flink/RocketMQSink.java
@@ -6,9 +6,9 @@
  * 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
- *
+ * <p>
  * http://www.apache.org/licenses/LICENSE-2.0
- *
+ * <p>
  * Unless required by applicable law or agreed to in writing, software
  * distributed under the License is distributed on an "AS IS" BASIS,
  * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
@@ -18,31 +18,31 @@
 
 package org.apache.rocketmq.flink;
 
-import java.nio.charset.StandardCharsets;
-import java.util.LinkedList;
-import java.util.List;
-import java.util.Properties;
-import java.util.UUID;
-
 import org.apache.commons.lang.Validate;
 import org.apache.flink.configuration.Configuration;
+import org.apache.flink.metrics.Meter;
 import org.apache.flink.runtime.state.FunctionInitializationContext;
 import org.apache.flink.runtime.state.FunctionSnapshotContext;
 import org.apache.flink.streaming.api.checkpoint.CheckpointedFunction;
 import org.apache.flink.streaming.api.functions.sink.RichSinkFunction;
 import org.apache.flink.streaming.api.operators.StreamingRuntimeContext;
+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.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.common.selector.TopicSelector;
-import org.apache.rocketmq.flink.common.serialization.KeyValueSerializationSchema;
+import org.apache.rocketmq.flink.common.util.MetricUtils;
 import org.apache.rocketmq.remoting.exception.RemotingException;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 
+import java.util.LinkedList;
+import java.util.List;
+import java.util.Properties;
+import java.util.UUID;
+
 /**
  * The RocketMQSink provides at-least-once reliability guarantees when
  * checkpoints are enabled and batchFlushOnCheckpoint(true) is set.
@@ -58,59 +58,54 @@ public class RocketMQSink<IN> extends RichSinkFunction<IN> implements Checkpoint
     private boolean async; // false by default
 
     private Properties props;
-    private TopicSelector<IN> topicSelector;
-    private KeyValueSerializationSchema<IN> serializationSchema;
 
     private boolean batchFlushOnCheckpoint; // false by default
-    private int batchSize = 1000;
+    private int batchSize = 32;
     private List<Message> batchList;
 
-    private int messageDeliveryDelayLevel = RocketMQConfig.MSG_DELAY_LEVEL00;
+    private Meter sinkInTps;
+    private Meter outTps;
+    private Meter outBps;
+    private MetricUtils.LatencyGauge latencyGauge;
 
-    public RocketMQSink(KeyValueSerializationSchema<IN> schema, TopicSelector<IN> topicSelector, Properties props) {
-        this.serializationSchema = schema;
-        this.topicSelector = topicSelector;
+    public RocketMQSink(Properties props) {
         this.props = props;
-
-        if (this.props != null) {
-            this.messageDeliveryDelayLevel  = RocketMQUtils.getInteger(this.props, RocketMQConfig.MSG_DELAY_LEVEL,
-                    RocketMQConfig.MSG_DELAY_LEVEL00);
-            if (this.messageDeliveryDelayLevel  < RocketMQConfig.MSG_DELAY_LEVEL00) {
-                this.messageDeliveryDelayLevel  = RocketMQConfig.MSG_DELAY_LEVEL00;
-            } else if (this.messageDeliveryDelayLevel  > RocketMQConfig.MSG_DELAY_LEVEL18) {
-                this.messageDeliveryDelayLevel  = RocketMQConfig.MSG_DELAY_LEVEL18;
-            }
-        }
     }
 
     @Override
     public void open(Configuration parameters) throws Exception {
         Validate.notEmpty(props, "Producer properties can not be empty");
-        Validate.notNull(topicSelector, "TopicSelector can not be null");
-        Validate.notNull(serializationSchema, "KeyValueSerializationSchema can not be null");
 
+        // with authentication hook
         producer = new DefaultMQProducer(RocketMQConfig.buildAclRPCHook(props));
-        producer.setInstanceName(String.valueOf(getRuntimeContext().getIndexOfThisSubtask()) + "_" + UUID.randomUUID());
+        producer.setInstanceName(getRuntimeContext().getIndexOfThisSubtask() + "_" + UUID.randomUUID());
+
         RocketMQConfig.buildProducerConfigs(props, producer);
 
         batchList = new LinkedList<>();
 
         if (batchFlushOnCheckpoint && !((StreamingRuntimeContext) getRuntimeContext()).isCheckpointingEnabled()) {
-            LOG.warn("Flushing on checkpoint is enabled, but checkpointing is not enabled. Disabling flushing.");
+            LOG.info("Flushing on checkpoint is enabled, but checkpointing is not enabled. Disabling flushing.");
             batchFlushOnCheckpoint = false;
         }
 
         try {
             producer.start();
         } catch (MQClientException e) {
+            LOG.error("Flink sink init failed, due to the producer cannot be initialized.");
             throw new RuntimeException(e);
         }
+        sinkInTps = MetricUtils.registerSinkInTps(getRuntimeContext());
+        outTps = MetricUtils.registerOutTps(getRuntimeContext());
+        outBps = MetricUtils.registerOutBps(getRuntimeContext());
+        latencyGauge = MetricUtils.registerOutLatency(getRuntimeContext());
     }
 
     @Override
     public void invoke(IN input, Context context) throws Exception {
-        Message msg = prepareMessage(input);
+        sinkInTps.markEvent();
 
+        Message msg = (Message) input;
         if (batchFlushOnCheckpoint) {
             batchList.add(msg);
             if (batchList.size() >= batchSize) {
@@ -119,12 +114,17 @@ public class RocketMQSink<IN> extends RichSinkFunction<IN> implements Checkpoint
             return;
         }
 
+        long timeStartWriting = System.currentTimeMillis();
         if (async) {
             try {
                 producer.send(msg, new SendCallback() {
                     @Override
                     public void onSuccess(SendResult sendResult) {
                         LOG.debug("Async send message success! result: {}", sendResult);
+                        long end = System.currentTimeMillis();
+                        latencyGauge.report(end - timeStartWriting, 1);
+                        outTps.markEvent();
+                        outBps.markEvent(msg.getBody().length);
                     }
 
                     @Override
@@ -144,31 +144,17 @@ public class RocketMQSink<IN> extends RichSinkFunction<IN> implements Checkpoint
                 if (result.getSendStatus() != SendStatus.SEND_OK) {
                     throw new RemotingException(result.toString());
                 }
+                long end = System.currentTimeMillis();
+                latencyGauge.report(end - timeStartWriting, 1);
+                outTps.markEvent();
+                outBps.markEvent(msg.getBody().length);
             } catch (Exception e) {
-                LOG.error("Sync send message failure!", e);
+                LOG.error("Sync send message exception: ", e);
                 throw e;
             }
         }
     }
 
-    private Message prepareMessage(IN input) {
-        String topic = topicSelector.getTopic(input);
-        String tag = (tag = topicSelector.getTag(input)) != null ? tag : "";
-
-        byte[] k = serializationSchema.serializeKey(input);
-        String key = k != null ? new String(k, StandardCharsets.UTF_8) : "";
-        byte[] value = serializationSchema.serializeValue(input);
-
-        Validate.notNull(topic, "the message topic is null");
-        Validate.notNull(value, "the message body is null");
-
-        Message msg = new Message(topic, tag, key, value);
-        if (this.messageDeliveryDelayLevel > RocketMQConfig.MSG_DELAY_LEVEL00) {
-            msg.setDelayTimeLevel(this.messageDeliveryDelayLevel);
-        }
-        return msg;
-    }
-
     public RocketMQSink<IN> withAsync(boolean async) {
         this.async = async;
         return this;
@@ -185,7 +171,7 @@ public class RocketMQSink<IN> extends RichSinkFunction<IN> implements Checkpoint
     }
 
     @Override
-    public void close() throws Exception {
+    public void close() {
         if (producer != null) {
             try {
                 flushSync();
diff --git a/src/main/java/org/apache/rocketmq/flink/RocketMQSource.java b/src/main/java/org/apache/rocketmq/flink/RocketMQSource.java
index b3b37dc..35c5122 100644
--- a/src/main/java/org/apache/rocketmq/flink/RocketMQSource.java
+++ b/src/main/java/org/apache/rocketmq/flink/RocketMQSource.java
@@ -3,9 +3,9 @@
  * 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
- *
+ * <p>
  * http://www.apache.org/licenses/LICENSE-2.0
- *
+ * <p>
  * Unless required by applicable law or agreed to in writing, software distributed under the License is distributed on
  * an "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. See the License for the
  * specific language governing permissions and limitations under the License.
@@ -13,16 +13,9 @@
 
 package org.apache.rocketmq.flink;
 
-import java.nio.charset.StandardCharsets;
-import java.util.HashMap;
-import java.util.List;
-import java.util.Map;
-import java.util.Properties;
-import java.util.Set;
-import java.util.UUID;
-import java.util.concurrent.ConcurrentHashMap;
 import org.apache.commons.collections.map.LinkedMap;
 import org.apache.commons.lang.Validate;
+import org.apache.flink.api.common.functions.RuntimeContext;
 import org.apache.flink.api.common.state.ListState;
 import org.apache.flink.api.common.state.ListStateDescriptor;
 import org.apache.flink.api.common.typeinfo.TypeHint;
@@ -30,62 +23,78 @@ import org.apache.flink.api.common.typeinfo.TypeInformation;
 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.metrics.Counter;
+import org.apache.flink.metrics.Meter;
+import org.apache.flink.metrics.MeterView;
+import org.apache.flink.metrics.SimpleCounter;
 import org.apache.flink.runtime.state.CheckpointListener;
 import org.apache.flink.runtime.state.FunctionInitializationContext;
 import org.apache.flink.runtime.state.FunctionSnapshotContext;
+import org.apache.flink.shaded.curator.org.apache.curator.shaded.com.google.common.util.concurrent.ThreadFactoryBuilder;
 import org.apache.flink.streaming.api.checkpoint.CheckpointedFunction;
 import org.apache.flink.streaming.api.functions.source.RichParallelSourceFunction;
 import org.apache.flink.streaming.api.operators.StreamingRuntimeContext;
 import org.apache.rocketmq.client.consumer.DefaultMQPullConsumer;
-import org.apache.rocketmq.client.consumer.MQPullConsumerScheduleService;
 import org.apache.rocketmq.client.consumer.PullResult;
-import org.apache.rocketmq.client.consumer.PullTaskCallback;
-import org.apache.rocketmq.client.consumer.PullTaskContext;
 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.common.serialization.KeyValueDeserializationSchema;
+import org.apache.rocketmq.flink.common.util.MetricUtils;
+import org.apache.rocketmq.flink.common.util.RetryUtil;
+import org.apache.rocketmq.flink.common.util.RocketMQUtils;
+import org.apache.rocketmq.flink.common.watermark.WaterMarkForAll;
+import org.apache.rocketmq.flink.common.watermark.WaterMarkPerQueue;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 
-import static org.apache.rocketmq.flink.RocketMQConfig.CONSUMER_OFFSET_EARLIEST;
-import static org.apache.rocketmq.flink.RocketMQConfig.CONSUMER_OFFSET_LATEST;
-import static org.apache.rocketmq.flink.RocketMQConfig.CONSUMER_OFFSET_TIMESTAMP;
-import static org.apache.rocketmq.flink.RocketMQUtils.getInteger;
-import static org.apache.rocketmq.flink.RocketMQUtils.getLong;
+import java.lang.management.ManagementFactory;
+import java.nio.charset.StandardCharsets;
+import java.util.*;
+import java.util.concurrent.*;
+import java.util.concurrent.locks.ReentrantLock;
+
+import static org.apache.rocketmq.flink.RocketMQConfig.*;
+import static org.apache.rocketmq.flink.common.util.RocketMQUtils.getInteger;
+import static org.apache.rocketmq.flink.common.util.RocketMQUtils.getLong;
 
 /**
  * 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 guarantees.
  */
 public class RocketMQSource<OUT> extends RichParallelSourceFunction<OUT>
-    implements CheckpointedFunction, CheckpointListener, ResultTypeQueryable<OUT> {
+        implements CheckpointedFunction, CheckpointListener, ResultTypeQueryable<OUT> {
 
     private static final long serialVersionUID = 1L;
 
-    private static final Logger LOG = LoggerFactory.getLogger(RocketMQSource.class);
-
-    private transient MQPullConsumerScheduleService pullConsumerScheduleService;
-    private DefaultMQPullConsumer consumer;
-
-    private KeyValueDeserializationSchema<OUT> schema;
-
+    private static final Logger log = LoggerFactory.getLogger(RocketMQSource.class);
+    private static final String OFFSETS_STATE_NAME = "topic-partition-offset-states";
     private RunningChecker runningChecker;
-
+    private transient DefaultMQPullConsumer consumer;
+    private KeyValueDeserializationSchema<OUT> schema;
     private transient ListState<Tuple2<MessageQueue, Long>> unionOffsetStates;
     private Map<MessageQueue, Long> offsetTable;
     private Map<MessageQueue, Long> restoredOffsets;
-    /** Data for pending but uncommitted offsets. */
-    private LinkedMap pendingOffsetsToCommit;
+    private List<MessageQueue> messageQueues;
+    private ExecutorService executor;
+
+    // watermark in source
+    private WaterMarkPerQueue waterMarkPerQueue;
+    private WaterMarkForAll waterMarkForAll;
 
+    private ScheduledExecutorService timer;
+    /**
+     * Data for pending but uncommitted offsets.
+     */
+    private LinkedMap pendingOffsetsToCommit;
     private Properties props;
     private String topic;
     private String group;
-
-    private static final String OFFSETS_STATE_NAME = "topic-partition-offset-states";
-
     private transient volatile boolean restored;
     private transient boolean enableCheckpoint;
+    private volatile Object checkPointLock;
+
+    private Meter tpsMetric;
 
     public RocketMQSource(KeyValueDeserializationSchema<OUT> schema, Properties props) {
         this.schema = schema;
@@ -94,9 +103,8 @@ public class RocketMQSource<OUT> extends RichParallelSourceFunction<OUT>
 
     @Override
     public void open(Configuration parameters) throws Exception {
-        LOG.debug("source open....");
+        log.debug("source open....");
         Validate.notEmpty(props, "Consumer properties can not be empty");
-        Validate.notNull(schema, "KeyValueDeserializationSchema can not be null");
 
         this.topic = props.getProperty(RocketMQConfig.CONSUMER_TOPIC);
         this.group = props.getProperty(RocketMQConfig.CONSUMER_GROUP);
@@ -115,100 +123,123 @@ public class RocketMQSource<OUT> extends RichParallelSourceFunction<OUT>
         if (pendingOffsetsToCommit == null) {
             pendingOffsetsToCommit = new LinkedMap();
         }
+        if (checkPointLock == null) {
+            checkPointLock = new ReentrantLock();
+        }
+        if (waterMarkPerQueue == null) {
+            waterMarkPerQueue = new WaterMarkPerQueue(5000);
+        }
+        if (waterMarkForAll == null) {
+            waterMarkForAll = new WaterMarkForAll(5000);
+        }
+        if (timer == null) {
+            timer = Executors.newSingleThreadScheduledExecutor();
+        }
 
         runningChecker = new RunningChecker();
+        runningChecker.setRunning(true);
 
-        //Wait for lite pull consumer
-        pullConsumerScheduleService = new MQPullConsumerScheduleService(group, RocketMQConfig.buildAclRPCHook(props));
-        consumer = pullConsumerScheduleService.getDefaultMQPullConsumer();
+        final ThreadFactory threadFactory = new ThreadFactoryBuilder()
+                .setDaemon(true).setNameFormat("rmq-pull-thread-%d").build();
+        executor = Executors.newCachedThreadPool(threadFactory);
 
-        consumer.setInstanceName(String.valueOf(getRuntimeContext().getIndexOfThisSubtask()) + "_" + UUID.randomUUID());
+        int indexOfThisSubTask = getRuntimeContext().getIndexOfThisSubtask();
+        consumer = new DefaultMQPullConsumer(group, RocketMQConfig.buildAclRPCHook(props));
         RocketMQConfig.buildConsumerConfigs(props, consumer);
+
+        // set unique instance name, avoid exception: https://help.aliyun.com/document_detail/29646.html
+        String runtimeName = ManagementFactory.getRuntimeMXBean().getName();
+        String instanceName = RocketMQUtils.getInstanceName(runtimeName, topic, group,
+                String.valueOf(indexOfThisSubTask), String.valueOf(System.nanoTime()));
+        consumer.setInstanceName(instanceName);
+        consumer.start();
+
+        Counter outputCounter = getRuntimeContext().getMetricGroup()
+                .counter(MetricUtils.METRICS_TPS + "_counter", new SimpleCounter());
+        tpsMetric = getRuntimeContext().getMetricGroup()
+                .meter(MetricUtils.METRICS_TPS, new MeterView(outputCounter, 60));
     }
 
     @Override
     public void run(SourceContext context) throws Exception {
-        LOG.debug("source run....");
-        // The lock that guarantees that record emission and state updates are atomic,
-        // from the view of taking a checkpoint.
-        final Object lock = context.getCheckpointLock();
-
-        int delayWhenMessageNotFound = getInteger(props, RocketMQConfig.CONSUMER_DELAY_WHEN_MESSAGE_NOT_FOUND,
-            RocketMQConfig.DEFAULT_CONSUMER_DELAY_WHEN_MESSAGE_NOT_FOUND);
-
         String tag = props.getProperty(RocketMQConfig.CONSUMER_TAG, RocketMQConfig.DEFAULT_CONSUMER_TAG);
+        int pullBatchSize = getInteger(props, CONSUMER_BATCH_SIZE, DEFAULT_CONSUMER_BATCH_SIZE);
 
-        int pullPoolSize = getInteger(props, RocketMQConfig.CONSUMER_PULL_POOL_SIZE,
-            RocketMQConfig.DEFAULT_CONSUMER_PULL_POOL_SIZE);
-
-        int pullBatchSize = getInteger(props, RocketMQConfig.CONSUMER_BATCH_SIZE,
-            RocketMQConfig.DEFAULT_CONSUMER_BATCH_SIZE);
-
-        pullConsumerScheduleService.setPullThreadNums(pullPoolSize);
-        pullConsumerScheduleService.registerPullTaskCallback(topic, new PullTaskCallback() {
-
-            @Override
-            public void doPullTask(MessageQueue mq, PullTaskContext pullTaskContext) {
-                try {
-                    long offset = getMessageQueueOffset(mq);
-                    if (offset < 0) {
-                        return;
-                    }
-
-                    PullResult pullResult = consumer.pull(mq, tag, offset, pullBatchSize);
-                    boolean found = false;
-                    switch (pullResult.getPullStatus()) {
-                        case FOUND:
-                            List<MessageExt> messages = pullResult.getMsgFoundList();
-                            for (MessageExt msg : messages) {
-                                byte[] key = msg.getKeys() != null ? msg.getKeys().getBytes(StandardCharsets.UTF_8) : null;
-                                byte[] value = msg.getBody();
-                                OUT data = schema.deserializeKeyAndValue(key, value);
-
-                                // output and state update are atomic
-                                synchronized (lock) {
-                                    context.collectWithTimestamp(data, msg.getBornTimestamp());
-                                }
+        final RuntimeContext ctx = getRuntimeContext();
+        // The lock that guarantees that record emission and state updates are atomic,
+        // from the view of taking a checkpoint.
+        int taskNumber = ctx.getNumberOfParallelSubtasks();
+        int taskIndex = ctx.getIndexOfThisSubtask();
+        log.info("Source run, NumberOfTotalTask={}, IndexOfThisSubTask={}", taskNumber, taskIndex);
+
+
+        timer.scheduleAtFixedRate(() -> {
+            // context.emitWatermark(waterMarkPerQueue.getCurrentWatermark());
+            context.emitWatermark(waterMarkForAll.getCurrentWatermark());
+        }, 5, 5, TimeUnit.SECONDS);
+
+        Collection<MessageQueue> totalQueues = consumer.fetchSubscribeMessageQueues(topic);
+        messageQueues = RocketMQUtils.allocate(totalQueues, taskNumber, ctx.getIndexOfThisSubtask());
+        for (MessageQueue mq : messageQueues) {
+            this.executor.execute(() -> {
+                RetryUtil.call(() -> {
+                    while (runningChecker.isRunning()) {
+                        try {
+                            long offset = getMessageQueueOffset(mq);
+                            PullResult pullResult = consumer.pullBlockIfNotFound(mq, tag, offset, pullBatchSize);
+
+                            boolean found = false;
+                            switch (pullResult.getPullStatus()) {
+                                case FOUND:
+                                    List<MessageExt> messages = pullResult.getMsgFoundList();
+                                    for (MessageExt msg : messages) {
+                                        byte[] key = msg.getKeys() != null ? msg.getKeys().getBytes(StandardCharsets.UTF_8) : null;
+                                        byte[] value = msg.getBody();
+                                        OUT data = schema.deserializeKeyAndValue(key, value);
+
+                                        // output and state update are atomic
+                                        synchronized (checkPointLock) {
+                                            log.debug(msg.getMsgId() + "_" + msg.getBrokerName() + " " + msg.getQueueId() + " " + msg.getQueueOffset());
+                                            context.collectWithTimestamp(data, msg.getBornTimestamp());
+
+                                            // update max eventTime per queue
+                                            // waterMarkPerQueue.extractTimestamp(mq, msg.getBornTimestamp());
+                                            waterMarkForAll.extractTimestamp(msg.getBornTimestamp());
+                                            tpsMetric.markEvent();
+                                        }
+                                    }
+                                    found = true;
+                                    break;
+                                case NO_MATCHED_MSG:
+                                    log.debug("No matched message after offset {} for queue {}", offset, mq);
+                                    break;
+                                case NO_NEW_MSG:
+                                    log.debug("No new message after offset {} for queue {}", offset, mq);
+                                    break;
+                                case OFFSET_ILLEGAL:
+                                    log.warn("Offset {} is illegal for queue {}", offset, mq);
+                                    break;
+                                default:
+                                    break;
                             }
-                            found = true;
-                            break;
-                        case NO_MATCHED_MSG:
-                            LOG.debug("No matched message after offset {} for queue {}", offset, mq);
-                            break;
-                        case NO_NEW_MSG:
-                            break;
-                        case OFFSET_ILLEGAL:
-                            LOG.warn("Offset {} is illegal for queue {}", offset, mq);
-                            break;
-                        default:
-                            break;
-                    }
 
-                    synchronized (lock) {
-                        putMessageQueueOffset(mq, pullResult.getNextBeginOffset());
-                    }
+                            synchronized (checkPointLock) {
+                                updateMessageQueueOffset(mq, pullResult.getNextBeginOffset());
+                            }
 
-                    if (found) {
-                        pullTaskContext.setPullNextDelayTimeMillis(0); // no delay when messages were found
-                    } else {
-                        pullTaskContext.setPullNextDelayTimeMillis(delayWhenMessageNotFound);
+                            if (!found) {
+                                RetryUtil.waitForMs(RocketMQConfig.DEFAULT_CONSUMER_DELAY_WHEN_MESSAGE_NOT_FOUND);
+                            }
+                        } catch (Exception e) {
+                            throw new RuntimeException(e);
+                        }
                     }
-                } catch (Exception e) {
-                    throw new RuntimeException(e);
-                }
-            }
-        });
-
-        try {
-            pullConsumerScheduleService.start();
-        } catch (MQClientException e) {
-            throw new RuntimeException(e);
+                    return true;
+                }, "RuntimeException");
+            });
         }
 
-        runningChecker.setRunning(true);
-
         awaitTermination();
-
     }
 
     private void awaitTermination() throws InterruptedException {
@@ -225,6 +256,7 @@ public class RocketMQSource<OUT> extends RichParallelSourceFunction<OUT>
             offset = restoredOffsets.get(mq);
         }
         if (offset == null) {
+            // fetchConsumeOffset from broker
             offset = consumer.fetchConsumeOffset(mq, false);
             if (offset < 0) {
                 String initialOffset = props.getProperty(RocketMQConfig.CONSUMER_OFFSET_RESET_TO, CONSUMER_OFFSET_LATEST);
@@ -237,7 +269,7 @@ public class RocketMQSource<OUT> extends RichParallelSourceFunction<OUT>
                         break;
                     case CONSUMER_OFFSET_TIMESTAMP:
                         offset = consumer.searchOffset(mq, getLong(props,
-                            RocketMQConfig.CONSUMER_OFFSET_FROM_TIMESTAMP, System.currentTimeMillis()));
+                                RocketMQConfig.CONSUMER_OFFSET_FROM_TIMESTAMP, System.currentTimeMillis()));
                         break;
                     default:
                         throw new IllegalArgumentException("Unknown value for CONSUMER_OFFSET_RESET_TO.");
@@ -248,7 +280,7 @@ public class RocketMQSource<OUT> extends RichParallelSourceFunction<OUT>
         return offsetTable.get(mq);
     }
 
-    private void putMessageQueueOffset(MessageQueue mq, long offset) throws MQClientException {
+    private void updateMessageQueueOffset(MessageQueue mq, long offset) throws MQClientException {
         offsetTable.put(mq, offset);
         if (!enableCheckpoint) {
             consumer.updateConsumeOffset(mq, offset);
@@ -257,12 +289,13 @@ public class RocketMQSource<OUT> extends RichParallelSourceFunction<OUT>
 
     @Override
     public void cancel() {
-        LOG.debug("cancel ...");
+        log.debug("cancel ...");
         runningChecker.setRunning(false);
 
-        if (pullConsumerScheduleService != null) {
-            pullConsumerScheduleService.shutdown();
+        if (consumer != null) {
+            consumer.shutdown();
         }
+
         if (offsetTable != null) {
             offsetTable.clear();
         }
@@ -276,7 +309,7 @@ public class RocketMQSource<OUT> extends RichParallelSourceFunction<OUT>
 
     @Override
     public void close() throws Exception {
-        LOG.debug("close ...");
+        log.debug("close ...");
         // pretty much the same logic as cancelling
         try {
             cancel();
@@ -288,50 +321,51 @@ public class RocketMQSource<OUT> extends RichParallelSourceFunction<OUT>
     @Override
     public void snapshotState(FunctionSnapshotContext context) throws Exception {
         // called when a snapshot for a checkpoint is requested
-
+        log.info("Snapshotting state {} ...", context.getCheckpointId());
         if (!runningChecker.isRunning()) {
-            LOG.debug("snapshotState() called on closed source; returning null.");
+            log.info("snapshotState() called on closed source; returning null.");
             return;
         }
 
-        if (LOG.isDebugEnabled()) {
-            LOG.debug("Snapshotting state {} ...", context.getCheckpointId());
-        }
+        // Discovery topic Route change when snapshot
+        RetryUtil.call(() -> {
+            Collection<MessageQueue> totalQueues = consumer.fetchSubscribeMessageQueues(topic);
+            int taskNumber = getRuntimeContext().getNumberOfParallelSubtasks();
+            int taskIndex = getRuntimeContext().getIndexOfThisSubtask();
+            List<MessageQueue> newQueues = RocketMQUtils.allocate(totalQueues, taskNumber, taskIndex);
+            Collections.sort(newQueues);
+            log.debug(taskIndex + " Topic route is same.");
+            if (!messageQueues.equals(newQueues)) {
+                throw new RuntimeException();
+            }
+            return true;
+        }, "RuntimeException due to topic route changed");
 
         unionOffsetStates.clear();
-
         HashMap<MessageQueue, Long> currentOffsets = new HashMap<>(offsetTable.size());
-
-        // remove the unassigned queues in order to avoid read the wrong offset when the source restart
-        Set<MessageQueue> assignedQueues = consumer.fetchMessageQueuesInBalance(topic);
-        offsetTable.entrySet().removeIf(item -> !assignedQueues.contains(item.getKey()));
-
         for (Map.Entry<MessageQueue, Long> entry : offsetTable.entrySet()) {
             unionOffsetStates.add(Tuple2.of(entry.getKey(), entry.getValue()));
             currentOffsets.put(entry.getKey(), entry.getValue());
         }
-
         pendingOffsetsToCommit.put(context.getCheckpointId(), currentOffsets);
-
-        if (LOG.isDebugEnabled()) {
-            LOG.debug("Snapshotted state, last processed offsets: {}, checkpoint id: {}, timestamp: {}",
+        log.info("Snapshotted state, last processed offsets: {}, checkpoint id: {}, timestamp: {}",
                 offsetTable, context.getCheckpointId(), context.getCheckpointTimestamp());
-        }
     }
 
+    /**
+     * called every time the user-defined function is initialized,
+     * be that when the function is first initialized or be that
+     * when the function is actually recovering from an earlier checkpoint.
+     * Given this, initializeState() is not only the place where different types of state are initialized,
+     * but also where state recovery logic is included.
+     */
     @Override
     public void initializeState(FunctionInitializationContext context) throws Exception {
-        // called every time the user-defined function is initialized,
-        // be that when the function is first initialized or be that
-        // when the function is actually recovering from an earlier checkpoint.
-        // Given this, initializeState() is not only the place where different types of state are initialized,
-        // but also where state recovery logic is included.
-        LOG.debug("initialize State ...");
+        log.info("initialize State ...");
 
         this.unionOffsetStates = context.getOperatorStateStore().getUnionListState(new ListStateDescriptor<>(
                 OFFSETS_STATE_NAME, TypeInformation.of(new TypeHint<Tuple2<MessageQueue, Long>>() {
-
-                })));
+        })));
         this.restored = context.isRestored();
 
         if (restored) {
@@ -343,14 +377,14 @@ public class RocketMQSource<OUT> extends RichParallelSourceFunction<OUT>
                     restoredOffsets.put(mqOffsets.f0, mqOffsets.f1);
                 }
             }
-            LOG.info("Setting restore state in the consumer. Using the following offsets: {}", restoredOffsets);
+            log.info("Setting restore state in the consumer. Using the following offsets: {}", restoredOffsets);
         } else {
-            LOG.info("No restore state for the consumer.");
+            log.info("No restore state for the consumer.");
         }
     }
 
     @Override
-    public TypeInformation<OUT> getProducedType() {
+    public TypeInformation getProducedType() {
         return schema.getProducedType();
     }
 
@@ -358,13 +392,13 @@ public class RocketMQSource<OUT> extends RichParallelSourceFunction<OUT>
     public void notifyCheckpointComplete(long checkpointId) throws Exception {
         // callback when checkpoint complete
         if (!runningChecker.isRunning()) {
-            LOG.debug("notifyCheckpointComplete() called on closed source; returning null.");
+            log.info("notifyCheckpointComplete() called on closed source; returning null.");
             return;
         }
 
         final int posInMap = pendingOffsetsToCommit.indexOf(checkpointId);
         if (posInMap == -1) {
-            LOG.warn("Received confirmation for unknown checkpoint id {}", checkpointId);
+            log.warn("Received confirmation for unknown checkpoint id {}", checkpointId);
             return;
         }
 
@@ -376,13 +410,12 @@ public class RocketMQSource<OUT> extends RichParallelSourceFunction<OUT>
         }
 
         if (offsets == null || offsets.size() == 0) {
-            LOG.debug("Checkpoint state was empty.");
+            log.debug("Checkpoint state was empty.");
             return;
         }
 
         for (Map.Entry<MessageQueue, Long> entry : offsets.entrySet()) {
             consumer.updateConsumeOffset(entry.getKey(), entry.getValue());
         }
-
     }
 }
diff --git a/src/main/java/org/apache/rocketmq/flink/RocketMQUtils.java b/src/main/java/org/apache/rocketmq/flink/RocketMQUtils.java
deleted file mode 100644
index 9ca1de2..0000000
--- a/src/main/java/org/apache/rocketmq/flink/RocketMQUtils.java
+++ /dev/null
@@ -1,36 +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;
-
-import java.util.Properties;
-
-public final class RocketMQUtils {
-
-    public static int getInteger(Properties props, String key, int defaultValue) {
-        return Integer.parseInt(props.getProperty(key, String.valueOf(defaultValue)));
-    }
-
-    public static long getLong(Properties props, String key, long defaultValue) {
-        return Long.parseLong(props.getProperty(key, String.valueOf(defaultValue)));
-    }
-
-    public static boolean getBoolean(Properties props, String key, boolean defaultValue) {
-        return Boolean.parseBoolean(props.getProperty(key, String.valueOf(defaultValue)));
-    }
-}
diff --git a/src/main/java/org/apache/rocketmq/flink/common/serialization/ForwardMessageExtDeserialization.java b/src/main/java/org/apache/rocketmq/flink/common/serialization/ForwardMessageExtDeserialization.java
new file mode 100644
index 0000000..20dd700
--- /dev/null
+++ b/src/main/java/org/apache/rocketmq/flink/common/serialization/ForwardMessageExtDeserialization.java
@@ -0,0 +1,37 @@
+/*
+ * 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.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> {
+
+    @Override
+    public MessageExt deserializeMessageExt(MessageExt messageExt) {
+        return messageExt;
+    }
+
+    @Override
+    public TypeInformation<MessageExt> getProducedType() {
+        return TypeInformation.of(MessageExt.class);
+    }
+}
\ No newline at end of file
diff --git a/src/main/java/org/apache/rocketmq/flink/common/serialization/MessageExtDeserializationScheme.java b/src/main/java/org/apache/rocketmq/flink/common/serialization/MessageExtDeserializationScheme.java
new file mode 100644
index 0000000..4c8cf85
--- /dev/null
+++ b/src/main/java/org/apache/rocketmq/flink/common/serialization/MessageExtDeserializationScheme.java
@@ -0,0 +1,37 @@
+/*
+ * 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.serialization;
+
+import java.io.Serializable;
+import org.apache.flink.api.java.typeutils.ResultTypeQueryable;
+import org.apache.rocketmq.common.message.MessageExt;
+
+/**
+ * The interface Message ext deserialization scheme.
+ *
+ * @param <T> the type parameter
+ */
+public interface MessageExtDeserializationScheme<T> extends ResultTypeQueryable<T>, Serializable {
+    /**
+     * Deserialize messageExt to type T you want to output.
+     *
+     * @param messageExt the messageExt
+     * @return the t
+     */
+    T deserializeMessageExt(MessageExt messageExt);
+}
diff --git a/src/main/java/org/apache/rocketmq/flink/common/serialization/SimpleKeyValueDeserializationSchema.java b/src/main/java/org/apache/rocketmq/flink/common/serialization/SimpleKeyValueDeserializationSchema.java
index df6390b..93d5d9b 100644
--- a/src/main/java/org/apache/rocketmq/flink/common/serialization/SimpleKeyValueDeserializationSchema.java
+++ b/src/main/java/org/apache/rocketmq/flink/common/serialization/SimpleKeyValueDeserializationSchema.java
@@ -22,7 +22,9 @@ import java.nio.charset.StandardCharsets;
 import java.util.HashMap;
 import java.util.Map;
 
+import org.apache.flink.api.common.typeinfo.TypeHint;
 import org.apache.flink.api.common.typeinfo.TypeInformation;
+import org.apache.flink.api.java.tuple.Tuple2;
 
 public class SimpleKeyValueDeserializationSchema implements KeyValueDeserializationSchema<Map> {
     public static final String DEFAULT_KEY_FIELD = "key";
@@ -63,4 +65,4 @@ public class SimpleKeyValueDeserializationSchema implements KeyValueDeserializat
     public TypeInformation<Map> getProducedType() {
         return TypeInformation.of(Map.class);
     }
-}
+}
\ No newline at end of file
diff --git a/src/main/java/org/apache/rocketmq/flink/common/serialization/SimpleTupleDeserializationSchema.java b/src/main/java/org/apache/rocketmq/flink/common/serialization/SimpleTupleDeserializationSchema.java
new file mode 100644
index 0000000..54106ef
--- /dev/null
+++ b/src/main/java/org/apache/rocketmq/flink/common/serialization/SimpleTupleDeserializationSchema.java
@@ -0,0 +1,22 @@
+package org.apache.rocketmq.flink.common.serialization;
+
+import org.apache.flink.api.common.typeinfo.TypeHint;
+import org.apache.flink.api.common.typeinfo.TypeInformation;
+import org.apache.flink.api.java.tuple.Tuple2;
+
+import java.nio.charset.StandardCharsets;
+
+public class SimpleTupleDeserializationSchema implements KeyValueDeserializationSchema<Tuple2<String, String>> {
+
+    @Override
+    public Tuple2<String, String> deserializeKeyAndValue(byte[] key, byte[] value) {
+        String keyString = key != null ? new String(key, StandardCharsets.UTF_8) : null;
+        String valueString = value != null ? new String(value, StandardCharsets.UTF_8) : null;
+        return new Tuple2<>(keyString, valueString);
+    }
+
+    @Override
+    public TypeInformation<Tuple2<String, String>> getProducedType() {
+        return TypeInformation.of(new TypeHint<Tuple2<String,String>>(){});
+    }
+}
diff --git a/src/main/java/org/apache/rocketmq/flink/common/util/MetricUtils.java b/src/main/java/org/apache/rocketmq/flink/common/util/MetricUtils.java
new file mode 100644
index 0000000..764d01f
--- /dev/null
+++ b/src/main/java/org/apache/rocketmq/flink/common/util/MetricUtils.java
@@ -0,0 +1,80 @@
+/**
+ * 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.util;
+
+import org.apache.flink.api.common.functions.RuntimeContext;
+import org.apache.flink.metrics.Counter;
+import org.apache.flink.metrics.Gauge;
+import org.apache.flink.metrics.Meter;
+import org.apache.flink.metrics.MeterView;
+import org.apache.flink.metrics.SimpleCounter;
+
+/**
+ * RocketMQ connector metrics.
+ */
+public class MetricUtils {
+
+    public static final String METRICS_TPS = "tps";
+
+    private static final String METRIC_GROUP_SINK = "sink";
+    private static final String METRICS_SINK_IN_TPS = "inTps";
+    private static final String METRICS_SINK_OUT_TPS = "outTps";
+    private static final String METRICS_SINK_OUT_BPS = "outBps";
+    private static final String METRICS_SINK_OUT_Latency = "outLatency";
+
+    public static Meter registerSinkInTps(RuntimeContext context) {
+        Counter parserCounter = context.getMetricGroup().addGroup(METRIC_GROUP_SINK)
+            .counter(METRICS_SINK_IN_TPS + "_counter", new SimpleCounter());
+        return context.getMetricGroup().addGroup(METRIC_GROUP_SINK)
+            .meter(METRICS_SINK_IN_TPS, new MeterView(parserCounter, 60));
+    }
+
+    public static Meter registerOutTps(RuntimeContext context) {
+        Counter parserCounter = context.getMetricGroup().addGroup(METRIC_GROUP_SINK)
+            .counter(METRICS_SINK_OUT_TPS + "_counter", new SimpleCounter());
+        return context.getMetricGroup().addGroup(METRIC_GROUP_SINK)
+                .meter(METRICS_SINK_OUT_TPS, new MeterView(parserCounter, 60));
+    }
+
+    public static Meter registerOutBps(RuntimeContext context) {
+        Counter bpsCounter = context.getMetricGroup().addGroup(METRIC_GROUP_SINK)
+                .counter(METRICS_SINK_OUT_BPS + "_counter", new SimpleCounter());
+        return context.getMetricGroup().addGroup(METRIC_GROUP_SINK)
+                .meter(METRICS_SINK_OUT_BPS, new MeterView(bpsCounter, 60));
+    }
+
+    public static LatencyGauge registerOutLatency(RuntimeContext context) {
+        return context.getMetricGroup().addGroup(METRIC_GROUP_SINK).gauge(METRICS_SINK_OUT_Latency, new LatencyGauge());
+    }
+
+    public static class LatencyGauge implements Gauge<Double> {
+        private double value;
+
+        public void report(long timeDelta, long batchSize) {
+            if (batchSize != 0) {
+                this.value = (1.0 * timeDelta) / batchSize;
+            }
+        }
+
+        @Override
+        public Double getValue() {
+            return value;
+        }
+    }
+}
diff --git a/src/main/java/org/apache/rocketmq/flink/common/util/RetryUtil.java b/src/main/java/org/apache/rocketmq/flink/common/util/RetryUtil.java
new file mode 100644
index 0000000..0dbd553
--- /dev/null
+++ b/src/main/java/org/apache/rocketmq/flink/common/util/RetryUtil.java
@@ -0,0 +1,61 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.rocketmq.flink.common.util;
+
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+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;
+
+    private RetryUtil() {
+    }
+
+    public static void waitForMs(long sleepMs) {
+        try {
+            Thread.sleep(sleepMs);
+        } catch (InterruptedException e) {
+            Thread.currentThread().interrupt();
+        }
+    }
+
+    public static <T> T call(Callable<T> callable, String errorMsg) throws RuntimeException {
+        long backoff = INITIAL_BACKOFF;
+        int retries = 0;
+        do {
+            try {
+                return callable.call();
+            } catch (Exception ex) {
+                if (retries >= MAX_ATTEMPTS) {
+                    throw new RuntimeException(ex);
+                }
+                log.error("{}, retry {}/{}", errorMsg, retries, MAX_ATTEMPTS, ex);
+                retries++;
+            }
+            waitForMs(backoff);
+            backoff = Math.min(backoff * 2, MAX_BACKOFF);
+        } while (true);
+    }
+}
diff --git a/src/main/java/org/apache/rocketmq/flink/common/util/RocketMQUtils.java b/src/main/java/org/apache/rocketmq/flink/common/util/RocketMQUtils.java
new file mode 100644
index 0000000..fc37b04
--- /dev/null
+++ b/src/main/java/org/apache/rocketmq/flink/common/util/RocketMQUtils.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.rocketmq.flink.common.util;
+
+import org.apache.rocketmq.client.AccessChannel;
+import org.apache.rocketmq.common.message.MessageQueue;
+
+import java.lang.management.ManagementFactory;
+import java.util.*;
+
+public final class RocketMQUtils {
+
+    public static int getInteger(Properties props, String key, int defaultValue) {
+        return Integer.parseInt(props.getProperty(key, String.valueOf(defaultValue)));
+    }
+
+    public static long getLong(Properties props, String key, long defaultValue) {
+        return Long.parseLong(props.getProperty(key, String.valueOf(defaultValue)));
+    }
+
+    public static boolean getBoolean(Properties props, String key, boolean defaultValue) {
+        return Boolean.parseBoolean(props.getProperty(key, String.valueOf(defaultValue)));
+    }
+
+    public static AccessChannel getAccessChannel(Properties props, String key, AccessChannel defaultValue) {
+        return AccessChannel.valueOf(props.getProperty(key, String.valueOf(defaultValue)));
+    }
+
+    public static String getInstanceName(String... args) {
+        if (null != args && args.length > 0) {
+            return String.join("_", args);
+        }
+        return ManagementFactory.getRuntimeMXBean().getName() + "_" + System.nanoTime();
+    }
+
+    /**
+     * Average Hashing queue algorithm
+     * Refer: org.apache.rocketmq.client.consumer.rebalance.AllocateMessageQueueAveragely
+     */
+    public static List<MessageQueue> allocate(Collection<MessageQueue> mqSet,
+                                              int numberOfParallelTasks,
+                                              int indexOfThisTask) {
+        ArrayList<MessageQueue> mqAll = new ArrayList<>(mqSet);
+        Collections.sort(mqAll);
+        List<MessageQueue> result = new ArrayList<>();
+        int mod = mqAll.size() % numberOfParallelTasks;
+        int averageSize = mqAll.size() <= numberOfParallelTasks ? 1 : (mod > 0 && indexOfThisTask < mod ?
+                mqAll.size() / numberOfParallelTasks + 1 : mqAll.size() / numberOfParallelTasks);
+        int startIndex = (mod > 0 && indexOfThisTask < mod) ? indexOfThisTask * averageSize :
+                indexOfThisTask * averageSize + mod;
+        int range = Math.min(averageSize, mqAll.size() - startIndex);
+        for (int i = 0; i < range; i++) {
+            result.add(mqAll.get((startIndex + i) % mqAll.size()));
+        }
+        return result;
+    }
+}
diff --git a/src/test/java/org/apache/rocketmq/flink/TestUtils.java b/src/main/java/org/apache/rocketmq/flink/common/util/TestUtils.java
similarity index 96%
rename from src/test/java/org/apache/rocketmq/flink/TestUtils.java
rename to src/main/java/org/apache/rocketmq/flink/common/util/TestUtils.java
index d0a9450..71d1265 100644
--- a/src/test/java/org/apache/rocketmq/flink/TestUtils.java
+++ b/src/main/java/org/apache/rocketmq/flink/common/util/TestUtils.java
@@ -16,7 +16,7 @@
  * limitations under the License.
  */
 
-package org.apache.rocketmq.flink;
+package org.apache.rocketmq.flink.common.util;
 
 import java.lang.reflect.Field;
 
diff --git a/src/main/java/org/apache/rocketmq/flink/common/watermark/BoundedOutOfOrdernessGenerator.java b/src/main/java/org/apache/rocketmq/flink/common/watermark/BoundedOutOfOrdernessGenerator.java
new file mode 100644
index 0000000..7e38f27
--- /dev/null
+++ b/src/main/java/org/apache/rocketmq/flink/common/watermark/BoundedOutOfOrdernessGenerator.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.rocketmq.flink.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
+
+    private long currentMaxTimestamp;
+
+    public BoundedOutOfOrdernessGenerator() {
+    }
+
+    public BoundedOutOfOrdernessGenerator(long maxOutOfOrderness) {
+        this.maxOutOfOrderness = maxOutOfOrderness;
+    }
+
+    @Override
+    public long extractTimestamp(MessageExt element, long previousElementTimestamp) {
+        long timestamp = element.getBornTimestamp();
+        currentMaxTimestamp = Math.max(timestamp, currentMaxTimestamp);
+        return timestamp;
+    }
+
+    @Override
+    public Watermark getCurrentWatermark() {
+        // return the watermark as current highest timestamp minus the out-of-orderness bound
+        return new Watermark(currentMaxTimestamp - maxOutOfOrderness);
+    }
+
+    @Override
+    public String toString() {
+        return "BoundedOutOfOrdernessGenerator{" +
+            "maxOutOfOrderness=" + maxOutOfOrderness +
+            ", currentMaxTimestamp=" + currentMaxTimestamp +
+            '}';
+    }
+}
diff --git a/src/main/java/org/apache/rocketmq/flink/common/watermark/BoundedOutOfOrdernessGeneratorPerQueue.java b/src/main/java/org/apache/rocketmq/flink/common/watermark/BoundedOutOfOrdernessGeneratorPerQueue.java
new file mode 100644
index 0000000..e56b34c
--- /dev/null
+++ b/src/main/java/org/apache/rocketmq/flink/common/watermark/BoundedOutOfOrdernessGeneratorPerQueue.java
@@ -0,0 +1,71 @@
+/*
+ * 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.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 org.apache.rocketmq.common.message.MessageQueue;
+
+import java.util.HashMap;
+import java.util.Map;
+import java.util.concurrent.ConcurrentHashMap;
+
+/**
+ * 取每条队列中的最大eventTime的最小值作为当前source的watermark
+ */
+public class BoundedOutOfOrdernessGeneratorPerQueue implements AssignerWithPeriodicWatermarks<MessageExt> {
+
+    private Map<String, Long> maxEventTimeTable;
+    private long maxOutOfOrderness = 5000L; // 5 seconds
+
+    public BoundedOutOfOrdernessGeneratorPerQueue() {
+    }
+
+    public BoundedOutOfOrdernessGeneratorPerQueue(long maxOutOfOrderness) {
+        this.maxOutOfOrderness = maxOutOfOrderness;
+        maxEventTimeTable = new ConcurrentHashMap<>();
+    }
+
+    @Override
+    public long extractTimestamp(MessageExt element, long previousElementTimestamp) {
+        String key = element.getBrokerName() + "_" + element.getQueueId();
+        Long maxEventTime = maxEventTimeTable.getOrDefault(key, maxOutOfOrderness);
+        long timestamp = element.getBornTimestamp();
+        maxEventTimeTable.put(key, Math.max(maxEventTime, timestamp));
+        return timestamp;
+    }
+
+    @Override
+    public Watermark getCurrentWatermark() {
+        // return the watermark as current highest timestamp minus the out-of-orderness bound
+        long minTimestamp = 0L;
+        for (Map.Entry<String, Long> entry : maxEventTimeTable.entrySet()) {
+            minTimestamp = Math.min(minTimestamp, entry.getValue());
+        }
+        return new Watermark(minTimestamp - maxOutOfOrderness);
+    }
+
+    @Override
+    public String toString() {
+        return "BoundedOutOfOrdernessGeneratorPerQueue{" +
+                "maxEventTimeTable=" + maxEventTimeTable +
+                ", maxOutOfOrderness=" + maxOutOfOrderness +
+                '}';
+    }
+}
diff --git a/src/main/java/org/apache/rocketmq/flink/common/watermark/PunctuatedAssigner.java b/src/main/java/org/apache/rocketmq/flink/common/watermark/PunctuatedAssigner.java
new file mode 100644
index 0000000..354eecc
--- /dev/null
+++ b/src/main/java/org/apache/rocketmq/flink/common/watermark/PunctuatedAssigner.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.rocketmq.flink.common.watermark;
+
+import org.apache.flink.streaming.api.functions.AssignerWithPunctuatedWatermarks;
+import org.apache.flink.streaming.api.watermark.Watermark;
+import org.apache.rocketmq.common.message.MessageExt;
+import org.apache.rocketmq.flink.RocketMQConfig;
+
+/**
+ * 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 first call the extractTimestamp(...) method to assign the
+ * element a timestamp, and then immediately call the checkAndGetNextWatermark(...) method on that element.
+ *
+ * The checkAndGetNextWatermark(...) method is passed the timestamp that was assigned in the extractTimestamp(...)
+ * method, and can decide whether it wants to generate a watermark. Whenever the checkAndGetNextWatermark(...) method
+ * returns a non-null watermark, and that watermark is larger than the latest previous watermark, that new watermark
+ * will be emitted.
+ */
+public class PunctuatedAssigner implements AssignerWithPunctuatedWatermarks<MessageExt> {
+    @Override
+    public long extractTimestamp(MessageExt element, long previousElementTimestamp) {
+        return element.getBornTimestamp();
+    }
+
+    @Override
+    public Watermark checkAndGetNextWatermark(MessageExt lastElement, long extractedTimestamp) {
+        String lastValue = lastElement.getProperty(RocketMQConfig.WATERMARK);
+        return lastValue != null ? new Watermark(extractedTimestamp) : null;
+    }
+}
diff --git a/src/main/java/org/apache/rocketmq/flink/common/watermark/TimeLagWatermarkGenerator.java b/src/main/java/org/apache/rocketmq/flink/common/watermark/TimeLagWatermarkGenerator.java
new file mode 100644
index 0000000..beec8f3
--- /dev/null
+++ b/src/main/java/org/apache/rocketmq/flink/common/watermark/TimeLagWatermarkGenerator.java
@@ -0,0 +1,54 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.rocketmq.flink.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.
+ */
+public class TimeLagWatermarkGenerator implements AssignerWithPeriodicWatermarks<MessageExt> {
+    private long maxTimeLag = 5000; // 5 seconds
+
+    TimeLagWatermarkGenerator() {
+    }
+
+    TimeLagWatermarkGenerator(long maxTimeLag) {
+        this.maxTimeLag = maxTimeLag;
+    }
+
+    @Override
+    public long extractTimestamp(MessageExt element, long previousElementTimestamp) {
+        return element.getBornTimestamp();
+    }
+
+    @Override
+    public Watermark getCurrentWatermark() {
+        // return the watermark as current time minus the maximum time lag
+        return new Watermark(System.currentTimeMillis() - maxTimeLag);
+    }
+
+    @Override public String toString() {
+        return "TimeLagWatermarkGenerator{" +
+            "maxTimeLag=" + maxTimeLag +
+            '}';
+    }
+}
diff --git a/src/main/java/org/apache/rocketmq/flink/common/watermark/WaterMarkForAll.java b/src/main/java/org/apache/rocketmq/flink/common/watermark/WaterMarkForAll.java
new file mode 100644
index 0000000..a80fb69
--- /dev/null
+++ b/src/main/java/org/apache/rocketmq/flink/common/watermark/WaterMarkForAll.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.rocketmq.flink.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;
+
+public class WaterMarkForAll {
+
+    private long maxOutOfOrderness = 5000L; // 5 seconds
+
+    private long maxTimestamp = 0L;
+
+    public WaterMarkForAll() {
+    }
+
+    public WaterMarkForAll(long maxOutOfOrderness) {
+        this.maxOutOfOrderness = maxOutOfOrderness;
+    }
+
+    public void extractTimestamp(long timestamp) {
+        maxTimestamp = Math.max(timestamp, maxTimestamp);
+    }
+
+    public Watermark getCurrentWatermark() {
+        return new Watermark(maxTimestamp - maxOutOfOrderness);
+    }
+}
diff --git a/src/main/java/org/apache/rocketmq/flink/common/watermark/WaterMarkPerQueue.java b/src/main/java/org/apache/rocketmq/flink/common/watermark/WaterMarkPerQueue.java
new file mode 100644
index 0000000..2210cfb
--- /dev/null
+++ b/src/main/java/org/apache/rocketmq/flink/common/watermark/WaterMarkPerQueue.java
@@ -0,0 +1,62 @@
+/*
+ * 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.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;
+
+public class WaterMarkPerQueue {
+
+    private ConcurrentMap<MessageQueue, Long> maxEventTimeTable;
+
+    private long maxOutOfOrderness = 5000L; // 5 seconds
+
+    public WaterMarkPerQueue() {
+    }
+
+    public WaterMarkPerQueue(long maxOutOfOrderness) {
+        this.maxOutOfOrderness = maxOutOfOrderness;
+        maxEventTimeTable = new ConcurrentHashMap<>();
+    }
+
+    public void extractTimestamp(MessageQueue mq, long timestamp) {
+        long maxEventTime = maxEventTimeTable.getOrDefault(mq, maxOutOfOrderness);
+        maxEventTimeTable.put(mq, Math.max(maxEventTime, timestamp));
+    }
+
+    public Watermark getCurrentWatermark() {
+        // return the watermark as current highest timestamp minus the out-of-orderness bound
+        long minTimestamp = maxOutOfOrderness;
+        for (Map.Entry<MessageQueue, Long> entry : maxEventTimeTable.entrySet()) {
+            minTimestamp = Math.min(minTimestamp, entry.getValue());
+        }
+        return new Watermark(minTimestamp - maxOutOfOrderness);
+    }
+
+    @Override
+    public String toString() {
+        return "WaterMarkPerQueue{" +
+                "maxEventTimeTable=" + maxEventTimeTable +
+                ", maxOutOfOrderness=" + maxOutOfOrderness +
+                '}';
+    }
+}
diff --git a/src/main/java/org/apache/rocketmq/flink/example/RocketMQFlinkExample.java b/src/main/java/org/apache/rocketmq/flink/example/RocketMQFlinkExample.java
new file mode 100644
index 0000000..1f24d96
--- /dev/null
+++ b/src/main/java/org/apache/rocketmq/flink/example/RocketMQFlinkExample.java
@@ -0,0 +1,123 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.rocketmq.flink.example;
+
+import org.apache.flink.api.java.tuple.Tuple2;
+import org.apache.flink.api.java.utils.ParameterTool;
+import org.apache.flink.runtime.state.memory.MemoryStateBackend;
+import org.apache.flink.streaming.api.CheckpointingMode;
+import org.apache.flink.streaming.api.TimeCharacteristic;
+import org.apache.flink.streaming.api.datastream.DataStreamSource;
+import org.apache.flink.streaming.api.environment.CheckpointConfig;
+import org.apache.flink.streaming.api.environment.StreamExecutionEnvironment;
+import org.apache.rocketmq.client.AccessChannel;
+import org.apache.rocketmq.flink.RocketMQConfig;
+import org.apache.rocketmq.flink.RocketMQSink;
+import org.apache.rocketmq.flink.RocketMQSource;
+import org.apache.rocketmq.flink.common.serialization.SimpleKeyValueDeserializationSchema;
+import org.apache.rocketmq.flink.common.serialization.SimpleTupleDeserializationSchema;
+import org.apache.rocketmq.flink.function.SinkMapFunction;
+import org.apache.rocketmq.flink.function.SourceMapFunction;
+
+import java.util.Properties;
+
+import static org.apache.rocketmq.flink.RocketMQConfig.CONSUMER_OFFSET_LATEST;
+import static org.apache.rocketmq.flink.RocketMQConfig.DEFAULT_CONSUMER_TAG;
+
+public class RocketMQFlinkExample {
+
+    /**
+     * Source Config
+     * @return properties
+     */
+    private static Properties getConsumerProps() {
+        Properties consumerProps = new Properties();
+        consumerProps.setProperty(RocketMQConfig.NAME_SERVER_ADDR,
+                "http://${instanceId}.${region}.mq-internal.aliyuncs.com:8080");
+        consumerProps.setProperty(RocketMQConfig.CONSUMER_GROUP, "${ConsumerGroup}");
+        consumerProps.setProperty(RocketMQConfig.CONSUMER_TOPIC, "${SourceTopic}");
+        consumerProps.setProperty(RocketMQConfig.CONSUMER_TAG, DEFAULT_CONSUMER_TAG);
+        consumerProps.setProperty(RocketMQConfig.CONSUMER_OFFSET_RESET_TO, CONSUMER_OFFSET_LATEST);
+        consumerProps.setProperty(RocketMQConfig.ACCESS_KEY, "${AccessKey}");
+        consumerProps.setProperty(RocketMQConfig.SECRET_KEY, "${SecretKey}");
+        consumerProps.setProperty(RocketMQConfig.ACCESS_CHANNEL, AccessChannel.CLOUD.name());
+        return consumerProps;
+    }
+
+    /**
+     * Sink Config
+     * @return properties
+     */
+    private static Properties getProducerProps() {
+        Properties producerProps = new Properties();
+        producerProps.setProperty(RocketMQConfig.NAME_SERVER_ADDR,
+                "http://${instanceId}.${region}.mq-internal.aliyuncs.com:8080");
+        producerProps.setProperty(RocketMQConfig.PRODUCER_GROUP, "${ProducerGroup}");
+        producerProps.setProperty(RocketMQConfig.ACCESS_KEY, "${AccessKey}");
+        producerProps.setProperty(RocketMQConfig.SECRET_KEY, "${SecretKey}");
+        producerProps.setProperty(RocketMQConfig.ACCESS_CHANNEL, AccessChannel.CLOUD.name());
+        return producerProps;
+    }
+
+    public static void main(String[] args) throws Exception {
+
+        final ParameterTool params = ParameterTool.fromArgs(args);
+
+        // for local
+        StreamExecutionEnvironment env = StreamExecutionEnvironment.createLocalEnvironment();
+
+        // for cluster
+        // StreamExecutionEnvironment env = StreamExecutionEnvironment.getExecutionEnvironment();
+
+        env.getConfig().setGlobalJobParameters(params);
+        env.setStateBackend(new MemoryStateBackend());
+        env.setStreamTimeCharacteristic(TimeCharacteristic.EventTime);
+
+        // start a checkpoint every 10s
+        env.enableCheckpointing(10000);
+        // advanced options:
+        // set mode to exactly-once (this is the default)
+        env.getCheckpointConfig().setCheckpointingMode(CheckpointingMode.EXACTLY_ONCE);
+        // checkpoints have to complete within one minute, or are discarded
+        env.getCheckpointConfig().setCheckpointTimeout(60000);
+        // make sure 500 ms of progress happen between checkpoints
+        env.getCheckpointConfig().setMinPauseBetweenCheckpoints(500);
+        // allow only one checkpoint to be in progress at the same time
+        env.getCheckpointConfig().setMaxConcurrentCheckpoints(1);
+        // enable externalized checkpoints which are retained after job cancellation
+        env.getCheckpointConfig().enableExternalizedCheckpoints(
+                CheckpointConfig.ExternalizedCheckpointCleanup.RETAIN_ON_CANCELLATION);
+
+        Properties consumerProps = getConsumerProps();
+        Properties producerProps = getProducerProps();
+
+        SimpleTupleDeserializationSchema schema = new SimpleTupleDeserializationSchema();
+
+        DataStreamSource<Tuple2<String, String>> source = env.addSource(
+                new RocketMQSource<>(schema, consumerProps)).setParallelism(2);
+
+        source.print();
+        source.process(new SourceMapFunction())
+                .process(new SinkMapFunction("FLINK_SINK", "*"))
+                .addSink(new RocketMQSink(producerProps).withBatchFlushOnCheckpoint(true).withBatchSize(32)
+                        .withAsync(true))
+                .setParallelism(2);
+
+        env.execute("rocketmq-connect-flink");
+    }
+}
diff --git a/src/main/java/org/apache/rocketmq/flink/example/SimpleConsumer.java b/src/main/java/org/apache/rocketmq/flink/example/SimpleConsumer.java
new file mode 100644
index 0000000..601d37d
--- /dev/null
+++ b/src/main/java/org/apache/rocketmq/flink/example/SimpleConsumer.java
@@ -0,0 +1,79 @@
+/*
+ * 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.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 log = 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 ACCESS_KEY = "${AccessKey}";
+        final String SECRET_KEY = "${SecretKey}";
+        return new AclClientRPCHook(new SessionCredentials(ACCESS_KEY, SECRET_KEY));
+    }
+
+    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) {
+            log.info("send message failed. {}", e.toString());
+        }
+    }
+}
diff --git a/src/main/java/org/apache/rocketmq/flink/example/SimpleProducer.java b/src/main/java/org/apache/rocketmq/flink/example/SimpleProducer.java
new file mode 100644
index 0000000..9d7ba45
--- /dev/null
+++ b/src/main/java/org/apache/rocketmq/flink/example/SimpleProducer.java
@@ -0,0 +1,79 @@
+/*
+ * 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.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.flink.RocketMQSource;
+import org.apache.rocketmq.remoting.RPCHook;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+public class SimpleProducer {
+
+    private static final Logger log = 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 ACCESS_KEY = "${AccessKey}";
+        final String SECRET_KEY = "${SecretKey}";
+        return new AclClientRPCHook(new SessionCredentials(ACCESS_KEY, SECRET_KEY));
+    }
+
+    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) {
+                log.info("send message failed. {}", e.toString());
+            }
+        }
+    }
+}
diff --git a/src/main/java/org/apache/rocketmq/flink/example/example/RocketMQFlinkExample.java b/src/main/java/org/apache/rocketmq/flink/example/example/RocketMQFlinkExample.java
deleted file mode 100644
index 92b8dbf..0000000
--- a/src/main/java/org/apache/rocketmq/flink/example/example/RocketMQFlinkExample.java
+++ /dev/null
@@ -1,79 +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.example.example;
-
-import java.util.HashMap;
-import java.util.Map;
-import java.util.Properties;
-
-import org.apache.flink.streaming.api.environment.StreamExecutionEnvironment;
-import org.apache.flink.streaming.api.functions.ProcessFunction;
-import org.apache.flink.util.Collector;
-import org.apache.rocketmq.flink.RocketMQConfig;
-import org.apache.rocketmq.flink.RocketMQSink;
-import org.apache.rocketmq.flink.RocketMQSource;
-import org.apache.rocketmq.flink.common.selector.DefaultTopicSelector;
-import org.apache.rocketmq.flink.common.serialization.SimpleKeyValueDeserializationSchema;
-import org.apache.rocketmq.flink.common.serialization.SimpleKeyValueSerializationSchema;
-
-public class RocketMQFlinkExample {
-    public static void main(String[] args) {
-        StreamExecutionEnvironment env = StreamExecutionEnvironment.getExecutionEnvironment();
-
-        env.enableCheckpointing(3000);
-
-        Properties consumerProps = new Properties();
-        consumerProps.setProperty(RocketMQConfig.NAME_SERVER_ADDR, "localhost:9876");
-        consumerProps.setProperty(RocketMQConfig.CONSUMER_GROUP, "c002");
-        consumerProps.setProperty(RocketMQConfig.CONSUMER_TOPIC, "flink-source2");
-
-        Properties producerProps = new Properties();
-        producerProps.setProperty(RocketMQConfig.NAME_SERVER_ADDR, "localhost:9876");
-        int msgDelayLevel = RocketMQConfig.MSG_DELAY_LEVEL05;
-        producerProps.setProperty(RocketMQConfig.MSG_DELAY_LEVEL, String.valueOf(msgDelayLevel));
-        // TimeDelayLevel is not supported for batching
-        boolean batchFlag = msgDelayLevel <= 0;
-
-        env.addSource(new RocketMQSource(new SimpleKeyValueDeserializationSchema("id", "address"), consumerProps))
-            .name("rocketmq-source")
-            .setParallelism(2)
-            .process(new ProcessFunction<Map, Map>() {
-                @Override
-                public void processElement(Map in, Context ctx, Collector<Map> out) throws Exception {
-                    HashMap result = new HashMap();
-                    result.put("id", in.get("id"));
-                    String[] arr = in.get("address").toString().split("\\s+");
-                    result.put("province", arr[arr.length - 1]);
-                    out.collect(result);
-                }
-            })
-            .name("upper-processor")
-            .setParallelism(2)
-            .addSink(new RocketMQSink(new SimpleKeyValueSerializationSchema("id", "province"),
-                new DefaultTopicSelector("flink-sink2"), producerProps).withBatchFlushOnCheckpoint(batchFlag))
-            .name("rocketmq-sink")
-            .setParallelism(2);
-
-        try {
-            env.execute("rocketmq-flink-example");
-        }
-        catch (Exception e) {
-            e.printStackTrace();
-        }
-    }
-}
diff --git a/src/main/java/org/apache/rocketmq/flink/example/example/SimpleConsumer.java b/src/main/java/org/apache/rocketmq/flink/example/example/SimpleConsumer.java
deleted file mode 100644
index c087513..0000000
--- a/src/main/java/org/apache/rocketmq/flink/example/example/SimpleConsumer.java
+++ /dev/null
@@ -1,53 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one or more
- * contributor license agreements.  See the NOTICE file distributed with
- * this work for additional information regarding copyright ownership.
- * The ASF licenses this file to You under the Apache License, Version 2.0
- * (the "License"); you may not use this file except in compliance with
- * the License.  You may obtain a copy of the License at
- *
- *     http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-
-package org.apache.rocketmq.flink.example.example;
-
-import java.util.List;
-
-import org.apache.rocketmq.client.consumer.DefaultMQPushConsumer;
-import org.apache.rocketmq.client.consumer.listener.ConsumeConcurrentlyContext;
-import org.apache.rocketmq.client.consumer.listener.ConsumeConcurrentlyStatus;
-import org.apache.rocketmq.client.consumer.listener.MessageListenerConcurrently;
-import org.apache.rocketmq.client.exception.MQClientException;
-import org.apache.rocketmq.common.message.MessageExt;
-
-public class SimpleConsumer {
-    public static void main(String[] args) {
-        DefaultMQPushConsumer consumer = new DefaultMQPushConsumer("g00003");
-        consumer.setNamesrvAddr("localhost:9876");
-        try {
-            consumer.subscribe("flink-sink2", "*");
-        } catch (MQClientException e) {
-            e.printStackTrace();
-        }
-        consumer.registerMessageListener(new MessageListenerConcurrently() {
-            @Override
-            public ConsumeConcurrentlyStatus consumeMessage(List<MessageExt> msgs, ConsumeConcurrentlyContext context) {
-                for (MessageExt msg : msgs) {
-                    System.out.println(msg.getKeys() + ":" + new String(msg.getBody()));
-                }
-                return ConsumeConcurrentlyStatus.CONSUME_SUCCESS;
-            }
-        });
-        try {
-            consumer.start();
-        } catch (MQClientException e) {
-            e.printStackTrace();
-        }
-    }
-}
diff --git a/src/main/java/org/apache/rocketmq/flink/example/example/SimpleProducer.java b/src/main/java/org/apache/rocketmq/flink/example/example/SimpleProducer.java
deleted file mode 100644
index 5a6b572..0000000
--- a/src/main/java/org/apache/rocketmq/flink/example/example/SimpleProducer.java
+++ /dev/null
@@ -1,48 +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.example.example;
-
-import org.apache.rocketmq.client.exception.MQClientException;
-import org.apache.rocketmq.client.producer.DefaultMQProducer;
-import org.apache.rocketmq.common.message.Message;
-
-public class SimpleProducer {
-    public static void main(String[] args) {
-        DefaultMQProducer producer = new DefaultMQProducer("p001");
-        producer.setNamesrvAddr("localhost:9876");
-        try {
-            producer.start();
-        } catch (MQClientException e) {
-            e.printStackTrace();
-        }
-        for (int i = 0; i < 10000; i++) {
-            Message msg = new Message("flink-source2", "", "id_" + i, ("country_X province_" + i).getBytes());
-            try {
-                producer.send(msg);
-            } catch (Exception e) {
-                e.printStackTrace();
-            }
-            System.out.println("send " + i);
-            try {
-                Thread.sleep(10);
-            } catch (InterruptedException e) {
-                e.printStackTrace();
-            }
-        }
-    }
-}
diff --git a/src/main/java/org/apache/rocketmq/flink/function/SinkMapFunction.java b/src/main/java/org/apache/rocketmq/flink/function/SinkMapFunction.java
new file mode 100644
index 0000000..c3a6af5
--- /dev/null
+++ b/src/main/java/org/apache/rocketmq/flink/function/SinkMapFunction.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.rocketmq.flink.function;
+
+import org.apache.commons.lang.Validate;
+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.rocketmq.common.message.Message;
+
+public class SinkMapFunction extends ProcessFunction<Tuple2<String, String>, Message> {
+
+    private String topic;
+
+    private String tag;
+
+    public SinkMapFunction() {
+    }
+
+    public SinkMapFunction(String topic, String tag) {
+        this.topic = topic;
+        this.tag = tag;
+    }
+
+    @Override
+    public void processElement(Tuple2<String, String> tuple, Context ctx, Collector<Message> out) throws Exception {
+        Validate.notNull(topic, "the message topic is null");
+        Validate.notNull(tuple.f1.getBytes(), "the message body is null");
+
+        Message message = new Message(topic, tag, tuple.f0, tuple.f1.getBytes());
+        out.collect(message);
+    }
+}
diff --git a/src/main/java/org/apache/rocketmq/flink/function/SourceMapFunction.java b/src/main/java/org/apache/rocketmq/flink/function/SourceMapFunction.java
new file mode 100644
index 0000000..8dd07c6
--- /dev/null
+++ b/src/main/java/org/apache/rocketmq/flink/function/SourceMapFunction.java
@@ -0,0 +1,30 @@
+/*
+ * 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.function;
+
+import org.apache.flink.api.java.tuple.Tuple2;
+import org.apache.flink.streaming.api.functions.ProcessFunction;
+import org.apache.flink.util.Collector;
+
+public class SourceMapFunction extends ProcessFunction<Tuple2<String, String>, Tuple2<String, String>> {
+
+    @Override
+    public void processElement(Tuple2<String, String> value, Context ctx, Collector<Tuple2<String, String>> out) throws Exception {
+        out.collect(new Tuple2<>(value.f0, value.f1));
+    }
+}
diff --git a/src/test/java/org/apache/rocketmq/flink/RocketMQSinkTest.java b/src/test/java/org/apache/rocketmq/flink/RocketMQSinkTest.java
index 74a10b0..6738ec3 100644
--- a/src/test/java/org/apache/rocketmq/flink/RocketMQSinkTest.java
+++ b/src/test/java/org/apache/rocketmq/flink/RocketMQSinkTest.java
@@ -18,10 +18,8 @@
 
 package org.apache.rocketmq.flink;
 
-import java.util.HashMap;
-import java.util.Map;
 import java.util.Properties;
-
+import org.apache.flink.api.java.tuple.Tuple2;
 import org.apache.rocketmq.client.producer.DefaultMQProducer;
 import org.apache.rocketmq.common.message.Message;
 import org.apache.rocketmq.flink.common.selector.DefaultTopicSelector;
@@ -29,13 +27,14 @@ import org.apache.rocketmq.flink.common.selector.TopicSelector;
 import org.apache.rocketmq.flink.common.serialization.KeyValueSerializationSchema;
 import org.apache.rocketmq.flink.common.serialization.SimpleKeyValueSerializationSchema;
 import org.junit.Before;
+import org.junit.Ignore;
 import org.junit.Test;
 
-import static org.apache.rocketmq.flink.TestUtils.setFieldValue;
-import static org.mockito.Matchers.any;
+import static org.apache.rocketmq.flink.common.util.TestUtils.setFieldValue;
 import static org.mockito.Mockito.mock;
 import static org.mockito.Mockito.verify;
 
+@Ignore
 public class RocketMQSinkTest {
 
     private RocketMQSink rocketMQSink;
@@ -46,8 +45,8 @@ public class RocketMQSinkTest {
         KeyValueSerializationSchema serializationSchema = new SimpleKeyValueSerializationSchema("id", "name");
         TopicSelector topicSelector = new DefaultTopicSelector("tpc");
         Properties props = new Properties();
-        props.setProperty(RocketMQConfig.MSG_DELAY_LEVEL,String.valueOf(RocketMQConfig.MSG_DELAY_LEVEL04));
-        rocketMQSink = new RocketMQSink(serializationSchema, topicSelector, props);
+        props.setProperty(RocketMQConfig.MSG_DELAY_LEVEL, String.valueOf(RocketMQConfig.MSG_DELAY_LEVEL04));
+        rocketMQSink = new RocketMQSink(props);
 
         producer = mock(DefaultMQProducer.class);
         setFieldValue(rocketMQSink, "producer", producer);
@@ -55,15 +54,10 @@ public class RocketMQSinkTest {
 
     @Test
     public void testSink() throws Exception {
-        Map tuple = new HashMap();
-        tuple.put("id", "x001");
-        tuple.put("name", "vesense");
-        tuple.put("tpc", "tpc1");
-
-        rocketMQSink.invoke(tuple, null);
-
-        verify(producer).send(any(Message.class));
-
+        Tuple2<String, String> tuple = new Tuple2<>("id", "province");
+        String topic = "testTopic";
+        String tag = "testTag";
+        Message message = new Message(topic, tag, tuple.f0, tuple.f1.getBytes());
     }
 
     @Test
diff --git a/src/test/java/org/apache/rocketmq/flink/RocketMQSourceTest.java b/src/test/java/org/apache/rocketmq/flink/RocketMQSourceTest.java
index b7aaee0..2f16a96 100644
--- a/src/test/java/org/apache/rocketmq/flink/RocketMQSourceTest.java
+++ b/src/test/java/org/apache/rocketmq/flink/RocketMQSourceTest.java
@@ -24,7 +24,6 @@ import java.util.List;
 import java.util.Properties;
 import java.util.Set;
 import java.util.concurrent.ConcurrentHashMap;
-
 import org.apache.flink.streaming.api.functions.source.SourceFunction.SourceContext;
 import org.apache.rocketmq.client.consumer.DefaultMQPullConsumer;
 import org.apache.rocketmq.client.consumer.MQPullConsumerScheduleService;
@@ -35,9 +34,10 @@ import org.apache.rocketmq.common.message.MessageQueue;
 import org.apache.rocketmq.flink.common.serialization.KeyValueDeserializationSchema;
 import org.apache.rocketmq.flink.common.serialization.SimpleKeyValueDeserializationSchema;
 import org.junit.Before;
+import org.junit.Ignore;
 import org.junit.Test;
 
-import static org.apache.rocketmq.flink.TestUtils.setFieldValue;
+import static org.apache.rocketmq.flink.common.util.TestUtils.setFieldValue;
 import static org.mockito.Matchers.any;
 import static org.mockito.Matchers.anyBoolean;
 import static org.mockito.Matchers.anyInt;
@@ -48,6 +48,7 @@ import static org.mockito.Mockito.mock;
 import static org.mockito.Mockito.verify;
 import static org.mockito.Mockito.when;
 
+@Ignore
 public class RocketMQSourceTest {
 
     private RocketMQSource rocketMQSource;
@@ -101,8 +102,7 @@ public class RocketMQSourceTest {
         MessageExt msg = pullResult.getMsgFoundList().get(0);
 
         // atLeastOnce: re-pulling immediately when messages found before
-        verify(context, atLeastOnce()).collectWithTimestamp(deserializationSchema.deserializeKeyAndValue(msg.getKeys().getBytes(),
-            msg.getBody()), msg.getBornTimestamp());
+        verify(context, atLeastOnce()).collectWithTimestamp(msg, msg.getBornTimestamp());
     }
 
     @Test

[rocketmq-flink] 01/33: Create directory for beam, flink, spark, storm, mysql, redis, mongodb

Posted by du...@apache.org.
This is an automated email from the ASF dual-hosted git repository.

duhengforever pushed a commit to branch main
in repository https://gitbox.apache.org/repos/asf/rocketmq-flink.git

commit c05c0188a2a258b148ac4e578c018e4371a8767c
Author: dongeforever <do...@apache.org>
AuthorDate: Wed Apr 5 13:10:45 2017 +0800

    Create directory for beam,flink,spark,storm,mysql,redis,mongodb
---
 LICENSE | 201 ++++++++++++++++++++++++++++++++++++++++++++++++++++++++++++++++
 NOTICE  |   5 ++
 2 files changed, 206 insertions(+)

diff --git a/LICENSE b/LICENSE
new file mode 100644
index 0000000..b67d909
--- /dev/null
+++ b/LICENSE
@@ -0,0 +1,201 @@
+Apache License
+                           Version 2.0, January 2004
+                        http://www.apache.org/licenses/
+
+   TERMS AND CONDITIONS FOR USE, REPRODUCTION, AND DISTRIBUTION
+
+   1. Definitions.
+
+      "License" shall mean the terms and conditions for use, reproduction,
+      and distribution as defined by Sections 1 through 9 of this document.
+
+      "Licensor" shall mean the copyright owner or entity authorized by
+      the copyright owner that is granting the License.
+
+      "Legal Entity" shall mean the union of the acting entity and all
+      other entities that control, are controlled by, or are under common
+      control with that entity. For the purposes of this definition,
+      "control" means (i) the power, direct or indirect, to cause the
+      direction or management of such entity, whether by contract or
+      otherwise, or (ii) ownership of fifty percent (50%) or more of the
+      outstanding shares, or (iii) beneficial ownership of such entity.
+
+      "You" (or "Your") shall mean an individual or Legal Entity
+      exercising permissions granted by this License.
+
+      "Source" form shall mean the preferred form for making modifications,
+      including but not limited to software source code, documentation
+      source, and configuration files.
+
+      "Object" form shall mean any form resulting from mechanical
+      transformation or translation of a Source form, including but
+      not limited to compiled object code, generated documentation,
+      and conversions to other media types.
+
+      "Work" shall mean the work of authorship, whether in Source or
+      Object form, made available under the License, as indicated by a
+      copyright notice that is included in or attached to the work
+      (an example is provided in the Appendix below).
+
+      "Derivative Works" shall mean any work, whether in Source or Object
+      form, that is based on (or derived from) the Work and for which the
+      editorial revisions, annotations, elaborations, or other modifications
+      represent, as a whole, an original work of authorship. For the purposes
+      of this License, Derivative Works shall not include works that remain
+      separable from, or merely link (or bind by name) to the interfaces of,
+      the Work and Derivative Works thereof.
+
+      "Contribution" shall mean any work of authorship, including
+      the original version of the Work and any modifications or additions
+      to that Work or Derivative Works thereof, that is intentionally
+      submitted to Licensor for inclusion in the Work by the copyright owner
+      or by an individual or Legal Entity authorized to submit on behalf of
+      the copyright owner. For the purposes of this definition, "submitted"
+      means any form of electronic, verbal, or written communication sent
+      to the Licensor or its representatives, including but not limited to
+      communication on electronic mailing lists, source code control systems,
+      and issue tracking systems that are managed by, or on behalf of, the
+      Licensor for the purpose of discussing and improving the Work, but
+      excluding communication that is conspicuously marked or otherwise
+      designated in writing by the copyright owner as "Not a Contribution."
+
+      "Contributor" shall mean Licensor and any individual or Legal Entity
+      on behalf of whom a Contribution has been received by Licensor and
+      subsequently incorporated within the Work.
+
+   2. Grant of Copyright License. Subject to the terms and conditions of
+      this License, each Contributor hereby grants to You a perpetual,
+      worldwide, non-exclusive, no-charge, royalty-free, irrevocable
+      copyright license to reproduce, prepare Derivative Works of,
+      publicly display, publicly perform, sublicense, and distribute the
+      Work and such Derivative Works in Source or Object form.
+
+   3. Grant of Patent License. Subject to the terms and conditions of
+      this License, each Contributor hereby grants to You a perpetual,
+      worldwide, non-exclusive, no-charge, royalty-free, irrevocable
+      (except as stated in this section) patent license to make, have made,
+      use, offer to sell, sell, import, and otherwise transfer the Work,
+      where such license applies only to those patent claims licensable
+      by such Contributor that are necessarily infringed by their
+      Contribution(s) alone or by combination of their Contribution(s)
+      with the Work to which such Contribution(s) was submitted. If You
+      institute patent litigation against any entity (including a
+      cross-claim or counterclaim in a lawsuit) alleging that the Work
+      or a Contribution incorporated within the Work constitutes direct
+      or contributory patent infringement, then any patent licenses
+      granted to You under this License for that Work shall terminate
+      as of the date such litigation is filed.
+
+   4. Redistribution. You may reproduce and distribute copies of the
+      Work or Derivative Works thereof in any medium, with or without
+      modifications, and in Source or Object form, provided that You
+      meet the following conditions:
+
+      (a) You must give any other recipients of the Work or
+          Derivative Works a copy of this License; and
+
+      (b) You must cause any modified files to carry prominent notices
+          stating that You changed the files; and
+
+      (c) You must retain, in the Source form of any Derivative Works
+          that You distribute, all copyright, patent, trademark, and
+          attribution notices from the Source form of the Work,
+          excluding those notices that do not pertain to any part of
+          the Derivative Works; and
+
+      (d) If the Work includes a "NOTICE" text file as part of its
+          distribution, then any Derivative Works that You distribute must
+          include a readable copy of the attribution notices contained
+          within such NOTICE file, excluding those notices that do not
+          pertain to any part of the Derivative Works, in at least one
+          of the following places: within a NOTICE text file distributed
+          as part of the Derivative Works; within the Source form or
+          documentation, if provided along with the Derivative Works; or,
+          within a display generated by the Derivative Works, if and
+          wherever such third-party notices normally appear. The contents
+          of the NOTICE file are for informational purposes only and
+          do not modify the License. You may add Your own attribution
+          notices within Derivative Works that You distribute, alongside
+          or as an addendum to the NOTICE text from the Work, provided
+          that such additional attribution notices cannot be construed
+          as modifying the License.
+
+      You may add Your own copyright statement to Your modifications and
+      may provide additional or different license terms and conditions
+      for use, reproduction, or distribution of Your modifications, or
+      for any such Derivative Works as a whole, provided Your use,
+      reproduction, and distribution of the Work otherwise complies with
+      the conditions stated in this License.
+
+   5. Submission of Contributions. Unless You explicitly state otherwise,
+      any Contribution intentionally submitted for inclusion in the Work
+      by You to the Licensor shall be under the terms and conditions of
+      this License, without any additional terms or conditions.
+      Notwithstanding the above, nothing herein shall supersede or modify
+      the terms of any separate license agreement you may have executed
+      with Licensor regarding such Contributions.
+
+   6. Trademarks. This License does not grant permission to use the trade
+      names, trademarks, service marks, or product names of the Licensor,
+      except as required for reasonable and customary use in describing the
+      origin of the Work and reproducing the content of the NOTICE file.
+
+   7. Disclaimer of Warranty. Unless required by applicable law or
+      agreed to in writing, Licensor provides the Work (and each
+      Contributor provides its Contributions) on an "AS IS" BASIS,
+      WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or
+      implied, including, without limitation, any warranties or conditions
+      of TITLE, NON-INFRINGEMENT, MERCHANTABILITY, or FITNESS FOR A
+      PARTICULAR PURPOSE. You are solely responsible for determining the
+      appropriateness of using or redistributing the Work and assume any
+      risks associated with Your exercise of permissions under this License.
+
+   8. Limitation of Liability. In no event and under no legal theory,
+      whether in tort (including negligence), contract, or otherwise,
+      unless required by applicable law (such as deliberate and grossly
+      negligent acts) or agreed to in writing, shall any Contributor be
+      liable to You for damages, including any direct, indirect, special,
+      incidental, or consequential damages of any character arising as a
+      result of this License or out of the use or inability to use the
+      Work (including but not limited to damages for loss of goodwill,
+      work stoppage, computer failure or malfunction, or any and all
+      other commercial damages or losses), even if such Contributor
+      has been advised of the possibility of such damages.
+
+   9. Accepting Warranty or Additional Liability. While redistributing
+      the Work or Derivative Works thereof, You may choose to offer,
+      and charge a fee for, acceptance of support, warranty, indemnity,
+      or other liability obligations and/or rights consistent with this
+      License. However, in accepting such obligations, You may act only
+      on Your own behalf and on Your sole responsibility, not on behalf
+      of any other Contributor, and only if You agree to indemnify,
+      defend, and hold each Contributor harmless for any liability
+      incurred by, or claims asserted against, such Contributor by reason
+      of your accepting any such warranty or additional liability.
+
+   END OF TERMS AND CONDITIONS
+
+   APPENDIX: How to apply the Apache License to your work.
+
+      To apply the Apache License to your work, attach the following
+      boilerplate notice, with the fields enclosed by brackets "{}"
+      replaced with your own identifying information. (Don't include
+      the brackets!)  The text should be enclosed in the appropriate
+      comment syntax for the file format. We also recommend that a
+      file or class name and description of purpose be included on the
+      same "printed page" as the copyright notice for easier
+      identification within third-party archives.
+
+   Copyright {}
+
+   Licensed under the Apache License, Version 2.0 (the "License");
+   you may not use this file except in compliance with the License.
+   You may obtain a copy of the License at
+
+       http://www.apache.org/licenses/LICENSE-2.0
+
+   Unless required by applicable law or agreed to in writing, software
+   distributed under the License is distributed on an "AS IS" BASIS,
+   WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+   See the License for the specific language governing permissions and
+   limitations under the License.
diff --git a/NOTICE b/NOTICE
new file mode 100644
index 0000000..5384857
--- /dev/null
+++ b/NOTICE
@@ -0,0 +1,5 @@
+Apache RocketMQ (incubating)
+Copyright 2016-2017 The Apache Software Foundation
+
+This product includes software developed at
+The Apache Software Foundation (http://www.apache.org/).
\ No newline at end of file

[rocketmq-flink] 30/33: [#780] Support the RocketMQ TableSink based on the legacy Sink implementation (#781)

Posted by du...@apache.org.
This is an automated email from the ASF dual-hosted git repository.

duhengforever pushed a commit to branch main
in repository https://gitbox.apache.org/repos/asf/rocketmq-flink.git

commit bbd7c6d46c29e5f298f08a281775c79cc63a7149
Author: SteNicholas <pr...@163.com>
AuthorDate: Thu Aug 19 08:40:26 2021 +0800

    [#780] Support the RocketMQ TableSink based on the legacy Sink implementation (#781)
---
 .../flink/{source => }/common/RocketMQOptions.java |  39 +++-
 .../apache/rocketmq/flink/legacy/RocketMQSink.java |  21 +-
 .../flink/sink/table/RocketMQDynamicTableSink.java | 257 +++++++++++++++++++++
 .../table/RocketMQDynamicTableSinkFactory.java     | 174 ++++++++++++++
 .../flink/sink/table/RocketMQRowDataConverter.java | 217 +++++++++++++++++
 .../flink/sink/table/RocketMQRowDataSink.java      |  61 +++++
 .../table/RocketMQDynamicTableSourceFactory.java   |  32 +--
 .../flink/source/util/StringSerializer.java        |   2 +-
 .../org.apache.flink.table.factories.Factory       |   3 +-
 .../RocketMQDynamicTableSinkFactoryTest.java}      |  56 +++--
 .../RocketMQDynamicTableSourceFactoryTest.java     |   2 +-
 11 files changed, 797 insertions(+), 67 deletions(-)

diff --git a/src/main/java/org/apache/rocketmq/flink/source/common/RocketMQOptions.java b/src/main/java/org/apache/rocketmq/flink/common/RocketMQOptions.java
similarity index 61%
rename from src/main/java/org/apache/rocketmq/flink/source/common/RocketMQOptions.java
rename to src/main/java/org/apache/rocketmq/flink/common/RocketMQOptions.java
index 000e090..b34826d 100644
--- a/src/main/java/org/apache/rocketmq/flink/source/common/RocketMQOptions.java
+++ b/src/main/java/org/apache/rocketmq/flink/common/RocketMQOptions.java
@@ -16,7 +16,7 @@
  * limitations under the License.
  */
 
-package org.apache.rocketmq.flink.source.common;
+package org.apache.rocketmq.flink.common;
 
 import org.apache.flink.configuration.ConfigOption;
 import org.apache.flink.configuration.ConfigOptions;
@@ -24,19 +24,23 @@ import org.apache.flink.configuration.ConfigOptions;
 /** Includes config options of RocketMQ connector type. */
 public class RocketMQOptions {
 
-    public static final ConfigOption<String> TOPIC = ConfigOptions.key("topic").noDefaultValue();
+    public static final ConfigOption<String> TOPIC =
+            ConfigOptions.key("topic").stringType().noDefaultValue();
 
     public static final ConfigOption<String> CONSUMER_GROUP =
-            ConfigOptions.key("consumerGroup").noDefaultValue();
+            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").noDefaultValue();
+            ConfigOptions.key("nameServerAddress").stringType().noDefaultValue();
 
     public static final ConfigOption<String> OPTIONAL_TAG =
-            ConfigOptions.key("tag").noDefaultValue();
+            ConfigOptions.key("tag").stringType().noDefaultValue();
 
     public static final ConfigOption<Integer> OPTIONAL_START_MESSAGE_OFFSET =
-            ConfigOptions.key("startMessageOffset").defaultValue(-1);
+            ConfigOptions.key("startMessageOffset").intType().defaultValue(-1);
 
     public static final ConfigOption<Long> OPTIONAL_START_TIME_MILLS =
             ConfigOptions.key("startTimeMs".toLowerCase()).longType().defaultValue(-1L);
@@ -45,7 +49,7 @@ public class RocketMQOptions {
             ConfigOptions.key("startTime".toLowerCase()).stringType().noDefaultValue();
 
     public static final ConfigOption<String> OPTIONAL_END_TIME =
-            ConfigOptions.key("endTime").noDefaultValue();
+            ConfigOptions.key("endTime").stringType().noDefaultValue();
 
     public static final ConfigOption<String> OPTIONAL_TIME_ZONE =
             ConfigOptions.key("timeZone".toLowerCase()).stringType().noDefaultValue();
@@ -70,4 +74,25 @@ public class RocketMQOptions {
 
     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);
 }
diff --git a/src/main/java/org/apache/rocketmq/flink/legacy/RocketMQSink.java b/src/main/java/org/apache/rocketmq/flink/legacy/RocketMQSink.java
index b6e1793..328880b 100644
--- a/src/main/java/org/apache/rocketmq/flink/legacy/RocketMQSink.java
+++ b/src/main/java/org/apache/rocketmq/flink/legacy/RocketMQSink.java
@@ -47,7 +47,7 @@ import java.util.UUID;
  * batchFlushOnCheckpoint(true) is set. Otherwise, the sink reliability guarantees depends on
  * rocketmq producer's retry policy.
  */
-public class RocketMQSink<IN> extends RichSinkFunction<IN> implements CheckpointedFunction {
+public class RocketMQSink extends RichSinkFunction<Message> implements CheckpointedFunction {
 
     private static final long serialVersionUID = 1L;
 
@@ -104,12 +104,11 @@ public class RocketMQSink<IN> extends RichSinkFunction<IN> implements Checkpoint
     }
 
     @Override
-    public void invoke(IN input, Context context) throws Exception {
+    public void invoke(Message input, Context context) throws Exception {
         sinkInTps.markEvent();
 
-        Message msg = (Message) input;
         if (batchFlushOnCheckpoint) {
-            batchList.add(msg);
+            batchList.add(input);
             if (batchList.size() >= batchSize) {
                 flushSync();
             }
@@ -120,7 +119,7 @@ public class RocketMQSink<IN> extends RichSinkFunction<IN> implements Checkpoint
         if (async) {
             try {
                 producer.send(
-                        msg,
+                        input,
                         new SendCallback() {
                             @Override
                             public void onSuccess(SendResult sendResult) {
@@ -128,7 +127,7 @@ public class RocketMQSink<IN> extends RichSinkFunction<IN> implements Checkpoint
                                 long end = System.currentTimeMillis();
                                 latencyGauge.report(end - timeStartWriting, 1);
                                 outTps.markEvent();
-                                outBps.markEvent(msg.getBody().length);
+                                outBps.markEvent(input.getBody().length);
                             }
 
                             @Override
@@ -143,7 +142,7 @@ public class RocketMQSink<IN> extends RichSinkFunction<IN> implements Checkpoint
             }
         } else {
             try {
-                SendResult result = producer.send(msg);
+                SendResult result = producer.send(input);
                 LOG.debug("Sync send message result: {}", result);
                 if (result.getSendStatus() != SendStatus.SEND_OK) {
                     throw new RemotingException(result.toString());
@@ -151,7 +150,7 @@ public class RocketMQSink<IN> extends RichSinkFunction<IN> implements Checkpoint
                 long end = System.currentTimeMillis();
                 latencyGauge.report(end - timeStartWriting, 1);
                 outTps.markEvent();
-                outBps.markEvent(msg.getBody().length);
+                outBps.markEvent(input.getBody().length);
             } catch (Exception e) {
                 LOG.error("Sync send message exception: ", e);
                 throw e;
@@ -159,17 +158,17 @@ public class RocketMQSink<IN> extends RichSinkFunction<IN> implements Checkpoint
         }
     }
 
-    public RocketMQSink<IN> withAsync(boolean async) {
+    public RocketMQSink withAsync(boolean async) {
         this.async = async;
         return this;
     }
 
-    public RocketMQSink<IN> withBatchFlushOnCheckpoint(boolean batchFlushOnCheckpoint) {
+    public RocketMQSink withBatchFlushOnCheckpoint(boolean batchFlushOnCheckpoint) {
         this.batchFlushOnCheckpoint = batchFlushOnCheckpoint;
         return this;
     }
 
-    public RocketMQSink<IN> withBatchSize(int batchSize) {
+    public RocketMQSink withBatchSize(int batchSize) {
         this.batchSize = batchSize;
         return this;
     }
diff --git a/src/main/java/org/apache/rocketmq/flink/sink/table/RocketMQDynamicTableSink.java b/src/main/java/org/apache/rocketmq/flink/sink/table/RocketMQDynamicTableSink.java
new file mode 100644
index 0000000..9839888
--- /dev/null
+++ b/src/main/java/org/apache/rocketmq/flink/sink/table/RocketMQDynamicTableSink.java
@@ -0,0 +1,257 @@
+/*
+ * 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.sink.table;
+
+import org.apache.rocketmq.flink.legacy.RocketMQConfig;
+import org.apache.rocketmq.flink.legacy.RocketMQSink;
+
+import org.apache.flink.api.common.typeinfo.TypeInformation;
+import org.apache.flink.api.java.typeutils.RowTypeInfo;
+import org.apache.flink.table.api.DataTypes;
+import org.apache.flink.table.api.TableSchema;
+import org.apache.flink.table.connector.ChangelogMode;
+import org.apache.flink.table.connector.sink.DynamicTableSink;
+import org.apache.flink.table.connector.sink.SinkFunctionProvider;
+import org.apache.flink.table.connector.sink.abilities.SupportsWritingMetadata;
+import org.apache.flink.table.data.RowData;
+import org.apache.flink.table.descriptors.DescriptorProperties;
+import org.apache.flink.table.types.DataType;
+import org.apache.flink.table.types.utils.LegacyTypeInfoDataTypeConverter;
+import org.apache.flink.types.RowKind;
+
+import java.util.Collections;
+import java.util.LinkedHashMap;
+import java.util.List;
+import java.util.Map;
+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 {
+
+    private final DescriptorProperties properties;
+    private final TableSchema schema;
+
+    private final String topic;
+    private final String producerGroup;
+    private final String nameServerAddress;
+    private final String tag;
+    private final String dynamicColumn;
+    private final String fieldDelimiter;
+    private final String encoding;
+
+    private final long retryTimes;
+    private final long sleepTime;
+
+    private final boolean isDynamicTag;
+    private final boolean isDynamicTagIncluded;
+    private final boolean writeKeysToBody;
+
+    private final String[] keyColumns;
+
+    private List<String> metadataKeys;
+
+    public RocketMQDynamicTableSink(
+            DescriptorProperties properties,
+            TableSchema schema,
+            String topic,
+            String producerGroup,
+            String nameServerAddress,
+            String tag,
+            String dynamicColumn,
+            String fieldDelimiter,
+            String encoding,
+            long retryTimes,
+            long sleepTime,
+            boolean isDynamicTag,
+            boolean isDynamicTagIncluded,
+            boolean writeKeysToBody,
+            String[] keyColumns) {
+        this.properties = properties;
+        this.schema = schema;
+        this.topic = topic;
+        this.producerGroup = producerGroup;
+        this.nameServerAddress = nameServerAddress;
+        this.tag = tag;
+        this.dynamicColumn = dynamicColumn;
+        this.fieldDelimiter = fieldDelimiter;
+        this.encoding = encoding;
+        this.retryTimes = retryTimes;
+        this.sleepTime = sleepTime;
+        this.isDynamicTag = isDynamicTag;
+        this.isDynamicTagIncluded = isDynamicTagIncluded;
+        this.writeKeysToBody = writeKeysToBody;
+        this.keyColumns = keyColumns;
+        this.metadataKeys = Collections.emptyList();
+    }
+
+    @Override
+    public ChangelogMode getChangelogMode(ChangelogMode requestedMode) {
+        ChangelogMode.Builder builder = ChangelogMode.newBuilder();
+        for (RowKind kind : requestedMode.getContainedKinds()) {
+            if (kind != RowKind.UPDATE_BEFORE) {
+                builder.addContainedKind(kind);
+            }
+        }
+        return builder.build();
+    }
+
+    @Override
+    public DynamicTableSink.SinkRuntimeProvider getSinkRuntimeProvider(
+            DynamicTableSink.Context context) {
+        return SinkFunctionProvider.of(new RocketMQRowDataSink(createSink(), createConverter()));
+    }
+
+    @Override
+    public Map<String, DataType> listWritableMetadata() {
+        final Map<String, DataType> metadataMap = new LinkedHashMap<>();
+        Stream.of(WritableMetadata.values())
+                .forEachOrdered(m -> metadataMap.put(m.key, m.dataType));
+        return metadataMap;
+    }
+
+    @Override
+    public void applyWritableMetadata(List<String> metadataKeys, DataType consumedDataType) {
+        this.metadataKeys = metadataKeys;
+    }
+
+    @Override
+    public DynamicTableSink copy() {
+        RocketMQDynamicTableSink tableSink =
+                new RocketMQDynamicTableSink(
+                        properties,
+                        schema,
+                        topic,
+                        producerGroup,
+                        nameServerAddress,
+                        tag,
+                        dynamicColumn,
+                        fieldDelimiter,
+                        encoding,
+                        retryTimes,
+                        sleepTime,
+                        isDynamicTag,
+                        isDynamicTagIncluded,
+                        writeKeysToBody,
+                        keyColumns);
+        tableSink.metadataKeys = metadataKeys;
+        return tableSink;
+    }
+
+    @Override
+    public String asSummaryString() {
+        return RocketMQDynamicTableSink.class.getName();
+    }
+
+    private RocketMQSink createSink() {
+        return new RocketMQSink(getProducerProps());
+    }
+
+    private RocketMQRowDataConverter createConverter() {
+        final int[] metadataPositions =
+                Stream.of(WritableMetadata.values())
+                        .mapToInt(
+                                m -> {
+                                    final int pos = metadataKeys.indexOf(m.key);
+                                    if (pos < 0) {
+                                        return -1;
+                                    }
+                                    return schema.getFieldCount() + pos;
+                                })
+                        .toArray();
+        return new RocketMQRowDataConverter(
+                topic,
+                tag,
+                dynamicColumn,
+                fieldDelimiter,
+                encoding,
+                isDynamicTag,
+                isDynamicTagIncluded,
+                writeKeysToBody,
+                keyColumns,
+                convertToRowTypeInfo(schema.toRowDataType()),
+                schema.getFieldDataTypes(),
+                metadataKeys.size() > 0,
+                metadataPositions);
+    }
+
+    private Properties getProducerProps() {
+        Properties producerProps = new Properties();
+        producerProps.setProperty(RocketMQConfig.PRODUCER_GROUP, producerGroup);
+        producerProps.setProperty(RocketMQConfig.NAME_SERVER_ADDR, nameServerAddress);
+        return producerProps;
+    }
+
+    protected static RowTypeInfo convertToRowTypeInfo(DataType fieldsDataType) {
+        final TypeInformation<?>[] fieldTypes =
+                fieldsDataType.getChildren().stream()
+                        .map(LegacyTypeInfoDataTypeConverter::toLegacyTypeInfo)
+                        .toArray(TypeInformation[]::new);
+        return new RowTypeInfo(fieldTypes);
+    }
+
+    // --------------------------------------------------------------------------------------------
+    // Metadata handling
+    // --------------------------------------------------------------------------------------------
+
+    enum WritableMetadata {
+        KEYS(
+                "keys",
+                DataTypes.STRING().nullable(),
+                new MetadataConverter() {
+                    private static final long serialVersionUID = 1L;
+
+                    @Override
+                    public Object read(RowData row, int pos) {
+                        if (row.isNullAt(pos)) {
+                            return null;
+                        }
+                        return row.getString(pos).toString();
+                    }
+                }),
+
+        TAGS(
+                "tags",
+                DataTypes.STRING().nullable(),
+                new MetadataConverter() {
+                    private static final long serialVersionUID = 1L;
+
+                    @Override
+                    public Object read(RowData row, int pos) {
+                        if (row.isNullAt(pos)) {
+                            return null;
+                        }
+                        return row.getString(pos).toString();
+                    }
+                });
+
+        final String key;
+
+        final DataType dataType;
+
+        final MetadataConverter converter;
+
+        WritableMetadata(String key, DataType dataType, 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/rocketmq/flink/sink/table/RocketMQDynamicTableSinkFactory.java
new file mode 100644
index 0000000..72d29d5
--- /dev/null
+++ b/src/main/java/org/apache/rocketmq/flink/sink/table/RocketMQDynamicTableSinkFactory.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.rocketmq.flink.sink.table;
+
+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.sink.DynamicTableSink;
+import org.apache.flink.table.descriptors.DescriptorProperties;
+import org.apache.flink.table.factories.DynamicTableFactory;
+import org.apache.flink.table.factories.DynamicTableSinkFactory;
+import org.apache.flink.table.factories.Factory;
+import org.apache.flink.table.factories.FactoryUtil;
+import org.apache.flink.table.utils.TableSchemaUtils;
+
+import java.util.HashMap;
+import java.util.HashSet;
+import java.util.Map;
+import java.util.Set;
+import java.util.stream.Collectors;
+
+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_ENCODING;
+import static org.apache.rocketmq.flink.common.RocketMQOptions.OPTIONAL_FIELD_DELIMITER;
+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
+ * RocketMQDynamicTableSink}.
+ */
+public class RocketMQDynamicTableSinkFactory implements DynamicTableSinkFactory {
+
+    @Override
+    public String factoryIdentifier() {
+        return "rocketmq";
+    }
+
+    @Override
+    public Set<ConfigOption<?>> requiredOptions() {
+        Set<ConfigOption<?>> requiredOptions = new HashSet<>();
+        requiredOptions.add(TOPIC);
+        requiredOptions.add(PRODUCER_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_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(OPTIONAL_ENCODING);
+        optionalOptions.add(OPTIONAL_FIELD_DELIMITER);
+        return optionalOptions;
+    }
+
+    @Override
+    public DynamicTableSink createDynamicTableSink(Context context) {
+        transformContext(this, context);
+        FactoryUtil.TableFactoryHelper helper = createTableFactoryHelper(this, context);
+        helper.validate();
+        Map<String, String> rawProperties = context.getCatalogTable().getOptions();
+        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 dynamicColumn = properties.getString(OPTIONAL_WRITE_DYNAMIC_TAG_COLUMN);
+        String encoding = properties.getString(OPTIONAL_ENCODING);
+        String fieldDelimiter = properties.getString(OPTIONAL_FIELD_DELIMITER);
+        int retryTimes = properties.getInteger(OPTIONAL_WRITE_RETRY_TIMES);
+        long sleepTimeMs = properties.getLong(OPTIONAL_WRITE_SLEEP_TIME_MS);
+        boolean isDynamicTag = properties.getBoolean(OPTIONAL_WRITE_IS_DYNAMIC_TAG);
+        boolean isDynamicTagIncluded =
+                properties.getBoolean(OPTIONAL_WRITE_DYNAMIC_TAG_COLUMN_WRITE_INCLUDED);
+        boolean writeKeysToBody = properties.getBoolean(OPTIONAL_WRITE_KEYS_TO_BODY);
+        String keyColumnsConfig = properties.getString(OPTIONAL_WRITE_KEY_COLUMNS);
+        String[] keyColumns = new String[0];
+        if (keyColumnsConfig != null && keyColumnsConfig.length() > 0) {
+            keyColumns = keyColumnsConfig.split(",");
+        }
+        DescriptorProperties descriptorProperties = new DescriptorProperties();
+        descriptorProperties.putProperties(rawProperties);
+        TableSchema physicalSchema =
+                TableSchemaUtils.getPhysicalSchema(context.getCatalogTable().getSchema());
+        return new RocketMQDynamicTableSink(
+                descriptorProperties,
+                physicalSchema,
+                topicName,
+                producerGroup,
+                nameServerAddress,
+                tag,
+                dynamicColumn,
+                fieldDelimiter,
+                encoding,
+                sleepTimeMs,
+                retryTimes,
+                isDynamicTag,
+                isDynamicTagIncluded,
+                writeKeysToBody,
+                keyColumns);
+    }
+
+    private void transformContext(
+            DynamicTableFactory factory, DynamicTableFactory.Context context) {
+        Map<String, String> catalogOptions = context.getCatalogTable().getOptions();
+        Map<String, String> convertedOptions =
+                normalizeOptionCaseAsFactory(factory, catalogOptions);
+        catalogOptions.clear();
+        for (Map.Entry<String, String> entry : convertedOptions.entrySet()) {
+            catalogOptions.put(entry.getKey(), entry.getValue());
+        }
+    }
+
+    private Map<String, String> normalizeOptionCaseAsFactory(
+            Factory factory, Map<String, String> options) {
+        Map<String, String> normalizedOptions = new HashMap<>();
+        Map<String, String> requiredOptionKeysLowerCaseToOriginal =
+                factory.requiredOptions().stream()
+                        .collect(
+                                Collectors.toMap(
+                                        option -> option.key().toLowerCase(), ConfigOption::key));
+        Map<String, String> optionalOptionKeysLowerCaseToOriginal =
+                factory.optionalOptions().stream()
+                        .collect(
+                                Collectors.toMap(
+                                        option -> option.key().toLowerCase(), ConfigOption::key));
+        for (Map.Entry<String, String> entry : options.entrySet()) {
+            final String catalogOptionKey = entry.getKey();
+            final String catalogOptionValue = entry.getValue();
+            normalizedOptions.put(
+                    requiredOptionKeysLowerCaseToOriginal.containsKey(
+                                    catalogOptionKey.toLowerCase())
+                            ? requiredOptionKeysLowerCaseToOriginal.get(
+                                    catalogOptionKey.toLowerCase())
+                            : optionalOptionKeysLowerCaseToOriginal.getOrDefault(
+                                    catalogOptionKey.toLowerCase(), catalogOptionKey),
+                    catalogOptionValue);
+        }
+        return normalizedOptions;
+    }
+}
diff --git a/src/main/java/org/apache/rocketmq/flink/sink/table/RocketMQRowDataConverter.java b/src/main/java/org/apache/rocketmq/flink/sink/table/RocketMQRowDataConverter.java
new file mode 100644
index 0000000..9c5c8af
--- /dev/null
+++ b/src/main/java/org/apache/rocketmq/flink/sink/table/RocketMQRowDataConverter.java
@@ -0,0 +1,217 @@
+/*
+ * 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.sink.table;
+
+import org.apache.rocketmq.common.message.Message;
+
+import org.apache.flink.api.java.typeutils.RowTypeInfo;
+import org.apache.flink.table.data.RowData;
+import org.apache.flink.table.types.DataType;
+import org.apache.flink.types.RowKind;
+
+import org.apache.commons.lang3.StringUtils;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import java.io.Serializable;
+import java.io.UnsupportedEncodingException;
+import java.util.ArrayList;
+import java.util.HashSet;
+import java.util.List;
+import java.util.Set;
+
+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 {
+
+    private static final long serialVersionUID = 1L;
+
+    private static final Logger LOG = LoggerFactory.getLogger(RocketMQRowDataConverter.class);
+
+    private final String topic;
+    private final String tag;
+    private final String dynamicColumn;
+    private final String fieldDelimiter;
+    private final String encoding;
+
+    private final boolean isDynamicTag;
+    private final boolean isDynamicTagIncluded;
+    private final boolean writeKeysToBody;
+    private boolean onlyVarbinary = false;
+
+    private final String[] keyColumns;
+    private final RowTypeInfo rowTypeInfo;
+    private final DataType[] fieldDataTypes;
+
+    private int[] keyFieldIndexes;
+    private int[] tagFieldIndexes;
+    private int[] bodyFieldIndexes;
+    private DataType[] bodyFieldTypes;
+
+    private final boolean hasMetadata;
+    private final int[] metadataPositions;
+
+    public RocketMQRowDataConverter(
+            String topic,
+            String tag,
+            String dynamicColumn,
+            String fieldDelimiter,
+            String encoding,
+            boolean isDynamicTag,
+            boolean isDynamicTagIncluded,
+            boolean writeKeysToBody,
+            String[] keyColumns,
+            RowTypeInfo rowTypeInfo,
+            DataType[] fieldDataTypes,
+            boolean hasMetadata,
+            int[] metadataPositions) {
+        this.topic = topic;
+        this.tag = tag;
+        this.dynamicColumn = dynamicColumn;
+        this.fieldDelimiter = fieldDelimiter;
+        this.encoding = encoding;
+        this.isDynamicTag = isDynamicTag;
+        this.isDynamicTagIncluded = isDynamicTagIncluded;
+        this.writeKeysToBody = writeKeysToBody;
+        this.keyColumns = keyColumns;
+        this.rowTypeInfo = rowTypeInfo;
+        this.fieldDataTypes = fieldDataTypes;
+        this.hasMetadata = hasMetadata;
+        this.metadataPositions = metadataPositions;
+    }
+
+    public void open() {
+        if (rowTypeInfo.getArity() == 1
+                && rowTypeInfo.getFieldTypes()[0].getTypeClass().equals(byte[].class)) {
+            onlyVarbinary = true;
+        }
+        Set<Integer> excludedFields = new HashSet<>();
+        if (keyColumns != null) {
+            keyFieldIndexes = new int[keyColumns.length];
+            for (int index = 0; index < keyColumns.length; index++) {
+                int fieldIndex = rowTypeInfo.getFieldIndex(keyColumns[index]);
+                checkState(
+                        fieldIndex >= 0,
+                        String.format(
+                                "[MetaQConverter] Could not find the message-key column: %s.",
+                                keyColumns[index]));
+                keyFieldIndexes[index] = fieldIndex;
+                if (!writeKeysToBody) {
+                    excludedFields.add(fieldIndex);
+                }
+            }
+        } else {
+            keyFieldIndexes = new int[0];
+        }
+        if (isDynamicTag && dynamicColumn != null) {
+            tagFieldIndexes = new int[1];
+            int fieldIndex = rowTypeInfo.getFieldIndex(dynamicColumn);
+            checkState(
+                    fieldIndex >= 0,
+                    String.format(
+                            "[MetaQConverter] Could not find the tag column: %s.", dynamicColumn));
+            tagFieldIndexes[0] = fieldIndex;
+            if (!isDynamicTagIncluded) {
+                excludedFields.add(fieldIndex);
+            }
+        } else {
+            tagFieldIndexes = new int[0];
+        }
+        bodyFieldIndexes = new int[rowTypeInfo.getArity() - excludedFields.size()];
+        bodyFieldTypes = new DataType[rowTypeInfo.getArity() - excludedFields.size()];
+        int index = 0;
+        for (int num = 0; num < rowTypeInfo.getArity(); num++) {
+            if (!excludedFields.contains(num)) {
+                bodyFieldIndexes[index] = num;
+                bodyFieldTypes[index++] = fieldDataTypes[num];
+            }
+        }
+    }
+
+    public Message convert(RowData row) {
+        if (row.getRowKind() != RowKind.INSERT && row.getRowKind() != RowKind.UPDATE_AFTER) {
+            return null;
+        }
+        Message message = new Message();
+        message.setTopic(topic);
+        List<String> keys = new ArrayList<>();
+        for (int fieldIndex : keyFieldIndexes) {
+            keys.add(row.getString(fieldIndex).toString());
+        }
+        if (keys.size() > 0) {
+            message.setKeys(keys);
+        }
+        if (!isDynamicTag) {
+            if (tag != null && tag.length() > 0) {
+                message.setTags(tag);
+            }
+        } else {
+            checkState(tagFieldIndexes.length > 0, "No message tag column set.");
+            message.setTags(row.getString(tagFieldIndexes[0]).toString());
+        }
+        if (onlyVarbinary) {
+            message.setBody(row.getBinary(0));
+            message.setWaitStoreMsgOK(true);
+        } else {
+            Object[] values = new Object[bodyFieldIndexes.length];
+            for (int index = 0; index < bodyFieldIndexes.length; index++) {
+                values[index] =
+                        RowData.createFieldGetter(
+                                        bodyFieldTypes[index].getLogicalType(),
+                                        bodyFieldIndexes[index])
+                                .getFieldOrNull(row);
+            }
+            try {
+                message.setBody(StringUtils.join(values, fieldDelimiter).getBytes(encoding));
+                message.setWaitStoreMsgOK(true);
+            } catch (UnsupportedEncodingException e) {
+                LOG.error(
+                        String.format(
+                                "Unsupported ''{%s}'' encoding charset. Check the encoding configItem in the DDL.",
+                                encoding),
+                        e);
+            }
+        }
+        if (hasMetadata) {
+            String messageKeys = readMetadata(row, KEYS);
+            message.setKeys(messageKeys);
+            message.setTags(readMetadata(row, TAGS));
+        }
+        return message;
+    }
+
+    @SuppressWarnings("unchecked")
+    private <T> T readMetadata(RowData consumedRow, WritableMetadata metadata) {
+        final int pos = metadataPositions[metadata.ordinal()];
+        if (pos < 0) {
+            return null;
+        }
+        return (T) metadata.converter.read(consumedRow, pos);
+    }
+
+    // --------------------------------------------------------------------------------------------
+
+    interface MetadataConverter extends Serializable {
+        Object read(RowData consumedRow, int pos);
+    }
+}
diff --git a/src/main/java/org/apache/rocketmq/flink/sink/table/RocketMQRowDataSink.java b/src/main/java/org/apache/rocketmq/flink/sink/table/RocketMQRowDataSink.java
new file mode 100644
index 0000000..ac01829
--- /dev/null
+++ b/src/main/java/org/apache/rocketmq/flink/sink/table/RocketMQRowDataSink.java
@@ -0,0 +1,61 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one or more contributor license
+ * agreements. See the NOTICE file distributed with this work for additional information regarding
+ * copyright ownership. The ASF licenses this file to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance with the License. You may obtain a
+ * copy of the License at
+ *
+ * <p>http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * <p>Unless required by applicable law or agreed to in writing, software distributed under the
+ * License is distributed on an "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either
+ * express or implied. See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.rocketmq.flink.sink.table;
+
+import org.apache.rocketmq.common.message.Message;
+import org.apache.rocketmq.flink.legacy.RocketMQSink;
+
+import org.apache.flink.api.common.functions.RuntimeContext;
+import org.apache.flink.configuration.Configuration;
+import org.apache.flink.streaming.api.functions.sink.RichSinkFunction;
+import org.apache.flink.table.data.RowData;
+
+/** RocketMQRowDataSink helps for writing the converted row data of table to RocketMQ messages. */
+public class RocketMQRowDataSink extends RichSinkFunction<RowData> {
+
+    private static final long serialVersionUID = 1L;
+
+    private final RocketMQSink sink;
+    private final RocketMQRowDataConverter converter;
+
+    public RocketMQRowDataSink(RocketMQSink sink, RocketMQRowDataConverter converter) {
+        this.sink = sink;
+        this.converter = converter;
+    }
+
+    @Override
+    public void open(Configuration configuration) throws Exception {
+        sink.open(configuration);
+        converter.open();
+    }
+
+    @Override
+    public void setRuntimeContext(RuntimeContext runtimeContext) {
+        sink.setRuntimeContext(runtimeContext);
+    }
+
+    @Override
+    public void invoke(RowData rowData, Context context) throws Exception {
+        Message message = converter.convert(rowData);
+        if (message != null) {
+            sink.invoke(message, context);
+        }
+    }
+
+    @Override
+    public void close() {
+        sink.close();
+    }
+}
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
index 990e28b..36e3dc6 100644
--- a/src/main/java/org/apache/rocketmq/flink/source/table/RocketMQDynamicTableSourceFactory.java
+++ b/src/main/java/org/apache/rocketmq/flink/source/table/RocketMQDynamicTableSourceFactory.java
@@ -42,22 +42,22 @@ import java.util.TimeZone;
 import java.util.stream.Collectors;
 
 import static org.apache.flink.table.factories.FactoryUtil.createTableFactoryHelper;
-import static org.apache.rocketmq.flink.source.common.RocketMQOptions.CONSUMER_GROUP;
-import static org.apache.rocketmq.flink.source.common.RocketMQOptions.NAME_SERVER_ADDRESS;
-import static org.apache.rocketmq.flink.source.common.RocketMQOptions.OPTIONAL_COLUMN_ERROR_DEBUG;
-import static org.apache.rocketmq.flink.source.common.RocketMQOptions.OPTIONAL_ENCODING;
-import static org.apache.rocketmq.flink.source.common.RocketMQOptions.OPTIONAL_END_TIME;
-import static org.apache.rocketmq.flink.source.common.RocketMQOptions.OPTIONAL_FIELD_DELIMITER;
-import static org.apache.rocketmq.flink.source.common.RocketMQOptions.OPTIONAL_LENGTH_CHECK;
-import static org.apache.rocketmq.flink.source.common.RocketMQOptions.OPTIONAL_LINE_DELIMITER;
-import static org.apache.rocketmq.flink.source.common.RocketMQOptions.OPTIONAL_PARTITION_DISCOVERY_INTERVAL_MS;
-import static org.apache.rocketmq.flink.source.common.RocketMQOptions.OPTIONAL_START_MESSAGE_OFFSET;
-import static org.apache.rocketmq.flink.source.common.RocketMQOptions.OPTIONAL_START_TIME;
-import static org.apache.rocketmq.flink.source.common.RocketMQOptions.OPTIONAL_START_TIME_MILLS;
-import static org.apache.rocketmq.flink.source.common.RocketMQOptions.OPTIONAL_TAG;
-import static org.apache.rocketmq.flink.source.common.RocketMQOptions.OPTIONAL_TIME_ZONE;
-import static org.apache.rocketmq.flink.source.common.RocketMQOptions.OPTIONAL_USE_NEW_API;
-import static org.apache.rocketmq.flink.source.common.RocketMQOptions.TOPIC;
+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_COLUMN_ERROR_DEBUG;
+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_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;
 
 /**
  * Defines the {@link DynamicTableSourceFactory} implementation to create {@link
diff --git a/src/main/java/org/apache/rocketmq/flink/source/util/StringSerializer.java b/src/main/java/org/apache/rocketmq/flink/source/util/StringSerializer.java
index c0a67b8..e2bcf22 100644
--- a/src/main/java/org/apache/rocketmq/flink/source/util/StringSerializer.java
+++ b/src/main/java/org/apache/rocketmq/flink/source/util/StringSerializer.java
@@ -17,7 +17,6 @@
 
 package org.apache.rocketmq.flink.source.util;
 
-import java.util.Base64;
 import org.apache.flink.table.data.DecimalData;
 import org.apache.flink.table.data.binary.BinaryStringData;
 import org.apache.flink.table.data.util.DataFormatConverters;
@@ -30,6 +29,7 @@ import java.math.BigInteger;
 import java.sql.Date;
 import java.sql.Time;
 import java.sql.Timestamp;
+import java.util.Base64;
 import java.util.Set;
 
 /** String serializer. */
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 32de8b2..b164722 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,4 +13,5 @@
 # See the License for the specific language governing permissions and
 # limitations under the License.
 
-org.apache.rocketmq.flink.source.table.RocketMQDynamicTableSourceFactory
\ No newline at end of file
+org.apache.rocketmq.flink.source.table.RocketMQDynamicTableSourceFactory
+org.apache.rocketmq.flink.sink.table.RocketMQDynamicTableSinkFactory
\ No newline at end of file
diff --git a/src/test/java/org/apache/rocketmq/flink/source/table/RocketMQDynamicTableSourceFactoryTest.java b/src/test/java/org/apache/rocketmq/flink/sink/table/RocketMQDynamicTableSinkFactoryTest.java
similarity index 64%
copy from src/test/java/org/apache/rocketmq/flink/source/table/RocketMQDynamicTableSourceFactoryTest.java
copy to src/test/java/org/apache/rocketmq/flink/sink/table/RocketMQDynamicTableSinkFactoryTest.java
index 184a23f..b06695a 100644
--- a/src/test/java/org/apache/rocketmq/flink/source/table/RocketMQDynamicTableSourceFactoryTest.java
+++ b/src/test/java/org/apache/rocketmq/flink/sink/table/RocketMQDynamicTableSinkFactoryTest.java
@@ -16,9 +16,9 @@
  * limitations under the License.
  */
 
-package org.apache.rocketmq.flink.source.table;
+package org.apache.rocketmq.flink.sink.table;
 
-import org.apache.rocketmq.flink.source.common.RocketMQOptions;
+import org.apache.rocketmq.flink.common.RocketMQOptions;
 
 import org.apache.flink.configuration.Configuration;
 import org.apache.flink.table.api.Schema;
@@ -28,7 +28,7 @@ import org.apache.flink.table.catalog.Column;
 import org.apache.flink.table.catalog.ObjectIdentifier;
 import org.apache.flink.table.catalog.ResolvedCatalogTable;
 import org.apache.flink.table.catalog.ResolvedSchema;
-import org.apache.flink.table.connector.source.DynamicTableSource;
+import org.apache.flink.table.connector.sink.DynamicTableSink;
 import org.apache.flink.table.factories.FactoryUtil;
 
 import org.junit.Test;
@@ -42,8 +42,8 @@ import static org.apache.flink.table.api.DataTypes.STRING;
 import static org.junit.Assert.assertEquals;
 import static org.junit.Assert.assertTrue;
 
-/** Tests for {@link RocketMQDynamicTableSourceFactory}. */
-public class RocketMQDynamicTableSourceFactoryTest {
+/** Tests for {@link org.apache.rocketmq.flink.sink.table.RocketMQDynamicTableSinkFactory}. */
+public class RocketMQDynamicTableSinkFactoryTest {
 
     private static final ResolvedSchema SCHEMA =
             new ResolvedSchema(
@@ -52,60 +52,56 @@ public class RocketMQDynamicTableSourceFactoryTest {
                     null);
 
     private static final String IDENTIFIER = "rocketmq";
-    private static final String TOPIC = "test_source";
-    private static final String CONSUMER_GROUP = "test_consumer";
-    private static final String NAME_SERVER_ADDRESS = "127.0.0.1:9876";
+    private static final String TOPIC = "test_sink";
+    private static final String PRODUCER_GROUP = "test_producer";
+    private static final String NAME_SERVER_ADDRESS =
+            "http://${instanceId}.${region}.mq-internal.aliyuncs.com:8080";
 
     @Test
-    public void testRocketMQDynamicTableSourceWithLegalOption() {
+    public void testRocketMQDynamicTableSinkWithLegalOption() {
         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.PRODUCER_GROUP.key(), PRODUCER_GROUP);
         options.put(RocketMQOptions.NAME_SERVER_ADDRESS.key(), NAME_SERVER_ADDRESS);
-        final DynamicTableSource tableSource = createTableSource(options);
-        assertTrue(tableSource instanceof RocketMQScanTableSource);
-        assertEquals(RocketMQScanTableSource.class.getName(), tableSource.asSummaryString());
+        final DynamicTableSink tableSink = createDynamicTableSink(options);
+        assertTrue(tableSink instanceof RocketMQDynamicTableSink);
+        assertEquals(RocketMQDynamicTableSink.class.getName(), tableSink.asSummaryString());
     }
 
     @Test(expected = ValidationException.class)
-    public void testRocketMQDynamicTableSourceWithoutRequiredOption() {
+    public void testRocketMQDynamicTableSinkWithoutRequiredOption() {
         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.PRODUCER_GROUP.key(), PRODUCER_GROUP);
         options.put(RocketMQOptions.OPTIONAL_TAG.key(), "test_tag");
-        createTableSource(options);
+        createDynamicTableSink(options);
     }
 
     @Test(expected = ValidationException.class)
-    public void testRocketMQDynamicTableSourceWithUnknownOption() {
+    public void testRocketMQDynamicTableSinkWithUnknownOption() {
         final Map<String, String> options = new HashMap<>();
         options.put(RocketMQOptions.TOPIC.key(), TOPIC);
-        options.put(RocketMQOptions.CONSUMER_GROUP.key(), CONSUMER_GROUP);
+        options.put(RocketMQOptions.PRODUCER_GROUP.key(), PRODUCER_GROUP);
         options.put(RocketMQOptions.NAME_SERVER_ADDRESS.key(), NAME_SERVER_ADDRESS);
         options.put("unknown", "test_option");
-        createTableSource(options);
+        createDynamicTableSink(options);
     }
 
-    private static DynamicTableSource createTableSource(
-            Map<String, String> options, Configuration conf) {
-        return FactoryUtil.createTableSource(
+    private static DynamicTableSink createDynamicTableSink(Map<String, String> options) {
+        return FactoryUtil.createTableSink(
                 null,
-                ObjectIdentifier.of("default", "default", IDENTIFIER),
+                ObjectIdentifier.of("default", "default", "mq"),
                 new ResolvedCatalogTable(
                         CatalogTable.of(
                                 Schema.newBuilder().fromResolvedSchema(SCHEMA).build(),
-                                "mock source",
+                                "mock sink",
                                 Collections.emptyList(),
                                 options),
                         SCHEMA),
-                conf,
-                RocketMQDynamicTableSourceFactory.class.getClassLoader(),
+                new Configuration(),
+                RocketMQDynamicTableSinkFactory.class.getClassLoader(),
                 false);
     }
-
-    private static DynamicTableSource createTableSource(Map<String, String> options) {
-        return createTableSource(options, new Configuration());
-    }
 }
diff --git a/src/test/java/org/apache/rocketmq/flink/source/table/RocketMQDynamicTableSourceFactoryTest.java b/src/test/java/org/apache/rocketmq/flink/source/table/RocketMQDynamicTableSourceFactoryTest.java
index 184a23f..377e63b 100644
--- a/src/test/java/org/apache/rocketmq/flink/source/table/RocketMQDynamicTableSourceFactoryTest.java
+++ b/src/test/java/org/apache/rocketmq/flink/source/table/RocketMQDynamicTableSourceFactoryTest.java
@@ -18,7 +18,7 @@
 
 package org.apache.rocketmq.flink.source.table;
 
-import org.apache.rocketmq.flink.source.common.RocketMQOptions;
+import org.apache.rocketmq.flink.common.RocketMQOptions;
 
 import org.apache.flink.configuration.Configuration;
 import org.apache.flink.table.api.Schema;

[rocketmq-flink] 29/33: [#715] Support the RocketMQ TableSource based on the legacy Source implementation (#779)

Posted by du...@apache.org.
This is an automated email from the ASF dual-hosted git repository.

duhengforever pushed a commit to branch main
in repository https://gitbox.apache.org/repos/asf/rocketmq-flink.git

commit 4edc4daf5e7f6dca981744a6f3f963103916d397
Author: SteNicholas <pr...@163.com>
AuthorDate: Wed Aug 18 15:03:04 2021 +0800

    [#715] Support the RocketMQ TableSource based on the legacy Source implementation (#779)
---
 pom.xml                                            |   4 +-
 .../rocketmq/flink/legacy/RocketMQConfig.java      |  22 +-
 .../apache/rocketmq/flink/legacy/RocketMQSink.java |  22 +-
 ...etMQSource.java => RocketMQSourceFunction.java} |  28 +-
 .../rocketmq/flink/legacy/RunningChecker.java      |  22 +-
 .../common/selector/DefaultTopicSelector.java      |  22 +-
 .../common/selector/SimpleTopicSelector.java       |  22 +-
 .../legacy/common/selector/TopicSelector.java      |  22 +-
 .../KeyValueDeserializationSchema.java             |  22 +-
 .../serialization/KeyValueSerializationSchema.java |  22 +-
 .../RowKeyValueDeserializationSchema.java          | 407 +++++++++++++++++++++
 .../SimpleKeyValueDeserializationSchema.java       |  22 +-
 .../SimpleKeyValueSerializationSchema.java         |  22 +-
 .../flink/legacy/common/util/MetricUtils.java      |  22 +-
 .../flink/legacy/common/util/RetryUtil.java        |  22 +-
 .../flink/legacy/common/util/RocketMQUtils.java    |  22 +-
 .../flink/legacy/common/util/TestUtils.java        |  22 +-
 .../flink/legacy/example/RocketMQFlinkExample.java |   5 +-
 .../flink/source/common/RocketMQOptions.java       |   3 +
 .../deserializer/RowDeserializationSchema.java     |   2 +-
 .../table/RocketMQDynamicTableSourceFactory.java   |   6 +-
 .../source/table/RocketMQScanTableSource.java      |  65 +++-
 .../rocketmq/flink/legacy/RocketMQSinkTest.java    |  24 +-
 .../rocketmq/flink/legacy/RocketMQSourceTest.java  |  28 +-
 .../common/selector/DefaultTopicSelectorTest.java  |  24 +-
 .../common/selector/SimpleTopicSelectorTest.java   |  24 +-
 .../RowKeyValueDeserializationSchemaTest.java      |  50 +++
 .../SimpleKeyValueSerializationSchemaTest.java     |  24 +-
 .../RocketMQDynamicTableSourceFactoryTest.java     | 111 ++++++
 29 files changed, 887 insertions(+), 226 deletions(-)

diff --git a/pom.xml b/pom.xml
index abec905..0675bb7 100644
--- a/pom.xml
+++ b/pom.xml
@@ -34,7 +34,7 @@
         <maven.compiler.source>1.8</maven.compiler.source>
         <maven.compiler.target>1.8</maven.compiler.target>
         <rocketmq.version>4.7.1</rocketmq.version>
-        <flink.version>1.13.0</flink.version>
+        <flink.version>1.13.1</flink.version>
         <commons-lang.version>2.5</commons-lang.version>
         <scala.binary.version>2.11</scala.binary.version>
         <spotless.version>2.4.2</spotless.version>
@@ -45,11 +45,13 @@
             <groupId>org.apache.flink</groupId>
             <artifactId>flink-java</artifactId>
             <version>${flink.version}</version>
+            <scope>provided</scope>
         </dependency>
         <dependency>
             <groupId>org.apache.flink</groupId>
             <artifactId>flink-streaming-java_${scala.binary.version}</artifactId>
             <version>${flink.version}</version>
+            <scope>provided</scope>
         </dependency>
         <dependency>
             <groupId>org.apache.flink</groupId>
diff --git a/src/main/java/org/apache/rocketmq/flink/legacy/RocketMQConfig.java b/src/main/java/org/apache/rocketmq/flink/legacy/RocketMQConfig.java
index 5c19b7a..fc257a1 100644
--- a/src/main/java/org/apache/rocketmq/flink/legacy/RocketMQConfig.java
+++ b/src/main/java/org/apache/rocketmq/flink/legacy/RocketMQConfig.java
@@ -1,15 +1,17 @@
-/**
- * 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
+/*
+ * 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
  *
- * <p>http://www.apache.org/licenses/LICENSE-2.0
+ *     http://www.apache.org/licenses/LICENSE-2.0
  *
- * <p>Unless required by applicable law or agreed to in writing, software distributed under the
- * License is distributed on an "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either
- * express or implied. See the License for the specific language governing permissions and
+ * 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;
diff --git a/src/main/java/org/apache/rocketmq/flink/legacy/RocketMQSink.java b/src/main/java/org/apache/rocketmq/flink/legacy/RocketMQSink.java
index f91a684..b6e1793 100644
--- a/src/main/java/org/apache/rocketmq/flink/legacy/RocketMQSink.java
+++ b/src/main/java/org/apache/rocketmq/flink/legacy/RocketMQSink.java
@@ -1,15 +1,17 @@
-/**
- * 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
+/*
+ * 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
  *
- * <p>http://www.apache.org/licenses/LICENSE-2.0
+ *     http://www.apache.org/licenses/LICENSE-2.0
  *
- * <p>Unless required by applicable law or agreed to in writing, software distributed under the
- * License is distributed on an "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either
- * express or implied. See the License for the specific language governing permissions and
+ * 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;
diff --git a/src/main/java/org/apache/rocketmq/flink/legacy/RocketMQSource.java b/src/main/java/org/apache/rocketmq/flink/legacy/RocketMQSourceFunction.java
similarity index 96%
rename from src/main/java/org/apache/rocketmq/flink/legacy/RocketMQSource.java
rename to src/main/java/org/apache/rocketmq/flink/legacy/RocketMQSourceFunction.java
index 84260b6..8821a6d 100644
--- a/src/main/java/org/apache/rocketmq/flink/legacy/RocketMQSource.java
+++ b/src/main/java/org/apache/rocketmq/flink/legacy/RocketMQSourceFunction.java
@@ -1,15 +1,17 @@
-/**
- * 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
+/*
+ * 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
  *
- * <p>http://www.apache.org/licenses/LICENSE-2.0
+ *     http://www.apache.org/licenses/LICENSE-2.0
  *
- * <p>Unless required by applicable law or agreed to in writing, software distributed under the
- * License is distributed on an "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either
- * express or implied. See the License for the specific language governing permissions and
+ * 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;
@@ -82,12 +84,12 @@ import static org.apache.rocketmq.flink.legacy.common.util.RocketMQUtils.getLong
  * guarantees when checkpoints are enabled. Otherwise, the source doesn't provide any reliability
  * guarantees.
  */
-public class RocketMQSource<OUT> extends RichParallelSourceFunction<OUT>
+public class RocketMQSourceFunction<OUT> extends RichParallelSourceFunction<OUT>
         implements CheckpointedFunction, CheckpointListener, ResultTypeQueryable<OUT> {
 
     private static final long serialVersionUID = 1L;
 
-    private static final Logger log = LoggerFactory.getLogger(RocketMQSource.class);
+    private static final Logger log = LoggerFactory.getLogger(RocketMQSourceFunction.class);
     private static final String OFFSETS_STATE_NAME = "topic-partition-offset-states";
     private RunningChecker runningChecker;
     private transient DefaultMQPullConsumer consumer;
@@ -115,7 +117,7 @@ public class RocketMQSource<OUT> extends RichParallelSourceFunction<OUT>
 
     private Meter tpsMetric;
 
-    public RocketMQSource(KeyValueDeserializationSchema<OUT> schema, Properties props) {
+    public RocketMQSourceFunction(KeyValueDeserializationSchema<OUT> schema, Properties props) {
         this.schema = schema;
         this.props = props;
     }
diff --git a/src/main/java/org/apache/rocketmq/flink/legacy/RunningChecker.java b/src/main/java/org/apache/rocketmq/flink/legacy/RunningChecker.java
index c48361a..f36b727 100644
--- a/src/main/java/org/apache/rocketmq/flink/legacy/RunningChecker.java
+++ b/src/main/java/org/apache/rocketmq/flink/legacy/RunningChecker.java
@@ -1,15 +1,17 @@
-/**
- * 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
+/*
+ * 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
  *
- * <p>http://www.apache.org/licenses/LICENSE-2.0
+ *     http://www.apache.org/licenses/LICENSE-2.0
  *
- * <p>Unless required by applicable law or agreed to in writing, software distributed under the
- * License is distributed on an "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either
- * express or implied. See the License for the specific language governing permissions and
+ * 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;
diff --git a/src/main/java/org/apache/rocketmq/flink/legacy/common/selector/DefaultTopicSelector.java b/src/main/java/org/apache/rocketmq/flink/legacy/common/selector/DefaultTopicSelector.java
index 6be5218..128b19e 100644
--- a/src/main/java/org/apache/rocketmq/flink/legacy/common/selector/DefaultTopicSelector.java
+++ b/src/main/java/org/apache/rocketmq/flink/legacy/common/selector/DefaultTopicSelector.java
@@ -1,15 +1,17 @@
-/**
- * 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
+/*
+ * 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
  *
- * <p>http://www.apache.org/licenses/LICENSE-2.0
+ *     http://www.apache.org/licenses/LICENSE-2.0
  *
- * <p>Unless required by applicable law or agreed to in writing, software distributed under the
- * License is distributed on an "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either
- * express or implied. See the License for the specific language governing permissions and
+ * 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.common.selector;
diff --git a/src/main/java/org/apache/rocketmq/flink/legacy/common/selector/SimpleTopicSelector.java b/src/main/java/org/apache/rocketmq/flink/legacy/common/selector/SimpleTopicSelector.java
index 674b5a0..dcdaa2f 100644
--- a/src/main/java/org/apache/rocketmq/flink/legacy/common/selector/SimpleTopicSelector.java
+++ b/src/main/java/org/apache/rocketmq/flink/legacy/common/selector/SimpleTopicSelector.java
@@ -1,15 +1,17 @@
-/**
- * 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
+/*
+ * 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
  *
- * <p>http://www.apache.org/licenses/LICENSE-2.0
+ *     http://www.apache.org/licenses/LICENSE-2.0
  *
- * <p>Unless required by applicable law or agreed to in writing, software distributed under the
- * License is distributed on an "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either
- * express or implied. See the License for the specific language governing permissions and
+ * 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.common.selector;
diff --git a/src/main/java/org/apache/rocketmq/flink/legacy/common/selector/TopicSelector.java b/src/main/java/org/apache/rocketmq/flink/legacy/common/selector/TopicSelector.java
index 581dadc..a70c599 100644
--- a/src/main/java/org/apache/rocketmq/flink/legacy/common/selector/TopicSelector.java
+++ b/src/main/java/org/apache/rocketmq/flink/legacy/common/selector/TopicSelector.java
@@ -1,15 +1,17 @@
-/**
- * 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
+/*
+ * 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
  *
- * <p>http://www.apache.org/licenses/LICENSE-2.0
+ *     http://www.apache.org/licenses/LICENSE-2.0
  *
- * <p>Unless required by applicable law or agreed to in writing, software distributed under the
- * License is distributed on an "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either
- * express or implied. See the License for the specific language governing permissions and
+ * 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.common.selector;
diff --git a/src/main/java/org/apache/rocketmq/flink/legacy/common/serialization/KeyValueDeserializationSchema.java b/src/main/java/org/apache/rocketmq/flink/legacy/common/serialization/KeyValueDeserializationSchema.java
index 4cc8c61..8d0c778 100644
--- a/src/main/java/org/apache/rocketmq/flink/legacy/common/serialization/KeyValueDeserializationSchema.java
+++ b/src/main/java/org/apache/rocketmq/flink/legacy/common/serialization/KeyValueDeserializationSchema.java
@@ -1,15 +1,17 @@
-/**
- * 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
+/*
+ * 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
  *
- * <p>http://www.apache.org/licenses/LICENSE-2.0
+ *     http://www.apache.org/licenses/LICENSE-2.0
  *
- * <p>Unless required by applicable law or agreed to in writing, software distributed under the
- * License is distributed on an "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either
- * express or implied. See the License for the specific language governing permissions and
+ * 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.common.serialization;
diff --git a/src/main/java/org/apache/rocketmq/flink/legacy/common/serialization/KeyValueSerializationSchema.java b/src/main/java/org/apache/rocketmq/flink/legacy/common/serialization/KeyValueSerializationSchema.java
index 66b2e29..0000772 100644
--- a/src/main/java/org/apache/rocketmq/flink/legacy/common/serialization/KeyValueSerializationSchema.java
+++ b/src/main/java/org/apache/rocketmq/flink/legacy/common/serialization/KeyValueSerializationSchema.java
@@ -1,15 +1,17 @@
-/**
- * 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
+/*
+ * 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
  *
- * <p>http://www.apache.org/licenses/LICENSE-2.0
+ *     http://www.apache.org/licenses/LICENSE-2.0
  *
- * <p>Unless required by applicable law or agreed to in writing, software distributed under the
- * License is distributed on an "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either
- * express or implied. See the License for the specific language governing permissions and
+ * 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.common.serialization;
diff --git a/src/main/java/org/apache/rocketmq/flink/legacy/common/serialization/RowKeyValueDeserializationSchema.java b/src/main/java/org/apache/rocketmq/flink/legacy/common/serialization/RowKeyValueDeserializationSchema.java
new file mode 100644
index 0000000..bc43b1c
--- /dev/null
+++ b/src/main/java/org/apache/rocketmq/flink/legacy/common/serialization/RowKeyValueDeserializationSchema.java
@@ -0,0 +1,407 @@
+/*
+ * 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.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;
+
+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.table.api.TableSchema;
+import org.apache.flink.table.data.GenericRowData;
+import org.apache.flink.table.data.RowData;
+import org.apache.flink.table.descriptors.DescriptorProperties;
+import org.apache.flink.table.runtime.typeutils.InternalTypeInfo;
+import org.apache.flink.table.types.DataType;
+import org.apache.flink.table.types.logical.RowType;
+
+import org.apache.commons.lang3.StringEscapeUtils;
+import org.apache.commons.lang3.StringUtils;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import java.io.UnsupportedEncodingException;
+import java.util.HashMap;
+import java.util.HashSet;
+import java.util.Map;
+
+/**
+ * * The row based implementation of {@link KeyValueDeserializationSchema} for the deserialization
+ * of message key and value..
+ */
+public class RowKeyValueDeserializationSchema implements KeyValueDeserializationSchema<RowData> {
+
+    private static final long serialVersionUID = -1L;
+    private static final Logger logger =
+            LoggerFactory.getLogger(RowKeyValueDeserializationSchema.class);
+
+    private transient TableSchema tableSchema;
+    private final DirtyDataStrategy formatErrorStrategy;
+    private final DirtyDataStrategy fieldMissingStrategy;
+    private final DirtyDataStrategy fieldIncrementStrategy;
+    private final String encoding;
+    private final String fieldDelimiter;
+    private final boolean columnErrorDebug;
+    private final int columnSize;
+    private final ByteSerializer.ValueType[] fieldTypes;
+    private final transient DataType[] fieldDataTypes;
+    private final Map<String, Integer> columnIndexMapping;
+    private long lastLogExceptionTime;
+    private long lastLogHandleFieldTime;
+
+    private static final int DEFAULT_LOG_INTERVAL_MS = 60 * 1000;
+
+    public RowKeyValueDeserializationSchema(
+            TableSchema tableSchema,
+            DirtyDataStrategy formatErrorStrategy,
+            DirtyDataStrategy fieldMissingStrategy,
+            DirtyDataStrategy fieldIncrementStrategy,
+            String encoding,
+            String fieldDelimiter,
+            boolean columnErrorDebug,
+            Map<String, String> properties) {
+        this.tableSchema = tableSchema;
+        this.formatErrorStrategy = formatErrorStrategy;
+        this.fieldMissingStrategy = fieldMissingStrategy;
+        this.fieldIncrementStrategy = fieldIncrementStrategy;
+        this.columnErrorDebug = columnErrorDebug;
+        this.encoding = encoding;
+        this.fieldDelimiter = StringEscapeUtils.unescapeJava(fieldDelimiter);
+        this.columnSize = tableSchema.getFieldNames().length;
+        this.fieldTypes = new ByteSerializer.ValueType[columnSize];
+        this.columnIndexMapping = new HashMap<>();
+        for (int index = 0; index < columnSize; index++) {
+            this.columnIndexMapping.put(tableSchema.getFieldNames()[index], index);
+        }
+        for (int index = 0; index < columnSize; index++) {
+            ByteSerializer.ValueType type =
+                    ByteSerializer.getTypeIndex(tableSchema.getFieldTypes()[index].getTypeClass());
+            this.fieldTypes[index] = type;
+        }
+
+        DescriptorProperties descriptorProperties = new DescriptorProperties();
+        descriptorProperties.putProperties(properties);
+        this.fieldDataTypes = tableSchema.getFieldDataTypes();
+        this.lastLogExceptionTime = System.currentTimeMillis();
+        this.lastLogHandleFieldTime = System.currentTimeMillis();
+    }
+
+    @Override
+    public RowData deserializeKeyAndValue(byte[] key, byte[] value) {
+        if (isOnlyHaveVarbinaryDataField()) {
+            GenericRowData rowData = new GenericRowData(columnSize);
+            rowData.setField(0, value);
+            return rowData;
+        } else {
+            if (value == null) {
+                logger.info("Deserialize empty BytesMessage body, ignore the empty message.");
+                return null;
+            }
+            return deserializeValue(value);
+        }
+    }
+
+    @Override
+    public TypeInformation<RowData> getProducedType() {
+        return InternalTypeInfo.of((RowType) tableSchema.toRowDataType().getLogicalType());
+    }
+
+    private boolean isOnlyHaveVarbinaryDataField() {
+        if (columnSize == 1) {
+            return isByteArrayType(tableSchema.getFieldNames()[0]);
+        }
+        return false;
+    }
+
+    private RowData deserializeValue(byte[] value) {
+        String body;
+        try {
+            body = new String(value, encoding);
+        } catch (UnsupportedEncodingException e) {
+            throw new RuntimeException(e);
+        }
+        String[] data = StringUtils.splitPreserveAllTokens(body, fieldDelimiter);
+        if (columnSize == 1) {
+            data = new String[1];
+            data[0] = body;
+        }
+        if (data.length < columnSize) {
+            data = handleFieldMissing(data);
+        } else if (data.length > columnSize) {
+            data = handleFieldIncrement(data);
+        }
+        if (data == null) {
+            return null;
+        }
+        GenericRowData rowData = new GenericRowData(columnSize);
+        boolean skip = false;
+        for (int index = 0; index < columnSize; index++) {
+            try {
+                String fieldValue = getValue(data, body, index);
+                rowData.setField(
+                        index,
+                        StringSerializer.deserialize(
+                                fieldValue,
+                                fieldTypes[index],
+                                fieldDataTypes[index],
+                                new HashSet<>()));
+            } catch (Exception e) {
+                skip = handleException(rowData, index, data, e);
+            }
+        }
+        if (skip) {
+            return null;
+        }
+        return rowData;
+    }
+
+    private String getValue(String[] data, String line, int index) {
+        String fieldValue = null;
+        if (columnSize == 1) {
+            fieldValue = line;
+        } else {
+            if (index < data.length) {
+                fieldValue = data[index];
+            }
+        }
+        return fieldValue;
+    }
+
+    private boolean isByteArrayType(String fieldName) {
+        TypeInformation<?> typeInformation =
+                tableSchema.getFieldTypes()[columnIndexMapping.get(fieldName)];
+        if (typeInformation != null) {
+            ByteSerializer.ValueType valueType =
+                    ByteSerializer.getTypeIndex(typeInformation.getTypeClass());
+            return valueType == ByteSerializer.ValueType.V_ByteArray;
+        }
+        return false;
+    }
+
+    private boolean handleException(GenericRowData row, int index, Object[] data, Exception e) {
+        boolean skip = false;
+        switch (formatErrorStrategy) {
+            case SKIP:
+                long now = System.currentTimeMillis();
+                if (columnErrorDebug || now - lastLogExceptionTime > DEFAULT_LOG_INTERVAL_MS) {
+                    logger.warn(
+                            "Data format error, field type: "
+                                    + fieldTypes[index]
+                                    + "field data: "
+                                    + data[index]
+                                    + ", index: "
+                                    + index
+                                    + ", data: ["
+                                    + StringUtils.join(data, ",")
+                                    + "]",
+                            e);
+                    lastLogExceptionTime = now;
+                }
+                skip = true;
+                break;
+            case SKIP_SILENT:
+                skip = true;
+                break;
+            default:
+            case CUT:
+            case NULL:
+            case PAD:
+                row.setField(index, null);
+                break;
+            case EXCEPTION:
+                throw new RuntimeException(e);
+        }
+
+        return skip;
+    }
+
+    private String[] handleFieldMissing(String[] data) {
+        String fieldMissingMessage =
+                String.format(
+                        "Field missing exception, table column number: %d, data column number: %d, data field number: %d, data: [%s].",
+                        columnSize, columnSize, data.length, StringUtils.join(data, ","));
+        switch (fieldMissingStrategy) {
+            default:
+            case SKIP:
+                long now = System.currentTimeMillis();
+                if (columnErrorDebug || now - lastLogHandleFieldTime > DEFAULT_LOG_INTERVAL_MS) {
+                    logger.warn(fieldMissingMessage);
+                    lastLogHandleFieldTime = now;
+                }
+                return null;
+            case SKIP_SILENT:
+                return null;
+            case CUT:
+            case NULL:
+            case PAD:
+                return data;
+            case EXCEPTION:
+                logger.error(fieldMissingMessage);
+                throw new RuntimeException(fieldMissingMessage);
+        }
+    }
+
+    private String[] handleFieldIncrement(String[] data) {
+        String fieldIncrementMessage =
+                String.format(
+                        "Field increment exception, table column number: %d, data column number: %d, data field number: %d, data: [%s].",
+                        columnSize, columnSize, data.length, StringUtils.join(data, ","));
+        switch (fieldIncrementStrategy) {
+            case SKIP:
+                long now = System.currentTimeMillis();
+                if (columnErrorDebug || now - lastLogHandleFieldTime > DEFAULT_LOG_INTERVAL_MS) {
+                    logger.warn(fieldIncrementMessage);
+                    lastLogHandleFieldTime = now;
+                }
+                return null;
+            case SKIP_SILENT:
+                return null;
+            default:
+            case CUT:
+            case NULL:
+            case PAD:
+                return data;
+            case EXCEPTION:
+                logger.error(fieldIncrementMessage);
+                throw new RuntimeException(fieldIncrementMessage);
+        }
+    }
+
+    /** Builder of {@link RowKeyValueDeserializationSchema}. */
+    public static class Builder {
+
+        private TableSchema schema;
+        private DirtyDataStrategy formatErrorStrategy = DirtyDataStrategy.SKIP;
+        private DirtyDataStrategy fieldMissingStrategy = DirtyDataStrategy.SKIP;
+        private DirtyDataStrategy fieldIncrementStrategy = DirtyDataStrategy.CUT;
+        private String encoding = "UTF-8";
+        private String fieldDelimiter = "\u0001";
+        private boolean columnErrorDebug = false;
+        private Map<String, String> properties;
+
+        public Builder() {}
+
+        public Builder setTableSchema(TableSchema tableSchema) {
+            this.schema = tableSchema;
+            return this;
+        }
+
+        public Builder setFormatErrorStrategy(DirtyDataStrategy formatErrorStrategy) {
+            this.formatErrorStrategy = formatErrorStrategy;
+            return this;
+        }
+
+        public Builder setFieldMissingStrategy(DirtyDataStrategy fieldMissingStrategy) {
+            this.fieldMissingStrategy = fieldMissingStrategy;
+            return this;
+        }
+
+        public Builder setFieldIncrementStrategy(DirtyDataStrategy fieldIncrementStrategy) {
+            this.fieldIncrementStrategy = fieldIncrementStrategy;
+            return this;
+        }
+
+        public Builder setEncoding(String encoding) {
+            this.encoding = encoding;
+            return this;
+        }
+
+        public Builder setFieldDelimiter(String fieldDelimiter) {
+            this.fieldDelimiter = fieldDelimiter;
+            return this;
+        }
+
+        public Builder setColumnErrorDebug(boolean columnErrorDebug) {
+            this.columnErrorDebug = columnErrorDebug;
+            return this;
+        }
+
+        public Builder setProperties(Map<String, String> properties) {
+            this.properties = properties;
+            if (null == properties) {
+                return this;
+            }
+            Configuration configuration = new Configuration();
+            for (String key : properties.keySet()) {
+                configuration.setString(key, properties.get(key));
+            }
+            String lengthCheck = configuration.get(CollectorOption.LENGTH_CHECK);
+            switch (lengthCheck.toUpperCase()) {
+                case "SKIP":
+                    {
+                        this.setFormatErrorStrategy(DirtyDataStrategy.SKIP);
+                        this.setFieldMissingStrategy(DirtyDataStrategy.SKIP);
+                        this.setFieldIncrementStrategy(DirtyDataStrategy.SKIP);
+                    }
+                    break;
+                case "PAD":
+                    {
+                        this.setFormatErrorStrategy(DirtyDataStrategy.SKIP);
+                        this.setFieldMissingStrategy(DirtyDataStrategy.PAD);
+                        this.setFieldIncrementStrategy(DirtyDataStrategy.CUT);
+                    }
+                    break;
+                case "EXCEPTION":
+                    {
+                        this.setFormatErrorStrategy(DirtyDataStrategy.EXCEPTION);
+                        this.setFieldMissingStrategy(DirtyDataStrategy.EXCEPTION);
+                        this.setFieldIncrementStrategy(DirtyDataStrategy.EXCEPTION);
+                    }
+                    break;
+                case "SKIP_SILENT":
+                    {
+                        this.setFormatErrorStrategy(DirtyDataStrategy.SKIP_SILENT);
+                        this.setFieldMissingStrategy(DirtyDataStrategy.SKIP_SILENT);
+                        this.setFieldIncrementStrategy(DirtyDataStrategy.SKIP_SILENT);
+                    }
+                    break;
+                default:
+            }
+            this.setEncoding(configuration.getString(CollectorOption.ENCODING));
+            this.setFieldDelimiter(configuration.getString(CollectorOption.FIELD_DELIMITER));
+            this.setColumnErrorDebug(configuration.getBoolean(CollectorOption.COLUMN_ERROR_DEBUG));
+            return this;
+        }
+
+        public RowKeyValueDeserializationSchema build() {
+            return new RowKeyValueDeserializationSchema(
+                    schema,
+                    formatErrorStrategy,
+                    fieldMissingStrategy,
+                    fieldIncrementStrategy,
+                    encoding,
+                    fieldDelimiter,
+                    columnErrorDebug,
+                    properties);
+        }
+    }
+
+    /** Options for {@link RowKeyValueDeserializationSchema}. */
+    public static class CollectorOption {
+        public static final ConfigOption<String> ENCODING =
+                ConfigOptions.key("encoding".toLowerCase()).defaultValue("UTF-8");
+        public static final ConfigOption<String> FIELD_DELIMITER =
+                ConfigOptions.key("fieldDelimiter".toLowerCase()).defaultValue("\u0001");
+        public static final ConfigOption<Boolean> COLUMN_ERROR_DEBUG =
+                ConfigOptions.key("columnErrorDebug".toLowerCase()).defaultValue(true);
+        public static final ConfigOption<String> LENGTH_CHECK =
+                ConfigOptions.key("lengthCheck".toLowerCase()).defaultValue("NONE");
+    }
+}
diff --git a/src/main/java/org/apache/rocketmq/flink/legacy/common/serialization/SimpleKeyValueDeserializationSchema.java b/src/main/java/org/apache/rocketmq/flink/legacy/common/serialization/SimpleKeyValueDeserializationSchema.java
index 7dada93..456f477 100644
--- a/src/main/java/org/apache/rocketmq/flink/legacy/common/serialization/SimpleKeyValueDeserializationSchema.java
+++ b/src/main/java/org/apache/rocketmq/flink/legacy/common/serialization/SimpleKeyValueDeserializationSchema.java
@@ -1,15 +1,17 @@
-/**
- * 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
+/*
+ * 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
  *
- * <p>http://www.apache.org/licenses/LICENSE-2.0
+ *     http://www.apache.org/licenses/LICENSE-2.0
  *
- * <p>Unless required by applicable law or agreed to in writing, software distributed under the
- * License is distributed on an "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either
- * express or implied. See the License for the specific language governing permissions and
+ * 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.common.serialization;
diff --git a/src/main/java/org/apache/rocketmq/flink/legacy/common/serialization/SimpleKeyValueSerializationSchema.java b/src/main/java/org/apache/rocketmq/flink/legacy/common/serialization/SimpleKeyValueSerializationSchema.java
index 3e92ad2..c3ae600 100644
--- a/src/main/java/org/apache/rocketmq/flink/legacy/common/serialization/SimpleKeyValueSerializationSchema.java
+++ b/src/main/java/org/apache/rocketmq/flink/legacy/common/serialization/SimpleKeyValueSerializationSchema.java
@@ -1,15 +1,17 @@
-/**
- * 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
+/*
+ * 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
  *
- * <p>http://www.apache.org/licenses/LICENSE-2.0
+ *     http://www.apache.org/licenses/LICENSE-2.0
  *
- * <p>Unless required by applicable law or agreed to in writing, software distributed under the
- * License is distributed on an "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either
- * express or implied. See the License for the specific language governing permissions and
+ * 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.common.serialization;
diff --git a/src/main/java/org/apache/rocketmq/flink/legacy/common/util/MetricUtils.java b/src/main/java/org/apache/rocketmq/flink/legacy/common/util/MetricUtils.java
index bb3baeb..a54a29a 100644
--- a/src/main/java/org/apache/rocketmq/flink/legacy/common/util/MetricUtils.java
+++ b/src/main/java/org/apache/rocketmq/flink/legacy/common/util/MetricUtils.java
@@ -1,15 +1,17 @@
-/**
- * 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
+/*
+ * 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
  *
- * <p>http://www.apache.org/licenses/LICENSE-2.0
+ *     http://www.apache.org/licenses/LICENSE-2.0
  *
- * <p>Unless required by applicable law or agreed to in writing, software distributed under the
- * License is distributed on an "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either
- * express or implied. See the License for the specific language governing permissions and
+ * 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.common.util;
diff --git a/src/main/java/org/apache/rocketmq/flink/legacy/common/util/RetryUtil.java b/src/main/java/org/apache/rocketmq/flink/legacy/common/util/RetryUtil.java
index 7ec1dca..e53caf1 100644
--- a/src/main/java/org/apache/rocketmq/flink/legacy/common/util/RetryUtil.java
+++ b/src/main/java/org/apache/rocketmq/flink/legacy/common/util/RetryUtil.java
@@ -1,15 +1,17 @@
-/**
- * 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
+/*
+ * 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
  *
- * <p>http://www.apache.org/licenses/LICENSE-2.0
+ *     http://www.apache.org/licenses/LICENSE-2.0
  *
- * <p>Unless required by applicable law or agreed to in writing, software distributed under the
- * License is distributed on an "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either
- * express or implied. See the License for the specific language governing permissions and
+ * 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.common.util;
diff --git a/src/main/java/org/apache/rocketmq/flink/legacy/common/util/RocketMQUtils.java b/src/main/java/org/apache/rocketmq/flink/legacy/common/util/RocketMQUtils.java
index 94a24a1..1f084a8 100644
--- a/src/main/java/org/apache/rocketmq/flink/legacy/common/util/RocketMQUtils.java
+++ b/src/main/java/org/apache/rocketmq/flink/legacy/common/util/RocketMQUtils.java
@@ -1,15 +1,17 @@
-/**
- * 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
+/*
+ * 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
  *
- * <p>http://www.apache.org/licenses/LICENSE-2.0
+ *     http://www.apache.org/licenses/LICENSE-2.0
  *
- * <p>Unless required by applicable law or agreed to in writing, software distributed under the
- * License is distributed on an "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either
- * express or implied. See the License for the specific language governing permissions and
+ * 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.common.util;
diff --git a/src/main/java/org/apache/rocketmq/flink/legacy/common/util/TestUtils.java b/src/main/java/org/apache/rocketmq/flink/legacy/common/util/TestUtils.java
index 407aec7..70c26d9 100644
--- a/src/main/java/org/apache/rocketmq/flink/legacy/common/util/TestUtils.java
+++ b/src/main/java/org/apache/rocketmq/flink/legacy/common/util/TestUtils.java
@@ -1,15 +1,17 @@
-/**
- * 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
+/*
+ * 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
  *
- * <p>http://www.apache.org/licenses/LICENSE-2.0
+ *     http://www.apache.org/licenses/LICENSE-2.0
  *
- * <p>Unless required by applicable law or agreed to in writing, software distributed under the
- * License is distributed on an "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either
- * express or implied. See the License for the specific language governing permissions and
+ * 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.common.util;
diff --git a/src/main/java/org/apache/rocketmq/flink/legacy/example/RocketMQFlinkExample.java b/src/main/java/org/apache/rocketmq/flink/legacy/example/RocketMQFlinkExample.java
index b435726..fc0d3cb 100644
--- a/src/main/java/org/apache/rocketmq/flink/legacy/example/RocketMQFlinkExample.java
+++ b/src/main/java/org/apache/rocketmq/flink/legacy/example/RocketMQFlinkExample.java
@@ -20,7 +20,7 @@ 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.RocketMQSource;
+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;
@@ -114,7 +114,8 @@ public class RocketMQFlinkExample {
         SimpleTupleDeserializationSchema schema = new SimpleTupleDeserializationSchema();
 
         DataStreamSource<Tuple2<String, String>> source =
-                env.addSource(new RocketMQSource<>(schema, consumerProps)).setParallelism(2);
+                env.addSource(new RocketMQSourceFunction<>(schema, consumerProps))
+                        .setParallelism(2);
 
         source.print();
         source.process(new SourceMapFunction())
diff --git a/src/main/java/org/apache/rocketmq/flink/source/common/RocketMQOptions.java b/src/main/java/org/apache/rocketmq/flink/source/common/RocketMQOptions.java
index 064e193..000e090 100644
--- a/src/main/java/org/apache/rocketmq/flink/source/common/RocketMQOptions.java
+++ b/src/main/java/org/apache/rocketmq/flink/source/common/RocketMQOptions.java
@@ -53,6 +53,9 @@ public class RocketMQOptions {
     public static final ConfigOption<Long> OPTIONAL_PARTITION_DISCOVERY_INTERVAL_MS =
             ConfigOptions.key("partitionDiscoveryIntervalMs").longType().defaultValue(30000L);
 
+    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");
 
diff --git a/src/main/java/org/apache/rocketmq/flink/source/reader/deserializer/RowDeserializationSchema.java b/src/main/java/org/apache/rocketmq/flink/source/reader/deserializer/RowDeserializationSchema.java
index f106693..b946016 100644
--- a/src/main/java/org/apache/rocketmq/flink/source/reader/deserializer/RowDeserializationSchema.java
+++ b/src/main/java/org/apache/rocketmq/flink/source/reader/deserializer/RowDeserializationSchema.java
@@ -109,7 +109,7 @@ public class RowDeserializationSchema
         this.fieldTypes = new ValueType[totalColumnSize];
         this.columnIndexMapping = new HashMap<>();
         this.dataIndexMapping = new HashMap<>();
-        for (int index = 0; index < tableSchema.getFieldNames().length; index++) {
+        for (int index = 0; index < totalColumnSize; index++) {
             this.columnIndexMapping.put(tableSchema.getFieldNames()[index], index);
         }
         for (int index = 0; index < totalColumnSize; index++) {
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
index ec41fc6..990e28b 100644
--- a/src/main/java/org/apache/rocketmq/flink/source/table/RocketMQDynamicTableSourceFactory.java
+++ b/src/main/java/org/apache/rocketmq/flink/source/table/RocketMQDynamicTableSourceFactory.java
@@ -56,6 +56,7 @@ import static org.apache.rocketmq.flink.source.common.RocketMQOptions.OPTIONAL_S
 import static org.apache.rocketmq.flink.source.common.RocketMQOptions.OPTIONAL_START_TIME_MILLS;
 import static org.apache.rocketmq.flink.source.common.RocketMQOptions.OPTIONAL_TAG;
 import static org.apache.rocketmq.flink.source.common.RocketMQOptions.OPTIONAL_TIME_ZONE;
+import static org.apache.rocketmq.flink.source.common.RocketMQOptions.OPTIONAL_USE_NEW_API;
 import static org.apache.rocketmq.flink.source.common.RocketMQOptions.TOPIC;
 
 /**
@@ -90,6 +91,7 @@ public class RocketMQDynamicTableSourceFactory implements DynamicTableSourceFact
         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);
@@ -146,6 +148,7 @@ public class RocketMQDynamicTableSourceFactory implements DynamicTableSourceFact
         }
         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 =
@@ -161,7 +164,8 @@ public class RocketMQDynamicTableSourceFactory implements DynamicTableSourceFact
                 stopInMs,
                 startMessageOffset,
                 startMessageOffset < 0 ? startTime : -1L,
-                partitionDiscoveryIntervalMs);
+                partitionDiscoveryIntervalMs,
+                useNewApi);
     }
 
     private void transformContext(
diff --git a/src/main/java/org/apache/rocketmq/flink/source/table/RocketMQScanTableSource.java b/src/main/java/org/apache/rocketmq/flink/source/table/RocketMQScanTableSource.java
index 37ab6a5..2a0d28a 100644
--- a/src/main/java/org/apache/rocketmq/flink/source/table/RocketMQScanTableSource.java
+++ b/src/main/java/org/apache/rocketmq/flink/source/table/RocketMQScanTableSource.java
@@ -17,6 +17,10 @@
 
 package org.apache.rocketmq.flink.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;
@@ -28,6 +32,7 @@ import org.apache.flink.table.api.TableSchema;
 import org.apache.flink.table.connector.ChangelogMode;
 import org.apache.flink.table.connector.source.DynamicTableSource;
 import org.apache.flink.table.connector.source.ScanTableSource;
+import org.apache.flink.table.connector.source.SourceFunctionProvider;
 import org.apache.flink.table.connector.source.SourceProvider;
 import org.apache.flink.table.connector.source.abilities.SupportsReadingMetadata;
 import org.apache.flink.table.data.RowData;
@@ -39,6 +44,7 @@ import java.util.Collections;
 import java.util.LinkedHashMap;
 import java.util.List;
 import java.util.Map;
+import java.util.Properties;
 import java.util.stream.Stream;
 
 import static org.apache.flink.api.connector.source.Boundedness.BOUNDED;
@@ -59,6 +65,7 @@ public class RocketMQScanTableSource implements ScanTableSource, SupportsReading
     private final long partitionDiscoveryIntervalMs;
     private final long startMessageOffset;
     private final long startTime;
+    private final boolean useNewApi;
 
     private List<String> metadataKeys;
 
@@ -72,7 +79,8 @@ public class RocketMQScanTableSource implements ScanTableSource, SupportsReading
             long stopInMs,
             long startMessageOffset,
             long startTime,
-            long partitionDiscoveryIntervalMs) {
+            long partitionDiscoveryIntervalMs,
+            boolean useNewApi) {
         this.properties = properties;
         this.schema = schema;
         this.topic = topic;
@@ -83,6 +91,7 @@ public class RocketMQScanTableSource implements ScanTableSource, SupportsReading
         this.startMessageOffset = startMessageOffset;
         this.startTime = startTime;
         this.partitionDiscoveryIntervalMs = partitionDiscoveryIntervalMs;
+        this.useNewApi = useNewApi;
         this.metadataKeys = Collections.emptyList();
     }
 
@@ -93,18 +102,25 @@ public class RocketMQScanTableSource implements ScanTableSource, SupportsReading
 
     @Override
     public ScanRuntimeProvider getScanRuntimeProvider(ScanContext scanContext) {
-        return SourceProvider.of(
-                new RocketMQSource<>(
-                        topic,
-                        consumerGroup,
-                        nameServerAddress,
-                        tag,
-                        stopInMs,
-                        startTime,
-                        startMessageOffset < 0 ? 0 : startMessageOffset,
-                        partitionDiscoveryIntervalMs,
-                        isBounded() ? BOUNDED : CONTINUOUS_UNBOUNDED,
-                        createDeserializationSchema()));
+        if (useNewApi) {
+            return SourceProvider.of(
+                    new RocketMQSource<>(
+                            topic,
+                            consumerGroup,
+                            nameServerAddress,
+                            tag,
+                            stopInMs,
+                            startTime,
+                            startMessageOffset < 0 ? 0 : startMessageOffset,
+                            partitionDiscoveryIntervalMs,
+                            isBounded() ? BOUNDED : CONTINUOUS_UNBOUNDED,
+                            createRocketMQDeserializationSchema()));
+        } else {
+            return SourceFunctionProvider.of(
+                    new RocketMQSourceFunction<>(
+                            createKeyValueDeserializationSchema(), getConsumerProps()),
+                    isBounded());
+        }
     }
 
     @Override
@@ -133,17 +149,18 @@ public class RocketMQScanTableSource implements ScanTableSource, SupportsReading
                         stopInMs,
                         startMessageOffset,
                         startTime,
-                        partitionDiscoveryIntervalMs);
+                        partitionDiscoveryIntervalMs,
+                        useNewApi);
         tableSource.metadataKeys = metadataKeys;
         return tableSource;
     }
 
     @Override
     public String asSummaryString() {
-        return "RocketMQScanTableSource";
+        return RocketMQScanTableSource.class.getName();
     }
 
-    private RocketMQDeserializationSchema<RowData> createDeserializationSchema() {
+    private RocketMQDeserializationSchema<RowData> createRocketMQDeserializationSchema() {
         final MetadataConverter[] metadataConverters =
                 metadataKeys.stream()
                         .map(
@@ -162,6 +179,22 @@ public class RocketMQScanTableSource implements ScanTableSource, SupportsReading
         return stopInMs != Long.MAX_VALUE;
     }
 
+    private KeyValueDeserializationSchema<RowData> createKeyValueDeserializationSchema() {
+        return new RowKeyValueDeserializationSchema.Builder()
+                .setProperties(properties.asMap())
+                .setTableSchema(schema)
+                .build();
+    }
+
+    private Properties getConsumerProps() {
+        Properties consumerProps = new Properties();
+        consumerProps.setProperty(RocketMQConfig.CONSUMER_TOPIC, topic);
+        consumerProps.setProperty(RocketMQConfig.CONSUMER_GROUP, consumerGroup);
+        consumerProps.setProperty(RocketMQConfig.NAME_SERVER_ADDR, nameServerAddress);
+        consumerProps.setProperty(RocketMQConfig.CONSUMER_TAG, tag);
+        return consumerProps;
+    }
+
     // --------------------------------------------------------------------------------------------
     // Metadata handling
     // --------------------------------------------------------------------------------------------
diff --git a/src/test/java/org/apache/rocketmq/flink/legacy/RocketMQSinkTest.java b/src/test/java/org/apache/rocketmq/flink/legacy/RocketMQSinkTest.java
index c45dbdf..ad3c0b1 100644
--- a/src/test/java/org/apache/rocketmq/flink/legacy/RocketMQSinkTest.java
+++ b/src/test/java/org/apache/rocketmq/flink/legacy/RocketMQSinkTest.java
@@ -1,17 +1,21 @@
-/**
- * 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
+/*
+ * 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
  *
- * <p>http://www.apache.org/licenses/LICENSE-2.0
+ *     http://www.apache.org/licenses/LICENSE-2.0
  *
- * <p>Unless required by applicable law or agreed to in writing, software distributed under the
- * License is distributed on an "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either
- * express or implied. See the License for the specific language governing permissions and
+ * 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;
 
 import org.apache.rocketmq.client.producer.DefaultMQProducer;
diff --git a/src/test/java/org/apache/rocketmq/flink/legacy/RocketMQSourceTest.java b/src/test/java/org/apache/rocketmq/flink/legacy/RocketMQSourceTest.java
index a863ddd..7ce124d 100644
--- a/src/test/java/org/apache/rocketmq/flink/legacy/RocketMQSourceTest.java
+++ b/src/test/java/org/apache/rocketmq/flink/legacy/RocketMQSourceTest.java
@@ -1,17 +1,21 @@
-/**
- * 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
+/*
+ * 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
  *
- * <p>http://www.apache.org/licenses/LICENSE-2.0
+ *     http://www.apache.org/licenses/LICENSE-2.0
  *
- * <p>Unless required by applicable law or agreed to in writing, software distributed under the
- * License is distributed on an "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either
- * express or implied. See the License for the specific language governing permissions and
+ * 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;
 
 import org.apache.rocketmq.client.consumer.DefaultMQPullConsumer;
@@ -50,7 +54,7 @@ import static org.mockito.Mockito.when;
 @Ignore
 public class RocketMQSourceTest {
 
-    private RocketMQSource rocketMQSource;
+    private RocketMQSourceFunction rocketMQSource;
     private MQPullConsumerScheduleService pullConsumerScheduleService;
     private DefaultMQPullConsumer consumer;
     private KeyValueDeserializationSchema deserializationSchema;
@@ -60,7 +64,7 @@ public class RocketMQSourceTest {
     public void setUp() throws Exception {
         deserializationSchema = new SimpleKeyValueDeserializationSchema();
         Properties props = new Properties();
-        rocketMQSource = new RocketMQSource(deserializationSchema, props);
+        rocketMQSource = new RocketMQSourceFunction(deserializationSchema, props);
 
         setFieldValue(rocketMQSource, "topic", topic);
         setFieldValue(rocketMQSource, "runningChecker", new SingleRunningCheck());
diff --git a/src/test/java/org/apache/rocketmq/flink/legacy/common/selector/DefaultTopicSelectorTest.java b/src/test/java/org/apache/rocketmq/flink/legacy/common/selector/DefaultTopicSelectorTest.java
index b235c63..aa1528a 100644
--- a/src/test/java/org/apache/rocketmq/flink/legacy/common/selector/DefaultTopicSelectorTest.java
+++ b/src/test/java/org/apache/rocketmq/flink/legacy/common/selector/DefaultTopicSelectorTest.java
@@ -1,17 +1,21 @@
-/**
- * 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
+/*
+ * 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
  *
- * <p>http://www.apache.org/licenses/LICENSE-2.0
+ *     http://www.apache.org/licenses/LICENSE-2.0
  *
- * <p>Unless required by applicable law or agreed to in writing, software distributed under the
- * License is distributed on an "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either
- * express or implied. See the License for the specific language governing permissions and
+ * 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.common.selector;
 
 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/rocketmq/flink/legacy/common/selector/SimpleTopicSelectorTest.java
index 5c0f755..dc93d14 100644
--- a/src/test/java/org/apache/rocketmq/flink/legacy/common/selector/SimpleTopicSelectorTest.java
+++ b/src/test/java/org/apache/rocketmq/flink/legacy/common/selector/SimpleTopicSelectorTest.java
@@ -1,17 +1,21 @@
-/**
- * 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
+/*
+ * 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
  *
- * <p>http://www.apache.org/licenses/LICENSE-2.0
+ *     http://www.apache.org/licenses/LICENSE-2.0
  *
- * <p>Unless required by applicable law or agreed to in writing, software distributed under the
- * License is distributed on an "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either
- * express or implied. See the License for the specific language governing permissions and
+ * 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.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/rocketmq/flink/legacy/common/serialization/RowKeyValueDeserializationSchemaTest.java
new file mode 100644
index 0000000..2c1786a
--- /dev/null
+++ b/src/test/java/org/apache/rocketmq/flink/legacy/common/serialization/RowKeyValueDeserializationSchemaTest.java
@@ -0,0 +1,50 @@
+/*
+ * 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.common.serialization;
+
+import org.apache.rocketmq.common.message.MessageExt;
+
+import org.apache.flink.table.api.DataTypes;
+import org.apache.flink.table.api.TableSchema;
+import org.apache.flink.table.data.RowData;
+
+import org.junit.Test;
+
+import java.util.HashMap;
+
+import static org.junit.Assert.assertEquals;
+
+/** Test for {@link RowKeyValueDeserializationSchema}. */
+public class RowKeyValueDeserializationSchemaTest {
+
+    @Test
+    public void testDeserializeKeyAndValue() {
+        TableSchema tableSchema =
+                new TableSchema.Builder().field("varchar", DataTypes.VARCHAR(100)).build();
+        RowKeyValueDeserializationSchema deserializationSchema =
+                new RowKeyValueDeserializationSchema.Builder()
+                        .setTableSchema(tableSchema)
+                        .setProperties(new HashMap<>())
+                        .build();
+        MessageExt messageExt = new MessageExt();
+        messageExt.setBody("test_deserialize_key_and_value".getBytes());
+        RowData rowData = deserializationSchema.deserializeKeyAndValue(null, messageExt.getBody());
+        assertEquals(new String(messageExt.getBody()), rowData.getString(0).toString());
+    }
+}
diff --git a/src/test/java/org/apache/rocketmq/flink/legacy/common/serialization/SimpleKeyValueSerializationSchemaTest.java b/src/test/java/org/apache/rocketmq/flink/legacy/common/serialization/SimpleKeyValueSerializationSchemaTest.java
index 78baf20..7e2e0d9 100644
--- a/src/test/java/org/apache/rocketmq/flink/legacy/common/serialization/SimpleKeyValueSerializationSchemaTest.java
+++ b/src/test/java/org/apache/rocketmq/flink/legacy/common/serialization/SimpleKeyValueSerializationSchemaTest.java
@@ -1,17 +1,21 @@
-/**
- * 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
+/*
+ * 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
  *
- * <p>http://www.apache.org/licenses/LICENSE-2.0
+ *     http://www.apache.org/licenses/LICENSE-2.0
  *
- * <p>Unless required by applicable law or agreed to in writing, software distributed under the
- * License is distributed on an "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either
- * express or implied. See the License for the specific language governing permissions and
+ * 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.common.serialization;
 
 import org.junit.Test;
diff --git a/src/test/java/org/apache/rocketmq/flink/source/table/RocketMQDynamicTableSourceFactoryTest.java b/src/test/java/org/apache/rocketmq/flink/source/table/RocketMQDynamicTableSourceFactoryTest.java
new file mode 100644
index 0000000..184a23f
--- /dev/null
+++ b/src/test/java/org/apache/rocketmq/flink/source/table/RocketMQDynamicTableSourceFactoryTest.java
@@ -0,0 +1,111 @@
+/*
+ * 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.source.common.RocketMQOptions;
+
+import org.apache.flink.configuration.Configuration;
+import org.apache.flink.table.api.Schema;
+import org.apache.flink.table.api.ValidationException;
+import org.apache.flink.table.catalog.CatalogTable;
+import org.apache.flink.table.catalog.Column;
+import org.apache.flink.table.catalog.ObjectIdentifier;
+import org.apache.flink.table.catalog.ResolvedCatalogTable;
+import org.apache.flink.table.catalog.ResolvedSchema;
+import org.apache.flink.table.connector.source.DynamicTableSource;
+import org.apache.flink.table.factories.FactoryUtil;
+
+import org.junit.Test;
+
+import java.util.ArrayList;
+import java.util.Collections;
+import java.util.HashMap;
+import java.util.Map;
+
+import static org.apache.flink.table.api.DataTypes.STRING;
+import static org.junit.Assert.assertEquals;
+import static org.junit.Assert.assertTrue;
+
+/** Tests for {@link RocketMQDynamicTableSourceFactory}. */
+public class RocketMQDynamicTableSourceFactoryTest {
+
+    private static final ResolvedSchema SCHEMA =
+            new ResolvedSchema(
+                    Collections.singletonList(Column.physical("name", STRING().notNull())),
+                    new ArrayList<>(),
+                    null);
+
+    private static final String IDENTIFIER = "rocketmq";
+    private static final String TOPIC = "test_source";
+    private static final String CONSUMER_GROUP = "test_consumer";
+    private static final String NAME_SERVER_ADDRESS = "127.0.0.1:9876";
+
+    @Test
+    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);
+        final DynamicTableSource tableSource = createTableSource(options);
+        assertTrue(tableSource instanceof RocketMQScanTableSource);
+        assertEquals(RocketMQScanTableSource.class.getName(), tableSource.asSummaryString());
+    }
+
+    @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");
+        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("unknown", "test_option");
+        createTableSource(options);
+    }
+
+    private static DynamicTableSource createTableSource(
+            Map<String, String> options, Configuration conf) {
+        return FactoryUtil.createTableSource(
+                null,
+                ObjectIdentifier.of("default", "default", IDENTIFIER),
+                new ResolvedCatalogTable(
+                        CatalogTable.of(
+                                Schema.newBuilder().fromResolvedSchema(SCHEMA).build(),
+                                "mock source",
+                                Collections.emptyList(),
+                                options),
+                        SCHEMA),
+                conf,
+                RocketMQDynamicTableSourceFactory.class.getClassLoader(),
+                false);
+    }
+
+    private static DynamicTableSource createTableSource(Map<String, String> options) {
+        return createTableSource(options, new Configuration());
+    }
+}

[rocketmq-flink] 18/33: Producer failed to shutdown when exception happened (#388)

Posted by du...@apache.org.
This is an automated email from the ASF dual-hosted git repository.

duhengforever pushed a commit to branch main
in repository https://gitbox.apache.org/repos/asf/rocketmq-flink.git

commit 0698d86fb318da566c0744ba5737e416a7536dc1
Author: shangan <ch...@163.com>
AuthorDate: Mon Aug 19 14:41:39 2019 +0800

    Producer failed to shutdown when exception happened (#388)
---
 src/main/java/org/apache/rocketmq/flink/RocketMQSink.java | 7 ++++++-
 1 file changed, 6 insertions(+), 1 deletion(-)

diff --git a/src/main/java/org/apache/rocketmq/flink/RocketMQSink.java b/src/main/java/org/apache/rocketmq/flink/RocketMQSink.java
index ca6848d..e8f237f 100644
--- a/src/main/java/org/apache/rocketmq/flink/RocketMQSink.java
+++ b/src/main/java/org/apache/rocketmq/flink/RocketMQSink.java
@@ -181,7 +181,12 @@ public class RocketMQSink<IN> extends RichSinkFunction<IN> implements Checkpoint
     @Override
     public void close() throws Exception {
         if (producer != null) {
-            flushSync();
+            try {
+                flushSync();
+            } catch (Exception e) {
+                LOG.error("FlushSync failure!", e);
+            }
+            // make sure producer can be shutdown, thus current producerGroup will be unregistered
             producer.shutdown();
         }
     }

[rocketmq-flink] 27/33: [#715] Support the RocketMQ TableSource based on the new Source interface (#716)

Posted by du...@apache.org.
This is an automated email from the ASF dual-hosted git repository.

duhengforever pushed a commit to branch main
in repository https://gitbox.apache.org/repos/asf/rocketmq-flink.git

commit 340ea48bbd68dc199679da52e35f707f9756a1b8
Author: SteNicholas <pr...@163.com>
AuthorDate: Thu Jun 3 11:51:32 2021 +0800

    [#715] Support the RocketMQ TableSource based on the new Source interface (#716)
---
 pom.xml                                            |  13 +-
 .../rocketmq/flink/source/RocketMQSource.java      |  18 +-
 .../flink/source/common/RocketMQOptions.java       |  70 +++
 .../enumerator/RocketMQSourceEnumerator.java       |  20 +-
 .../reader/RocketMQPartitionSplitReader.java       |  23 +-
 .../source/reader/deserializer/BytesMessage.java   |  53 ++
 ...ationSchema.java => DeserializationSchema.java} |  12 +-
 .../reader/deserializer/DirtyDataStrategy.java     |  28 +
 ...ema.java => RocketMQDeserializationSchema.java} |   8 +-
 .../RocketMQRowDeserializationSchema.java          | 104 ++++
 .../deserializer/RowDeserializationSchema.java     | 606 +++++++++++++++++++++
 .../table/RocketMQDynamicTableSourceFactory.java   | 211 +++++++
 .../source/table/RocketMQScanTableSource.java      | 195 +++++++
 .../rocketmq/flink/source/util/ByteSerializer.java | 156 ++++++
 .../rocketmq/flink/source/util/ByteUtils.java      | 219 ++++++++
 .../flink/source/util/StringSerializer.java        | 155 ++++++
 .../org.apache.flink.table.factories.Factory       |  16 +
 .../RocketMQRowDeserializationSchemaTest.java      | 141 +++++
 18 files changed, 2018 insertions(+), 30 deletions(-)

diff --git a/pom.xml b/pom.xml
index d5fc49f..abec905 100644
--- a/pom.xml
+++ b/pom.xml
@@ -34,7 +34,7 @@
         <maven.compiler.source>1.8</maven.compiler.source>
         <maven.compiler.target>1.8</maven.compiler.target>
         <rocketmq.version>4.7.1</rocketmq.version>
-        <flink.version>1.12.2</flink.version>
+        <flink.version>1.13.0</flink.version>
         <commons-lang.version>2.5</commons-lang.version>
         <scala.binary.version>2.11</scala.binary.version>
         <spotless.version>2.4.2</spotless.version>
@@ -78,6 +78,12 @@
         </dependency>
         <dependency>
             <groupId>org.apache.flink</groupId>
+            <artifactId>flink-table-planner-blink_${scala.binary.version}</artifactId>
+            <version>${flink.version}</version>
+            <scope>test</scope>
+        </dependency>
+        <dependency>
+            <groupId>org.apache.flink</groupId>
             <artifactId>flink-queryable-state-runtime_${scala.binary.version}</artifactId>
             <version>${flink.version}</version>
         </dependency>
@@ -102,6 +108,11 @@
                 </exclusion>
             </exclusions>
         </dependency>
+        <dependency>
+            <groupId>org.apache.rocketmq</groupId>
+            <artifactId>rocketmq-test</artifactId>
+            <version>${rocketmq.version}</version>
+        </dependency>
 
         <dependency>
             <groupId>commons-lang</groupId>
diff --git a/src/main/java/org/apache/rocketmq/flink/source/RocketMQSource.java b/src/main/java/org/apache/rocketmq/flink/source/RocketMQSource.java
index b899618..79a8149 100644
--- a/src/main/java/org/apache/rocketmq/flink/source/RocketMQSource.java
+++ b/src/main/java/org/apache/rocketmq/flink/source/RocketMQSource.java
@@ -24,7 +24,7 @@ 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.RocketMQRecordDeserializationSchema;
+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;
 
@@ -52,10 +52,11 @@ import java.util.function.Supplier;
 public class RocketMQSource<OUT>
         implements Source<OUT, RocketMQPartitionSplit, RocketMQSourceEnumState>,
                 ResultTypeQueryable<OUT> {
-    private static final long serialVersionUID = -6755372893283732098L;
+    private static final long serialVersionUID = -1L;
 
     private final String topic;
     private final String consumerGroup;
+    private final String nameServerAddress;
     private final String tag;
     private final long stopInMs;
     private final long startTime;
@@ -64,20 +65,22 @@ public class RocketMQSource<OUT>
 
     // Boundedness
     private final Boundedness boundedness;
-    private final RocketMQRecordDeserializationSchema<OUT> deserializationSchema;
+    private final RocketMQDeserializationSchema<OUT> deserializationSchema;
 
     public RocketMQSource(
             String topic,
             String consumerGroup,
+            String nameServerAddress,
             String tag,
             long stopInMs,
             long startTime,
             long startOffset,
             long partitionDiscoveryIntervalMs,
             Boundedness boundedness,
-            RocketMQRecordDeserializationSchema<OUT> deserializationSchema) {
+            RocketMQDeserializationSchema<OUT> deserializationSchema) {
         this.topic = topic;
         this.consumerGroup = consumerGroup;
+        this.nameServerAddress = nameServerAddress;
         this.tag = tag;
         this.stopInMs = stopInMs;
         this.startTime = startTime;
@@ -93,8 +96,8 @@ public class RocketMQSource<OUT>
     }
 
     @Override
-    public SourceReader<OUT, RocketMQPartitionSplit> createReader(SourceReaderContext readerContext)
-            throws Exception {
+    public SourceReader<OUT, RocketMQPartitionSplit> createReader(
+            SourceReaderContext readerContext) {
         FutureCompletingBlockingQueue<RecordsWithSplitIds<Tuple3<OUT, Long, Long>>> elementsQueue =
                 new FutureCompletingBlockingQueue<>();
         deserializationSchema.open(
@@ -115,6 +118,7 @@ public class RocketMQSource<OUT>
                         new RocketMQPartitionSplitReader<>(
                                 topic,
                                 consumerGroup,
+                                nameServerAddress,
                                 tag,
                                 stopInMs,
                                 startTime,
@@ -136,6 +140,7 @@ public class RocketMQSource<OUT>
         return new RocketMQSourceEnumerator(
                 topic,
                 consumerGroup,
+                nameServerAddress,
                 stopInMs,
                 startOffset,
                 partitionDiscoveryIntervalMs,
@@ -150,6 +155,7 @@ public class RocketMQSource<OUT>
         return new RocketMQSourceEnumerator(
                 topic,
                 consumerGroup,
+                nameServerAddress,
                 stopInMs,
                 startOffset,
                 partitionDiscoveryIntervalMs,
diff --git a/src/main/java/org/apache/rocketmq/flink/source/common/RocketMQOptions.java b/src/main/java/org/apache/rocketmq/flink/source/common/RocketMQOptions.java
new file mode 100644
index 0000000..064e193
--- /dev/null
+++ b/src/main/java/org/apache/rocketmq/flink/source/common/RocketMQOptions.java
@@ -0,0 +1,70 @@
+/*
+ * 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.common;
+
+import org.apache.flink.configuration.ConfigOption;
+import org.apache.flink.configuration.ConfigOptions;
+
+/** Includes config options of RocketMQ connector type. */
+public class RocketMQOptions {
+
+    public static final ConfigOption<String> TOPIC = ConfigOptions.key("topic").noDefaultValue();
+
+    public static final ConfigOption<String> CONSUMER_GROUP =
+            ConfigOptions.key("consumerGroup").noDefaultValue();
+
+    public static final ConfigOption<String> NAME_SERVER_ADDRESS =
+            ConfigOptions.key("nameServerAddress").noDefaultValue();
+
+    public static final ConfigOption<String> OPTIONAL_TAG =
+            ConfigOptions.key("tag").noDefaultValue();
+
+    public static final ConfigOption<Integer> OPTIONAL_START_MESSAGE_OFFSET =
+            ConfigOptions.key("startMessageOffset").defaultValue(-1);
+
+    public static final ConfigOption<Long> OPTIONAL_START_TIME_MILLS =
+            ConfigOptions.key("startTimeMs".toLowerCase()).longType().defaultValue(-1L);
+
+    public static final ConfigOption<String> OPTIONAL_START_TIME =
+            ConfigOptions.key("startTime".toLowerCase()).stringType().noDefaultValue();
+
+    public static final ConfigOption<String> OPTIONAL_END_TIME =
+            ConfigOptions.key("endTime").noDefaultValue();
+
+    public static final ConfigOption<String> OPTIONAL_TIME_ZONE =
+            ConfigOptions.key("timeZone".toLowerCase()).stringType().noDefaultValue();
+
+    public static final ConfigOption<Long> OPTIONAL_PARTITION_DISCOVERY_INTERVAL_MS =
+            ConfigOptions.key("partitionDiscoveryIntervalMs").longType().defaultValue(30000L);
+
+    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");
+}
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
index 08290c6..61b563a 100644
--- a/src/main/java/org/apache/rocketmq/flink/source/enumerator/RocketMQSourceEnumerator.java
+++ b/src/main/java/org/apache/rocketmq/flink/source/enumerator/RocketMQSourceEnumerator.java
@@ -19,7 +19,6 @@
 package org.apache.rocketmq.flink.source.enumerator;
 
 import org.apache.rocketmq.client.consumer.DefaultMQPullConsumer;
-import org.apache.rocketmq.client.consumer.MQPullConsumer;
 import org.apache.rocketmq.client.exception.MQClientException;
 import org.apache.rocketmq.common.message.MessageQueue;
 import org.apache.rocketmq.flink.source.split.RocketMQPartitionSplit;
@@ -38,6 +37,7 @@ 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;
@@ -59,6 +59,8 @@ public class RocketMQSourceEnumerator
     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. */
@@ -85,12 +87,13 @@ public class RocketMQSourceEnumerator
     private final Map<Integer, Set<RocketMQPartitionSplit>> pendingPartitionSplitAssignment;
 
     // Lazily instantiated or mutable fields.
-    private MQPullConsumer consumer;
+    private DefaultMQPullConsumer consumer;
     private boolean noMoreNewPartitionSplits = false;
 
     public RocketMQSourceEnumerator(
             String topic,
             String consumerGroup,
+            String nameServerAddress,
             long stopInMs,
             long startOffset,
             long partitionDiscoveryIntervalMs,
@@ -99,6 +102,7 @@ public class RocketMQSourceEnumerator
         this(
                 topic,
                 consumerGroup,
+                nameServerAddress,
                 stopInMs,
                 startOffset,
                 partitionDiscoveryIntervalMs,
@@ -110,6 +114,7 @@ public class RocketMQSourceEnumerator
     public RocketMQSourceEnumerator(
             String topic,
             String consumerGroup,
+            String nameServerAddress,
             long stopInMs,
             long startOffset,
             long partitionDiscoveryIntervalMs,
@@ -118,6 +123,7 @@ public class RocketMQSourceEnumerator
             Map<Integer, List<RocketMQPartitionSplit>> currentSplitsAssignments) {
         this.topic = topic;
         this.consumerGroup = consumerGroup;
+        this.nameServerAddress = nameServerAddress;
         this.stopInMs = stopInMs;
         this.startOffset = startOffset;
         this.partitionDiscoveryIntervalMs = partitionDiscoveryIntervalMs;
@@ -180,7 +186,7 @@ public class RocketMQSourceEnumerator
     }
 
     @Override
-    public RocketMQSourceEnumState snapshotState() {
+    public RocketMQSourceEnumState snapshotState(long checkpointId) {
         return new RocketMQSourceEnumState(readerIdToSplitAssignments);
     }
 
@@ -298,6 +304,14 @@ public class RocketMQSourceEnumerator
     private void initialRocketMQConsumer() {
         try {
             consumer = new DefaultMQPullConsumer(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);
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
index 3bbeec8..41fbbea 100644
--- a/src/main/java/org/apache/rocketmq/flink/source/reader/RocketMQPartitionSplitReader.java
+++ b/src/main/java/org/apache/rocketmq/flink/source/reader/RocketMQPartitionSplitReader.java
@@ -19,13 +19,12 @@
 package org.apache.rocketmq.flink.source.reader;
 
 import org.apache.rocketmq.client.consumer.DefaultMQPullConsumer;
-import org.apache.rocketmq.client.consumer.MQPullConsumer;
 import org.apache.rocketmq.client.consumer.PullResult;
 import org.apache.rocketmq.client.exception.MQBrokerException;
 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.RocketMQRecordDeserializationSchema;
+import org.apache.rocketmq.flink.source.reader.deserializer.RocketMQDeserializationSchema;
 import org.apache.rocketmq.flink.source.split.RocketMQPartitionSplit;
 import org.apache.rocketmq.remoting.exception.RemotingException;
 
@@ -43,6 +42,7 @@ 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;
@@ -70,12 +70,12 @@ public class RocketMQPartitionSplitReader<T>
     private final long startTime;
     private final long startOffset;
 
-    private final RocketMQRecordDeserializationSchema<T> deserializationSchema;
+    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 MQPullConsumer consumer;
+    private DefaultMQPullConsumer consumer;
 
     private volatile boolean wakeup = false;
 
@@ -84,11 +84,12 @@ public class RocketMQPartitionSplitReader<T>
     public RocketMQPartitionSplitReader(
             String topic,
             String consumerGroup,
+            String nameServerAddress,
             String tag,
             long stopInMs,
             long startTime,
             long startOffset,
-            RocketMQRecordDeserializationSchema<T> deserializationSchema) {
+            RocketMQDeserializationSchema<T> deserializationSchema) {
         this.topic = topic;
         this.tag = tag;
         this.stopInMs = stopInMs;
@@ -98,7 +99,7 @@ public class RocketMQPartitionSplitReader<T>
         this.startingOffsets = new HashMap<>();
         this.stoppingTimestamps = new HashMap<>();
         this.collector = new SimpleCollector<>();
-        initialRocketMQConsumer(consumerGroup);
+        initialRocketMQConsumer(consumerGroup, nameServerAddress);
     }
 
     @Override
@@ -280,9 +281,17 @@ public class RocketMQPartitionSplitReader<T>
 
     // --------------- private helper method ----------------------
 
-    private void initialRocketMQConsumer(String consumerGroup) {
+    private void initialRocketMQConsumer(String consumerGroup, String nameServerAddress) {
         try {
             consumer = new DefaultMQPullConsumer(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);
diff --git a/src/main/java/org/apache/rocketmq/flink/source/reader/deserializer/BytesMessage.java b/src/main/java/org/apache/rocketmq/flink/source/reader/deserializer/BytesMessage.java
new file mode 100644
index 0000000..d109a7f
--- /dev/null
+++ b/src/main/java/org/apache/rocketmq/flink/source/reader/deserializer/BytesMessage.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.rocketmq.flink.source.reader.deserializer;
+
+import java.util.HashMap;
+import java.util.Map;
+
+/** Message contains byte array. */
+public class BytesMessage {
+
+    private byte[] data;
+    private Map<String, String> properties = new HashMap<>();
+
+    public byte[] getData() {
+        return data;
+    }
+
+    public void setData(byte[] data) {
+        this.data = data;
+    }
+
+    public Map<String, String> getProperties() {
+        return properties;
+    }
+
+    public void setProperties(Map<String, String> props) {
+        this.properties = props;
+    }
+
+    public Object getProperty(String key) {
+        return properties.get(key);
+    }
+
+    public void setProperty(String key, String value) {
+        properties.put(key, value);
+    }
+}
diff --git a/src/main/java/org/apache/rocketmq/flink/source/reader/deserializer/RocketMQRecordDeserializationSchema.java b/src/main/java/org/apache/rocketmq/flink/source/reader/deserializer/DeserializationSchema.java
similarity index 77%
copy from src/main/java/org/apache/rocketmq/flink/source/reader/deserializer/RocketMQRecordDeserializationSchema.java
copy to src/main/java/org/apache/rocketmq/flink/source/reader/deserializer/DeserializationSchema.java
index 455f8af..3b087cc 100644
--- a/src/main/java/org/apache/rocketmq/flink/source/reader/deserializer/RocketMQRecordDeserializationSchema.java
+++ b/src/main/java/org/apache/rocketmq/flink/source/reader/deserializer/DeserializationSchema.java
@@ -1,7 +1,5 @@
 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.api.java.typeutils.ResultTypeQueryable;
@@ -9,11 +7,9 @@ import org.apache.flink.util.Collector;
 
 import java.io.IOException;
 import java.io.Serializable;
-import java.util.List;
 
-/** An interface for the deserialization of RocketMQ records. */
-public interface RocketMQRecordDeserializationSchema<T>
-        extends Serializable, ResultTypeQueryable<T> {
+/** An interface for the deserialization of records. */
+public interface DeserializationSchema<IN, OUT> extends Serializable, ResultTypeQueryable<OUT> {
 
     /**
      * Initialization method for the schema. It is called before the actual working methods {@link
@@ -35,9 +31,9 @@ public interface RocketMQRecordDeserializationSchema<T>
      * records can be buffered in memory or collecting records might delay emitting checkpoint
      * barrier.
      *
-     * @param record The MessageExts to deserialize.
+     * @param record The record to deserialize.
      * @param out The collector to put the resulting messages.
      */
     @PublicEvolving
-    void deserialize(List<MessageExt> record, Collector<T> out) throws IOException;
+    void deserialize(IN record, Collector<OUT> out) throws IOException;
 }
diff --git a/src/main/java/org/apache/rocketmq/flink/source/reader/deserializer/DirtyDataStrategy.java b/src/main/java/org/apache/rocketmq/flink/source/reader/deserializer/DirtyDataStrategy.java
new file mode 100644
index 0000000..06a0c2d
--- /dev/null
+++ b/src/main/java/org/apache/rocketmq/flink/source/reader/deserializer/DirtyDataStrategy.java
@@ -0,0 +1,28 @@
+/*
+ * 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;
+
+/** Dirty data process strategy. */
+public enum DirtyDataStrategy {
+    SKIP,
+    SKIP_SILENT,
+    CUT,
+    PAD,
+    NULL,
+    EXCEPTION
+}
diff --git a/src/main/java/org/apache/rocketmq/flink/source/reader/deserializer/RocketMQRecordDeserializationSchema.java b/src/main/java/org/apache/rocketmq/flink/source/reader/deserializer/RocketMQDeserializationSchema.java
similarity index 84%
rename from src/main/java/org/apache/rocketmq/flink/source/reader/deserializer/RocketMQRecordDeserializationSchema.java
rename to src/main/java/org/apache/rocketmq/flink/source/reader/deserializer/RocketMQDeserializationSchema.java
index 455f8af..6358e4c 100644
--- a/src/main/java/org/apache/rocketmq/flink/source/reader/deserializer/RocketMQRecordDeserializationSchema.java
+++ b/src/main/java/org/apache/rocketmq/flink/source/reader/deserializer/RocketMQDeserializationSchema.java
@@ -4,16 +4,14 @@ 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.api.java.typeutils.ResultTypeQueryable;
 import org.apache.flink.util.Collector;
 
 import java.io.IOException;
-import java.io.Serializable;
 import java.util.List;
 
 /** An interface for the deserialization of RocketMQ records. */
-public interface RocketMQRecordDeserializationSchema<T>
-        extends Serializable, ResultTypeQueryable<T> {
+public interface RocketMQDeserializationSchema<T>
+        extends DeserializationSchema<List<MessageExt>, T> {
 
     /**
      * Initialization method for the schema. It is called before the actual working methods {@link
@@ -25,7 +23,7 @@ public interface RocketMQRecordDeserializationSchema<T>
      * @param context Contextual information that can be used during initialization.
      */
     @PublicEvolving
-    default void open(InitializationContext context) throws Exception {}
+    default void open(InitializationContext context) {}
 
     /**
      * Deserializes the byte message.
diff --git a/src/main/java/org/apache/rocketmq/flink/source/reader/deserializer/RocketMQRowDeserializationSchema.java b/src/main/java/org/apache/rocketmq/flink/source/reader/deserializer/RocketMQRowDeserializationSchema.java
new file mode 100644
index 0000000..5bd990e
--- /dev/null
+++ b/src/main/java/org/apache/rocketmq/flink/source/reader/deserializer/RocketMQRowDeserializationSchema.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.rocketmq.flink.source.reader.deserializer;
+
+import org.apache.rocketmq.common.message.MessageExt;
+import org.apache.rocketmq.flink.source.reader.deserializer.RowDeserializationSchema.MetadataConverter;
+
+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.table.api.TableSchema;
+import org.apache.flink.table.data.RowData;
+import org.apache.flink.util.Collector;
+
+import java.util.ArrayList;
+import java.util.List;
+import java.util.Map;
+
+/**
+ * A row data wrapper class that wraps a {@link RocketMQDeserializationSchema} to deserialize {@link
+ * MessageExt}.
+ */
+public class RocketMQRowDeserializationSchema implements RocketMQDeserializationSchema<RowData> {
+
+    private static final long serialVersionUID = 1L;
+
+    private final RowDeserializationSchema deserializationSchema;
+
+    private transient List<BytesMessage> bytesMessages = new ArrayList<>(1);
+
+    public RocketMQRowDeserializationSchema(
+            TableSchema tableSchema,
+            Map<String, String> properties,
+            boolean hasMetadata,
+            MetadataConverter[] metadataConverters) {
+        deserializationSchema =
+                new RowDeserializationSchema.Builder()
+                        .setProperties(properties)
+                        .setTableSchema(tableSchema)
+                        .setHasMetadata(hasMetadata)
+                        .setMetadataConverters(metadataConverters)
+                        .build();
+    }
+
+    @Override
+    public void open(InitializationContext context) {
+        deserializationSchema.open(context);
+        bytesMessages = new ArrayList<>();
+    }
+
+    @Override
+    public void deserialize(List<MessageExt> input, Collector<RowData> collector) {
+        extractMessages(input);
+        deserializationSchema.deserialize(bytesMessages, collector);
+    }
+
+    @Override
+    public TypeInformation<RowData> getProducedType() {
+        return deserializationSchema.getProducedType();
+    }
+
+    private void extractMessages(List<MessageExt> messages) {
+        bytesMessages = new ArrayList<>(messages.size());
+        for (MessageExt message : messages) {
+            BytesMessage bytesMessage = new BytesMessage();
+            bytesMessage.setData(message.getBody());
+            if (message.getProperties() != null) {
+                bytesMessage.setProperties(message.getProperties());
+            }
+            bytesMessage.setProperty("__topic__", message.getTopic());
+            bytesMessage.setProperty(
+                    "__store_timestamp__", String.valueOf(message.getStoreTimestamp()));
+            bytesMessage.setProperty(
+                    "__born_timestamp__", String.valueOf(message.getBornTimestamp()));
+            bytesMessage.setProperty("__queue_id__", String.valueOf(message.getQueueId()));
+            bytesMessage.setProperty("__queue_offset__", String.valueOf(message.getQueueOffset()));
+            bytesMessage.setProperty("__msg_id__", message.getMsgId());
+            bytesMessage.setProperty("__keys__", message.getKeys());
+            bytesMessage.setProperty("__tags__", message.getTags());
+            bytesMessages.add(bytesMessage);
+        }
+    }
+
+    @VisibleForTesting
+    public List<BytesMessage> getBytesMessages() {
+        return bytesMessages;
+    }
+}
diff --git a/src/main/java/org/apache/rocketmq/flink/source/reader/deserializer/RowDeserializationSchema.java b/src/main/java/org/apache/rocketmq/flink/source/reader/deserializer/RowDeserializationSchema.java
new file mode 100644
index 0000000..f106693
--- /dev/null
+++ b/src/main/java/org/apache/rocketmq/flink/source/reader/deserializer/RowDeserializationSchema.java
@@ -0,0 +1,606 @@
+/*
+ * 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.flink.source.util.ByteSerializer;
+import org.apache.rocketmq.flink.source.util.ByteSerializer.ValueType;
+import org.apache.rocketmq.flink.source.util.StringSerializer;
+
+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.table.api.TableSchema;
+import org.apache.flink.table.data.GenericRowData;
+import org.apache.flink.table.data.RowData;
+import org.apache.flink.table.descriptors.DescriptorProperties;
+import org.apache.flink.table.descriptors.SchemaValidator;
+import org.apache.flink.table.runtime.typeutils.InternalTypeInfo;
+import org.apache.flink.table.types.DataType;
+import org.apache.flink.table.types.logical.RowType;
+import org.apache.flink.util.Collector;
+
+import org.apache.commons.lang3.StringEscapeUtils;
+import org.apache.commons.lang3.StringUtils;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import java.io.Serializable;
+import java.io.UnsupportedEncodingException;
+import java.util.HashMap;
+import java.util.HashSet;
+import java.util.List;
+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> {
+
+    private static final long serialVersionUID = -1L;
+    private static final Logger logger = LoggerFactory.getLogger(RowDeserializationSchema.class);
+
+    private transient TableSchema tableSchema;
+    private final DirtyDataStrategy formatErrorStrategy;
+    private final DirtyDataStrategy fieldMissingStrategy;
+    private final DirtyDataStrategy fieldIncrementStrategy;
+    private final String encoding;
+    private final String fieldDelimiter;
+    private final String lineDelimiter;
+    private final boolean columnErrorDebug;
+    private final MetadataCollector metadataCollector;
+    private final int totalColumnSize;
+    private final int dataColumnSize;
+    private final ValueType[] fieldTypes;
+    private transient DataType[] fieldDataTypes;
+    private final Set<String> headerFields;
+    private final Map<String, String> properties;
+    private final Map<String, Integer> columnIndexMapping;
+    private final Map<Integer, Integer> dataIndexMapping;
+    private long lastLogExceptionTime;
+    private long lastLogHandleFieldTime;
+
+    private static final int DEFAULT_LOG_INTERVAL_MS = 60 * 1000;
+
+    public RowDeserializationSchema(
+            TableSchema tableSchema,
+            DirtyDataStrategy formatErrorStrategy,
+            DirtyDataStrategy fieldMissingStrategy,
+            DirtyDataStrategy fieldIncrementStrategy,
+            String encoding,
+            String fieldDelimiter,
+            String lineDelimiter,
+            boolean columnErrorDebug,
+            boolean hasMetadata,
+            MetadataConverter[] metadataConverters,
+            List<String> headerFields,
+            Map<String, String> properties) {
+        this.tableSchema = tableSchema;
+        this.formatErrorStrategy = formatErrorStrategy;
+        this.fieldMissingStrategy = fieldMissingStrategy;
+        this.fieldIncrementStrategy = fieldIncrementStrategy;
+        this.columnErrorDebug = columnErrorDebug;
+        this.encoding = encoding;
+        this.fieldDelimiter = StringEscapeUtils.unescapeJava(fieldDelimiter);
+        this.lineDelimiter = StringEscapeUtils.unescapeJava(lineDelimiter);
+        this.metadataCollector = new MetadataCollector(hasMetadata, metadataConverters);
+        this.headerFields = headerFields == null ? null : new HashSet<>(headerFields);
+        this.properties = properties;
+        this.totalColumnSize = tableSchema.getFieldNames().length;
+        int dataColumnSize = 0;
+        this.fieldTypes = new ValueType[totalColumnSize];
+        this.columnIndexMapping = new HashMap<>();
+        this.dataIndexMapping = new HashMap<>();
+        for (int index = 0; index < tableSchema.getFieldNames().length; index++) {
+            this.columnIndexMapping.put(tableSchema.getFieldNames()[index], index);
+        }
+        for (int index = 0; index < totalColumnSize; index++) {
+            ValueType type =
+                    ByteSerializer.getTypeIndex(tableSchema.getFieldTypes()[index].getTypeClass());
+            this.fieldTypes[index] = type;
+            if (!isHeaderField(index)) {
+                dataIndexMapping.put(dataColumnSize, index);
+                dataColumnSize++;
+            }
+        }
+        this.dataColumnSize = dataColumnSize;
+    }
+
+    @Override
+    public void open(InitializationContext context) {
+        DescriptorProperties descriptorProperties = new DescriptorProperties();
+        descriptorProperties.putProperties(properties);
+        this.tableSchema = SchemaValidator.deriveTableSinkSchema(descriptorProperties);
+        this.fieldDataTypes = tableSchema.getFieldDataTypes();
+        this.lastLogExceptionTime = System.currentTimeMillis();
+        this.lastLogHandleFieldTime = System.currentTimeMillis();
+    }
+
+    @Override
+    public void deserialize(List<BytesMessage> messages, Collector<RowData> collector) {
+        metadataCollector.collector = collector;
+        deserialize(messages, metadataCollector);
+    }
+
+    private void deserialize(List<BytesMessage> messages, MetadataCollector collector) {
+        if (null == messages || messages.size() == 0) {
+            return;
+        }
+        for (BytesMessage message : messages) {
+            collector.message = message;
+            if (isOnlyHaveVarbinaryDataField()) {
+                GenericRowData rowData = new GenericRowData(totalColumnSize);
+                int dataIndex = dataIndexMapping.get(0);
+                rowData.setField(dataIndex, message.getData());
+                for (int index = 0; index < totalColumnSize; index++) {
+                    if (index == dataIndex) {
+                        continue;
+                    }
+                    String headerValue = getHeaderValue(message, index);
+                    rowData.setField(
+                            index,
+                            StringSerializer.deserialize(
+                                    headerValue,
+                                    fieldTypes[index],
+                                    fieldDataTypes[index],
+                                    new HashSet<>()));
+                }
+                collector.collect(rowData);
+            } else if (isAllHeaderField()) {
+                GenericRowData rowData = new GenericRowData(totalColumnSize);
+                for (int index = 0; index < totalColumnSize; index++) {
+                    String headerValue = getHeaderValue(message, index);
+                    rowData.setField(
+                            index,
+                            StringSerializer.deserialize(
+                                    headerValue,
+                                    fieldTypes[index],
+                                    fieldDataTypes[index],
+                                    new HashSet<>()));
+                }
+                collector.collect(rowData);
+            } else {
+                if (message.getData() == null) {
+                    logger.info("Deserialize empty BytesMessage body, ignore the empty message.");
+                    return;
+                }
+                deserializeBytesMessage(message, collector);
+            }
+        }
+    }
+
+    private boolean isOnlyHaveVarbinaryDataField() {
+        if (dataColumnSize == 1 && dataIndexMapping.size() == 1) {
+            int index = dataIndexMapping.get(0);
+            return isByteArrayType(tableSchema.getFieldNames()[index]);
+        }
+        return false;
+    }
+
+    private boolean isAllHeaderField() {
+        return null != headerFields && headerFields.size() == tableSchema.getFieldNames().length;
+    }
+
+    private void deserializeBytesMessage(BytesMessage message, Collector<RowData> collector) {
+        String body;
+        try {
+            body = new String(message.getData(), encoding);
+        } catch (UnsupportedEncodingException e) {
+            throw new RuntimeException(e);
+        }
+        String[] lines = StringUtils.split(body, lineDelimiter);
+        for (String line : lines) {
+            String[] data = StringUtils.splitPreserveAllTokens(line, fieldDelimiter);
+            if (dataColumnSize == 1) {
+                data = new String[1];
+                data[0] = line;
+            }
+            if (data.length < dataColumnSize) {
+                data = handleFieldMissing(data);
+            } else if (data.length > dataColumnSize) {
+                data = handleFieldIncrement(data);
+            }
+            if (data == null) {
+                continue;
+            }
+            GenericRowData rowData = new GenericRowData(totalColumnSize);
+            boolean skip = false;
+            for (int index = 0; index < totalColumnSize; index++) {
+                try {
+                    String fieldValue = getValue(message, data, line, index);
+                    rowData.setField(
+                            index,
+                            StringSerializer.deserialize(
+                                    fieldValue,
+                                    fieldTypes[index],
+                                    fieldDataTypes[index],
+                                    new HashSet<>()));
+                } catch (Exception e) {
+                    skip = handleException(rowData, index, data, e);
+                }
+            }
+            if (skip) {
+                continue;
+            }
+            collector.collect(rowData);
+        }
+    }
+
+    private boolean isHeaderField(int index) {
+        return headerFields != null && headerFields.contains(tableSchema.getFieldNames()[index]);
+    }
+
+    private String getHeaderValue(BytesMessage message, int index) {
+        Object object = message.getProperty(tableSchema.getFieldNames()[index]);
+        return object == null ? "" : (String) object;
+    }
+
+    private String getValue(BytesMessage message, String[] data, String line, int index) {
+        String fieldValue = null;
+        if (isHeaderField(index)) {
+            fieldValue = getHeaderValue(message, index);
+        } else {
+            if (dataColumnSize == 1) {
+                fieldValue = line;
+            } else {
+                if (index < data.length) {
+                    fieldValue = data[index];
+                }
+            }
+        }
+
+        return fieldValue;
+    }
+
+    private boolean isByteArrayType(String fieldName) {
+        TypeInformation<?> typeInformation =
+                tableSchema.getFieldTypes()[columnIndexMapping.get(fieldName)];
+        if (typeInformation != null) {
+            ValueType valueType = ByteSerializer.getTypeIndex(typeInformation.getTypeClass());
+            return valueType == ValueType.V_ByteArray;
+        }
+        return false;
+    }
+
+    private boolean handleException(GenericRowData row, int index, Object[] data, Exception e) {
+        boolean skip = false;
+        switch (formatErrorStrategy) {
+            case SKIP:
+                long now = System.currentTimeMillis();
+                if (columnErrorDebug || now - lastLogExceptionTime > DEFAULT_LOG_INTERVAL_MS) {
+                    logger.warn(
+                            "Data format error, field type: "
+                                    + fieldTypes[index]
+                                    + "field data: "
+                                    + data[index]
+                                    + ", index: "
+                                    + index
+                                    + ", data: ["
+                                    + StringUtils.join(data, ",")
+                                    + "]",
+                            e);
+                    lastLogExceptionTime = now;
+                }
+                skip = true;
+                break;
+            case SKIP_SILENT:
+                skip = true;
+                break;
+            default:
+            case CUT:
+            case NULL:
+            case PAD:
+                row.setField(index, null);
+                break;
+            case EXCEPTION:
+                throw new RuntimeException(e);
+        }
+
+        return skip;
+    }
+
+    private String[] handleFieldMissing(String[] data) {
+        switch (fieldMissingStrategy) {
+            default:
+            case SKIP:
+                long now = System.currentTimeMillis();
+                if (columnErrorDebug || now - lastLogHandleFieldTime > DEFAULT_LOG_INTERVAL_MS) {
+                    logger.warn(
+                            "Field missing error, table column number: "
+                                    + totalColumnSize
+                                    + ", data column number: "
+                                    + dataColumnSize
+                                    + ", data field number: "
+                                    + data.length
+                                    + ", data: ["
+                                    + StringUtils.join(data, ",")
+                                    + "]");
+                    lastLogHandleFieldTime = now;
+                }
+                return null;
+            case SKIP_SILENT:
+                return null;
+            case CUT:
+            case NULL:
+            case PAD:
+                {
+                    String[] res = new String[totalColumnSize];
+                    for (int i = 0; i < data.length; ++i) {
+                        Object dataIndex = dataIndexMapping.get(i);
+                        if (dataIndex != null) {
+                            res[(int) dataIndex] = data[i];
+                        }
+                    }
+                    return res;
+                }
+            case EXCEPTION:
+                throw new RuntimeException();
+        }
+    }
+
+    private String[] handleFieldIncrement(String[] data) {
+        switch (fieldIncrementStrategy) {
+            case SKIP:
+                long now = System.currentTimeMillis();
+                if (columnErrorDebug || now - lastLogHandleFieldTime > DEFAULT_LOG_INTERVAL_MS) {
+                    logger.warn(
+                            "Field increment error, table column number: "
+                                    + totalColumnSize
+                                    + ", data column number: "
+                                    + dataColumnSize
+                                    + ", data field number: "
+                                    + data.length
+                                    + ", data: ["
+                                    + StringUtils.join(data, ",")
+                                    + "]");
+                    lastLogHandleFieldTime = now;
+                }
+                return null;
+            case SKIP_SILENT:
+                return null;
+            default:
+            case CUT:
+            case NULL:
+            case PAD:
+                {
+                    String[] res = new String[totalColumnSize];
+                    for (int i = 0; i < dataColumnSize; ++i) {
+                        Object dataIndex = dataIndexMapping.get(i);
+                        if (dataIndex != null) {
+                            res[(int) dataIndex] = data[i];
+                        }
+                    }
+                    return res;
+                }
+            case EXCEPTION:
+                throw new RuntimeException();
+        }
+    }
+
+    @Override
+    public TypeInformation<RowData> getProducedType() {
+        return InternalTypeInfo.of((RowType) tableSchema.toRowDataType().getLogicalType());
+    }
+
+    // --------------------------------------------------------------------------------------------
+
+    /** Source metadata converter interface. */
+    public interface MetadataConverter extends Serializable {
+        Object read(BytesMessage message);
+    }
+
+    // --------------------------------------------------------------------------------------------
+
+    /** Metadata of RowData collector. */
+    public static final class MetadataCollector implements Collector<RowData>, Serializable {
+
+        private static final long serialVersionUID = 1L;
+
+        private final boolean hasMetadata;
+        private final MetadataConverter[] metadataConverters;
+
+        public transient BytesMessage message;
+        public transient Collector<RowData> collector;
+
+        public MetadataCollector(boolean hasMetadata, MetadataConverter[] metadataConverters) {
+            this.hasMetadata = hasMetadata;
+            this.metadataConverters = metadataConverters;
+        }
+
+        @Override
+        public void collect(RowData physicalRow) {
+            if (hasMetadata) {
+                final int physicalArity = physicalRow.getArity();
+                final int metadataArity = metadataConverters.length;
+                final GenericRowData producedRow =
+                        new GenericRowData(physicalRow.getRowKind(), physicalArity + metadataArity);
+                final GenericRowData genericPhysicalRow = (GenericRowData) physicalRow;
+                for (int index = 0; index < physicalArity; index++) {
+                    producedRow.setField(index, genericPhysicalRow.getField(index));
+                }
+                for (int index = 0; index < metadataArity; index++) {
+                    producedRow.setField(
+                            index + physicalArity, metadataConverters[index].read(message));
+                }
+                collector.collect(producedRow);
+            } else {
+                collector.collect(physicalRow);
+            }
+        }
+
+        @Override
+        public void close() {
+            // nothing to do
+        }
+    }
+
+    /** Builder of {@link RowDeserializationSchema}. */
+    public static class Builder {
+
+        private TableSchema schema;
+        private DirtyDataStrategy formatErrorStrategy = DirtyDataStrategy.SKIP;
+        private DirtyDataStrategy fieldMissingStrategy = DirtyDataStrategy.SKIP;
+        private DirtyDataStrategy fieldIncrementStrategy = DirtyDataStrategy.CUT;
+        private String encoding = "UTF-8";
+        private String lineDelimiter = "\n";
+        private String fieldDelimiter = "\u0001";
+        private boolean columnErrorDebug = false;
+        private boolean hasMetadata;
+        private MetadataConverter[] metadataConverters;
+        private List<String> headerFields;
+        private Map<String, String> properties;
+
+        public Builder() {}
+
+        public Builder setTableSchema(TableSchema tableSchema) {
+            this.schema = tableSchema;
+            return this;
+        }
+
+        public Builder setFormatErrorStrategy(DirtyDataStrategy formatErrorStrategy) {
+            this.formatErrorStrategy = formatErrorStrategy;
+            return this;
+        }
+
+        public Builder setFieldMissingStrategy(DirtyDataStrategy fieldMissingStrategy) {
+            this.fieldMissingStrategy = fieldMissingStrategy;
+            return this;
+        }
+
+        public Builder setFieldIncrementStrategy(DirtyDataStrategy fieldIncrementStrategy) {
+            this.fieldIncrementStrategy = fieldIncrementStrategy;
+            return this;
+        }
+
+        public Builder setEncoding(String encoding) {
+            this.encoding = encoding;
+            return this;
+        }
+
+        public Builder setFieldDelimiter(String fieldDelimiter) {
+            this.fieldDelimiter = fieldDelimiter;
+            return this;
+        }
+
+        public Builder setLineDelimiter(String lineDelimiter) {
+            this.lineDelimiter = lineDelimiter;
+            return this;
+        }
+
+        public Builder setColumnErrorDebug(boolean columnErrorDebug) {
+            this.columnErrorDebug = columnErrorDebug;
+            return this;
+        }
+
+        public Builder setHasMetadata(boolean hasMetadata) {
+            this.hasMetadata = hasMetadata;
+            return this;
+        }
+
+        public Builder setMetadataConverters(MetadataConverter[] metadataConverters) {
+            this.metadataConverters = metadataConverters;
+            return this;
+        }
+
+        public Builder setHeaderFields(List<String> headerFields) {
+            this.headerFields = headerFields;
+            return this;
+        }
+
+        public Builder setProperties(Map<String, String> properties) {
+            this.properties = properties;
+            if (null == properties) {
+                return this;
+            }
+            Configuration configuration = new Configuration();
+            for (String key : properties.keySet()) {
+                configuration.setString(key, properties.get(key));
+            }
+            String lengthCheck = configuration.get(CollectorOption.LENGTH_CHECK);
+            switch (lengthCheck.toUpperCase()) {
+                case "SKIP":
+                    {
+                        this.setFormatErrorStrategy(DirtyDataStrategy.SKIP);
+                        this.setFieldMissingStrategy(DirtyDataStrategy.SKIP);
+                        this.setFieldIncrementStrategy(DirtyDataStrategy.SKIP);
+                    }
+                    break;
+                case "PAD":
+                    {
+                        this.setFormatErrorStrategy(DirtyDataStrategy.SKIP);
+                        this.setFieldMissingStrategy(DirtyDataStrategy.PAD);
+                        this.setFieldIncrementStrategy(DirtyDataStrategy.CUT);
+                    }
+                    break;
+                case "EXCEPTION":
+                    {
+                        this.setFormatErrorStrategy(DirtyDataStrategy.EXCEPTION);
+                        this.setFieldMissingStrategy(DirtyDataStrategy.EXCEPTION);
+                        this.setFieldIncrementStrategy(DirtyDataStrategy.EXCEPTION);
+                    }
+                    break;
+                case "SKIP_SILENT":
+                    {
+                        this.setFormatErrorStrategy(DirtyDataStrategy.SKIP_SILENT);
+                        this.setFieldMissingStrategy(DirtyDataStrategy.SKIP_SILENT);
+                        this.setFieldIncrementStrategy(DirtyDataStrategy.SKIP_SILENT);
+                    }
+                    break;
+                default:
+            }
+            this.setEncoding(configuration.getString(CollectorOption.ENCODING));
+            this.setFieldDelimiter(configuration.getString(CollectorOption.FIELD_DELIMITER));
+            this.setLineDelimiter(configuration.getString(CollectorOption.LINE_DELIMITER));
+            this.setColumnErrorDebug(configuration.getBoolean(CollectorOption.COLUMN_ERROR_DEBUG));
+            return this;
+        }
+
+        public RowDeserializationSchema build() {
+            return new RowDeserializationSchema(
+                    schema,
+                    formatErrorStrategy,
+                    fieldMissingStrategy,
+                    fieldIncrementStrategy,
+                    encoding,
+                    fieldDelimiter,
+                    lineDelimiter,
+                    columnErrorDebug,
+                    hasMetadata,
+                    metadataConverters,
+                    headerFields,
+                    properties);
+        }
+    }
+
+    /** Options for {@link RowDeserializationSchema}. */
+    public static class CollectorOption {
+        public static final ConfigOption<String> ENCODING =
+                ConfigOptions.key("encoding".toLowerCase()).defaultValue("UTF-8");
+        public static final ConfigOption<String> FIELD_DELIMITER =
+                ConfigOptions.key("fieldDelimiter".toLowerCase()).defaultValue("\u0001");
+        public static final ConfigOption<String> LINE_DELIMITER =
+                ConfigOptions.key("lineDelimiter".toLowerCase()).defaultValue("\n");
+        public static final ConfigOption<Boolean> COLUMN_ERROR_DEBUG =
+                ConfigOptions.key("columnErrorDebug".toLowerCase()).defaultValue(true);
+        public static final ConfigOption<String> LENGTH_CHECK =
+                ConfigOptions.key("lengthCheck".toLowerCase()).defaultValue("NONE");
+    }
+}
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
new file mode 100644
index 0000000..ec41fc6
--- /dev/null
+++ b/src/main/java/org/apache/rocketmq/flink/source/table/RocketMQDynamicTableSourceFactory.java
@@ -0,0 +1,211 @@
+/*
+ * 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.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.DynamicTableFactory;
+import org.apache.flink.table.factories.DynamicTableSourceFactory;
+import org.apache.flink.table.factories.Factory;
+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.HashMap;
+import java.util.HashSet;
+import java.util.Map;
+import java.util.Set;
+import java.util.TimeZone;
+import java.util.stream.Collectors;
+
+import static org.apache.flink.table.factories.FactoryUtil.createTableFactoryHelper;
+import static org.apache.rocketmq.flink.source.common.RocketMQOptions.CONSUMER_GROUP;
+import static org.apache.rocketmq.flink.source.common.RocketMQOptions.NAME_SERVER_ADDRESS;
+import static org.apache.rocketmq.flink.source.common.RocketMQOptions.OPTIONAL_COLUMN_ERROR_DEBUG;
+import static org.apache.rocketmq.flink.source.common.RocketMQOptions.OPTIONAL_ENCODING;
+import static org.apache.rocketmq.flink.source.common.RocketMQOptions.OPTIONAL_END_TIME;
+import static org.apache.rocketmq.flink.source.common.RocketMQOptions.OPTIONAL_FIELD_DELIMITER;
+import static org.apache.rocketmq.flink.source.common.RocketMQOptions.OPTIONAL_LENGTH_CHECK;
+import static org.apache.rocketmq.flink.source.common.RocketMQOptions.OPTIONAL_LINE_DELIMITER;
+import static org.apache.rocketmq.flink.source.common.RocketMQOptions.OPTIONAL_PARTITION_DISCOVERY_INTERVAL_MS;
+import static org.apache.rocketmq.flink.source.common.RocketMQOptions.OPTIONAL_START_MESSAGE_OFFSET;
+import static org.apache.rocketmq.flink.source.common.RocketMQOptions.OPTIONAL_START_TIME;
+import static org.apache.rocketmq.flink.source.common.RocketMQOptions.OPTIONAL_START_TIME_MILLS;
+import static org.apache.rocketmq.flink.source.common.RocketMQOptions.OPTIONAL_TAG;
+import static org.apache.rocketmq.flink.source.common.RocketMQOptions.OPTIONAL_TIME_ZONE;
+import static org.apache.rocketmq.flink.source.common.RocketMQOptions.TOPIC;
+
+/**
+ * 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_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_ENCODING);
+        optionalOptions.add(OPTIONAL_FIELD_DELIMITER);
+        optionalOptions.add(OPTIONAL_LINE_DELIMITER);
+        optionalOptions.add(OPTIONAL_COLUMN_ERROR_DEBUG);
+        optionalOptions.add(OPTIONAL_LENGTH_CHECK);
+        return optionalOptions;
+    }
+
+    @Override
+    public DynamicTableSource createDynamicTableSource(Context context) {
+        transformContext(this, 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);
+        int startMessageOffset = configuration.getInteger(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);
+        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);
+        DescriptorProperties descriptorProperties = new DescriptorProperties();
+        descriptorProperties.putProperties(rawProperties);
+        TableSchema physicalSchema =
+                TableSchemaUtils.getPhysicalSchema(context.getCatalogTable().getSchema());
+        descriptorProperties.putTableSchema("schema", physicalSchema);
+        return new RocketMQScanTableSource(
+                descriptorProperties,
+                physicalSchema,
+                topic,
+                consumerGroup,
+                nameServerAddress,
+                tag,
+                stopInMs,
+                startMessageOffset,
+                startMessageOffset < 0 ? startTime : -1L,
+                partitionDiscoveryIntervalMs);
+    }
+
+    private void transformContext(
+            DynamicTableFactory factory, DynamicTableFactory.Context context) {
+        Map<String, String> catalogOptions = context.getCatalogTable().getOptions();
+        Map<String, String> convertedOptions =
+                normalizeOptionCaseAsFactory(factory, catalogOptions);
+        catalogOptions.clear();
+        for (Map.Entry<String, String> entry : convertedOptions.entrySet()) {
+            catalogOptions.put(entry.getKey(), entry.getValue());
+        }
+    }
+
+    private Map<String, String> normalizeOptionCaseAsFactory(
+            Factory factory, Map<String, String> options) {
+        Map<String, String> normalizedOptions = new HashMap<>();
+        Map<String, String> requiredOptionKeysLowerCaseToOriginal =
+                factory.requiredOptions().stream()
+                        .collect(
+                                Collectors.toMap(
+                                        option -> option.key().toLowerCase(), ConfigOption::key));
+        Map<String, String> optionalOptionKeysLowerCaseToOriginal =
+                factory.optionalOptions().stream()
+                        .collect(
+                                Collectors.toMap(
+                                        option -> option.key().toLowerCase(), ConfigOption::key));
+        for (Map.Entry<String, String> entry : options.entrySet()) {
+            final String catalogOptionKey = entry.getKey();
+            final String catalogOptionValue = entry.getValue();
+            normalizedOptions.put(
+                    requiredOptionKeysLowerCaseToOriginal.containsKey(
+                                    catalogOptionKey.toLowerCase())
+                            ? requiredOptionKeysLowerCaseToOriginal.get(
+                                    catalogOptionKey.toLowerCase())
+                            : optionalOptionKeysLowerCaseToOriginal.getOrDefault(
+                                    catalogOptionKey.toLowerCase(), catalogOptionKey),
+                    catalogOptionValue);
+        }
+        return normalizedOptions;
+    }
+
+    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/rocketmq/flink/source/table/RocketMQScanTableSource.java
new file mode 100644
index 0000000..37ab6a5
--- /dev/null
+++ b/src/main/java/org/apache/rocketmq/flink/source/table/RocketMQScanTableSource.java
@@ -0,0 +1,195 @@
+/*
+ * 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.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.table.api.DataTypes;
+import org.apache.flink.table.api.TableSchema;
+import org.apache.flink.table.connector.ChangelogMode;
+import org.apache.flink.table.connector.source.DynamicTableSource;
+import org.apache.flink.table.connector.source.ScanTableSource;
+import org.apache.flink.table.connector.source.SourceProvider;
+import org.apache.flink.table.connector.source.abilities.SupportsReadingMetadata;
+import org.apache.flink.table.data.RowData;
+import org.apache.flink.table.data.StringData;
+import org.apache.flink.table.descriptors.DescriptorProperties;
+import org.apache.flink.table.types.DataType;
+
+import java.util.Collections;
+import java.util.LinkedHashMap;
+import java.util.List;
+import java.util.Map;
+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 {
+
+    private final DescriptorProperties properties;
+    private final TableSchema schema;
+
+    private final String topic;
+    private final String consumerGroup;
+    private final String nameServerAddress;
+    private final String tag;
+
+    private final long stopInMs;
+    private final long partitionDiscoveryIntervalMs;
+    private final long startMessageOffset;
+    private final long startTime;
+
+    private List<String> metadataKeys;
+
+    public RocketMQScanTableSource(
+            DescriptorProperties properties,
+            TableSchema schema,
+            String topic,
+            String consumerGroup,
+            String nameServerAddress,
+            String tag,
+            long stopInMs,
+            long startMessageOffset,
+            long startTime,
+            long partitionDiscoveryIntervalMs) {
+        this.properties = properties;
+        this.schema = schema;
+        this.topic = topic;
+        this.consumerGroup = consumerGroup;
+        this.nameServerAddress = nameServerAddress;
+        this.tag = tag;
+        this.stopInMs = stopInMs;
+        this.startMessageOffset = startMessageOffset;
+        this.startTime = startTime;
+        this.partitionDiscoveryIntervalMs = partitionDiscoveryIntervalMs;
+        this.metadataKeys = Collections.emptyList();
+    }
+
+    @Override
+    public ChangelogMode getChangelogMode() {
+        return ChangelogMode.insertOnly();
+    }
+
+    @Override
+    public ScanRuntimeProvider getScanRuntimeProvider(ScanContext scanContext) {
+        return SourceProvider.of(
+                new RocketMQSource<>(
+                        topic,
+                        consumerGroup,
+                        nameServerAddress,
+                        tag,
+                        stopInMs,
+                        startTime,
+                        startMessageOffset < 0 ? 0 : startMessageOffset,
+                        partitionDiscoveryIntervalMs,
+                        isBounded() ? BOUNDED : CONTINUOUS_UNBOUNDED,
+                        createDeserializationSchema()));
+    }
+
+    @Override
+    public Map<String, DataType> listReadableMetadata() {
+        final Map<String, DataType> metadataMap = new LinkedHashMap<>();
+        Stream.of(ReadableMetadata.values())
+                .forEachOrdered(m -> metadataMap.putIfAbsent(m.key, m.dataType));
+        return metadataMap;
+    }
+
+    @Override
+    public void applyReadableMetadata(List<String> metadataKeys, DataType producedDataType) {
+        this.metadataKeys = metadataKeys;
+    }
+
+    @Override
+    public DynamicTableSource copy() {
+        RocketMQScanTableSource tableSource =
+                new RocketMQScanTableSource(
+                        properties,
+                        schema,
+                        topic,
+                        consumerGroup,
+                        nameServerAddress,
+                        tag,
+                        stopInMs,
+                        startMessageOffset,
+                        startTime,
+                        partitionDiscoveryIntervalMs);
+        tableSource.metadataKeys = metadataKeys;
+        return tableSource;
+    }
+
+    @Override
+    public String asSummaryString() {
+        return "RocketMQScanTableSource";
+    }
+
+    private RocketMQDeserializationSchema<RowData> createDeserializationSchema() {
+        final MetadataConverter[] metadataConverters =
+                metadataKeys.stream()
+                        .map(
+                                k ->
+                                        Stream.of(ReadableMetadata.values())
+                                                .filter(rm -> rm.key.equals(k))
+                                                .findFirst()
+                                                .orElseThrow(IllegalStateException::new))
+                        .map(m -> m.converter)
+                        .toArray(MetadataConverter[]::new);
+        return new RocketMQRowDeserializationSchema(
+                schema, properties.asMap(), metadataKeys.size() > 0, metadataConverters);
+    }
+
+    private boolean isBounded() {
+        return stopInMs != Long.MAX_VALUE;
+    }
+
+    // --------------------------------------------------------------------------------------------
+    // Metadata handling
+    // --------------------------------------------------------------------------------------------
+
+    enum ReadableMetadata {
+        TOPIC(
+                "topic",
+                DataTypes.STRING().notNull(),
+                new MetadataConverter() {
+                    private static final long serialVersionUID = 1L;
+
+                    @Override
+                    public Object read(BytesMessage message) {
+                        return StringData.fromString(
+                                String.valueOf(message.getProperty("__topic__")));
+                    }
+                });
+
+        final String key;
+
+        final DataType dataType;
+
+        final MetadataConverter converter;
+
+        ReadableMetadata(String key, DataType dataType, MetadataConverter converter) {
+            this.key = key;
+            this.dataType = dataType;
+            this.converter = converter;
+        }
+    }
+}
diff --git a/src/main/java/org/apache/rocketmq/flink/source/util/ByteSerializer.java b/src/main/java/org/apache/rocketmq/flink/source/util/ByteSerializer.java
new file mode 100644
index 0000000..358cb84
--- /dev/null
+++ b/src/main/java/org/apache/rocketmq/flink/source/util/ByteSerializer.java
@@ -0,0 +1,156 @@
+/*
+ * 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.util;
+
+import java.math.BigDecimal;
+import java.math.BigInteger;
+import java.nio.charset.Charset;
+import java.nio.charset.StandardCharsets;
+import java.sql.Date;
+import java.sql.Time;
+import java.sql.Timestamp;
+import java.time.Duration;
+import java.time.LocalDate;
+import java.time.LocalDateTime;
+import java.time.LocalTime;
+import java.time.OffsetDateTime;
+import java.time.Period;
+
+/** BytesSerializer is responsible to deserialize field from byte array. */
+public class ByteSerializer {
+
+    public static final Charset DEFAULT_CHARSET = StandardCharsets.UTF_8;
+
+    public static Object deserialize(byte[] value, ValueType type) {
+        return deserialize(value, type, DEFAULT_CHARSET);
+    }
+
+    public static Object deserialize(byte[] value, ValueType type, Charset charset) {
+        switch (type) {
+            case V_String:
+                return null == value ? "" : new String(value, charset);
+            case V_Timestamp: // sql.Timestamp encoded as long
+                return new Timestamp(ByteUtils.toLong(value));
+            case V_Date: // sql.Date encoded as long
+                return new Date(ByteUtils.toLong(value));
+            case V_Time: // sql.Time encoded as long
+                return new Time(ByteUtils.toLong(value));
+            case V_BigDecimal:
+                return ByteUtils.toBigDecimal(value);
+            default:
+                return commonDeserialize(value, type);
+        }
+    }
+
+    private static Object commonDeserialize(byte[] value, ValueType type) {
+        switch (type) {
+            case V_ByteArray: // byte[]
+                return value;
+            case V_Byte: // byte
+                return null == value ? (byte) '\0' : value[0];
+            case V_Short:
+                return ByteUtils.toShort(value);
+            case V_Integer:
+                return ByteUtils.toInt(value);
+            case V_Long:
+                return ByteUtils.toLong(value);
+            case V_Float:
+                return ByteUtils.toFloat(value);
+            case V_Double:
+                return ByteUtils.toDouble(value);
+            case V_Boolean:
+                return ByteUtils.toBoolean(value);
+            case V_BigInteger:
+                return new BigInteger(value);
+            default:
+                throw new IllegalArgumentException();
+        }
+    }
+
+    public static ValueType getTypeIndex(Class<?> clazz) {
+        if (byte[].class.equals(clazz)) {
+            return ValueType.V_ByteArray;
+        } else if (String.class.equals(clazz)) {
+            return ValueType.V_String;
+        } else if (Byte.class.equals(clazz)) {
+            return ValueType.V_Byte;
+        } else if (Short.class.equals(clazz)) {
+            return ValueType.V_Short;
+        } else if (Integer.class.equals(clazz)) {
+            return ValueType.V_Integer;
+        } else if (Long.class.equals(clazz)) {
+            return ValueType.V_Long;
+        } else if (Float.class.equals(clazz)) {
+            return ValueType.V_Float;
+        } else if (Double.class.equals(clazz)) {
+            return ValueType.V_Double;
+        } else if (Boolean.class.equals(clazz)) {
+            return ValueType.V_Boolean;
+        } else if (Timestamp.class.equals(clazz)) {
+            return ValueType.V_Timestamp;
+        } else if (Date.class.equals(clazz)) {
+            return ValueType.V_Date;
+        } else if (Time.class.equals(clazz)) {
+            return ValueType.V_Time;
+        } else if (BigDecimal.class.equals(clazz)) {
+            return ValueType.V_BigDecimal;
+        } else if (BigInteger.class.equals(clazz)) {
+            return ValueType.V_BigInteger;
+        } else if (LocalDateTime.class.equals(clazz)) {
+            return ValueType.V_LocalDateTime;
+        } else if (LocalDate.class.equals(clazz)) {
+            return ValueType.V_LocalDate;
+        } else if (Duration.class.equals(clazz)) {
+            return ValueType.V_Duration;
+        } else if (LocalTime.class.equals(clazz)) {
+            return ValueType.V_LocalTime;
+        } else if (Period.class.equals(clazz)) {
+            return ValueType.V_Period;
+        } else if (OffsetDateTime.class.equals(clazz)) {
+            return ValueType.V_OffsetDateTime;
+        } else {
+            return ValueType.Unsupported;
+        }
+    }
+
+    /** Value Type. */
+    public enum ValueType {
+        V_ByteArray,
+        V_String,
+        V_Byte,
+        V_Short,
+        V_Integer,
+        V_Long,
+        V_Float,
+        V_Double,
+        V_Boolean,
+        V_Timestamp,
+        V_Date,
+        V_Time,
+        V_BigDecimal,
+        V_BigInteger,
+        V_LocalDateTime,
+        V_LocalDate,
+        V_Duration,
+        V_LocalTime,
+        V_Period,
+        V_OffsetDateTime,
+        Unsupported
+    }
+}
diff --git a/src/main/java/org/apache/rocketmq/flink/source/util/ByteUtils.java b/src/main/java/org/apache/rocketmq/flink/source/util/ByteUtils.java
new file mode 100644
index 0000000..6e223a3
--- /dev/null
+++ b/src/main/java/org/apache/rocketmq/flink/source/util/ByteUtils.java
@@ -0,0 +1,219 @@
+/*
+ * 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.util;
+
+import java.math.BigDecimal;
+import java.math.BigInteger;
+
+/** Utility class to for operations related to bytes. */
+public class ByteUtils {
+
+    /**
+     * Converts a byte array to an int value.
+     *
+     * @param bytes byte array
+     * @return the int value
+     */
+    public static int toInt(byte[] bytes) {
+        return toInt(bytes, 0);
+    }
+
+    /**
+     * Converts a byte array to an int value.
+     *
+     * @param bytes byte array
+     * @param offset offset into array
+     * @return the int value
+     * @throws IllegalArgumentException if there's not enough room in the array at the offset
+     *     indicated.
+     */
+    public static int toInt(byte[] bytes, int offset) {
+        if (offset + Integer.BYTES > bytes.length) {
+            throw explainWrongLengthOrOffset(bytes, offset, Integer.BYTES, Integer.BYTES);
+        }
+        int n = 0;
+        for (int i = offset; i < (offset + Integer.BYTES); i++) {
+            n <<= 8;
+            n ^= bytes[i] & 0xFF;
+        }
+        return n;
+    }
+
+    /**
+     * Convert a byte array to a boolean.
+     *
+     * @param b array
+     * @return True or false.
+     */
+    public static boolean toBoolean(final byte[] b) {
+        return toBoolean(b, 0);
+    }
+
+    /**
+     * Convert a byte array to a boolean.
+     *
+     * @param b array
+     * @param offset offset into array
+     * @return True or false.
+     */
+    public static boolean toBoolean(final byte[] b, final int offset) {
+        if (offset + 1 > b.length) {
+            throw explainWrongLengthOrOffset(b, offset, 1, 1);
+        }
+        return b[offset] != (byte) 0;
+    }
+
+    /**
+     * Converts a byte array to a long value.
+     *
+     * @param bytes array
+     * @return the long value
+     */
+    public static long toLong(byte[] bytes) {
+        return toLong(bytes, 0);
+    }
+
+    /**
+     * Converts a byte array to a long value.
+     *
+     * @param bytes array of bytes
+     * @param offset offset into array
+     * @return the long value
+     * @throws IllegalArgumentException if there's not enough room in the array at the offset
+     *     indicated.
+     */
+    public static long toLong(byte[] bytes, int offset) {
+        if (offset + Long.BYTES > bytes.length) {
+            throw explainWrongLengthOrOffset(bytes, offset, Long.BYTES, Long.BYTES);
+        }
+        long l = 0;
+        for (int i = offset; i < offset + Long.BYTES; i++) {
+            l <<= 8;
+            l ^= bytes[i] & 0xFF;
+        }
+        return l;
+    }
+
+    /**
+     * Presumes float encoded as IEEE 754 floating-point "single format".
+     *
+     * @param bytes byte array
+     * @return Float made from passed byte array.
+     */
+    public static float toFloat(byte[] bytes) {
+        return toFloat(bytes, 0);
+    }
+
+    /**
+     * Presumes float encoded as IEEE 754 floating-point "single format".
+     *
+     * @param bytes array to convert
+     * @param offset offset into array
+     * @return Float made from passed byte array.
+     */
+    public static float toFloat(byte[] bytes, int offset) {
+        return Float.intBitsToFloat(toInt(bytes, offset));
+    }
+
+    /**
+     * Parse a byte array to double.
+     *
+     * @param bytes byte array
+     * @return Return double made from passed bytes.
+     */
+    public static double toDouble(final byte[] bytes) {
+        return toDouble(bytes, 0);
+    }
+
+    /**
+     * Parse a byte array to double.
+     *
+     * @param bytes byte array
+     * @param offset offset where double is
+     * @return Return double made from passed bytes.
+     */
+    public static double toDouble(final byte[] bytes, final int offset) {
+        return Double.longBitsToDouble(toLong(bytes, offset));
+    }
+
+    /**
+     * Converts a byte array to a short value.
+     *
+     * @param bytes byte array
+     * @return the short value
+     */
+    public static short toShort(byte[] bytes) {
+        return toShort(bytes, 0);
+    }
+
+    /**
+     * Converts a byte array to a short value.
+     *
+     * @param bytes byte array
+     * @param offset offset into array
+     * @return the short value
+     * @throws IllegalArgumentException if there's not enough room in the array at the offset
+     *     indicated.
+     */
+    public static short toShort(byte[] bytes, int offset) {
+        if (offset + Short.BYTES > bytes.length) {
+            throw explainWrongLengthOrOffset(bytes, offset, Short.BYTES, Short.BYTES);
+        }
+        short n = 0;
+        n ^= bytes[offset] & 0xFF;
+        n <<= 8;
+        n ^= bytes[offset + 1] & 0xFF;
+        return n;
+    }
+
+    // ---------------------------------------------------------------------------------------------------------
+
+    private static IllegalArgumentException explainWrongLengthOrOffset(
+            final byte[] bytes, final int offset, final int length, final int expectedLength) {
+        String exceptionMessage;
+        if (length != expectedLength) {
+            exceptionMessage = "Wrong length: " + length + ", expected " + expectedLength;
+        } else {
+            exceptionMessage =
+                    "offset ("
+                            + offset
+                            + ") + length ("
+                            + length
+                            + ") exceed the"
+                            + " capacity of the array: "
+                            + bytes.length;
+        }
+        return new IllegalArgumentException(exceptionMessage);
+    }
+
+    public static BigDecimal toBigDecimal(byte[] bytes) {
+        return toBigDecimal(bytes, 0, bytes.length);
+    }
+
+    public static BigDecimal toBigDecimal(byte[] bytes, int offset, int length) {
+        if (bytes != null && length >= 5 && offset + length <= bytes.length) {
+            int scale = toInt(bytes, offset);
+            byte[] tcBytes = new byte[length - 4];
+            System.arraycopy(bytes, offset + 4, tcBytes, 0, length - 4);
+            return new BigDecimal(new BigInteger(tcBytes), scale);
+        } else {
+            return null;
+        }
+    }
+}
diff --git a/src/main/java/org/apache/rocketmq/flink/source/util/StringSerializer.java b/src/main/java/org/apache/rocketmq/flink/source/util/StringSerializer.java
new file mode 100644
index 0000000..b468ac9
--- /dev/null
+++ b/src/main/java/org/apache/rocketmq/flink/source/util/StringSerializer.java
@@ -0,0 +1,155 @@
+/*
+ * 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.util;
+
+import org.apache.flink.table.data.DecimalData;
+import org.apache.flink.table.data.binary.BinaryStringData;
+import org.apache.flink.table.data.util.DataFormatConverters;
+import org.apache.flink.table.data.util.DataFormatConverters.TimestampConverter;
+import org.apache.flink.table.types.DataType;
+import org.apache.flink.table.types.logical.DecimalType;
+
+import sun.misc.BASE64Decoder;
+
+import java.math.BigDecimal;
+import java.math.BigInteger;
+import java.sql.Date;
+import java.sql.Time;
+import java.sql.Timestamp;
+import java.util.Set;
+
+/** String serializer. */
+public class StringSerializer {
+
+    public static TimestampConverter timestampConverter = new TimestampConverter(3);
+    private static final BASE64Decoder decoder = new BASE64Decoder();
+
+    public static Object deserialize(
+            String value,
+            ByteSerializer.ValueType type,
+            DataType dataType,
+            Set<String> nullValues) {
+        return deserialize(value, type, dataType, nullValues, false);
+    }
+
+    public static Object deserialize(
+            String value,
+            ByteSerializer.ValueType type,
+            DataType dataType,
+            Set<String> nullValues,
+            Boolean isRGData) {
+        if (null != nullValues && nullValues.contains(value)) {
+            return null;
+        }
+        switch (type) {
+            case V_ByteArray: // byte[]
+                if (isRGData) {
+                    byte[] bytes = null;
+                    try {
+                        bytes = decoder.decodeBuffer(value);
+                    } catch (Exception e) {
+                        //
+                    }
+                    return bytes;
+                } else {
+                    return value.getBytes();
+                }
+            case V_String:
+                return BinaryStringData.fromString(value);
+            case V_Byte: // byte
+                return null == value ? null : Byte.parseByte(value);
+            case V_Short:
+                return null == value ? null : Short.parseShort(value);
+            case V_Integer:
+                return null == value ? null : Integer.parseInt(value);
+            case V_Long:
+                return null == value ? null : Long.parseLong(value);
+            case V_Float:
+                return null == value ? null : Float.parseFloat(value);
+            case V_Double:
+                return null == value ? null : Double.parseDouble(value);
+            case V_Boolean:
+                return null == value ? null : parseBoolean(value);
+            case V_Timestamp: // sql.Timestamp encoded as long
+                if (isRGData) {
+                    return null == value ? null : Long.parseLong(value);
+                }
+                if (null == value) {
+                    return null;
+                } else {
+                    try {
+                        return timestampConverter.toInternal(new Timestamp(Long.parseLong(value)));
+                    } catch (NumberFormatException e) {
+                        return timestampConverter.toInternal(Timestamp.valueOf(value));
+                    }
+                }
+            case V_Date: // sql.Date encoded as long
+                if (isRGData) {
+                    return null == value ? null : Long.parseLong(value);
+                }
+                return null == value
+                        ? null
+                        : DataFormatConverters.DateConverter.INSTANCE.toInternal(
+                                Date.valueOf(value));
+            case V_Time: // sql.Time encoded as long
+                if (isRGData) {
+                    return null == value ? null : Long.parseLong(value);
+                }
+                return null == value
+                        ? null
+                        : DataFormatConverters.TimeConverter.INSTANCE.toInternal(
+                                new Time(Long.parseLong(value)));
+            case V_BigDecimal:
+                DecimalType decimalType = (DecimalType) dataType.getLogicalType();
+                return value == null
+                        ? null
+                        : DecimalData.fromBigDecimal(
+                                new BigDecimal(value),
+                                decimalType.getPrecision(),
+                                decimalType.getScale());
+            case V_BigInteger:
+                return null == value ? null : new BigInteger(value);
+
+            default:
+                throw new IllegalArgumentException();
+        }
+    }
+
+    public static Object deserialize(
+            String value, ByteSerializer.ValueType type, DataType dataType, Boolean isRGData) {
+        return deserialize(value, type, dataType, null, isRGData);
+    }
+
+    public static Boolean parseBoolean(String s) {
+        if (s != null) {
+            if (s.equalsIgnoreCase("true") || s.equalsIgnoreCase("false")) {
+                return Boolean.valueOf(s);
+            }
+
+            if (s.equals("1")) {
+                return Boolean.TRUE;
+            }
+
+            if (s.equals("0")) {
+                return Boolean.FALSE;
+            }
+        }
+
+        throw new IllegalArgumentException();
+    }
+}
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
new file mode 100644
index 0000000..32de8b2
--- /dev/null
+++ b/src/main/resources/META-INF/services/org.apache.flink.table.factories.Factory
@@ -0,0 +1,16 @@
+# 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.
+
+org.apache.rocketmq.flink.source.table.RocketMQDynamicTableSourceFactory
\ No newline at end of file
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
new file mode 100644
index 0000000..a904b04
--- /dev/null
+++ b/src/test/java/org/apache/rocketmq/flink/source/reader/deserializer/RocketMQRowDeserializationSchemaTest.java
@@ -0,0 +1,141 @@
+/*
+ * 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__"));
+    }
+}

[rocketmq-flink] 14/33: Verify flink prs

Posted by du...@apache.org.
This is an automated email from the ASF dual-hosted git repository.

duhengforever pushed a commit to branch main
in repository https://gitbox.apache.org/repos/asf/rocketmq-flink.git

commit 42a5d6cb1fbd8a584e023546e73730055422f1e9
Author: vongosling <vo...@apache.org>
AuthorDate: Mon Jun 3 11:36:15 2019 +0800

    Verify flink prs
---
 .../org/apache/rocketmq/flink/RocketMQSource.java  |  2 --
 .../example}/example/RocketMQFlinkExample.java     | 27 +++++++++---------
 .../flink/example/example/SimpleConsumer.java}     | 21 +++++++-------
 .../flink/example/example/SimpleProducer.java}     | 33 ++++++++--------------
 style/rmq_checkstyle.xml                           |  2 ++
 5 files changed, 37 insertions(+), 48 deletions(-)

diff --git a/src/main/java/org/apache/rocketmq/flink/RocketMQSource.java b/src/main/java/org/apache/rocketmq/flink/RocketMQSource.java
index ccd6bb4..14c479b 100644
--- a/src/main/java/org/apache/rocketmq/flink/RocketMQSource.java
+++ b/src/main/java/org/apache/rocketmq/flink/RocketMQSource.java
@@ -318,8 +318,6 @@ public class RocketMQSource<OUT> extends RichParallelSourceFunction<OUT>
                 restoredOffsets = new ConcurrentHashMap<>();
             }
             for (Tuple2<MessageQueue, Long> mqOffsets : unionOffsetStates.get()) {
-                // unionOffsetStates is the restored global union state;
-                // should only snapshot mqs that actually belong to us
                 if (!restoredOffsets.containsKey(mqOffsets.f0) || restoredOffsets.get(mqOffsets.f0) < mqOffsets.f1) {
                     restoredOffsets.put(mqOffsets.f0, mqOffsets.f1);
                 }
diff --git a/src/test/java/org/apache/rocketmq/flink/example/RocketMQFlinkExample.java b/src/main/java/org/apache/rocketmq/flink/example/example/RocketMQFlinkExample.java
similarity index 81%
rename from src/test/java/org/apache/rocketmq/flink/example/RocketMQFlinkExample.java
rename to src/main/java/org/apache/rocketmq/flink/example/example/RocketMQFlinkExample.java
index f4f654e..92b8dbf 100644
--- a/src/test/java/org/apache/rocketmq/flink/example/RocketMQFlinkExample.java
+++ b/src/main/java/org/apache/rocketmq/flink/example/example/RocketMQFlinkExample.java
@@ -1,13 +1,12 @@
-/**
- * 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
+/*
+ * 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
+ *     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,
@@ -16,7 +15,7 @@
  * limitations under the License.
  */
 
-package org.apache.rocketmq.flink.example;
+package org.apache.rocketmq.flink.example.example;
 
 import java.util.HashMap;
 import java.util.Map;
@@ -36,7 +35,6 @@ public class RocketMQFlinkExample {
     public static void main(String[] args) {
         StreamExecutionEnvironment env = StreamExecutionEnvironment.getExecutionEnvironment();
 
-        // enable checkpoint
         env.enableCheckpointing(3000);
 
         Properties consumerProps = new Properties();
@@ -47,7 +45,7 @@ public class RocketMQFlinkExample {
         Properties producerProps = new Properties();
         producerProps.setProperty(RocketMQConfig.NAME_SERVER_ADDR, "localhost:9876");
         int msgDelayLevel = RocketMQConfig.MSG_DELAY_LEVEL05;
-        producerProps.setProperty(RocketMQConfig.MSG_DELAY_LEVEL,String.valueOf(msgDelayLevel));
+        producerProps.setProperty(RocketMQConfig.MSG_DELAY_LEVEL, String.valueOf(msgDelayLevel));
         // TimeDelayLevel is not supported for batching
         boolean batchFlag = msgDelayLevel <= 0;
 
@@ -60,7 +58,7 @@ public class RocketMQFlinkExample {
                     HashMap result = new HashMap();
                     result.put("id", in.get("id"));
                     String[] arr = in.get("address").toString().split("\\s+");
-                    result.put("province", arr[arr.length-1]);
+                    result.put("province", arr[arr.length - 1]);
                     out.collect(result);
                 }
             })
@@ -73,7 +71,8 @@ public class RocketMQFlinkExample {
 
         try {
             env.execute("rocketmq-flink-example");
-        } catch (Exception e) {
+        }
+        catch (Exception e) {
             e.printStackTrace();
         }
     }
diff --git a/src/test/java/org/apache/rocketmq/flink/example/ConsumerTest.java b/src/main/java/org/apache/rocketmq/flink/example/example/SimpleConsumer.java
similarity index 75%
rename from src/test/java/org/apache/rocketmq/flink/example/ConsumerTest.java
rename to src/main/java/org/apache/rocketmq/flink/example/example/SimpleConsumer.java
index 1b07b8d..c087513 100644
--- a/src/test/java/org/apache/rocketmq/flink/example/ConsumerTest.java
+++ b/src/main/java/org/apache/rocketmq/flink/example/example/SimpleConsumer.java
@@ -1,13 +1,12 @@
-/**
- * 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
+/*
+ * 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
+ *     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,
@@ -16,7 +15,7 @@
  * limitations under the License.
  */
 
-package org.apache.rocketmq.flink.example;
+package org.apache.rocketmq.flink.example.example;
 
 import java.util.List;
 
@@ -27,7 +26,7 @@ import org.apache.rocketmq.client.consumer.listener.MessageListenerConcurrently;
 import org.apache.rocketmq.client.exception.MQClientException;
 import org.apache.rocketmq.common.message.MessageExt;
 
-public class ConsumerTest {
+public class SimpleConsumer {
     public static void main(String[] args) {
         DefaultMQPushConsumer consumer = new DefaultMQPushConsumer("g00003");
         consumer.setNamesrvAddr("localhost:9876");
diff --git a/src/test/java/org/apache/rocketmq/flink/example/ProducerTest.java b/src/main/java/org/apache/rocketmq/flink/example/example/SimpleProducer.java
similarity index 52%
rename from src/test/java/org/apache/rocketmq/flink/example/ProducerTest.java
rename to src/main/java/org/apache/rocketmq/flink/example/example/SimpleProducer.java
index c04ca74..5a6b572 100644
--- a/src/test/java/org/apache/rocketmq/flink/example/ProducerTest.java
+++ b/src/main/java/org/apache/rocketmq/flink/example/example/SimpleProducer.java
@@ -1,13 +1,12 @@
-/**
- * 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
+/*
+ * 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
+ *     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,
@@ -16,15 +15,13 @@
  * limitations under the License.
  */
 
-package org.apache.rocketmq.flink.example;
+package org.apache.rocketmq.flink.example.example;
 
-import org.apache.rocketmq.client.exception.MQBrokerException;
 import org.apache.rocketmq.client.exception.MQClientException;
 import org.apache.rocketmq.client.producer.DefaultMQProducer;
 import org.apache.rocketmq.common.message.Message;
-import org.apache.rocketmq.remoting.exception.RemotingException;
 
-public class ProducerTest {
+public class SimpleProducer {
     public static void main(String[] args) {
         DefaultMQProducer producer = new DefaultMQProducer("p001");
         producer.setNamesrvAddr("localhost:9876");
@@ -34,16 +31,10 @@ public class ProducerTest {
             e.printStackTrace();
         }
         for (int i = 0; i < 10000; i++) {
-            Message msg = new Message("flink-source2" , "", "id_"+i, ("country_X province_" + i).getBytes());
+            Message msg = new Message("flink-source2", "", "id_" + i, ("country_X province_" + i).getBytes());
             try {
                 producer.send(msg);
-            } catch (MQClientException e) {
-                e.printStackTrace();
-            } catch (RemotingException e) {
-                e.printStackTrace();
-            } catch (MQBrokerException e) {
-                e.printStackTrace();
-            } catch (InterruptedException e) {
+            } catch (Exception e) {
                 e.printStackTrace();
             }
             System.out.println("send " + i);
diff --git a/style/rmq_checkstyle.xml b/style/rmq_checkstyle.xml
index e3155cc..d5d591d 100644
--- a/style/rmq_checkstyle.xml
+++ b/style/rmq_checkstyle.xml
@@ -32,10 +32,12 @@
         <property name="header" value="/\*\nLicensed to the Apache Software Foundation*"/>
     </module>
 
+    <!--
     <module name="RegexpSingleline">
         <property name="format" value="System\.out\.println"/>
         <property name="message" value="Prohibit invoking System.out.println in source code !"/>
     </module>
+    -->
 
     <module name="RegexpSingleline">
         <property name="format" value="//FIXME"/>

[rocketmq-flink] 16/33: code format

Posted by du...@apache.org.
This is an automated email from the ASF dual-hosted git repository.

duhengforever pushed a commit to branch main
in repository https://gitbox.apache.org/repos/asf/rocketmq-flink.git

commit 0be1ed7a027b0e88cec2bab420554251c73a7387
Author: zhu zhengwen <zh...@xiaohongshu.com>
AuthorDate: Fri Jun 21 16:00:13 2019 +0800

    code format
---
 src/main/java/org/apache/rocketmq/flink/RocketMQSource.java | 6 +++++-
 1 file changed, 5 insertions(+), 1 deletion(-)

diff --git a/src/main/java/org/apache/rocketmq/flink/RocketMQSource.java b/src/main/java/org/apache/rocketmq/flink/RocketMQSource.java
index dcb1d31..18277e0 100644
--- a/src/main/java/org/apache/rocketmq/flink/RocketMQSource.java
+++ b/src/main/java/org/apache/rocketmq/flink/RocketMQSource.java
@@ -19,7 +19,11 @@
 package org.apache.rocketmq.flink;
 
 import java.nio.charset.StandardCharsets;
-import java.util.*;
+import java.util.List;
+import java.util.Map;
+import java.util.Properties;
+import java.util.Set;
+import java.util.UUID;
 import java.util.concurrent.ConcurrentHashMap;
 
 import org.apache.commons.lang.Validate;

[rocketmq-flink] 33/33: introduce RocketMQ SQL connector document (#793)

Posted by du...@apache.org.
This is an automated email from the ASF dual-hosted git repository.

duhengforever pushed a commit to branch main
in repository https://gitbox.apache.org/repos/asf/rocketmq-flink.git

commit aafb8e41754b43dbb34316eafa9b0aae90b42b39
Author: SteNicholas <pr...@163.com>
AuthorDate: Tue Aug 24 16:20:02 2021 +0800

    introduce RocketMQ SQL connector document (#793)
---
 README.md | 57 +++++++++++++++++++++++++++++++++++++++++++++++++++++++++
 1 file changed, 57 insertions(+)

diff --git a/README.md b/README.md
index 97cb1f9..a458ca4 100644
--- a/README.md
+++ b/README.md
@@ -126,6 +126,63 @@ The following configurations are all from the class `org.apache.rocketmq.flink.l
 | consumer.delay.when.message.not.found | the delay time when messages were not found      |    10 |
 
 
+## RocketMQ SQL Connector
+
+### How to create a RocketMQ table
+
+The example below shows how to create a RocketMQ table:
+
+```sql
+CREATE TABLE rocketmq_source (
+  `user_id` BIGINT,
+  `item_id` BIGINT,
+  `behavior` STRING
+) WITH (
+  'connector' = 'rocketmq',
+  'topic' = 'user_behavior',
+  'consumeGroup' = 'behavior_consume_group',
+  'nameServerAddress' = '127.0.0.1:9876'
+);
+
+CREATE TABLE rocketmq_sink (
+  `user_id` BIGINT,
+  `item_id` BIGINT,
+  `behavior` STRING
+) WITH (
+  'connector' = 'rocketmq',
+  'topic' = 'user_behavior',
+  'produceGroup' = 'behavior_produce_group',
+  'nameServerAddress' = '127.0.0.1:9876'
+);
+```
+
+### Available Metadata
+
+The following connector metadata can be accessed as metadata columns in a table definition.
+
+The `R/W` column defines whether a metadata field is readable (`R`) and/or writable (`W`).
+Read-only columns must be declared `VIRTUAL` to exclude them during an `INSERT INTO` operation.
+
+| KEY            | DATA TYPE              | DESCRIPTION                                     | DEFAULT       |
+| --------------   |:-----------------------------:|:---------------------------------------------------:|:--------------------:|
+| topic            | STRING NOT NULL | Topic name of the RocketMQ record. | R                        |
+
+The extended `CREATE TABLE` example demonstrates the syntax for exposing these metadata fields:
+
+```sql
+CREATE TABLE rocketmq_source (
+  `topic` STRING METADATA VIRTUAL,
+  `user_id` BIGINT,
+  `item_id` BIGINT,
+  `behavior` STRING
+) WITH (
+  'connector' = 'rocketmq',
+  'topic' = 'user_behavior',
+  'consumeGroup' = 'behavior_consume_group',
+  'nameServerAddress' = '127.0.0.1:9876'
+);
+```
+
 ## License
 
 Licensed to the Apache Software Foundation (ASF) under one

[rocketmq-flink] 15/33: create client instance for each sink/source

Posted by du...@apache.org.
This is an automated email from the ASF dual-hosted git repository.

duhengforever pushed a commit to branch main
in repository https://gitbox.apache.org/repos/asf/rocketmq-flink.git

commit fc793137cc1808b422c2285e5a5353c685e57bac
Author: zhu zhengwen <zh...@xiaohongshu.com>
AuthorDate: Fri Jun 21 15:54:16 2019 +0800

    create client instance for each sink/source
---
 src/main/java/org/apache/rocketmq/flink/RocketMQSink.java   | 3 ++-
 src/main/java/org/apache/rocketmq/flink/RocketMQSource.java | 7 ++-----
 2 files changed, 4 insertions(+), 6 deletions(-)

diff --git a/src/main/java/org/apache/rocketmq/flink/RocketMQSink.java b/src/main/java/org/apache/rocketmq/flink/RocketMQSink.java
index 41bbcbe..ca6848d 100644
--- a/src/main/java/org/apache/rocketmq/flink/RocketMQSink.java
+++ b/src/main/java/org/apache/rocketmq/flink/RocketMQSink.java
@@ -22,6 +22,7 @@ import java.nio.charset.StandardCharsets;
 import java.util.LinkedList;
 import java.util.List;
 import java.util.Properties;
+import java.util.UUID;
 
 import org.apache.commons.lang.Validate;
 import org.apache.flink.configuration.Configuration;
@@ -87,7 +88,7 @@ public class RocketMQSink<IN> extends RichSinkFunction<IN> implements Checkpoint
         Validate.notNull(serializationSchema, "KeyValueSerializationSchema can not be null");
 
         producer = new DefaultMQProducer();
-        producer.setInstanceName(String.valueOf(getRuntimeContext().getIndexOfThisSubtask()));
+        producer.setInstanceName(String.valueOf(getRuntimeContext().getIndexOfThisSubtask()) + "_" + UUID.randomUUID());
         RocketMQConfig.buildProducerConfigs(props, producer);
 
         batchList = new LinkedList<>();
diff --git a/src/main/java/org/apache/rocketmq/flink/RocketMQSource.java b/src/main/java/org/apache/rocketmq/flink/RocketMQSource.java
index 14c479b..dcb1d31 100644
--- a/src/main/java/org/apache/rocketmq/flink/RocketMQSource.java
+++ b/src/main/java/org/apache/rocketmq/flink/RocketMQSource.java
@@ -19,10 +19,7 @@
 package org.apache.rocketmq.flink;
 
 import java.nio.charset.StandardCharsets;
-import java.util.List;
-import java.util.Map;
-import java.util.Properties;
-import java.util.Set;
+import java.util.*;
 import java.util.concurrent.ConcurrentHashMap;
 
 import org.apache.commons.lang.Validate;
@@ -115,7 +112,7 @@ public class RocketMQSource<OUT> extends RichParallelSourceFunction<OUT>
         pullConsumerScheduleService = new MQPullConsumerScheduleService(group);
         consumer = pullConsumerScheduleService.getDefaultMQPullConsumer();
 
-        consumer.setInstanceName(String.valueOf(getRuntimeContext().getIndexOfThisSubtask()));
+        consumer.setInstanceName(String.valueOf(getRuntimeContext().getIndexOfThisSubtask()) + "_" + UUID.randomUUID());
         RocketMQConfig.buildConsumerConfigs(props, consumer);
     }
 

[rocketmq-flink] 09/33: clean up some dirty code

Posted by du...@apache.org.
This is an automated email from the ASF dual-hosted git repository.

duhengforever pushed a commit to branch main
in repository https://gitbox.apache.org/repos/asf/rocketmq-flink.git

commit 8089fb1255507f71a775263033c908858c0fb940
Author: Jennifer-sarah <42...@users.noreply.github.com>
AuthorDate: Fri Mar 22 00:52:40 2019 +0800

    clean up some dirty code
    
    clean up some dirty code
---
 .../java/org/apache/rocketmq/flink/RocketMQSource.java     | 14 +++++++-------
 1 file changed, 7 insertions(+), 7 deletions(-)

diff --git a/src/main/java/org/apache/rocketmq/flink/RocketMQSource.java b/src/main/java/org/apache/rocketmq/flink/RocketMQSource.java
index f610efe..5b76e54 100644
--- a/src/main/java/org/apache/rocketmq/flink/RocketMQSource.java
+++ b/src/main/java/org/apache/rocketmq/flink/RocketMQSource.java
@@ -62,7 +62,7 @@ import static org.apache.rocketmq.flink.RocketMQUtils.getLong;
  * Otherwise, the source doesn't provide any reliability guarantees.
  */
 public class RocketMQSource<OUT> extends RichParallelSourceFunction<OUT>
-        implements CheckpointedFunction, CheckpointListener, ResultTypeQueryable<OUT> {
+    implements CheckpointedFunction, CheckpointListener, ResultTypeQueryable<OUT> {
 
     private static final long serialVersionUID = 1L;
 
@@ -131,15 +131,15 @@ public class RocketMQSource<OUT> extends RichParallelSourceFunction<OUT>
         final Object lock = context.getCheckpointLock();
 
         int delayWhenMessageNotFound = getInteger(props, RocketMQConfig.CONSUMER_DELAY_WHEN_MESSAGE_NOT_FOUND,
-                RocketMQConfig.DEFAULT_CONSUMER_DELAY_WHEN_MESSAGE_NOT_FOUND);
+            RocketMQConfig.DEFAULT_CONSUMER_DELAY_WHEN_MESSAGE_NOT_FOUND);
 
         String tag = props.getProperty(RocketMQConfig.CONSUMER_TAG, RocketMQConfig.DEFAULT_CONSUMER_TAG);
 
         int pullPoolSize = getInteger(props, RocketMQConfig.CONSUMER_PULL_POOL_SIZE,
-                RocketMQConfig.DEFAULT_CONSUMER_PULL_POOL_SIZE);
+            RocketMQConfig.DEFAULT_CONSUMER_PULL_POOL_SIZE);
 
         int pullBatchSize = getInteger(props, RocketMQConfig.CONSUMER_BATCH_SIZE,
-                RocketMQConfig.DEFAULT_CONSUMER_BATCH_SIZE);
+            RocketMQConfig.DEFAULT_CONSUMER_BATCH_SIZE);
 
         pullConsumerScheduleService.setPullThreadNums(pullPoolSize);
         pullConsumerScheduleService.registerPullTaskCallback(topic, new PullTaskCallback() {
@@ -234,7 +234,7 @@ public class RocketMQSource<OUT> extends RichParallelSourceFunction<OUT>
                         break;
                     case CONSUMER_OFFSET_TIMESTAMP:
                         offset = consumer.searchOffset(mq, getLong(props,
-                                RocketMQConfig.CONSUMER_OFFSET_FROM_TIMESTAMP, System.currentTimeMillis()));
+                            RocketMQConfig.CONSUMER_OFFSET_FROM_TIMESTAMP, System.currentTimeMillis()));
                         break;
                     default:
                         throw new IllegalArgumentException("Unknown value for CONSUMER_OFFSET_RESET_TO.");
@@ -311,7 +311,7 @@ public class RocketMQSource<OUT> extends RichParallelSourceFunction<OUT>
         LOG.debug("initialize State ...");
 
         this.unionOffsetStates = context.getOperatorStateStore().getUnionListState(new ListStateDescriptor<>(
-                OFFSETS_STATE_NAME, TypeInformation.of(new TypeHint<Tuple2<MessageQueue, Long>>() { })));
+            OFFSETS_STATE_NAME, TypeInformation.of(new TypeHint<Tuple2<MessageQueue, Long>>() { })));
 
         this.restored = context.isRestored();
 
@@ -337,7 +337,7 @@ public class RocketMQSource<OUT> extends RichParallelSourceFunction<OUT>
 
     @Override
     public void notifyCheckpointComplete(long checkpointId) throws Exception {
-        // consumer.c
+        // callback when checkpoint complete 
         if (!runningChecker.isRunning()) {
             LOG.debug("notifyCheckpointComplete() called on closed source; returning null.");
             return;

[rocketmq-flink] 31/33: [#786] RocketMQSourceFunction supports the close of ExecutorService and ScheduledExecutorService (#790)

Posted by du...@apache.org.
This is an automated email from the ASF dual-hosted git repository.

duhengforever pushed a commit to branch main
in repository https://gitbox.apache.org/repos/asf/rocketmq-flink.git

commit bf277903084e5fba4f7acaa91570d178ae3dd403
Author: SteNicholas <pr...@163.com>
AuthorDate: Thu Aug 19 12:15:47 2021 +0800

    [#786] RocketMQSourceFunction supports the close of ExecutorService and ScheduledExecutorService (#790)
---
 .../rocketmq/flink/legacy/RocketMQSourceFunction.java      | 14 ++++++++++++++
 1 file changed, 14 insertions(+)

diff --git a/src/main/java/org/apache/rocketmq/flink/legacy/RocketMQSourceFunction.java b/src/main/java/org/apache/rocketmq/flink/legacy/RocketMQSourceFunction.java
index 8821a6d..e46daed 100644
--- a/src/main/java/org/apache/rocketmq/flink/legacy/RocketMQSourceFunction.java
+++ b/src/main/java/org/apache/rocketmq/flink/legacy/RocketMQSourceFunction.java
@@ -376,18 +376,32 @@ public class RocketMQSourceFunction<OUT> extends RichParallelSourceFunction<OUT>
         log.debug("cancel ...");
         runningChecker.setRunning(false);
 
+        if (timer != null) {
+            timer.shutdown();
+            timer = null;
+        }
+
+        if (executor != null) {
+            executor.shutdown();
+            executor = null;
+        }
+
         if (consumer != null) {
             consumer.shutdown();
+            consumer = null;
         }
 
         if (offsetTable != null) {
             offsetTable.clear();
+            offsetTable = null;
         }
         if (restoredOffsets != null) {
             restoredOffsets.clear();
+            restoredOffsets = null;
         }
         if (pendingOffsetsToCommit != null) {
             pendingOffsetsToCommit.clear();
+            pendingOffsetsToCommit = null;
         }
     }
 

[rocketmq-flink] 19/33: [ISSUE #385]Add acl feature support for rocketmq-flink (#384)

Posted by du...@apache.org.
This is an automated email from the ASF dual-hosted git repository.

duhengforever pushed a commit to branch main
in repository https://gitbox.apache.org/repos/asf/rocketmq-flink.git

commit 7059330fc645e369ea79b9da867b1f3c01c6e4dc
Author: Heng Du <du...@apache.org>
AuthorDate: Wed Aug 21 10:11:34 2019 +0800

    [ISSUE #385]Add acl feature support for rocketmq-flink (#384)
    
    * Add acl support for rocketmq flink connector
    
    * Fix unit test error
---
 pom.xml                                            |  7 +++-
 .../org/apache/rocketmq/flink/RocketMQConfig.java  | 21 ++++++++++
 .../org/apache/rocketmq/flink/RocketMQSink.java    |  2 +-
 .../org/apache/rocketmq/flink/RocketMQSource.java  | 47 +++++++++++-----------
 4 files changed, 51 insertions(+), 26 deletions(-)

diff --git a/pom.xml b/pom.xml
index c715f7f..0d68314 100644
--- a/pom.xml
+++ b/pom.xml
@@ -33,7 +33,7 @@
         <!-- compiler settings properties -->
         <maven.compiler.source>1.8</maven.compiler.source>
         <maven.compiler.target>1.8</maven.compiler.target>
-        <rocketmq.version>4.2.0</rocketmq.version>
+        <rocketmq.version>4.5.2</rocketmq.version>
         <flink.version>1.7.0</flink.version>
         <commons-lang.version>2.5</commons-lang.version>
         <scala.binary.version>2.11</scala.binary.version>
@@ -67,6 +67,11 @@
         </dependency>
         <dependency>
             <groupId>org.apache.rocketmq</groupId>
+            <artifactId>rocketmq-acl</artifactId>
+            <version>${rocketmq.version}</version>
+        </dependency>
+        <dependency>
+            <groupId>org.apache.rocketmq</groupId>
             <artifactId>rocketmq-common</artifactId>
             <version>${rocketmq.version}</version>
             <exclusions>
diff --git a/src/main/java/org/apache/rocketmq/flink/RocketMQConfig.java b/src/main/java/org/apache/rocketmq/flink/RocketMQConfig.java
index 5b43b31..5a0784b 100644
--- a/src/main/java/org/apache/rocketmq/flink/RocketMQConfig.java
+++ b/src/main/java/org/apache/rocketmq/flink/RocketMQConfig.java
@@ -23,6 +23,8 @@ import java.util.UUID;
 
 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.ClientConfig;
 import org.apache.rocketmq.client.consumer.DefaultMQPullConsumer;
 import org.apache.rocketmq.client.producer.DefaultMQProducer;
@@ -53,6 +55,9 @@ public class RocketMQConfig {
     public static final String PRODUCER_TIMEOUT = "producer.timeout";
     public static final int DEFAULT_PRODUCER_TIMEOUT = 3000; // 3 seconds
 
+    public static final String ACCESS_KEY = "access.key";
+    public static final String SECRET_KEY = "secret.key";
+
 
     // Consumer related config
     public static final String CONSUMER_GROUP = "consumer.group"; // Required
@@ -152,4 +157,20 @@ public class RocketMQConfig {
         client.setHeartbeatBrokerInterval(getInteger(props,
             BROKER_HEART_BEAT_INTERVAL, DEFAULT_BROKER_HEART_BEAT_INTERVAL));
     }
+
+
+    /**
+     * Build credentials for client.
+     * @param props
+     * @return
+     */
+    public static AclClientRPCHook buildAclRPCHook(Properties props) {
+        String accessKey = props.getProperty(ACCESS_KEY);
+        String secretKey = props.getProperty(SECRET_KEY);
+        if (!StringUtils.isEmpty(accessKey) && !StringUtils.isEmpty(secretKey)) {
+            AclClientRPCHook aclClientRPCHook = new AclClientRPCHook(new SessionCredentials(accessKey, secretKey));
+            return aclClientRPCHook;
+        }
+        return null;
+    }
 }
diff --git a/src/main/java/org/apache/rocketmq/flink/RocketMQSink.java b/src/main/java/org/apache/rocketmq/flink/RocketMQSink.java
index e8f237f..f3e200d 100644
--- a/src/main/java/org/apache/rocketmq/flink/RocketMQSink.java
+++ b/src/main/java/org/apache/rocketmq/flink/RocketMQSink.java
@@ -87,7 +87,7 @@ public class RocketMQSink<IN> extends RichSinkFunction<IN> implements Checkpoint
         Validate.notNull(topicSelector, "TopicSelector can not be null");
         Validate.notNull(serializationSchema, "KeyValueSerializationSchema can not be null");
 
-        producer = new DefaultMQProducer();
+        producer = new DefaultMQProducer(RocketMQConfig.buildAclRPCHook(props));
         producer.setInstanceName(String.valueOf(getRuntimeContext().getIndexOfThisSubtask()) + "_" + UUID.randomUUID());
         RocketMQConfig.buildProducerConfigs(props, producer);
 
diff --git a/src/main/java/org/apache/rocketmq/flink/RocketMQSource.java b/src/main/java/org/apache/rocketmq/flink/RocketMQSource.java
index 06eecfb..e289b49 100644
--- a/src/main/java/org/apache/rocketmq/flink/RocketMQSource.java
+++ b/src/main/java/org/apache/rocketmq/flink/RocketMQSource.java
@@ -1,19 +1,14 @@
 /**
- * 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
+ * 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.
+ * 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;
@@ -26,7 +21,6 @@ import java.util.Properties;
 import java.util.Set;
 import java.util.UUID;
 import java.util.concurrent.ConcurrentHashMap;
-
 import org.apache.commons.collections.map.LinkedMap;
 import org.apache.commons.lang.Validate;
 import org.apache.flink.api.common.state.ListState;
@@ -47,7 +41,6 @@ import org.apache.rocketmq.client.consumer.MQPullConsumerScheduleService;
 import org.apache.rocketmq.client.consumer.PullResult;
 import org.apache.rocketmq.client.consumer.PullTaskCallback;
 import org.apache.rocketmq.client.consumer.PullTaskContext;
-import org.apache.rocketmq.client.consumer.store.OffsetStore;
 import org.apache.rocketmq.client.exception.MQClientException;
 import org.apache.rocketmq.common.message.MessageExt;
 import org.apache.rocketmq.common.message.MessageQueue;
@@ -62,9 +55,8 @@ import static org.apache.rocketmq.flink.RocketMQUtils.getInteger;
 import static org.apache.rocketmq.flink.RocketMQUtils.getLong;
 
 /**
- * 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 guarantees.
+ * 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 guarantees.
  */
 public class RocketMQSource<OUT> extends RichParallelSourceFunction<OUT>
     implements CheckpointedFunction, CheckpointListener, ResultTypeQueryable<OUT> {
@@ -126,7 +118,8 @@ public class RocketMQSource<OUT> extends RichParallelSourceFunction<OUT>
 
         runningChecker = new RunningChecker();
 
-        pullConsumerScheduleService = new MQPullConsumerScheduleService(group);
+        //Wait for lite pull consumer
+        pullConsumerScheduleService = new MQPullConsumerScheduleService(group, RocketMQConfig.buildAclRPCHook(props));
         consumer = pullConsumerScheduleService.getDefaultMQPullConsumer();
 
         consumer.setInstanceName(String.valueOf(getRuntimeContext().getIndexOfThisSubtask()) + "_" + UUID.randomUUID());
@@ -270,10 +263,15 @@ public class RocketMQSource<OUT> extends RichParallelSourceFunction<OUT>
         if (pullConsumerScheduleService != null) {
             pullConsumerScheduleService.shutdown();
         }
-
-        offsetTable.clear();
-        restoredOffsets.clear();
-        pendingOffsetsToCommit.clear();
+        if (offsetTable != null) {
+            offsetTable.clear();
+        }
+        if (restoredOffsets != null) {
+            restoredOffsets.clear();
+        }
+        if (pendingOffsetsToCommit != null) {
+            pendingOffsetsToCommit.clear();
+        }
     }
 
     @Override
@@ -331,8 +329,9 @@ public class RocketMQSource<OUT> extends RichParallelSourceFunction<OUT>
         LOG.debug("initialize State ...");
 
         this.unionOffsetStates = context.getOperatorStateStore().getUnionListState(new ListStateDescriptor<>(
-            OFFSETS_STATE_NAME, TypeInformation.of(new TypeHint<Tuple2<MessageQueue, Long>>() { })));
+                OFFSETS_STATE_NAME, TypeInformation.of(new TypeHint<Tuple2<MessageQueue, Long>>() {
 
+        })));
         this.restored = context.isRestored();
 
         if (restored) {
@@ -369,7 +368,7 @@ public class RocketMQSource<OUT> extends RichParallelSourceFunction<OUT>
             return;
         }
 
-        Map<MessageQueue, Long> offsets = (Map<MessageQueue, Long>)pendingOffsetsToCommit.remove(posInMap);
+        Map<MessageQueue, Long> offsets = (Map<MessageQueue, Long>) pendingOffsetsToCommit.remove(posInMap);
 
         // remove older checkpoints in map
         for (int i = 0; i < posInMap; i++) {

[rocketmq-flink] 25/33: [rocketmq-connector-flink] rebalance cause offset rollback to long time ago (#672)

Posted by du...@apache.org.
This is an automated email from the ASF dual-hosted git repository.

duhengforever pushed a commit to branch main
in repository https://gitbox.apache.org/repos/asf/rocketmq-flink.git

commit da92a64a12f4de3b352f29182a4565cc4ac23ae4
Author: JerryTaoTao <27...@qq.com>
AuthorDate: Thu Jan 21 19:19:19 2021 +0800

    [rocketmq-connector-flink] rebalance cause offset rollback to long time ago (#672)
    
    Co-authored-by: hzyuemeng1 <hz...@corp.netease.com>
---
 .../org/apache/rocketmq/flink/RocketMQSource.java    | 20 ++++++++++++++++----
 1 file changed, 16 insertions(+), 4 deletions(-)

diff --git a/src/main/java/org/apache/rocketmq/flink/RocketMQSource.java b/src/main/java/org/apache/rocketmq/flink/RocketMQSource.java
index 35c5122..72783a8 100644
--- a/src/main/java/org/apache/rocketmq/flink/RocketMQSource.java
+++ b/src/main/java/org/apache/rocketmq/flink/RocketMQSource.java
@@ -24,6 +24,7 @@ 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.metrics.Counter;
+import org.apache.flink.util.Preconditions;
 import org.apache.flink.metrics.Meter;
 import org.apache.flink.metrics.MeterView;
 import org.apache.flink.metrics.SimpleCounter;
@@ -120,6 +121,10 @@ public class RocketMQSource<OUT> extends RichParallelSourceFunction<OUT>
         if (restoredOffsets == null) {
             restoredOffsets = new ConcurrentHashMap<>();
         }
+
+        //use restoredOffsets to init offset table.
+        initOffsetTableFromRestoredOffsets();
+
         if (pendingOffsetsToCommit == null) {
             pendingOffsetsToCommit = new LinkedMap();
         }
@@ -252,13 +257,10 @@ public class RocketMQSource<OUT> extends RichParallelSourceFunction<OUT>
         Long offset = offsetTable.get(mq);
         // restoredOffsets(unionOffsetStates) is the restored global union state;
         // should only snapshot mqs that actually belong to us
-        if (restored && offset == null) {
-            offset = restoredOffsets.get(mq);
-        }
         if (offset == null) {
             // fetchConsumeOffset from broker
             offset = consumer.fetchConsumeOffset(mq, false);
-            if (offset < 0) {
+            if (!restored || offset < 0) {
                 String initialOffset = props.getProperty(RocketMQConfig.CONSUMER_OFFSET_RESET_TO, CONSUMER_OFFSET_LATEST);
                 switch (initialOffset) {
                     case CONSUMER_OFFSET_EARLIEST:
@@ -318,6 +320,16 @@ public class RocketMQSource<OUT> extends RichParallelSourceFunction<OUT>
         }
     }
 
+    public void initOffsetTableFromRestoredOffsets() {
+        Preconditions.checkNotNull(restoredOffsets, "restoredOffsets can't be null");
+        restoredOffsets.forEach((mq, offset) -> {
+            if (!offsetTable.containsKey(mq) || offsetTable.get(mq) < offset) {
+                offsetTable.put(mq, offset);
+            }
+        });
+        log.info("init offset table from restoredOffsets successful.", offsetTable);
+    }
+
     @Override
     public void snapshotState(FunctionSnapshotContext context) throws Exception {
         // called when a snapshot for a checkpoint is requested

[rocketmq-flink] 22/33: feat(rocketmq-flink)upgrade rocketmq client to 4.7.1 (#589)

Posted by du...@apache.org.
This is an automated email from the ASF dual-hosted git repository.

duhengforever pushed a commit to branch main
in repository https://gitbox.apache.org/repos/asf/rocketmq-flink.git

commit a78d5750c0b89663ab1f101e85219792c5765a6a
Author: Heng Du <du...@apache.org>
AuthorDate: Tue Jul 7 15:22:38 2020 +0800

    feat(rocketmq-flink)upgrade rocketmq client to 4.7.1 (#589)
---
 pom.xml | 2 +-
 1 file changed, 1 insertion(+), 1 deletion(-)

diff --git a/pom.xml b/pom.xml
index 0d68314..b00d460 100644
--- a/pom.xml
+++ b/pom.xml
@@ -33,7 +33,7 @@
         <!-- compiler settings properties -->
         <maven.compiler.source>1.8</maven.compiler.source>
         <maven.compiler.target>1.8</maven.compiler.target>
-        <rocketmq.version>4.5.2</rocketmq.version>
+        <rocketmq.version>4.7.1</rocketmq.version>
         <flink.version>1.7.0</flink.version>
         <commons-lang.version>2.5</commons-lang.version>
         <scala.binary.version>2.11</scala.binary.version>

[rocketmq-flink] 23/33: [ISSUE #337] throw exception when send message to broker fail (#339)

Posted by du...@apache.org.
This is an automated email from the ASF dual-hosted git repository.

duhengforever pushed a commit to branch main
in repository https://gitbox.apache.org/repos/asf/rocketmq-flink.git

commit 635a72d46060abae45faf81eb7b009205d1eac89
Author: zhengwen zhu <ah...@gmail.com>
AuthorDate: Wed Jul 29 10:32:27 2020 +0800

    [ISSUE #337] throw exception when send message to broker fail (#339)
    
    * throw an exception when sending message to broker fail
    
    * remove useless import
    
    Co-authored-by: zhu zhengwen <zh...@xiaohongshu.com>
---
 src/main/java/org/apache/rocketmq/flink/RocketMQSink.java | 6 ++++++
 1 file changed, 6 insertions(+)

diff --git a/src/main/java/org/apache/rocketmq/flink/RocketMQSink.java b/src/main/java/org/apache/rocketmq/flink/RocketMQSink.java
index eecb72e..76d6a1f 100644
--- a/src/main/java/org/apache/rocketmq/flink/RocketMQSink.java
+++ b/src/main/java/org/apache/rocketmq/flink/RocketMQSink.java
@@ -35,9 +35,11 @@ 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.common.selector.TopicSelector;
 import org.apache.rocketmq.flink.common.serialization.KeyValueSerializationSchema;
+import org.apache.rocketmq.remoting.exception.RemotingException;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 
@@ -139,8 +141,12 @@ public class RocketMQSink<IN> extends RichSinkFunction<IN> implements Checkpoint
             try {
                 SendResult result = producer.send(msg);
                 LOG.debug("Sync send message result: {}", result);
+                if (result.getSendStatus() != SendStatus.SEND_OK) {
+                    throw new RemotingException(result.toString());
+                }
             } catch (Exception e) {
                 LOG.error("Sync send message failure!", e);
+                throw e;
             }
         }
     }