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

[rocketmq-connect] 02/08: Add Redis connector. (#403)

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

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

commit 14da1d84a26d98e66a075c3ac59faaeb5e8573fe
Author: liuboyan69 <69...@qq.com>
AuthorDate: Wed Aug 28 19:21:40 2019 +0800

    Add Redis connector. (#403)
    
    org.apache.rocketmq.connect.redis.connector.RedisSourceConnector
---
 README-CN.md                                       |   19 +
 README.md                                          |   18 +
 pom.xml                                            |  103 ++
 .../rocketmq/connect/redis/common/Config.java      |  237 +++++
 .../rocketmq/connect/redis/common/Options.java     |  123 +++
 .../connect/redis/common/RedisConstants.java       |   45 +
 .../rocketmq/connect/redis/common/SyncMod.java     |   37 +
 .../redis/connector/RedisSourceConnector.java      |   70 ++
 .../connect/redis/connector/RedisSourceTask.java   |  148 +++
 .../connect/redis/converter/KVEntryConverter.java  |   28 +
 .../redis/converter/RedisEntryConverter.java       |  134 +++
 .../redis/converter/RedisPositionConverter.java    |   54 +
 .../redis/handler/DefaultRedisEventHandler.java    |  174 +++
 .../connect/redis/handler/RedisEventHandler.java   |   72 ++
 .../redis/parser/AbstractCommandParser.java        |   82 ++
 .../connect/redis/parser/AppendParser.java         |   38 +
 .../connect/redis/parser/BitFieldParser.java       |   32 +
 .../rocketmq/connect/redis/parser/BitOpParser.java |   41 +
 .../connect/redis/parser/BrPopLPushParser.java     |   41 +
 .../connect/redis/parser/CommonParser.java         |  117 +++
 .../connect/redis/parser/DecrByParser.java         |   38 +
 .../rocketmq/connect/redis/parser/DecrParser.java  |   32 +
 .../redis/parser/DefaultRedisRdbParser.java        |  197 ++++
 .../rocketmq/connect/redis/parser/DelParser.java   |   46 +
 .../rocketmq/connect/redis/parser/EvalParser.java  |   38 +
 .../connect/redis/parser/EvalShaParser.java        |   38 +
 .../rocketmq/connect/redis/parser/ExecParser.java  |   32 +
 .../connect/redis/parser/ExpireAtParser.java       |   38 +
 .../connect/redis/parser/ExpireParser.java         |   40 +
 .../connect/redis/parser/FlushAllParser.java       |   32 +
 .../connect/redis/parser/FlushDbParser.java        |   33 +
 .../connect/redis/parser/GeoAddParser.java         |   49 +
 .../connect/redis/parser/GetsetParser.java         |   37 +
 .../rocketmq/connect/redis/parser/HDelParser.java  |   41 +
 .../connect/redis/parser/HIncrByParser.java        |   41 +
 .../connect/redis/parser/HSetNxParser.java         |   37 +
 .../rocketmq/connect/redis/parser/HSetParser.java  |   37 +
 .../rocketmq/connect/redis/parser/HmSetParser.java |   37 +
 .../connect/redis/parser/IncrByParser.java         |   38 +
 .../rocketmq/connect/redis/parser/IncrParser.java  |   32 +
 .../rocketmq/connect/redis/parser/LPopParser.java  |   32 +
 .../rocketmq/connect/redis/parser/LPushParser.java |   41 +
 .../connect/redis/parser/LPushXParser.java         |   37 +
 .../rocketmq/connect/redis/parser/LRemParser.java  |   41 +
 .../rocketmq/connect/redis/parser/LSetParser.java  |   41 +
 .../rocketmq/connect/redis/parser/LTrimParser.java |   41 +
 .../connect/redis/parser/LinsertParser.java        |   51 +
 .../connect/redis/parser/MSetNxParser.java         |   37 +
 .../rocketmq/connect/redis/parser/MSetParser.java  |   37 +
 .../rocketmq/connect/redis/parser/MoveParser.java  |   38 +
 .../rocketmq/connect/redis/parser/MultiParser.java |   32 +
 .../connect/redis/parser/PExpireAtParser.java      |   38 +
 .../connect/redis/parser/PExpireParser.java        |   38 +
 .../connect/redis/parser/PSetExParser.java         |   41 +
 .../rocketmq/connect/redis/parser/Parser.java      |   24 +
 .../connect/redis/parser/PersistParser.java        |   32 +
 .../rocketmq/connect/redis/parser/PfAddParser.java |   41 +
 .../connect/redis/parser/PfCountParser.java        |   37 +
 .../connect/redis/parser/PfMergeParser.java        |   41 +
 .../connect/redis/parser/PublishParser.java        |   37 +
 .../connect/redis/parser/RPopLPushParser.java      |   37 +
 .../rocketmq/connect/redis/parser/RPopParser.java  |   32 +
 .../rocketmq/connect/redis/parser/RPushParser.java |   41 +
 .../connect/redis/parser/RPushXParser.java         |   37 +
 .../connect/redis/parser/RedisRdbParser.java       |   67 ++
 .../connect/redis/parser/RenameNxParser.java       |   37 +
 .../connect/redis/parser/RenameParser.java         |   37 +
 .../connect/redis/parser/RestoreParser.java        |   45 +
 .../rocketmq/connect/redis/parser/SAddParser.java  |   41 +
 .../connect/redis/parser/SDiffStoreParser.java     |   41 +
 .../connect/redis/parser/SInterStoreParser.java    |   41 +
 .../rocketmq/connect/redis/parser/SMoveParser.java |   41 +
 .../rocketmq/connect/redis/parser/SRemParser.java  |   41 +
 .../connect/redis/parser/SUnionStoreParser.java    |   41 +
 .../connect/redis/parser/ScriptParser.java         |   32 +
 .../connect/redis/parser/SelectParser.java         |   38 +
 .../connect/redis/parser/SetBitParser.java         |   41 +
 .../rocketmq/connect/redis/parser/SetExParser.java |   42 +
 .../rocketmq/connect/redis/parser/SetNxParser.java |   37 +
 .../rocketmq/connect/redis/parser/SetParser.java   |   70 ++
 .../connect/redis/parser/SetRangeParser.java       |   41 +
 .../rocketmq/connect/redis/parser/SortParser.java  |   32 +
 .../connect/redis/parser/SwapDbParser.java         |   38 +
 .../connect/redis/parser/UnLinkParser.java         |   37 +
 .../rocketmq/connect/redis/parser/XAckParser.java  |   49 +
 .../rocketmq/connect/redis/parser/XAddParser.java  |   46 +
 .../connect/redis/parser/XClaimParser.java         |   88 ++
 .../rocketmq/connect/redis/parser/XDelParser.java  |   41 +
 .../connect/redis/parser/XGroupParser.java         |   32 +
 .../connect/redis/parser/XSetIdParser.java         |   37 +
 .../rocketmq/connect/redis/parser/XTrimParser.java |   43 +
 .../rocketmq/connect/redis/parser/ZAddParser.java  |   69 ++
 .../connect/redis/parser/ZIncrByParser.java        |   41 +
 .../connect/redis/parser/ZInterStoreParser.java    |   37 +
 .../connect/redis/parser/ZPopMaxParser.java        |   38 +
 .../connect/redis/parser/ZPopMinParser.java        |   38 +
 .../rocketmq/connect/redis/parser/ZRemParser.java  |   41 +
 .../connect/redis/parser/ZRemRangeByLexParser.java |   41 +
 .../redis/parser/ZRemRangeByRankParser.java        |   41 +
 .../redis/parser/ZRemRangeByScoreParser.java       |   41 +
 .../connect/redis/parser/ZUnionStoreParser.java    |   37 +
 .../apache/rocketmq/connect/redis/pojo/Geo.java    |   48 +
 .../rocketmq/connect/redis/pojo/KVEntry.java       |   71 ++
 .../rocketmq/connect/redis/pojo/RedisEntry.java    |  171 +++
 .../rocketmq/connect/redis/pojo/RedisEvent.java    |   59 ++
 .../processor/DefaultRedisEventProcessor.java      |  453 ++++++++
 .../redis/processor/RedisClosedListener.java       |   37 +
 .../redis/processor/RedisEventListener.java        |  121 +++
 .../redis/processor/RedisEventProcessor.java       |   40 +
 .../processor/RedisEventProcessorCallback.java     |   23 +
 .../redis/processor/RedisExceptionListener.java    |   44 +
 .../connect/redis/util/ParseStringUtils.java       |   54 +
 .../connect/redis/util/PropertyToObjectUtils.java  |   64 ++
 .../rocketmq/redis/test/common/ConfigTest.java     |  107 ++
 .../rocketmq/redis/test/common/OptionsTest.java    |   34 +
 .../test/connector/RedisSourceConnectorTest.java   |   35 +
 .../redis/test/connector/RedisSourceTaskTest.java  |  150 +++
 .../test/converter/RedisEntryConverterTest.java    |  118 +++
 .../test/converter/RedisPositionConverterTest.java |   51 +
 .../redis/test/handler/RedisEventHandlerTest.java  |  468 +++++++++
 .../rocketmq/redis/test/parser/ParserTest.java     | 1108 ++++++++++++++++++++
 .../apache/rocketmq/redis/test/pojo/GeoTest.java   |   19 +
 .../rocketmq/redis/test/pojo/KVEntryTest.java      |   85 ++
 .../rocketmq/redis/test/pojo/RedisEventTest.java   |   33 +
 .../redis/test/processor/ListenerTest.java         |  135 +++
 .../redis/test/processor/ProcessorTest.java        |  193 ++++
 .../redis/test/util/ParseStringUtilsTest.java      |   45 +
 .../redis/test/util/PropertyToObjectUtilsTest.java |  118 +++
 style/copyright/Apache.xml                         |   23 +
 style/copyright/profiles_settings.xml              |   64 ++
 style/rmq_checkstyle.xml                           |  134 +++
 style/rmq_codeStyle.xml                            |  143 +++
 132 files changed, 9278 insertions(+)

diff --git a/README-CN.md b/README-CN.md
new file mode 100644
index 0000000..786b0d6
--- /dev/null
+++ b/README-CN.md
@@ -0,0 +1,19 @@
+# RocketMQ-connect-redis
+##### RedisSourceConnector 完全限定名
+org.apache.rocketmq.connect.redis.connector.RedisSourceConnector
+
+
+##### 配置参数
+
+参数 | 作用 | 是否必填 | 默认值
+---|--- |--- | ---
+redisAddr | Redis服务IP地址。 | 是 | null
+redisPort | Redis服务端口。 | 是 | null
+redisPassword | Redis auth 密码。 | 是 | null
+timeout | 连接Redis超时时间,单位毫秒。 | 否 | 3000
+syncMod | 数据同步模式。 | 否 | CUSTOM_OFFSET
+offset | Redis位点信息。 | 否 | -1 
+replId | Redis master_replId,可用redis-cli通过info命令获取。 | 否  | null 
+commands | 需要监听的Redis操作命令。  | 否 | *
+eventCommitRetryTimes | 收到Redis event后提交到RedisEventProcessor的失败重试次数。 | 否 | 5
+eventCommitRetryInterval | 收到Redis event后提交到RedisEventProcessor的失败重试的时间间隔,单位毫秒。 | 否 | 100
diff --git a/README.md b/README.md
index 961f77a..6da01d7 100644
--- a/README.md
+++ b/README.md
@@ -1 +1,19 @@
 # RocketMQ-connect-redis
+##### RedisSourceConnector fully-qualified name
+org.apache.rocketmq.connect.redis.connector.RedisSourceConnector
+
+
+##### parameter configuration
+
+parameter | effect | required |default
+---|--- |--- | ---
+redisAddr | The address of the Redis. | yes | null
+redisPort | The port fo the Redis address. | yes | null
+redisPassword | The password to use when connecting to Redis. | yes | null
+timeout | The waiting time before connect to Redis success. | no | 3000
+syncMod | The mod for how to get data from redis. | no | CUSTOM_OFFSET
+offset | The position of Redis data. | no | -1 
+replId | The master replyId of Redis, which can get it with command "info" by redis-cli from Redis. | no  | null 
+commands | The Redis commands you want to sync, they are useful only for increment Redis data, multiple commands are separated by commas. | 否 | *
+eventCommitRetryTimes | The retry time when receive Redis change event, failed to commit to RedisEventProcessor. | no | 5
+eventCommitRetryInterval | The time when receive Redis change, failed to commit to RedisEventProcessor and retry commit. | no | 100
diff --git a/pom.xml b/pom.xml
new file mode 100644
index 0000000..c6a2586
--- /dev/null
+++ b/pom.xml
@@ -0,0 +1,103 @@
+<?xml version="1.0" encoding="UTF-8"?>
+<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">
+
+    <groupId>org.apache.rocketmq</groupId>
+    <packaging>pom</packaging>
+    <version>0.0.1-SNAPSHOT</version>
+    <modelVersion>4.0.0</modelVersion>
+
+    <artifactId>rocketmq-connect-redis</artifactId>
+
+    <properties>
+        <redis.replicator.version>3.3.0</redis.replicator.version>
+        <!-- Compiler settings properties -->
+        <maven.compiler.source>1.8</maven.compiler.source>
+        <maven.compiler.target>1.8</maven.compiler.target>
+    </properties>
+
+    <dependencies>
+        <dependency>
+            <groupId>commons-lang</groupId>
+            <artifactId>commons-lang</artifactId>
+            <version>2.6</version>
+        </dependency>
+        <dependency>
+            <groupId>io.openmessaging</groupId>
+            <artifactId>openmessaging-api</artifactId>
+            <version>0.3.1-alpha</version>
+        </dependency>
+        <dependency>
+            <groupId>org.slf4j</groupId>
+            <artifactId>slf4j-api</artifactId>
+            <version>1.7.5</version>
+        </dependency>
+        <dependency>
+            <groupId>ch.qos.logback</groupId>
+            <artifactId>logback-classic</artifactId>
+            <version>1.0.13</version>
+        </dependency>
+        <dependency>
+            <groupId>commons-codec</groupId>
+            <artifactId>commons-codec</artifactId>
+            <version>1.9</version>
+        </dependency>
+        <dependency>
+            <groupId>junit</groupId>
+            <artifactId>junit</artifactId>
+            <version>4.11</version>
+            <scope>test</scope>
+        </dependency>
+        <dependency>
+            <groupId>com.alibaba</groupId>
+            <artifactId>fastjson</artifactId>
+            <version>1.2.51</version>
+        </dependency>
+        <dependency>
+            <groupId>io.openmessaging</groupId>
+            <artifactId>openmessaging-connect</artifactId>
+            <version>0.1.0-beta</version>
+        </dependency>
+        <dependency>
+            <groupId>com.moilioncircle</groupId>
+            <artifactId>redis-replicator</artifactId>
+            <version>${redis.replicator.version}</version>
+            <exclusions>
+                <exclusion>
+                    <groupId>org.slf4j</groupId>
+                    <artifactId>slf4j-api</artifactId>
+                </exclusion>
+            </exclusions>
+        </dependency>
+        <dependency>
+            <groupId>redis.clients</groupId>
+            <artifactId>jedis</artifactId>
+            <version>2.9.0</version>
+        </dependency>
+        <dependency>
+            <groupId>org.mockito</groupId>
+            <artifactId>mockito-core</artifactId>
+            <version>2.6.3</version>
+            <scope>test</scope>
+        </dependency>
+    </dependencies>
+
+
+    <build>
+        <plugins>
+            <plugin>
+                <artifactId>maven-compiler-plugin</artifactId>
+                <version>3.6.1</version>
+                <configuration>
+                    <source>${maven.compiler.source}</source>
+                    <target>${maven.compiler.target}</target>
+                    <compilerVersion>${maven.compiler.source}</compilerVersion>
+                    <showDeprecation>true</showDeprecation>
+                    <showWarnings>true</showWarnings>
+                </configuration>
+            </plugin>
+        </plugins>
+    </build>
+
+</project>
\ No newline at end of file
diff --git a/src/main/java/org/apache/rocketmq/connect/redis/common/Config.java b/src/main/java/org/apache/rocketmq/connect/redis/common/Config.java
new file mode 100644
index 0000000..fe41075
--- /dev/null
+++ b/src/main/java/org/apache/rocketmq/connect/redis/common/Config.java
@@ -0,0 +1,237 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.rocketmq.connect.redis.common;
+
+import java.net.URISyntaxException;
+import java.nio.ByteBuffer;
+import java.util.Arrays;
+import java.util.Collections;
+import java.util.HashSet;
+import java.util.Set;
+
+import com.moilioncircle.redis.replicator.RedisURI;
+import org.apache.commons.lang.StringUtils;
+import org.apache.rocketmq.connect.redis.util.PropertyToObjectUtils;
+import io.openmessaging.KeyValue;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+/**
+ * Config info.
+ */
+public class Config {
+    private final Logger LOGGER = LoggerFactory.getLogger(Config.class);
+    /**
+     * Redis base info.
+     */
+    private String redisAddr;
+    private Integer redisPort;
+    private String redisPassword;
+    /**
+     * Timeout for jedis pool.
+     */
+    private Integer timeout = 3000;
+    private SyncMod syncMod = SyncMod.CUSTOM_OFFSET;
+    private Long offset = -1L;
+    /**
+     * Redis master_replid
+     */
+    private String replId;
+    /**
+     * Specify synchronized redis commands that are useful
+     * only for increment redis data.
+     *
+     * Multiple commands are separated by commas.
+     *
+     */
+    private String commands = RedisConstants.ALL_COMMAND;
+
+    /**
+     * Position info from connector runtime.
+     */
+    private Long position;
+
+    private Integer eventCommitRetryTimes = RedisConstants.EVENT_COMMIT_RETRY_TIMES;
+    private Long eventCommitRetryInterval = RedisConstants.EVENT_COMMIT_RETRY_INTERVAL;
+
+    public static final Set<String> REQUEST_CONFIG = Collections.unmodifiableSet(new HashSet<>(Arrays.asList(
+        "redisAddr",
+        "redisPort",
+        "redisPassword"
+    )));
+
+    @Override
+    public String toString() {
+        StringBuilder sb = new StringBuilder();
+        sb.append("redisAddr: ")
+            .append(redisPassword)
+            .append(",")
+            .append("redisPort: ")
+            .append(redisPort)
+            .append(",")
+            .append("redisPassword: ")
+            .append(redisPassword)
+            .append(",")
+            .append("syncMod: ")
+            .append(syncMod)
+            .append(",")
+            .append("offset: ")
+            .append(offset)
+            .append(",")
+            .append("replId: ")
+            .append(replId)
+            .append(",")
+            .append("commands: ")
+            .append(commands)
+        ;
+        return sb.toString();
+    }
+
+    public static String checkConfig(KeyValue config) {
+        for (String requestKey : Config.REQUEST_CONFIG) {
+            if (!config.containsKey(requestKey)) {
+                return "Request lost config key: " + requestKey;
+            }
+        }
+        return null;
+    }
+
+    public ByteBuffer getPositionPartitionKey() {
+        if (StringUtils.isBlank(redisAddr) || redisPort == null) {
+            return null;
+        }
+        return ByteBuffer.wrap((this.getRedisAddr() + this.getRedisPort()).getBytes());
+    }
+
+    public RedisURI getRedisUri() {
+        StringBuilder sb = new StringBuilder("redis://");
+        if (StringUtils.isBlank(redisAddr) || redisPort == null) {
+            return null;
+        }
+        sb.append(redisAddr).append(":").append(redisPort);
+        try {
+            return new RedisURI(sb.toString());
+        } catch (URISyntaxException e) {
+            LOGGER.error("redis uri error. {}", e);
+        }
+        return null;
+    }
+
+    public String load(KeyValue pros) {
+        try {
+            PropertyToObjectUtils.properties2Object(pros, this);
+        } catch (Exception e) {
+            LOGGER.error("load config failed. {}", e);
+            return "load config failed.";
+        }
+        return null;
+    }
+
+    public static Set<String> getRequestConfig() {
+        return REQUEST_CONFIG;
+    }
+
+    public String getRedisAddr() {
+        return redisAddr;
+    }
+
+    public void setRedisAddr(String redisAddr) {
+        this.redisAddr = redisAddr;
+    }
+
+    public Integer getRedisPort() {
+        return redisPort;
+    }
+
+    public void setRedisPort(Integer redisPort) {
+        this.redisPort = redisPort;
+    }
+
+    public String getRedisPassword() {
+        return redisPassword;
+    }
+
+    public void setRedisPassword(String redisPassword) {
+        this.redisPassword = redisPassword;
+    }
+
+    public Long getPosition() {
+        return position;
+    }
+
+    public void setPosition(Long position) {
+        this.position = position;
+    }
+
+    public SyncMod getSyncMod() {
+        return syncMod;
+    }
+
+    public void setSyncMod(String syncMod) {
+        this.syncMod = SyncMod.valueOf(syncMod);
+    }
+
+    public Long getOffset() {
+        return offset;
+    }
+
+    public void setOffset(Long offset) {
+        this.offset = offset;
+    }
+
+    public String getReplId() {
+        return replId;
+    }
+
+    public void setReplId(String replId) {
+        this.replId = replId;
+    }
+
+    public Integer getTimeout() {
+        return timeout;
+    }
+
+    public void setTimeout(Integer timeout) {
+        this.timeout = timeout;
+    }
+
+    public String getCommands() {
+        return commands;
+    }
+
+    public void setCommands(String commands) {
+        this.commands = commands;
+    }
+
+    public Integer getEventCommitRetryTimes() {
+        return eventCommitRetryTimes;
+    }
+
+    public void setEventCommitRetryTimes(Integer eventCommitRetryTimes) {
+        this.eventCommitRetryTimes = eventCommitRetryTimes;
+    }
+
+    public Long getEventCommitRetryInterval() {
+        return eventCommitRetryInterval;
+    }
+
+    public void setEventCommitRetryInterval(Long eventCommitRetryInterval) {
+        this.eventCommitRetryInterval = eventCommitRetryInterval;
+    }
+
+}
diff --git a/src/main/java/org/apache/rocketmq/connect/redis/common/Options.java b/src/main/java/org/apache/rocketmq/connect/redis/common/Options.java
new file mode 100644
index 0000000..0a02bd4
--- /dev/null
+++ b/src/main/java/org/apache/rocketmq/connect/redis/common/Options.java
@@ -0,0 +1,123 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.rocketmq.connect.redis.common;
+
+import java.util.concurrent.ConcurrentHashMap;
+import java.util.concurrent.ConcurrentMap;
+import com.moilioncircle.redis.replicator.rdb.datatype.EvictType;
+import com.moilioncircle.redis.replicator.rdb.datatype.ExpiredType;
+
+public class Options<T>{
+    private static ConcurrentMap<String, Options<Object>> pool = new ConcurrentHashMap<>(32);
+
+    public static final Options<Void> REDIS_PARTITION = newOption("DEFAULT_PARTITION");
+    public static final Options<Void> REDIS_QEUEUE = newOption("redis");
+    public static final Options<Void> REDIS_DATASOURCE = newOption("redis");
+    public static final Options<Void> REDIS_COMMAND = newOption("command");
+    public static final Options<Void> REDIS_KEY = newOption("key");
+    public static final Options<Void> REDIS_VALUE = newOption("value");
+    public static final Options<Void> REDIS_PARAMS = newOption("params");
+
+    public static final Options<String> REDIS_REPLID = newOption("replId");
+
+    public static final Options<Long> EXPIRED_TIME = newOption("EXPIRED_TIME");
+    public static final Options<ExpiredType> EXPIRED_TYPE = newOption("EXPIRED_TYPE");
+    public static final Options<Long> EVICT_VALUE = newOption("EVICT_VALUE");
+    public static final Options<EvictType> EVICT_TYPE = newOption("EVICT_TYPE");
+
+    public static final Options<Integer> REDIS_EX = newOption("EX");
+    public static final Options<Long> REDIS_PX = newOption("PX");
+    public static final Options<Boolean> REDIS_NX = newOption("NX");
+    public static final Options<Boolean> REDIS_XX = newOption("XX");
+    public static final Options<Long> REDIS_INDEX = newOption("INDEX");
+    public static final Options<Long> REDIS_OFFSET = newOption("CUSTOM_OFFSET");
+    public static final Options<Long> REDIS_COUNT = newOption("COUNT");
+    public static final Options<Long> REDIS_INCREMENT = newOption("LAST_OFFSET");
+    public static final Options<Integer> REDIS_DB_INDEX = newOption("DB_INDEX");
+    public static final Options<Long> REDIS_TIMEOUT = newOption("TIMEOUT");
+    public static final Options<Integer> REDIS_EX_TIMESTAMP = newOption("EX_TIMESTAMP");
+    public static final Options<Long> REDIS_PX_TIMESTAMP = newOption("PX_TIMESTAMP");
+    public static final Options<String> REDIS_GROUP = newOption("GROUP");
+    public static final Options<String> REDIS_ID = newOption("ID");
+    public static final Options<Boolean> REDIS_CH = newOption("CH");
+    public static final Options<Boolean> REDIS_INCR = newOption("INCR");
+    public static final Options<Boolean> REDIS_WEIGHTS = newOption("WEIGHTS");
+    public static final Options<String> REDIS_AGGREGATE = newOption("AGGREGATE");
+    public static final Options<Boolean> REDIS_BEFORE = newOption("BEFORE");
+    public static final Options<Boolean> REDIS_AFTER = newOption("AFTER");
+    public static final Options<Long> REDIS_TTL = newOption("TTL");
+    public static final Options<Boolean> REDIS_REPLACE = newOption("REPLACE");
+    public static final Options<Boolean> REDIS_XTRIM = newOption("~");
+    public static final Options<String> REDIS_CONSUMER = newOption("CONSUMER");
+    public static final Options<Long> REDIS_MIN_IDLE_TIME = newOption("MIN-IDLE-TIME");
+    public static final Options<Long> REDIS_IDLE = newOption("IDLE");
+    public static final Options<Long> REDIS_TIME = newOption("TIME");
+    public static final Options<Integer> REDIS_RETRYCOUNT = newOption("RETRYCOUNT");
+    public static final Options<Boolean> REDIS_FORCE = newOption("FORCE");
+    public static final Options<Boolean> REDIS_JUSTID = newOption("JUSTID");
+
+
+    private final String name;
+
+    public Options(String name) {
+        this.name = name;
+    }
+
+    public String name(){
+        return this.name;
+    }
+
+    public static <T> Options<T> valueOf(String name){
+        if(name == null || name.isEmpty()){
+            throw new IllegalArgumentException("empty name");
+        }
+        return (Options<T>)get(name);
+    }
+
+
+    private static <T> Options<T> newOption(String name){
+        if(name == null || name.isEmpty()){
+            throw new IllegalArgumentException("empty name");
+        }
+        return (Options<T>)getOrCreate(name);
+    }
+
+    private static Options<Object> getOrCreate(String name){
+        Options<Object> option = get(name);
+        if(option == null){
+            final Options<Object> tempOption = new Options<>(name);
+            option = save(name, tempOption);
+            if(option == null){
+                return tempOption;
+            }
+        }
+        return option;
+    }
+
+    private static Options<Object> get(String name){
+        Options<Object> option = pool.get(name);
+        return option;
+    }
+
+    private static Options<Object> save(String name, Options<Object> options){
+        return pool.putIfAbsent(name, options);
+    }
+
+}
+
+
diff --git a/src/main/java/org/apache/rocketmq/connect/redis/common/RedisConstants.java b/src/main/java/org/apache/rocketmq/connect/redis/common/RedisConstants.java
new file mode 100644
index 0000000..a6d1c0d
--- /dev/null
+++ b/src/main/java/org/apache/rocketmq/connect/redis/common/RedisConstants.java
@@ -0,0 +1,45 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.rocketmq.connect.redis.common;
+
+public class RedisConstants {
+    public static final String DB_INDEX = "dbIndex";
+    public static final String POSITION = "nextPosition";
+
+    public static final String NX = "NX";
+    public static final String PX = "PX";
+    public static final String EX = "EX";
+    public static final String XX = "XX";
+    public static final String CH = "CH";
+    public static final String INCR = "INCR";
+    public static final String IDLE = "IDLE";
+    public static final String TIME = "TIME";
+    public static final String RETRYCOUNT = "RETRYCOUNT";
+    public static final String FORCE = "FORCE";
+    public static final String JUSTID = "JUSTID";
+
+    public static final String REDIS_INFO_REPLICATION = "Replication";
+    public static final String REDIS_INFO_REPLICATION_MASTER_REPLID = "master_replid";
+    public static final String REDIS_INFO_REPLICATION_MASTER_REPL_OFFSET = "master_repl_offset";
+
+    public static final String ALL_COMMAND = "*";
+
+    public static final Integer EVENT_COMMIT_RETRY_TIMES = 5;
+    public static final Long EVENT_COMMIT_RETRY_INTERVAL = 100L;
+
+}
diff --git a/src/main/java/org/apache/rocketmq/connect/redis/common/SyncMod.java b/src/main/java/org/apache/rocketmq/connect/redis/common/SyncMod.java
new file mode 100644
index 0000000..9ad2093
--- /dev/null
+++ b/src/main/java/org/apache/rocketmq/connect/redis/common/SyncMod.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.connect.redis.common;
+
+public enum SyncMod {
+    /**
+     * sync data from the last offset.
+     */
+    LAST_OFFSET,
+    /**
+     * sync data from the custom offset, offset for - 1 synchronization of all existing data.
+     */
+    CUSTOM_OFFSET,
+    /**
+     * sync data form last offset, ignore the position from connector runtime.
+     */
+    LAST_OFFSET_FORCE,
+    /**
+     * sync data from the custom offset, ignore the position from connector runtime.
+     */
+    CUSTOM_OFFSET_FORCE
+}
diff --git a/src/main/java/org/apache/rocketmq/connect/redis/connector/RedisSourceConnector.java b/src/main/java/org/apache/rocketmq/connect/redis/connector/RedisSourceConnector.java
new file mode 100644
index 0000000..fde9f92
--- /dev/null
+++ b/src/main/java/org/apache/rocketmq/connect/redis/connector/RedisSourceConnector.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.connect.redis.connector;
+
+import java.util.ArrayList;
+import java.util.List;
+
+import io.openmessaging.KeyValue;
+import io.openmessaging.connector.api.Task;
+import io.openmessaging.connector.api.source.SourceConnector;
+import org.apache.rocketmq.connect.redis.common.Config;
+
+public class RedisSourceConnector extends SourceConnector {
+    private KeyValue keyValue;
+
+    @Override public String verifyAndSetConfig(KeyValue keyValue) {
+        this.keyValue = keyValue;
+        String msg = Config.checkConfig(keyValue);
+        if (msg != null) {
+            return msg;
+        }
+        return null;
+    }
+
+    @Override public void start() {
+
+    }
+
+
+    @Override public void stop() {
+
+    }
+
+
+    @Override public void pause() {
+
+    }
+
+
+    @Override public void resume() {
+
+    }
+
+
+    @Override public Class<? extends Task> taskClass() {
+        return RedisSourceTask.class;
+    }
+
+
+    @Override public List<KeyValue> taskConfigs() {
+        List<KeyValue> keyValues = new ArrayList<>();
+        keyValues.add(this.keyValue);
+        return keyValues;
+    }
+}
diff --git a/src/main/java/org/apache/rocketmq/connect/redis/connector/RedisSourceTask.java b/src/main/java/org/apache/rocketmq/connect/redis/connector/RedisSourceTask.java
new file mode 100644
index 0000000..1b97836
--- /dev/null
+++ b/src/main/java/org/apache/rocketmq/connect/redis/connector/RedisSourceTask.java
@@ -0,0 +1,148 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.rocketmq.connect.redis.connector;
+
+import io.openmessaging.connector.api.data.EntryType;
+import java.io.IOException;
+import java.nio.ByteBuffer;
+import java.util.Collection;
+
+import io.openmessaging.KeyValue;
+import io.openmessaging.connector.api.data.SourceDataEntry;
+import io.openmessaging.connector.api.source.SourceTask;
+import org.apache.rocketmq.connect.redis.common.Config;
+import org.apache.rocketmq.connect.redis.common.Options;
+import org.apache.rocketmq.connect.redis.converter.KVEntryConverter;
+import org.apache.rocketmq.connect.redis.converter.RedisEntryConverter;
+import org.apache.rocketmq.connect.redis.handler.DefaultRedisEventHandler;
+import org.apache.rocketmq.connect.redis.handler.RedisEventHandler;
+import org.apache.rocketmq.connect.redis.pojo.KVEntry;
+import org.apache.rocketmq.connect.redis.processor.DefaultRedisEventProcessor;
+import org.apache.rocketmq.connect.redis.processor.RedisEventProcessor;
+import org.apache.rocketmq.connect.redis.converter.RedisPositionConverter;
+import org.apache.rocketmq.connect.redis.processor.RedisEventProcessorCallback;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+public class RedisSourceTask extends SourceTask {
+    private static final Logger LOGGER = LoggerFactory.getLogger(RedisSourceTask.class);
+    /**
+     * listening and handle Redis event.
+     */
+    private RedisEventProcessor eventProcessor;
+    private Config config;
+    /**
+     * convert kVEntry to list of sourceDataEntry
+     */
+    private KVEntryConverter kvEntryConverter;
+
+    public RedisEventProcessor getEventProcessor() {
+        return eventProcessor;
+    }
+
+    public void setEventProcessor(RedisEventProcessor eventProcessor) {
+        this.eventProcessor = eventProcessor;
+    }
+
+    public Config getConfig() {
+        return config;
+    }
+
+
+    @Override public Collection<SourceDataEntry> poll() {
+        try {
+            KVEntry event = this.eventProcessor.poll();
+            if (event == null) {
+                return null;
+            }
+            event.queueName(Options.REDIS_QEUEUE.name());
+            event.entryType(EntryType.UPDATE);
+
+            Collection<SourceDataEntry> res = this.kvEntryConverter.kVEntryToDataEntries(event);
+            LOGGER.info("send data entries: {}", res);
+            return res;
+        } catch (InterruptedException e) {
+            LOGGER.error("redis task interrupted. {}", e);
+            this.stop();
+        } catch (Exception e) {
+            LOGGER.error("redis task error. {}", e);
+            this.stop();
+        }
+        return null;
+    }
+
+
+    @Override public void start(KeyValue keyValue) {
+        this.kvEntryConverter = new RedisEntryConverter();
+
+        this.config = new Config();
+        this.config.load(keyValue);
+        LOGGER.info("task config msg: {}", this.config.toString());
+
+        // get position info
+        ByteBuffer byteBuffer = this.context.positionStorageReader().getPosition(
+            this.config.getPositionPartitionKey()
+        );
+        Long position = RedisPositionConverter.jsonToLong(byteBuffer);
+        if (position != null && position >= -1) {
+            this.config.setPosition(position);
+        }
+        LOGGER.info("task load connector runtime position: {}", this.config.getPosition());
+
+        this.eventProcessor = new DefaultRedisEventProcessor(config);
+        RedisEventHandler eventHandler = new DefaultRedisEventHandler(this.config);
+        this.eventProcessor.registEventHandler(eventHandler);
+        this.eventProcessor.registProcessorCallback(new DefaultRedisEventProcessorCallback());
+        try {
+            this.eventProcessor.start();
+            LOGGER.info("Redis task start.");
+        } catch (IOException e) {
+            LOGGER.error("processor start error: {}", e);
+            this.stop();
+        }
+    }
+
+
+    @Override public void stop() {
+        if (this.eventProcessor != null) {
+            try {
+                this.eventProcessor.stop();
+                LOGGER.info("Redis task is stopped.");
+            } catch (IOException e) {
+                LOGGER.error("processor stop error: {}", e);
+            }
+        }
+    }
+
+
+    @Override public void pause() {
+
+    }
+
+
+    @Override public void resume() {
+
+    }
+
+    private class DefaultRedisEventProcessorCallback implements RedisEventProcessorCallback {
+        @Override public void onStop(RedisEventProcessor eventProcessor) {
+            stop();
+        }
+    }
+
+}
diff --git a/src/main/java/org/apache/rocketmq/connect/redis/converter/KVEntryConverter.java b/src/main/java/org/apache/rocketmq/connect/redis/converter/KVEntryConverter.java
new file mode 100644
index 0000000..de1b8ca
--- /dev/null
+++ b/src/main/java/org/apache/rocketmq/connect/redis/converter/KVEntryConverter.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.connect.redis.converter;
+
+import io.openmessaging.connector.api.data.SourceDataEntry;
+import java.util.List;
+import org.apache.rocketmq.connect.redis.pojo.KVEntry;
+
+public interface KVEntryConverter {
+
+    List<SourceDataEntry> kVEntryToDataEntries(KVEntry kvEntry);
+
+}
diff --git a/src/main/java/org/apache/rocketmq/connect/redis/converter/RedisEntryConverter.java b/src/main/java/org/apache/rocketmq/connect/redis/converter/RedisEntryConverter.java
new file mode 100644
index 0000000..23de075
--- /dev/null
+++ b/src/main/java/org/apache/rocketmq/connect/redis/converter/RedisEntryConverter.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.connect.redis.converter;
+
+import io.openmessaging.connector.api.data.DataEntryBuilder;
+import io.openmessaging.connector.api.data.Field;
+import io.openmessaging.connector.api.data.FieldType;
+import io.openmessaging.connector.api.data.Schema;
+import io.openmessaging.connector.api.data.SourceDataEntry;
+import java.nio.ByteBuffer;
+import java.util.ArrayList;
+import java.util.HashMap;
+import java.util.List;
+import java.util.Map;
+import java.util.concurrent.atomic.AtomicInteger;
+import org.apache.rocketmq.connect.redis.common.Options;
+import org.apache.rocketmq.connect.redis.converter.KVEntryConverter;
+import org.apache.rocketmq.connect.redis.pojo.KVEntry;
+
+public class RedisEntryConverter implements KVEntryConverter {
+    private final int maxValueSize = 500;
+
+    @Override public List<SourceDataEntry> kVEntryToDataEntries(KVEntry kvEntry) {
+        Schema schema = getRedisSchema(kvEntry.getValueType());
+
+        String partition = kvEntry.getPartition();
+        if (partition == null) {
+            throw new IllegalStateException("partition info error.");
+        }
+        List<SourceDataEntry> res = new ArrayList<>();
+        List<Object> values = splitValue(kvEntry.getValueType(), kvEntry.getValue(), this.maxValueSize);
+        for (int i = 0; i < values.size(); i++) {
+            DataEntryBuilder builder = newDataEntryBuilderWithoutValue(schema, kvEntry);
+
+            builder.putFiled(Options.REDIS_VALUE.name(), values.get(i));
+            builder.timestamp(System.currentTimeMillis());
+
+            SourceDataEntry entry = builder.buildSourceDataEntry(
+                ByteBuffer.wrap(kvEntry.getPartition().getBytes()),
+                ByteBuffer.wrap(RedisPositionConverter.longToJson(kvEntry.getOffset()).toJSONString().getBytes())
+            );
+            res.add(entry);
+        }
+        return res;
+    }
+
+
+    private List<Object> splitValue(FieldType valueType, Object value, Integer maxValueSize) {
+        List<Object> res = new ArrayList<>();
+        if (valueType.equals(FieldType.ARRAY) && value instanceof List) {
+            List<Object> list = (List)value;
+            if (list.size() < maxValueSize) {
+                res.add(list);
+            } else {
+                int num = list.size() / maxValueSize + 1;
+                for (int i = 0; i < num; i++) {
+                    List<Object> v = new ArrayList<>();
+                    for (int j = i * maxValueSize; j < Math.min((i + 1) * maxValueSize, list.size()); j++) {
+                        v.add(list.get(j));
+                    }
+                    if(!v.isEmpty()){
+                        res.add(v);
+                    }
+                }
+            }
+            return res;
+        }
+
+        if (valueType.equals(FieldType.MAP) && value instanceof Map) {
+            Map<Object, Object> map = (Map<Object, Object>)value;
+            if (map.size() < maxValueSize) {
+                res.add(map);
+            } else {
+                AtomicInteger num = new AtomicInteger(0);
+                Map<Object, Object> v = new HashMap<>();
+                for (Object k : map.keySet()) {
+                    v.put(k, map.get(k));
+                    if (num.incrementAndGet() == maxValueSize) {
+                        res.add(v);
+                        v = new HashMap<>();
+                        num = new AtomicInteger(0);
+                    }
+                }
+                if(!v.isEmpty()){
+                    res.add(v);
+                }
+            }
+            return res;
+        }
+
+        res.add(value);
+        return res;
+    }
+
+    private DataEntryBuilder newDataEntryBuilderWithoutValue(Schema schema, KVEntry kvEntry) {
+        DataEntryBuilder dataEntryBuilder = new DataEntryBuilder(schema);
+        dataEntryBuilder.queue(kvEntry.getQueueName());
+        dataEntryBuilder.entryType(kvEntry.getEntryType());
+        dataEntryBuilder.putFiled(Options.REDIS_COMMAND.name(), kvEntry.getCommand());
+        dataEntryBuilder.putFiled(Options.REDIS_KEY.name(), kvEntry.getKey());
+        dataEntryBuilder.putFiled(Options.REDIS_PARAMS.name(), kvEntry.getParams());
+        return dataEntryBuilder;
+    }
+
+    private Schema getRedisSchema(FieldType valueType) {
+        Schema schema = new Schema();
+        schema.setDataSource(Options.REDIS_DATASOURCE.name());
+        List<Field> fields = new ArrayList<>();
+
+        fields.add(new Field(0, Options.REDIS_COMMAND.name(), FieldType.STRING));
+        fields.add(new Field(1, Options.REDIS_KEY.name(), FieldType.STRING));
+        fields.add(new Field(2, Options.REDIS_VALUE.name(), valueType));
+        fields.add(new Field(3, Options.REDIS_PARAMS.name(), FieldType.MAP));
+
+        schema.setFields(fields);
+        return schema;
+    }
+
+}
diff --git a/src/main/java/org/apache/rocketmq/connect/redis/converter/RedisPositionConverter.java b/src/main/java/org/apache/rocketmq/connect/redis/converter/RedisPositionConverter.java
new file mode 100644
index 0000000..37e9e98
--- /dev/null
+++ b/src/main/java/org/apache/rocketmq/connect/redis/converter/RedisPositionConverter.java
@@ -0,0 +1,54 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.rocketmq.connect.redis.converter;
+
+import java.io.UnsupportedEncodingException;
+import java.nio.ByteBuffer;
+
+import com.alibaba.fastjson.JSONObject;
+
+import org.apache.rocketmq.connect.redis.common.RedisConstants;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+public class RedisPositionConverter {
+    private static final Logger LOGGER = LoggerFactory.getLogger(RedisPositionConverter.class);
+
+    public static Long jsonToLong(ByteBuffer byteBuffer){
+        if(byteBuffer == null){
+            return null;
+        }
+        try {
+            String positionJson = new String(byteBuffer.array(), "UTF-8");
+            JSONObject jsonObject = JSONObject.parseObject(positionJson);
+            if(jsonObject != null){
+                Long position = jsonObject.getLong(RedisConstants.POSITION);
+                return position;
+            }
+        } catch (UnsupportedEncodingException e) {
+            LOGGER.error("position encoding error. {}", e);
+        }
+        return null;
+    }
+
+    public static JSONObject longToJson(Long replOffset) {
+        JSONObject jsonObject = new JSONObject();
+        jsonObject.put(RedisConstants.POSITION, replOffset);
+        return jsonObject;
+    }
+}
diff --git a/src/main/java/org/apache/rocketmq/connect/redis/handler/DefaultRedisEventHandler.java b/src/main/java/org/apache/rocketmq/connect/redis/handler/DefaultRedisEventHandler.java
new file mode 100644
index 0000000..14362b9
--- /dev/null
+++ b/src/main/java/org/apache/rocketmq/connect/redis/handler/DefaultRedisEventHandler.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.connect.redis.handler;
+
+import java.io.IOException;
+
+import com.moilioncircle.redis.replicator.cmd.Command;
+import com.moilioncircle.redis.replicator.event.Event;
+import com.moilioncircle.redis.replicator.rdb.datatype.KeyValuePair;
+import com.moilioncircle.redis.replicator.rdb.iterable.datatype.BatchedKeyValuePair;
+import org.apache.rocketmq.connect.redis.common.Config;
+import org.apache.rocketmq.connect.redis.common.SyncMod;
+import org.apache.rocketmq.connect.redis.parser.DefaultRedisRdbParser;
+import org.apache.rocketmq.connect.redis.parser.RedisRdbParser;
+import org.apache.rocketmq.connect.redis.pojo.KVEntry;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import static com.moilioncircle.redis.replicator.Constants.RDB_TYPE_HASH;
+import static com.moilioncircle.redis.replicator.Constants.RDB_TYPE_HASH_ZIPLIST;
+import static com.moilioncircle.redis.replicator.Constants.RDB_TYPE_HASH_ZIPMAP;
+import static com.moilioncircle.redis.replicator.Constants.RDB_TYPE_LIST;
+import static com.moilioncircle.redis.replicator.Constants.RDB_TYPE_LIST_QUICKLIST;
+import static com.moilioncircle.redis.replicator.Constants.RDB_TYPE_LIST_ZIPLIST;
+import static com.moilioncircle.redis.replicator.Constants.RDB_TYPE_MODULE;
+import static com.moilioncircle.redis.replicator.Constants.RDB_TYPE_MODULE_2;
+import static com.moilioncircle.redis.replicator.Constants.RDB_TYPE_SET;
+import static com.moilioncircle.redis.replicator.Constants.RDB_TYPE_SET_INTSET;
+import static com.moilioncircle.redis.replicator.Constants.RDB_TYPE_STREAM_LISTPACKS;
+import static com.moilioncircle.redis.replicator.Constants.RDB_TYPE_STRING;
+import static com.moilioncircle.redis.replicator.Constants.RDB_TYPE_ZSET;
+import static com.moilioncircle.redis.replicator.Constants.RDB_TYPE_ZSET_2;
+import static com.moilioncircle.redis.replicator.Constants.RDB_TYPE_ZSET_ZIPLIST;
+
+/**
+ * Handling various types of redis events
+ */
+public class DefaultRedisEventHandler implements RedisEventHandler {
+    protected final Logger LOGGER = LoggerFactory.getLogger(DefaultRedisEventHandler.class);
+    /**
+     * config info
+     */
+    private Config config;
+
+    private RedisRdbParser<KVEntry> redisRdbParser;
+
+    public DefaultRedisEventHandler(Config config) {
+        this.config = config;
+        this.redisRdbParser = new DefaultRedisRdbParser();
+    }
+
+    /**
+     * Handle redis commands
+     *
+     * @param replId
+     * @param replOffset
+     * @param command
+     * @return
+     * @throws Exception
+     */
+    @Override public KVEntry handleCommand(String replId, Long replOffset, Command command) throws Exception {
+        if (command instanceof KVEntry) {
+            return ((KVEntry)command).sourceId(replId).offset(replOffset);
+        } else {
+            return this.handleOtherEvent(replId, replOffset, command);
+        }
+    }
+
+    /**
+     * Handle kV data in RDB files
+     *
+     * @param replId
+     * @param replOffset
+     * @param keyValuePair
+     * @return
+     * @throws Exception
+     */
+    @Override public KVEntry handleKVString(String replId, Long replOffset, KeyValuePair keyValuePair) throws Exception {
+        //Increment mode asynchronous RDB files
+        if (SyncMod.LAST_OFFSET.equals(this.config.getSyncMod())) {
+            return null;
+        }
+        KVEntry entry = parseRdbData(this.redisRdbParser, keyValuePair);
+        if(entry == null){
+            return null;
+        }
+        return entry.sourceId(replId).offset(replOffset);
+    }
+
+    /**
+     * Handle batch kV event
+     *
+     * @param replId
+     * @param replOffset
+     * @param batchedKeyValuePair
+     * @return
+     * @throws Exception
+     */
+    @Override public KVEntry handleBatchKVString(String replId, Long replOffset, BatchedKeyValuePair batchedKeyValuePair)
+        throws Exception {
+        LOGGER.warn("skip handle batch event: {}", batchedKeyValuePair);
+        return null;
+    }
+
+    /**
+     * Handle redis other operation instructions
+     *
+     * @param replId
+     * @param replOffset
+     * @param event
+     * @return
+     * @throws Exception
+     */
+
+    @Override public KVEntry handleOtherEvent(String replId, Long replOffset, Event event) throws Exception {
+        LOGGER.warn("skip handle other event: {}", event.getClass());
+        return null;
+    }
+
+    private <T extends KVEntry> T parseRdbData(RedisRdbParser<T> redisRdbParser, KeyValuePair keyValuePair)
+        throws IOException {
+        int rdbValueType = keyValuePair.getValueRdbType();
+
+        switch (rdbValueType) {
+            case RDB_TYPE_STRING:
+                return (T)redisRdbParser.applyString(keyValuePair).command("SET");
+            case RDB_TYPE_LIST:
+                return (T)redisRdbParser.applyList(keyValuePair).command("RPUSH");
+            case RDB_TYPE_SET:
+                return (T)redisRdbParser.applySet(keyValuePair).command("SADD");
+            case RDB_TYPE_ZSET:
+                return (T)redisRdbParser.applyZSet(keyValuePair).command("ZADD");
+            case RDB_TYPE_ZSET_2:
+                return (T)redisRdbParser.applyZSet2(keyValuePair).command("ZADD");
+            case RDB_TYPE_HASH:
+                return (T)redisRdbParser.applyHash(keyValuePair).command("HMSET");
+            case RDB_TYPE_HASH_ZIPMAP:
+                return (T)redisRdbParser.applyHashZipMap(keyValuePair).command("HMSET");
+            case RDB_TYPE_LIST_ZIPLIST:
+                return (T)redisRdbParser.applyListZipList(keyValuePair).command("RPUSH");
+            case RDB_TYPE_SET_INTSET:
+                return (T)redisRdbParser.applySetIntSet(keyValuePair).command("ZADD");
+            case RDB_TYPE_ZSET_ZIPLIST:
+                return (T)redisRdbParser.applyZSetZipList(keyValuePair).command("ZADD");
+            case RDB_TYPE_HASH_ZIPLIST:
+                return (T)redisRdbParser.applyHashZipList(keyValuePair).command("HMSET");
+            case RDB_TYPE_LIST_QUICKLIST:
+                return (T)redisRdbParser.applyListQuickList(keyValuePair).command("RPUSH");
+            case RDB_TYPE_MODULE:
+                return (T)redisRdbParser.applyModule(keyValuePair);
+            case RDB_TYPE_MODULE_2:
+                return (T)redisRdbParser.applyModule2(keyValuePair);
+            case RDB_TYPE_STREAM_LISTPACKS:
+                return (T)redisRdbParser.applyStreamListPacks(keyValuePair);
+            default:
+                throw new AssertionError("unexpected value type:" + rdbValueType);
+        }
+    }
+}
diff --git a/src/main/java/org/apache/rocketmq/connect/redis/handler/RedisEventHandler.java b/src/main/java/org/apache/rocketmq/connect/redis/handler/RedisEventHandler.java
new file mode 100644
index 0000000..6f93b2d
--- /dev/null
+++ b/src/main/java/org/apache/rocketmq/connect/redis/handler/RedisEventHandler.java
@@ -0,0 +1,72 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.rocketmq.connect.redis.handler;
+
+import com.moilioncircle.redis.replicator.cmd.Command;
+import com.moilioncircle.redis.replicator.event.Event;
+import com.moilioncircle.redis.replicator.rdb.datatype.KeyValuePair;
+import com.moilioncircle.redis.replicator.rdb.iterable.datatype.BatchedKeyValuePair;
+import org.apache.rocketmq.connect.redis.pojo.KVEntry;
+
+public interface RedisEventHandler {
+
+    /**
+     * handle redis command
+     *
+     * @param replId
+     * @param replOffset
+     * @param command
+     * @return
+     * @throws Exception
+     */
+    KVEntry handleCommand(String replId, Long replOffset, Command command) throws Exception;
+
+    /**
+     * handle data from RDB file
+     *
+     * @param replId
+     * @param replOffset
+     * @param keyValuePair
+     * @return
+     * @throws Exception
+     */
+    KVEntry handleKVString(String replId, Long replOffset, KeyValuePair keyValuePair) throws Exception;
+
+    /**
+     * handle redis batch kv data
+     *
+     * @param replId
+     * @param replOffset
+     * @param batchedKeyValuePair
+     * @return
+     * @throws Exception
+     */
+    KVEntry handleBatchKVString(String replId, Long replOffset, BatchedKeyValuePair batchedKeyValuePair) throws Exception;
+
+    /**
+     * handle other redis command
+     *
+     * @param replId
+     * @param replOffset
+     * @param event
+     * @return
+     * @throws Exception
+     */
+    KVEntry handleOtherEvent(String replId, Long replOffset, Event event) throws Exception;
+
+}
diff --git a/src/main/java/org/apache/rocketmq/connect/redis/parser/AbstractCommandParser.java b/src/main/java/org/apache/rocketmq/connect/redis/parser/AbstractCommandParser.java
new file mode 100644
index 0000000..bf67b7d
--- /dev/null
+++ b/src/main/java/org/apache/rocketmq/connect/redis/parser/AbstractCommandParser.java
@@ -0,0 +1,82 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.rocketmq.connect.redis.parser;
+
+import java.util.Arrays;
+import java.util.stream.Collectors;
+
+import com.moilioncircle.redis.replicator.cmd.CommandParser;
+import com.moilioncircle.redis.replicator.cmd.CommandParsers;
+import org.apache.rocketmq.connect.redis.pojo.KVEntry;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import static com.moilioncircle.redis.replicator.cmd.CommandParsers.toRune;
+
+public abstract class AbstractCommandParser implements CommandParser<KVEntry> {
+    protected static final Logger LOGGER = LoggerFactory.getLogger(AbstractCommandParser.class);
+
+    @Override public KVEntry parse(Object[] command) {
+        KVEntry builder = createBuilder();
+
+        if (builder == null) {
+            return null;
+        }
+
+        if (command.length > 0) {
+            String commandStr = toRune(command[0]);
+            builder.command(commandStr);
+        }
+
+        if (command.length > 1) {
+            String commandKeyStr = toRune(command[1]);
+            builder.key(commandKeyStr);
+        }
+
+        // 有其他参数
+        if (command.length > 2) {
+            byte[][] real_byte_args = new byte[command.length - 2][];
+            System.arraycopy(command, 2, real_byte_args, 0, real_byte_args.length);
+            String[] real_args = Arrays.stream(real_byte_args).map(CommandParsers::toRune).toArray(String[]::new);
+            try {
+                builder = handleValue(builder, real_args);
+            } catch (Exception e) {
+                LOGGER.error("parser value error: {} {}", Arrays.stream(command).map(CommandParsers::toRune)
+                    .collect(Collectors.joining(" ")), e);
+            }
+        }else {
+            try {
+                builder = handleNoArgValue(builder);
+            } catch (Exception e) {
+                LOGGER.error("parser value error: {} {}", Arrays.stream(command).map(CommandParsers::toRune)
+                    .collect(Collectors.joining(" ")), e);
+            }
+        }
+        return builder;
+    }
+
+    public abstract KVEntry createBuilder();
+
+    public KVEntry handleValue(KVEntry builder, String[] args){
+        return builder;
+    }
+
+    public KVEntry handleNoArgValue(KVEntry builder){
+        return builder;
+    }
+}
diff --git a/src/main/java/org/apache/rocketmq/connect/redis/parser/AppendParser.java b/src/main/java/org/apache/rocketmq/connect/redis/parser/AppendParser.java
new file mode 100644
index 0000000..67c5a76
--- /dev/null
+++ b/src/main/java/org/apache/rocketmq/connect/redis/parser/AppendParser.java
@@ -0,0 +1,38 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.rocketmq.connect.redis.parser;
+
+import io.openmessaging.connector.api.data.FieldType;
+import org.apache.rocketmq.connect.redis.pojo.KVEntry;
+import org.apache.rocketmq.connect.redis.pojo.RedisEntry;
+
+/**
+ * APPEND key value
+ */
+public class AppendParser extends AbstractCommandParser {
+
+    @Override
+    public KVEntry createBuilder() {
+        return RedisEntry.newEntry(FieldType.STRING);
+    }
+
+    @Override
+    public KVEntry handleValue(KVEntry builder, String[] args) {
+        return builder.value(args[0]);
+    }
+}
diff --git a/src/main/java/org/apache/rocketmq/connect/redis/parser/BitFieldParser.java b/src/main/java/org/apache/rocketmq/connect/redis/parser/BitFieldParser.java
new file mode 100644
index 0000000..55c7ce0
--- /dev/null
+++ b/src/main/java/org/apache/rocketmq/connect/redis/parser/BitFieldParser.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
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.rocketmq.connect.redis.parser;
+
+import io.openmessaging.connector.api.data.FieldType;
+import org.apache.rocketmq.connect.redis.pojo.KVEntry;
+import org.apache.rocketmq.connect.redis.pojo.RedisEntry;
+
+/**
+ * BITFIELD key [GET type offset] [SET type offset value] [INCRBY type offset increment] [OVERFLOW WRAP|SAT|FAIL]
+ */
+public class BitFieldParser extends AbstractCommandParser {
+    @Override
+    public KVEntry createBuilder() {
+        return RedisEntry.newEntry(FieldType.STRING);
+    }
+}
diff --git a/src/main/java/org/apache/rocketmq/connect/redis/parser/BitOpParser.java b/src/main/java/org/apache/rocketmq/connect/redis/parser/BitOpParser.java
new file mode 100644
index 0000000..38b1b71
--- /dev/null
+++ b/src/main/java/org/apache/rocketmq/connect/redis/parser/BitOpParser.java
@@ -0,0 +1,41 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.rocketmq.connect.redis.parser;
+
+import java.util.ArrayList;
+import java.util.Arrays;
+import java.util.stream.Collectors;
+
+import io.openmessaging.connector.api.data.FieldType;
+import org.apache.rocketmq.connect.redis.pojo.KVEntry;
+import org.apache.rocketmq.connect.redis.pojo.RedisEntry;
+
+/**
+ * BITOP operation destkey key [key ...]
+ */
+public class BitOpParser extends AbstractCommandParser {
+    @Override
+    public KVEntry createBuilder() {
+        return RedisEntry.newEntry(FieldType.ARRAY);
+    }
+
+    @Override
+    public KVEntry handleValue(KVEntry builder, String[] args) {
+        return builder.value(Arrays.stream(args).collect(Collectors.toCollection(ArrayList::new)));
+    }
+}
diff --git a/src/main/java/org/apache/rocketmq/connect/redis/parser/BrPopLPushParser.java b/src/main/java/org/apache/rocketmq/connect/redis/parser/BrPopLPushParser.java
new file mode 100644
index 0000000..bd8a090
--- /dev/null
+++ b/src/main/java/org/apache/rocketmq/connect/redis/parser/BrPopLPushParser.java
@@ -0,0 +1,41 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.rocketmq.connect.redis.parser;
+
+import io.openmessaging.connector.api.data.FieldType;
+import org.apache.rocketmq.connect.redis.pojo.KVEntry;
+import org.apache.rocketmq.connect.redis.common.Options;
+import org.apache.rocketmq.connect.redis.pojo.RedisEntry;
+
+/**
+ * BRPOPLPUSH source destination timeout
+ */
+public class BrPopLPushParser extends AbstractCommandParser {
+    @Override
+    public KVEntry createBuilder() {
+        return RedisEntry.newEntry(FieldType.STRING);
+    }
+
+    @Override
+    public KVEntry handleValue(KVEntry builder, String[] args) {
+        if (args.length == 2) {
+            builder.value(args[0]).param(Options.REDIS_TIMEOUT, Long.parseLong(args[1]));
+        }
+        return builder;
+    }
+}
diff --git a/src/main/java/org/apache/rocketmq/connect/redis/parser/CommonParser.java b/src/main/java/org/apache/rocketmq/connect/redis/parser/CommonParser.java
new file mode 100644
index 0000000..f8dca37
--- /dev/null
+++ b/src/main/java/org/apache/rocketmq/connect/redis/parser/CommonParser.java
@@ -0,0 +1,117 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.rocketmq.connect.redis.parser;
+
+import java.util.ArrayList;
+import java.util.Arrays;
+import java.util.HashMap;
+import java.util.List;
+import java.util.Map;
+import java.util.stream.Collectors;
+
+import org.apache.rocketmq.connect.redis.pojo.KVEntry;
+import org.apache.rocketmq.connect.redis.common.Options;
+
+public class CommonParser {
+
+    public static KVEntry commonEvalParser(KVEntry builder, String[] args) {
+        int numberKeys = Integer.parseInt(args[0]);
+        if (numberKeys > 0) {
+            List<List<String>> values = new ArrayList<>();
+
+            String[] keys_arr = new String[numberKeys];
+            System.arraycopy(args, 1, keys_arr, 0, numberKeys);
+            List<String> keysList = new ArrayList<>(numberKeys);
+            keysList.addAll(Arrays.asList(keys_arr));
+            values.add(keysList);
+
+            if (numberKeys + 1 < args.length) {
+                String[] args_arr = new String[args.length - numberKeys - 1];
+                System.arraycopy(args, numberKeys + 1, args_arr, 0, args.length - numberKeys - 1);
+                List<String> argsList = new ArrayList<>(numberKeys);
+                argsList.addAll(Arrays.asList(args_arr));
+                values.add(argsList);
+            }
+
+            builder.value(values);
+        }
+
+        return builder;
+    }
+
+    public static KVEntry commonMapParser(KVEntry builder, String[] args) {
+        Map<String, String> kvMap = new HashMap<>(args.length / 2);
+        for (int i = 0, j = i + 1; j < args.length; i++, i++, j++, j++) {
+            kvMap.put(args[i], args[j]);
+        }
+        builder.value(kvMap);
+        return builder;
+    }
+
+    public static KVEntry noCommandKeyMapParser(KVEntry builder, String[] args) {
+        String firKey = builder.getKey();
+        Map<String, String> kvMap = new HashMap<>((args.length + 1) / 2);
+        kvMap.put(firKey, args[0]);
+        for (int i = 1, j = i + 1; j < args.length; i++, i++, j++, j++) {
+            kvMap.put(args[i], args[j]);
+        }
+        builder.value(kvMap);
+        return builder;
+    }
+
+    public static KVEntry noCommandKeyArrayParser(KVEntry builder, String[] args) {
+        List<String> keys = new ArrayList<>(args.length + 1);
+        keys.add(builder.getKey());
+        keys.addAll(Arrays.stream(args).collect(Collectors.toCollection(ArrayList::new)));
+        return builder.value(keys);
+    }
+
+    public static KVEntry commonZStoreHandler(KVEntry builder, String[] args) {
+        int numberKey = Integer.parseInt(args[0]);
+        Map<String, String> map = new HashMap<>();
+        String[] keys = new String[numberKey];
+        System.arraycopy(args, 1, keys, 0, numberKey);
+        int idx = numberKey + 1;
+        if (idx < args.length) {
+            if (Options.REDIS_WEIGHTS.name().equals(args[idx].toUpperCase())) {
+                String[] values = new String[numberKey];
+                System.arraycopy(args, idx + 1, values, 0, numberKey);
+                for (int i = 0; i < numberKey; i++) {
+                    map.put(keys[i], values[i]);
+                }
+                builder.value(map);
+                idx = idx + numberKey + 1;
+            } else {
+                for (int i = 0; i < numberKey; i++) {
+                    map.put(keys[i], Integer.toString(1));
+                }
+                builder.value(map);
+            }
+            if (idx < args.length && Options.REDIS_AGGREGATE.name().equals(args[idx].toUpperCase())) {
+                builder.param(Options.REDIS_AGGREGATE, args[idx + 1]);
+            }
+        } else {
+            for (int i = 0; i < numberKey; i++) {
+                map.put(keys[i], Integer.toString(1));
+            }
+            builder.value(map);
+        }
+        return builder;
+    }
+
+}
diff --git a/src/main/java/org/apache/rocketmq/connect/redis/parser/DecrByParser.java b/src/main/java/org/apache/rocketmq/connect/redis/parser/DecrByParser.java
new file mode 100644
index 0000000..edfad48
--- /dev/null
+++ b/src/main/java/org/apache/rocketmq/connect/redis/parser/DecrByParser.java
@@ -0,0 +1,38 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.rocketmq.connect.redis.parser;
+
+import io.openmessaging.connector.api.data.FieldType;
+import org.apache.rocketmq.connect.redis.pojo.KVEntry;
+import org.apache.rocketmq.connect.redis.common.Options;
+import org.apache.rocketmq.connect.redis.pojo.RedisEntry;
+
+/**
+ * DECRBY key increment
+ */
+public class DecrByParser extends AbstractCommandParser{
+    @Override
+    public KVEntry createBuilder() {
+        return RedisEntry.newEntry(FieldType.STRING);
+    }
+
+    @Override
+    public KVEntry handleValue(KVEntry builder, String[] args) {
+        return builder.param(Options.REDIS_INCREMENT, Long.parseLong(args[0]));
+    }
+}
diff --git a/src/main/java/org/apache/rocketmq/connect/redis/parser/DecrParser.java b/src/main/java/org/apache/rocketmq/connect/redis/parser/DecrParser.java
new file mode 100644
index 0000000..c632603
--- /dev/null
+++ b/src/main/java/org/apache/rocketmq/connect/redis/parser/DecrParser.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
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.rocketmq.connect.redis.parser;
+
+import io.openmessaging.connector.api.data.FieldType;
+import org.apache.rocketmq.connect.redis.pojo.KVEntry;
+import org.apache.rocketmq.connect.redis.pojo.RedisEntry;
+
+/**
+ * DECR key
+ */
+public class DecrParser extends AbstractCommandParser {
+    @Override
+    public KVEntry createBuilder() {
+        return RedisEntry.newEntry(FieldType.STRING);
+    }
+}
diff --git a/src/main/java/org/apache/rocketmq/connect/redis/parser/DefaultRedisRdbParser.java b/src/main/java/org/apache/rocketmq/connect/redis/parser/DefaultRedisRdbParser.java
new file mode 100644
index 0000000..9a4c3ad
--- /dev/null
+++ b/src/main/java/org/apache/rocketmq/connect/redis/parser/DefaultRedisRdbParser.java
@@ -0,0 +1,197 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.rocketmq.connect.redis.parser;
+
+import java.io.IOException;
+import java.util.ArrayList;
+import java.util.Collection;
+import java.util.List;
+import java.util.Map;
+import java.util.Set;
+import java.util.stream.Collectors;
+
+import com.moilioncircle.redis.replicator.rdb.datatype.KeyValuePair;
+import com.moilioncircle.redis.replicator.rdb.datatype.Module;
+import com.moilioncircle.redis.replicator.rdb.datatype.Stream;
+import com.moilioncircle.redis.replicator.rdb.datatype.ZSetEntry;
+import io.openmessaging.connector.api.data.FieldType;
+import org.apache.rocketmq.connect.redis.pojo.KVEntry;
+import org.apache.rocketmq.connect.redis.common.Options;
+import org.apache.rocketmq.connect.redis.pojo.RedisEntry;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+/**
+ *
+ */
+public class DefaultRedisRdbParser implements RedisRdbParser<KVEntry> {
+    protected static final Logger LOGGER = LoggerFactory.getLogger(DefaultRedisRdbParser.class);
+
+    @Override
+    public KVEntry applyString(KeyValuePair<byte[], byte[]> keyValuePair) throws IOException {
+        RedisEntry builder = RedisEntry.newEntry(FieldType.STRING);
+        return builder
+            .param(Options.EXPIRED_TYPE, keyValuePair.getExpiredType())
+            .param(Options.EXPIRED_TIME, keyValuePair.getExpiredValue())
+            .param(Options.EVICT_TYPE, keyValuePair.getEvictType())
+            .param(Options.EVICT_VALUE, keyValuePair.getEvictValue())
+            .key(new String(keyValuePair.getKey()))
+            .value(new String(keyValuePair.getValue()))
+            ;
+    }
+
+    @Override
+    public KVEntry applyList(KeyValuePair<byte[], List<byte[]>> keyValuePair) throws IOException {
+        return commonArray(keyValuePair);
+    }
+
+    @Override
+    public KVEntry applySet(KeyValuePair<byte[], Set<byte[]>> keyValuePair) throws IOException {
+        return commonArray(keyValuePair);
+    }
+
+    @Override
+    public KVEntry applyZSet(KeyValuePair<byte[], Set<ZSetEntry>> keyValuePair) throws IOException {
+        return commonZSet(keyValuePair);
+    }
+
+    @Override
+    public KVEntry applyZSet2(KeyValuePair<byte[], Set<ZSetEntry>> keyValuePair) throws IOException {
+        return commonZSet(keyValuePair);
+    }
+
+    @Override
+    public KVEntry applyHash(KeyValuePair<byte[], Map<byte[], byte[]>> keyValuePair) throws IOException {
+        return commonHash(keyValuePair);
+    }
+
+    @Override
+    public KVEntry applyHashZipMap(KeyValuePair<byte[], Map<byte[], byte[]>> keyValuePair) throws IOException {
+        return commonHash(keyValuePair);
+    }
+
+    @Override
+    public KVEntry applyListZipList(KeyValuePair<byte[], List<byte[]>> keyValuePair) throws IOException {
+        return commonArray(keyValuePair);
+    }
+
+    @Override
+    public KVEntry applySetIntSet(KeyValuePair<byte[], Set<byte[]>> keyValuePair) throws IOException {
+        return commonArray(keyValuePair);
+    }
+
+    @Override
+    public KVEntry applyZSetZipList(KeyValuePair<byte[], Set<ZSetEntry>> keyValuePair) throws IOException {
+        return commonZSet(keyValuePair);
+    }
+
+    @Override
+    public KVEntry applyHashZipList(KeyValuePair<byte[], Map<byte[], byte[]>> keyValuePair) throws IOException {
+        return commonHash(keyValuePair);
+    }
+
+    @Override
+    public KVEntry applyListQuickList(KeyValuePair<byte[], List<byte[]>> keyValuePair) throws IOException {
+        return commonArray(keyValuePair);
+    }
+
+    @Override
+    public KVEntry applyModule(KeyValuePair<byte[], Module> keyValuePair) throws IOException {
+        return commonModule(keyValuePair);
+    }
+
+    @Override
+    public KVEntry applyModule2(KeyValuePair<byte[], Module> keyValuePair) throws IOException {
+        return commonModule(keyValuePair);
+    }
+
+    @Override
+    public KVEntry applyStreamListPacks(KeyValuePair<byte[], Stream> keyValuePair) throws IOException {
+        return commonStream(keyValuePair);
+    }
+
+    private KVEntry commonStream(KeyValuePair<byte[], ? extends Stream> keyValuePair){
+        RedisEntry builder = RedisEntry.newEntry(FieldType.ARRAY);
+        return builder
+            .param(Options.EXPIRED_TYPE, keyValuePair.getExpiredType())
+            .param(Options.EXPIRED_TIME, keyValuePair.getExpiredValue())
+            .param(Options.EVICT_TYPE, keyValuePair.getEvictType())
+            .param(Options.EVICT_VALUE, keyValuePair.getEvictValue())
+            .key(new String(keyValuePair.getKey()))
+            .value(keyValuePair.getValue())
+            ;
+    }
+
+    private KVEntry commonModule(KeyValuePair<byte[], ? extends Module> keyValuePair){
+        RedisEntry builder = RedisEntry.newEntry(FieldType.ARRAY);
+        return builder
+            .param(Options.EXPIRED_TYPE, keyValuePair.getExpiredType())
+            .param(Options.EXPIRED_TIME, keyValuePair.getExpiredValue())
+            .param(Options.EVICT_TYPE, keyValuePair.getEvictType())
+            .param(Options.EVICT_VALUE, keyValuePair.getEvictValue())
+            .key(new String(keyValuePair.getKey()))
+            .value(keyValuePair.getValue())
+            ;
+    }
+
+    private KVEntry commonArray(KeyValuePair<byte[], ? extends Collection<byte[]>> keyValuePair){
+        RedisEntry builder = RedisEntry.newEntry(FieldType.ARRAY);
+        return builder
+            .param(Options.EXPIRED_TYPE, keyValuePair.getExpiredType())
+            .param(Options.EXPIRED_TIME, keyValuePair.getExpiredValue())
+            .param(Options.EVICT_TYPE, keyValuePair.getEvictType())
+            .param(Options.EVICT_VALUE, keyValuePair.getEvictValue())
+            .key(new String(keyValuePair.getKey()))
+            .value(keyValuePair.getValue().stream().map(String::new).collect(Collectors.toCollection(ArrayList::new)))
+            ;
+    }
+
+    private KVEntry commonZSet(KeyValuePair<byte[], ? extends Collection<ZSetEntry>> keyValuePair){
+        RedisEntry builder = RedisEntry.newEntry(FieldType.MAP);
+        return builder
+            .param(Options.EXPIRED_TYPE, keyValuePair.getExpiredType())
+            .param(Options.EXPIRED_TIME, keyValuePair.getExpiredValue())
+            .param(Options.EVICT_TYPE, keyValuePair.getEvictType())
+            .param(Options.EVICT_VALUE, keyValuePair.getEvictValue())
+            .key(new String(keyValuePair.getKey()))
+            .value(
+                keyValuePair.getValue().stream().collect(Collectors.toMap(
+                    entry -> new String(entry.getElement()),
+                    ZSetEntry::getScore)
+                )
+            )
+            ;
+    }
+
+    private KVEntry commonHash(KeyValuePair<byte[], ? extends Map<byte[], byte[]>> keyValuePair){
+        RedisEntry builder = RedisEntry.newEntry(FieldType.MAP);
+        return builder
+            .param(Options.EXPIRED_TYPE, keyValuePair.getExpiredType())
+            .param(Options.EXPIRED_TIME, keyValuePair.getExpiredValue())
+            .param(Options.EVICT_TYPE, keyValuePair.getEvictType())
+            .param(Options.EVICT_VALUE, keyValuePair.getEvictValue())
+            .key(new String(keyValuePair.getKey()))
+            .value(
+                keyValuePair.getValue().entrySet().stream().collect(Collectors.toMap(
+                    entry -> new String(entry.getKey()),
+                    entry -> new String(entry.getValue())
+                ))
+            )
+            ;
+    }
+}
diff --git a/src/main/java/org/apache/rocketmq/connect/redis/parser/DelParser.java b/src/main/java/org/apache/rocketmq/connect/redis/parser/DelParser.java
new file mode 100644
index 0000000..17ae917
--- /dev/null
+++ b/src/main/java/org/apache/rocketmq/connect/redis/parser/DelParser.java
@@ -0,0 +1,46 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.rocketmq.connect.redis.parser;
+
+import java.util.ArrayList;
+import java.util.List;
+
+import io.openmessaging.connector.api.data.FieldType;
+import org.apache.rocketmq.connect.redis.pojo.KVEntry;
+import org.apache.rocketmq.connect.redis.pojo.RedisEntry;
+
+/**
+ * del key [key ...]
+ */
+public class DelParser extends AbstractCommandParser {
+    @Override
+    public KVEntry createBuilder() {
+        return RedisEntry.newEntry(FieldType.ARRAY);
+    }
+
+    @Override
+    public KVEntry handleValue(KVEntry builder, String[] args) {
+        List<String> delList = new ArrayList<>(args.length + 1);
+                delList.add(builder.getKey());
+        for (int i = 0; i < args.length; i++) {
+            delList.add(args[i]);
+        }
+        builder.value(delList);
+        return builder;
+    }
+}
diff --git a/src/main/java/org/apache/rocketmq/connect/redis/parser/EvalParser.java b/src/main/java/org/apache/rocketmq/connect/redis/parser/EvalParser.java
new file mode 100644
index 0000000..deb36c1
--- /dev/null
+++ b/src/main/java/org/apache/rocketmq/connect/redis/parser/EvalParser.java
@@ -0,0 +1,38 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.rocketmq.connect.redis.parser;
+
+import io.openmessaging.connector.api.data.FieldType;
+import org.apache.rocketmq.connect.redis.pojo.KVEntry;
+import org.apache.rocketmq.connect.redis.pojo.RedisEntry;
+
+/**
+ * EVAL script numkeys key [key ...] arg [arg ...]
+ */
+public class EvalParser extends AbstractCommandParser {
+
+    @Override
+    public KVEntry createBuilder() {
+        return RedisEntry.newEntry(FieldType.ARRAY);
+    }
+
+    @Override
+    public KVEntry handleValue(KVEntry builder, String[] args) {
+        return CommonParser.commonEvalParser(builder, args);
+    }
+}
diff --git a/src/main/java/org/apache/rocketmq/connect/redis/parser/EvalShaParser.java b/src/main/java/org/apache/rocketmq/connect/redis/parser/EvalShaParser.java
new file mode 100644
index 0000000..38b564e
--- /dev/null
+++ b/src/main/java/org/apache/rocketmq/connect/redis/parser/EvalShaParser.java
@@ -0,0 +1,38 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.rocketmq.connect.redis.parser;
+
+import io.openmessaging.connector.api.data.FieldType;
+import org.apache.rocketmq.connect.redis.pojo.KVEntry;
+import org.apache.rocketmq.connect.redis.pojo.RedisEntry;
+
+/**
+ * EVAL sha1 numkeys key [key ...] arg [arg ...]
+ */
+public class EvalShaParser extends AbstractCommandParser {
+
+    @Override
+    public KVEntry createBuilder() {
+        return RedisEntry.newEntry(FieldType.ARRAY);
+    }
+
+    @Override
+    public KVEntry handleValue(KVEntry builder, String[] args) {
+        return CommonParser.commonEvalParser(builder, args);
+    }
+}
diff --git a/src/main/java/org/apache/rocketmq/connect/redis/parser/ExecParser.java b/src/main/java/org/apache/rocketmq/connect/redis/parser/ExecParser.java
new file mode 100644
index 0000000..0b48a59
--- /dev/null
+++ b/src/main/java/org/apache/rocketmq/connect/redis/parser/ExecParser.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
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.rocketmq.connect.redis.parser;
+
+import io.openmessaging.connector.api.data.FieldType;
+import org.apache.rocketmq.connect.redis.pojo.KVEntry;
+import org.apache.rocketmq.connect.redis.pojo.RedisEntry;
+
+/**
+ * exec
+ */
+public class ExecParser extends AbstractCommandParser {
+    @Override
+    public KVEntry createBuilder() {
+        return RedisEntry.newEntry(FieldType.BOOLEAN);
+    }
+}
diff --git a/src/main/java/org/apache/rocketmq/connect/redis/parser/ExpireAtParser.java b/src/main/java/org/apache/rocketmq/connect/redis/parser/ExpireAtParser.java
new file mode 100644
index 0000000..e04606c
--- /dev/null
+++ b/src/main/java/org/apache/rocketmq/connect/redis/parser/ExpireAtParser.java
@@ -0,0 +1,38 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.rocketmq.connect.redis.parser;
+
+import io.openmessaging.connector.api.data.FieldType;
+import org.apache.rocketmq.connect.redis.pojo.KVEntry;
+import org.apache.rocketmq.connect.redis.common.Options;
+import org.apache.rocketmq.connect.redis.pojo.RedisEntry;
+
+/**
+ * expireat timestamp
+ */
+public class ExpireAtParser extends AbstractCommandParser {
+    @Override
+    public KVEntry createBuilder() {
+        return RedisEntry.newEntry(FieldType.STRING);
+    }
+
+    @Override
+    public KVEntry handleNoArgValue(KVEntry builder) {
+        return builder.param(Options.REDIS_EX_TIMESTAMP, Integer.parseInt(builder.getKey()));
+    }
+}
diff --git a/src/main/java/org/apache/rocketmq/connect/redis/parser/ExpireParser.java b/src/main/java/org/apache/rocketmq/connect/redis/parser/ExpireParser.java
new file mode 100644
index 0000000..7a47de7
--- /dev/null
+++ b/src/main/java/org/apache/rocketmq/connect/redis/parser/ExpireParser.java
@@ -0,0 +1,40 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.rocketmq.connect.redis.parser;
+
+import com.moilioncircle.redis.replicator.rdb.datatype.ExpiredType;
+import io.openmessaging.connector.api.data.FieldType;
+import org.apache.rocketmq.connect.redis.pojo.KVEntry;
+import org.apache.rocketmq.connect.redis.common.Options;
+import org.apache.rocketmq.connect.redis.pojo.RedisEntry;
+
+/**
+ * expire key seconds
+ */
+public class ExpireParser extends AbstractCommandParser {
+    @Override
+    public KVEntry createBuilder() {
+        return RedisEntry.newEntry(FieldType.STRING);
+    }
+
+    @Override
+    public KVEntry handleValue(KVEntry builder, String[] args) {
+        return builder.param(Options.EXPIRED_TYPE, ExpiredType.SECOND)
+                    .param(Options.EXPIRED_TIME, Long.parseLong(args[0]));
+    }
+}
diff --git a/src/main/java/org/apache/rocketmq/connect/redis/parser/FlushAllParser.java b/src/main/java/org/apache/rocketmq/connect/redis/parser/FlushAllParser.java
new file mode 100644
index 0000000..7db9603
--- /dev/null
+++ b/src/main/java/org/apache/rocketmq/connect/redis/parser/FlushAllParser.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
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.rocketmq.connect.redis.parser;
+
+import io.openmessaging.connector.api.data.FieldType;
+import org.apache.rocketmq.connect.redis.pojo.KVEntry;
+import org.apache.rocketmq.connect.redis.pojo.RedisEntry;
+
+/**
+ * FLUSHALL [ASYNC]
+ */
+public class FlushAllParser extends AbstractCommandParser {
+    @Override
+    public KVEntry createBuilder() {
+        return RedisEntry.newEntry(FieldType.STRING);
+    }
+}
diff --git a/src/main/java/org/apache/rocketmq/connect/redis/parser/FlushDbParser.java b/src/main/java/org/apache/rocketmq/connect/redis/parser/FlushDbParser.java
new file mode 100644
index 0000000..908d456
--- /dev/null
+++ b/src/main/java/org/apache/rocketmq/connect/redis/parser/FlushDbParser.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.connect.redis.parser;
+
+import io.openmessaging.connector.api.data.FieldType;
+import org.apache.rocketmq.connect.redis.pojo.KVEntry;
+import org.apache.rocketmq.connect.redis.pojo.RedisEntry;
+
+/**
+ * FLUSHDB [async]
+ */
+public class FlushDbParser extends AbstractCommandParser {
+    @Override
+    public KVEntry createBuilder() {
+        return RedisEntry.newEntry(FieldType.STRING);
+    }
+
+}
diff --git a/src/main/java/org/apache/rocketmq/connect/redis/parser/GeoAddParser.java b/src/main/java/org/apache/rocketmq/connect/redis/parser/GeoAddParser.java
new file mode 100644
index 0000000..7136787
--- /dev/null
+++ b/src/main/java/org/apache/rocketmq/connect/redis/parser/GeoAddParser.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.connect.redis.parser;
+
+import java.util.ArrayList;
+import java.util.List;
+
+import io.openmessaging.connector.api.data.FieldType;
+import org.apache.rocketmq.connect.redis.pojo.Geo;
+import org.apache.rocketmq.connect.redis.pojo.KVEntry;
+import org.apache.rocketmq.connect.redis.pojo.RedisEntry;
+
+/**
+ * GEOADD key longitude latitude member [longitude latitude member ...]
+ */
+public class GeoAddParser extends AbstractCommandParser {
+    @Override
+    public KVEntry createBuilder() {
+        return RedisEntry.newEntry(FieldType.ARRAY);
+    }
+
+    @Override
+    public KVEntry handleValue(KVEntry builder, String[] args) {
+        List<Geo> geos = new ArrayList<>();
+        for (int i = 0, j = i + 1, k = i + 2; i < args.length; i += 3, j += 3, k += 3) {
+            Geo geo = new Geo();
+            geo.setLongitude(Long.parseLong(args[i]));
+            geo.setLatitude(Long.parseLong(args[j]));
+            geo.setMember(args[k]);
+            geos.add(geo);
+        }
+        return builder.value(geos);
+    }
+}
diff --git a/src/main/java/org/apache/rocketmq/connect/redis/parser/GetsetParser.java b/src/main/java/org/apache/rocketmq/connect/redis/parser/GetsetParser.java
new file mode 100644
index 0000000..1c816fc
--- /dev/null
+++ b/src/main/java/org/apache/rocketmq/connect/redis/parser/GetsetParser.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.connect.redis.parser;
+
+import io.openmessaging.connector.api.data.FieldType;
+import org.apache.rocketmq.connect.redis.pojo.KVEntry;
+import org.apache.rocketmq.connect.redis.pojo.RedisEntry;
+
+/**
+ * Getset key value
+ */
+public class GetsetParser extends AbstractCommandParser {
+    @Override
+    public KVEntry createBuilder() {
+        return RedisEntry.newEntry(FieldType.STRING);
+    }
+
+    @Override
+    public KVEntry handleValue(KVEntry builder, String[] args) {
+        return builder.value(args[0]);
+    }
+}
diff --git a/src/main/java/org/apache/rocketmq/connect/redis/parser/HDelParser.java b/src/main/java/org/apache/rocketmq/connect/redis/parser/HDelParser.java
new file mode 100644
index 0000000..9b17875
--- /dev/null
+++ b/src/main/java/org/apache/rocketmq/connect/redis/parser/HDelParser.java
@@ -0,0 +1,41 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.rocketmq.connect.redis.parser;
+
+import java.util.ArrayList;
+import java.util.Arrays;
+import java.util.stream.Collectors;
+
+import io.openmessaging.connector.api.data.FieldType;
+import org.apache.rocketmq.connect.redis.pojo.KVEntry;
+import org.apache.rocketmq.connect.redis.pojo.RedisEntry;
+
+/**
+ * HDEL key field [field ...]
+ */
+public class HDelParser extends AbstractCommandParser {
+    @Override
+    public KVEntry createBuilder() {
+        return RedisEntry.newEntry(FieldType.ARRAY);
+    }
+
+    @Override
+    public KVEntry handleValue(KVEntry builder, String[] args) {
+        return builder.value(Arrays.stream(args).collect(Collectors.toCollection(ArrayList::new)));
+    }
+}
diff --git a/src/main/java/org/apache/rocketmq/connect/redis/parser/HIncrByParser.java b/src/main/java/org/apache/rocketmq/connect/redis/parser/HIncrByParser.java
new file mode 100644
index 0000000..7547740
--- /dev/null
+++ b/src/main/java/org/apache/rocketmq/connect/redis/parser/HIncrByParser.java
@@ -0,0 +1,41 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.rocketmq.connect.redis.parser;
+
+import io.openmessaging.connector.api.data.FieldType;
+import org.apache.rocketmq.connect.redis.pojo.KVEntry;
+import org.apache.rocketmq.connect.redis.common.Options;
+import org.apache.rocketmq.connect.redis.pojo.RedisEntry;
+
+/**
+ * HINCRBY key field increment
+ */
+public class HIncrByParser extends AbstractCommandParser {
+    @Override
+    public KVEntry createBuilder() {
+        return RedisEntry.newEntry(FieldType.STRING);
+    }
+
+    @Override
+    public KVEntry handleValue(KVEntry builder, String[] args) {
+        if (args.length == 2) {
+            builder.value(args[0]).param(Options.REDIS_INCREMENT, Long.parseLong(args[1]));
+        }
+        return builder;
+    }
+}
diff --git a/src/main/java/org/apache/rocketmq/connect/redis/parser/HSetNxParser.java b/src/main/java/org/apache/rocketmq/connect/redis/parser/HSetNxParser.java
new file mode 100644
index 0000000..02268d0
--- /dev/null
+++ b/src/main/java/org/apache/rocketmq/connect/redis/parser/HSetNxParser.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.connect.redis.parser;
+
+import io.openmessaging.connector.api.data.FieldType;
+import org.apache.rocketmq.connect.redis.pojo.KVEntry;
+import org.apache.rocketmq.connect.redis.pojo.RedisEntry;
+
+/**
+ * hsetnx key field value
+ */
+public class HSetNxParser extends AbstractCommandParser {
+    @Override
+    public KVEntry createBuilder() {
+        return RedisEntry.newEntry(FieldType.MAP);
+    }
+
+    @Override
+    public KVEntry handleValue(KVEntry builder, String[] args) {
+        return CommonParser.commonMapParser(builder, args);
+    }
+}
diff --git a/src/main/java/org/apache/rocketmq/connect/redis/parser/HSetParser.java b/src/main/java/org/apache/rocketmq/connect/redis/parser/HSetParser.java
new file mode 100644
index 0000000..87dcc33
--- /dev/null
+++ b/src/main/java/org/apache/rocketmq/connect/redis/parser/HSetParser.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.connect.redis.parser;
+
+import io.openmessaging.connector.api.data.FieldType;
+import org.apache.rocketmq.connect.redis.pojo.KVEntry;
+import org.apache.rocketmq.connect.redis.pojo.RedisEntry;
+
+/**
+ * hset key field value
+ */
+public class HSetParser extends AbstractCommandParser {
+    @Override
+    public KVEntry createBuilder() {
+        return RedisEntry.newEntry(FieldType.MAP);
+    }
+
+    @Override
+    public KVEntry handleValue(KVEntry builder, String[] args) {
+        return CommonParser.commonMapParser(builder, args);
+    }
+}
diff --git a/src/main/java/org/apache/rocketmq/connect/redis/parser/HmSetParser.java b/src/main/java/org/apache/rocketmq/connect/redis/parser/HmSetParser.java
new file mode 100644
index 0000000..4e317e4
--- /dev/null
+++ b/src/main/java/org/apache/rocketmq/connect/redis/parser/HmSetParser.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.connect.redis.parser;
+
+import io.openmessaging.connector.api.data.FieldType;
+import org.apache.rocketmq.connect.redis.pojo.KVEntry;
+import org.apache.rocketmq.connect.redis.pojo.RedisEntry;
+
+/**
+ * hmset key field value [field value ...]
+ */
+public class HmSetParser extends AbstractCommandParser {
+    @Override
+    public KVEntry createBuilder() {
+        return RedisEntry.newEntry(FieldType.MAP);
+    }
+
+    @Override
+    public KVEntry handleValue(KVEntry builder, String[] args) {
+        return CommonParser.commonMapParser(builder, args);
+    }
+}
diff --git a/src/main/java/org/apache/rocketmq/connect/redis/parser/IncrByParser.java b/src/main/java/org/apache/rocketmq/connect/redis/parser/IncrByParser.java
new file mode 100644
index 0000000..d691955
--- /dev/null
+++ b/src/main/java/org/apache/rocketmq/connect/redis/parser/IncrByParser.java
@@ -0,0 +1,38 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.rocketmq.connect.redis.parser;
+
+import io.openmessaging.connector.api.data.FieldType;
+import org.apache.rocketmq.connect.redis.pojo.KVEntry;
+import org.apache.rocketmq.connect.redis.common.Options;
+import org.apache.rocketmq.connect.redis.pojo.RedisEntry;
+
+/**
+ * INCRBY key increment
+ */
+public class IncrByParser extends AbstractCommandParser {
+    @Override
+    public KVEntry createBuilder() {
+        return RedisEntry.newEntry(FieldType.INT64);
+    }
+
+    @Override
+    public KVEntry handleValue(KVEntry builder, String[] args) {
+        return  builder.param(Options.REDIS_INCREMENT, Long.parseLong(args[0]));
+    }
+}
diff --git a/src/main/java/org/apache/rocketmq/connect/redis/parser/IncrParser.java b/src/main/java/org/apache/rocketmq/connect/redis/parser/IncrParser.java
new file mode 100644
index 0000000..db0402b
--- /dev/null
+++ b/src/main/java/org/apache/rocketmq/connect/redis/parser/IncrParser.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
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.rocketmq.connect.redis.parser;
+
+import io.openmessaging.connector.api.data.FieldType;
+import org.apache.rocketmq.connect.redis.pojo.KVEntry;
+import org.apache.rocketmq.connect.redis.pojo.RedisEntry;
+
+/**
+ * INCR key
+ */
+public class IncrParser extends AbstractCommandParser {
+    @Override
+    public KVEntry createBuilder() {
+        return RedisEntry.newEntry(FieldType.STRING);
+    }
+}
diff --git a/src/main/java/org/apache/rocketmq/connect/redis/parser/LPopParser.java b/src/main/java/org/apache/rocketmq/connect/redis/parser/LPopParser.java
new file mode 100644
index 0000000..cc92668
--- /dev/null
+++ b/src/main/java/org/apache/rocketmq/connect/redis/parser/LPopParser.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
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.rocketmq.connect.redis.parser;
+
+import io.openmessaging.connector.api.data.FieldType;
+import org.apache.rocketmq.connect.redis.pojo.KVEntry;
+import org.apache.rocketmq.connect.redis.pojo.RedisEntry;
+
+/**
+ * LPOP key
+ */
+public class LPopParser extends AbstractCommandParser {
+    @Override
+    public KVEntry createBuilder() {
+        return RedisEntry.newEntry(FieldType.STRING);
+    }
+}
diff --git a/src/main/java/org/apache/rocketmq/connect/redis/parser/LPushParser.java b/src/main/java/org/apache/rocketmq/connect/redis/parser/LPushParser.java
new file mode 100644
index 0000000..68672cf
--- /dev/null
+++ b/src/main/java/org/apache/rocketmq/connect/redis/parser/LPushParser.java
@@ -0,0 +1,41 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.rocketmq.connect.redis.parser;
+
+import java.util.ArrayList;
+import java.util.Arrays;
+import java.util.stream.Collectors;
+
+import io.openmessaging.connector.api.data.FieldType;
+import org.apache.rocketmq.connect.redis.pojo.KVEntry;
+import org.apache.rocketmq.connect.redis.pojo.RedisEntry;
+
+/**
+ * LPUSH key value [value ...]
+ */
+public class LPushParser extends AbstractCommandParser {
+    @Override
+    public KVEntry createBuilder() {
+        return RedisEntry.newEntry(FieldType.ARRAY);
+    }
+
+    @Override
+    public KVEntry handleValue(KVEntry builder, String[] args) {
+        return builder.value(Arrays.stream(args).collect(Collectors.toCollection(ArrayList::new)));
+    }
+}
diff --git a/src/main/java/org/apache/rocketmq/connect/redis/parser/LPushXParser.java b/src/main/java/org/apache/rocketmq/connect/redis/parser/LPushXParser.java
new file mode 100644
index 0000000..bc301a5
--- /dev/null
+++ b/src/main/java/org/apache/rocketmq/connect/redis/parser/LPushXParser.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.connect.redis.parser;
+
+import io.openmessaging.connector.api.data.FieldType;
+import org.apache.rocketmq.connect.redis.pojo.KVEntry;
+import org.apache.rocketmq.connect.redis.pojo.RedisEntry;
+
+/**
+ * LPUSHX key value
+ */
+public class LPushXParser extends AbstractCommandParser {
+    @Override
+    public KVEntry createBuilder() {
+        return RedisEntry.newEntry(FieldType.STRING);
+    }
+
+    @Override
+    public KVEntry handleValue(KVEntry builder, String[] args) {
+        return  builder.value(args[0]);
+    }
+}
diff --git a/src/main/java/org/apache/rocketmq/connect/redis/parser/LRemParser.java b/src/main/java/org/apache/rocketmq/connect/redis/parser/LRemParser.java
new file mode 100644
index 0000000..6716061
--- /dev/null
+++ b/src/main/java/org/apache/rocketmq/connect/redis/parser/LRemParser.java
@@ -0,0 +1,41 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.rocketmq.connect.redis.parser;
+
+import io.openmessaging.connector.api.data.FieldType;
+import org.apache.rocketmq.connect.redis.pojo.KVEntry;
+import org.apache.rocketmq.connect.redis.common.Options;
+import org.apache.rocketmq.connect.redis.pojo.RedisEntry;
+
+/**
+ * LRem key count value
+ */
+public class LRemParser extends AbstractCommandParser {
+    @Override
+    public KVEntry createBuilder() {
+        return RedisEntry.newEntry(FieldType.STRING);
+    }
+
+    @Override
+    public KVEntry handleValue(KVEntry builder, String[] args) {
+        if (args.length == 2) {
+            builder.value(args[1]).param(Options.REDIS_COUNT, Long.parseLong(args[0]));
+        }
+        return builder;
+    }
+}
diff --git a/src/main/java/org/apache/rocketmq/connect/redis/parser/LSetParser.java b/src/main/java/org/apache/rocketmq/connect/redis/parser/LSetParser.java
new file mode 100644
index 0000000..be6fed4
--- /dev/null
+++ b/src/main/java/org/apache/rocketmq/connect/redis/parser/LSetParser.java
@@ -0,0 +1,41 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.rocketmq.connect.redis.parser;
+
+import io.openmessaging.connector.api.data.FieldType;
+import org.apache.rocketmq.connect.redis.pojo.KVEntry;
+import org.apache.rocketmq.connect.redis.common.Options;
+import org.apache.rocketmq.connect.redis.pojo.RedisEntry;
+
+/**
+ * LSet key index value
+ */
+public class LSetParser extends AbstractCommandParser {
+    @Override
+    public KVEntry createBuilder() {
+        return RedisEntry.newEntry(FieldType.STRING);
+    }
+
+    @Override
+    public KVEntry handleValue(KVEntry builder, String[] args) {
+        if (args.length == 2) {
+            builder.value(args[1]).param(Options.REDIS_INDEX, Long.parseLong(args[0]));
+        }
+        return builder;
+    }
+}
diff --git a/src/main/java/org/apache/rocketmq/connect/redis/parser/LTrimParser.java b/src/main/java/org/apache/rocketmq/connect/redis/parser/LTrimParser.java
new file mode 100644
index 0000000..39a4386
--- /dev/null
+++ b/src/main/java/org/apache/rocketmq/connect/redis/parser/LTrimParser.java
@@ -0,0 +1,41 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.rocketmq.connect.redis.parser;
+
+import java.util.ArrayList;
+import java.util.Arrays;
+import java.util.stream.Collectors;
+
+import io.openmessaging.connector.api.data.FieldType;
+import org.apache.rocketmq.connect.redis.pojo.KVEntry;
+import org.apache.rocketmq.connect.redis.pojo.RedisEntry;
+
+/**
+ * LTrim key start stop
+ */
+public class LTrimParser extends AbstractCommandParser {
+    @Override
+    public KVEntry createBuilder() {
+        return RedisEntry.newEntry(FieldType.ARRAY);
+    }
+
+    @Override
+    public KVEntry handleValue(KVEntry builder, String[] args) {
+        return builder.value(Arrays.stream(args).collect(Collectors.toCollection(ArrayList::new)));
+    }
+}
diff --git a/src/main/java/org/apache/rocketmq/connect/redis/parser/LinsertParser.java b/src/main/java/org/apache/rocketmq/connect/redis/parser/LinsertParser.java
new file mode 100644
index 0000000..c3d65b6
--- /dev/null
+++ b/src/main/java/org/apache/rocketmq/connect/redis/parser/LinsertParser.java
@@ -0,0 +1,51 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.rocketmq.connect.redis.parser;
+
+import java.util.ArrayList;
+
+import io.openmessaging.connector.api.data.FieldType;
+import org.apache.rocketmq.connect.redis.pojo.KVEntry;
+import org.apache.rocketmq.connect.redis.common.Options;
+import org.apache.rocketmq.connect.redis.pojo.RedisEntry;
+
+/**
+ * Linsert key BEFORE|AFTER pivot value
+ */
+public class LinsertParser extends AbstractCommandParser {
+    @Override
+    public KVEntry createBuilder() {
+        return RedisEntry.newEntry(FieldType.ARRAY);
+    }
+
+    @Override
+    public KVEntry handleValue(KVEntry builder, String[] args) {
+        if (args.length == 3) {
+            if (Options.REDIS_BEFORE.name().equals(args[0].toUpperCase())) {
+                builder.param(Options.REDIS_BEFORE, Boolean.TRUE);
+            } else {
+                builder.param(Options.REDIS_AFTER, Boolean.TRUE);
+            }
+            builder.value(new ArrayList<String>() {{
+                add(args[1]);
+                add(args[2]);
+            }});
+        }
+        return builder;
+    }
+}
diff --git a/src/main/java/org/apache/rocketmq/connect/redis/parser/MSetNxParser.java b/src/main/java/org/apache/rocketmq/connect/redis/parser/MSetNxParser.java
new file mode 100644
index 0000000..8b13652
--- /dev/null
+++ b/src/main/java/org/apache/rocketmq/connect/redis/parser/MSetNxParser.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.connect.redis.parser;
+
+import io.openmessaging.connector.api.data.FieldType;
+import org.apache.rocketmq.connect.redis.pojo.KVEntry;
+import org.apache.rocketmq.connect.redis.pojo.RedisEntry;
+
+/**
+ * msetnx key value [key value ...]
+ */
+public class MSetNxParser extends AbstractCommandParser {
+    @Override
+    public KVEntry createBuilder() {
+        return RedisEntry.newEntry(FieldType.MAP);
+    }
+
+    @Override
+    public KVEntry handleValue(KVEntry builder, String[] args) {
+        return CommonParser.noCommandKeyMapParser(builder, args);
+    }
+}
diff --git a/src/main/java/org/apache/rocketmq/connect/redis/parser/MSetParser.java b/src/main/java/org/apache/rocketmq/connect/redis/parser/MSetParser.java
new file mode 100644
index 0000000..34f9f11
--- /dev/null
+++ b/src/main/java/org/apache/rocketmq/connect/redis/parser/MSetParser.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.connect.redis.parser;
+
+import io.openmessaging.connector.api.data.FieldType;
+import org.apache.rocketmq.connect.redis.pojo.KVEntry;
+import org.apache.rocketmq.connect.redis.pojo.RedisEntry;
+
+/**
+ * mset key value [key value ...]
+ */
+public class MSetParser extends AbstractCommandParser {
+    @Override
+    public KVEntry createBuilder() {
+        return RedisEntry.newEntry(FieldType.MAP);
+    }
+
+    @Override
+    public KVEntry handleValue(KVEntry builder, String[] args) {
+        return CommonParser.noCommandKeyMapParser(builder, args);
+    }
+}
diff --git a/src/main/java/org/apache/rocketmq/connect/redis/parser/MoveParser.java b/src/main/java/org/apache/rocketmq/connect/redis/parser/MoveParser.java
new file mode 100644
index 0000000..0173086
--- /dev/null
+++ b/src/main/java/org/apache/rocketmq/connect/redis/parser/MoveParser.java
@@ -0,0 +1,38 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.rocketmq.connect.redis.parser;
+
+import io.openmessaging.connector.api.data.FieldType;
+import org.apache.rocketmq.connect.redis.pojo.KVEntry;
+import org.apache.rocketmq.connect.redis.common.Options;
+import org.apache.rocketmq.connect.redis.pojo.RedisEntry;
+
+/**
+ * move key db
+ */
+public class MoveParser extends AbstractCommandParser {
+    @Override
+    public KVEntry createBuilder() {
+        return RedisEntry.newEntry(FieldType.STRING);
+    }
+
+    @Override
+    public KVEntry handleValue(KVEntry builder, String[] args) {
+        return  builder.param(Options.REDIS_DB_INDEX, Integer.parseInt(args[0]));
+    }
+}
diff --git a/src/main/java/org/apache/rocketmq/connect/redis/parser/MultiParser.java b/src/main/java/org/apache/rocketmq/connect/redis/parser/MultiParser.java
new file mode 100644
index 0000000..cd32b16
--- /dev/null
+++ b/src/main/java/org/apache/rocketmq/connect/redis/parser/MultiParser.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
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.rocketmq.connect.redis.parser;
+
+import io.openmessaging.connector.api.data.FieldType;
+import org.apache.rocketmq.connect.redis.pojo.KVEntry;
+import org.apache.rocketmq.connect.redis.pojo.RedisEntry;
+
+/**
+ * Multi
+ */
+public class MultiParser extends AbstractCommandParser {
+    @Override
+    public KVEntry createBuilder() {
+        return RedisEntry.newEntry(FieldType.STRING);
+    }
+}
diff --git a/src/main/java/org/apache/rocketmq/connect/redis/parser/PExpireAtParser.java b/src/main/java/org/apache/rocketmq/connect/redis/parser/PExpireAtParser.java
new file mode 100644
index 0000000..0390de0
--- /dev/null
+++ b/src/main/java/org/apache/rocketmq/connect/redis/parser/PExpireAtParser.java
@@ -0,0 +1,38 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.rocketmq.connect.redis.parser;
+
+import io.openmessaging.connector.api.data.FieldType;
+import org.apache.rocketmq.connect.redis.pojo.KVEntry;
+import org.apache.rocketmq.connect.redis.common.Options;
+import org.apache.rocketmq.connect.redis.pojo.RedisEntry;
+
+/**
+ * PExpireAt key milliseconds-timestamp
+ */
+public class PExpireAtParser extends AbstractCommandParser {
+    @Override
+    public KVEntry createBuilder() {
+        return RedisEntry.newEntry(FieldType.STRING);
+    }
+
+    @Override
+    public KVEntry handleValue(KVEntry builder, String[] args) {
+        return builder.param(Options.REDIS_PX_TIMESTAMP, Long.parseLong(args[0]));
+    }
+}
diff --git a/src/main/java/org/apache/rocketmq/connect/redis/parser/PExpireParser.java b/src/main/java/org/apache/rocketmq/connect/redis/parser/PExpireParser.java
new file mode 100644
index 0000000..efeee21
--- /dev/null
+++ b/src/main/java/org/apache/rocketmq/connect/redis/parser/PExpireParser.java
@@ -0,0 +1,38 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.rocketmq.connect.redis.parser;
+
+import io.openmessaging.connector.api.data.FieldType;
+import org.apache.rocketmq.connect.redis.pojo.KVEntry;
+import org.apache.rocketmq.connect.redis.common.Options;
+import org.apache.rocketmq.connect.redis.pojo.RedisEntry;
+
+/**
+ * PExpire key milliseconds
+ */
+public class PExpireParser extends AbstractCommandParser {
+    @Override
+    public KVEntry createBuilder() {
+        return RedisEntry.newEntry(FieldType.STRING);
+    }
+
+    @Override
+    public KVEntry handleValue(KVEntry builder, String[] args) {
+        return builder.param(Options.REDIS_PX, Long.parseLong(args[0]));
+    }
+}
diff --git a/src/main/java/org/apache/rocketmq/connect/redis/parser/PSetExParser.java b/src/main/java/org/apache/rocketmq/connect/redis/parser/PSetExParser.java
new file mode 100644
index 0000000..7ed476e
--- /dev/null
+++ b/src/main/java/org/apache/rocketmq/connect/redis/parser/PSetExParser.java
@@ -0,0 +1,41 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.rocketmq.connect.redis.parser;
+
+import io.openmessaging.connector.api.data.FieldType;
+import org.apache.rocketmq.connect.redis.pojo.KVEntry;
+import org.apache.rocketmq.connect.redis.common.Options;
+import org.apache.rocketmq.connect.redis.pojo.RedisEntry;
+
+/**
+ * PSETEX key milliseconds value
+ */
+public class PSetExParser extends AbstractCommandParser {
+    @Override
+    public KVEntry createBuilder() {
+        return RedisEntry.newEntry(FieldType.STRING);
+    }
+
+    @Override
+    public KVEntry handleValue(KVEntry builder, String[] args) {
+        if (args.length == 2) {
+            builder.value(args[1]).param(Options.REDIS_PX, Long.parseLong(args[0]));
+        }
+        return builder;
+    }
+}
diff --git a/src/main/java/org/apache/rocketmq/connect/redis/parser/Parser.java b/src/main/java/org/apache/rocketmq/connect/redis/parser/Parser.java
new file mode 100644
index 0000000..b9a4b12
--- /dev/null
+++ b/src/main/java/org/apache/rocketmq/connect/redis/parser/Parser.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
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.rocketmq.connect.redis.parser;
+
+import com.moilioncircle.redis.replicator.cmd.Command;
+
+public interface Parser<T> {
+    T parse(Command command);
+}
diff --git a/src/main/java/org/apache/rocketmq/connect/redis/parser/PersistParser.java b/src/main/java/org/apache/rocketmq/connect/redis/parser/PersistParser.java
new file mode 100644
index 0000000..6bfb2d0
--- /dev/null
+++ b/src/main/java/org/apache/rocketmq/connect/redis/parser/PersistParser.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
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.rocketmq.connect.redis.parser;
+
+import io.openmessaging.connector.api.data.FieldType;
+import org.apache.rocketmq.connect.redis.pojo.KVEntry;
+import org.apache.rocketmq.connect.redis.pojo.RedisEntry;
+
+/**
+ * PERSIST key
+ */
+public class PersistParser extends AbstractCommandParser {
+    @Override
+    public KVEntry createBuilder() {
+        return RedisEntry.newEntry(FieldType.STRING);
+    }
+}
diff --git a/src/main/java/org/apache/rocketmq/connect/redis/parser/PfAddParser.java b/src/main/java/org/apache/rocketmq/connect/redis/parser/PfAddParser.java
new file mode 100644
index 0000000..51cfda2
--- /dev/null
+++ b/src/main/java/org/apache/rocketmq/connect/redis/parser/PfAddParser.java
@@ -0,0 +1,41 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.rocketmq.connect.redis.parser;
+
+import java.util.ArrayList;
+import java.util.Arrays;
+import java.util.stream.Collectors;
+
+import io.openmessaging.connector.api.data.FieldType;
+import org.apache.rocketmq.connect.redis.pojo.KVEntry;
+import org.apache.rocketmq.connect.redis.pojo.RedisEntry;
+
+/**
+ * PFADD key element [element ...]
+ */
+public class PfAddParser extends AbstractCommandParser {
+    @Override
+    public KVEntry createBuilder() {
+        return RedisEntry.newEntry(FieldType.ARRAY);
+    }
+
+    @Override
+    public KVEntry handleValue(KVEntry builder, String[] args) {
+        return builder.value(Arrays.stream(args).collect(Collectors.toCollection(ArrayList::new)));
+    }
+}
diff --git a/src/main/java/org/apache/rocketmq/connect/redis/parser/PfCountParser.java b/src/main/java/org/apache/rocketmq/connect/redis/parser/PfCountParser.java
new file mode 100644
index 0000000..fe674d9
--- /dev/null
+++ b/src/main/java/org/apache/rocketmq/connect/redis/parser/PfCountParser.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.connect.redis.parser;
+
+import io.openmessaging.connector.api.data.FieldType;
+import org.apache.rocketmq.connect.redis.pojo.KVEntry;
+import org.apache.rocketmq.connect.redis.pojo.RedisEntry;
+
+/**
+ * PFCOUNT key [key ...]
+ */
+public class PfCountParser extends AbstractCommandParser {
+    @Override
+    public KVEntry createBuilder() {
+        return RedisEntry.newEntry(FieldType.ARRAY);
+    }
+
+    @Override
+    public KVEntry handleValue(KVEntry builder, String[] args) {
+        return CommonParser.noCommandKeyArrayParser(builder, args);
+    }
+}
diff --git a/src/main/java/org/apache/rocketmq/connect/redis/parser/PfMergeParser.java b/src/main/java/org/apache/rocketmq/connect/redis/parser/PfMergeParser.java
new file mode 100644
index 0000000..650c75b
--- /dev/null
+++ b/src/main/java/org/apache/rocketmq/connect/redis/parser/PfMergeParser.java
@@ -0,0 +1,41 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.rocketmq.connect.redis.parser;
+
+import java.util.ArrayList;
+import java.util.Arrays;
+import java.util.stream.Collectors;
+
+import io.openmessaging.connector.api.data.FieldType;
+import org.apache.rocketmq.connect.redis.pojo.KVEntry;
+import org.apache.rocketmq.connect.redis.pojo.RedisEntry;
+
+/**
+ * PFMERGE destkey sourcekey [sourcekey ...]
+ */
+public class PfMergeParser extends AbstractCommandParser {
+    @Override
+    public KVEntry createBuilder() {
+        return RedisEntry.newEntry(FieldType.ARRAY);
+    }
+
+    @Override
+    public KVEntry handleValue(KVEntry builder, String[] args) {
+        return builder.value(Arrays.stream(args).collect(Collectors.toCollection(ArrayList::new)));
+    }
+}
diff --git a/src/main/java/org/apache/rocketmq/connect/redis/parser/PublishParser.java b/src/main/java/org/apache/rocketmq/connect/redis/parser/PublishParser.java
new file mode 100644
index 0000000..0d5d235
--- /dev/null
+++ b/src/main/java/org/apache/rocketmq/connect/redis/parser/PublishParser.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.connect.redis.parser;
+
+import io.openmessaging.connector.api.data.FieldType;
+import org.apache.rocketmq.connect.redis.pojo.KVEntry;
+import org.apache.rocketmq.connect.redis.pojo.RedisEntry;
+
+/**
+ * Publish channel message
+ */
+public class PublishParser extends AbstractCommandParser {
+    @Override
+    public KVEntry createBuilder() {
+        return RedisEntry.newEntry(FieldType.STRING);
+    }
+
+    @Override
+    public KVEntry handleValue(KVEntry builder, String[] args) {
+        return builder.value(args[0]);
+    }
+}
diff --git a/src/main/java/org/apache/rocketmq/connect/redis/parser/RPopLPushParser.java b/src/main/java/org/apache/rocketmq/connect/redis/parser/RPopLPushParser.java
new file mode 100644
index 0000000..38edce3
--- /dev/null
+++ b/src/main/java/org/apache/rocketmq/connect/redis/parser/RPopLPushParser.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.connect.redis.parser;
+
+import io.openmessaging.connector.api.data.FieldType;
+import org.apache.rocketmq.connect.redis.pojo.KVEntry;
+import org.apache.rocketmq.connect.redis.pojo.RedisEntry;
+
+/**
+ * RPOPLPUSH source destination
+ */
+public class RPopLPushParser extends AbstractCommandParser {
+    @Override
+    public KVEntry createBuilder() {
+        return RedisEntry.newEntry(FieldType.STRING);
+    }
+
+    @Override
+    public KVEntry handleValue(KVEntry builder, String[] args) {
+        return builder.value(args[0]);
+    }
+}
diff --git a/src/main/java/org/apache/rocketmq/connect/redis/parser/RPopParser.java b/src/main/java/org/apache/rocketmq/connect/redis/parser/RPopParser.java
new file mode 100644
index 0000000..85f122a
--- /dev/null
+++ b/src/main/java/org/apache/rocketmq/connect/redis/parser/RPopParser.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
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.rocketmq.connect.redis.parser;
+
+import io.openmessaging.connector.api.data.FieldType;
+import org.apache.rocketmq.connect.redis.pojo.KVEntry;
+import org.apache.rocketmq.connect.redis.pojo.RedisEntry;
+
+/**
+ * RPOP key
+ */
+public class RPopParser extends AbstractCommandParser {
+    @Override
+    public KVEntry createBuilder() {
+        return RedisEntry.newEntry(FieldType.STRING);
+    }
+}
diff --git a/src/main/java/org/apache/rocketmq/connect/redis/parser/RPushParser.java b/src/main/java/org/apache/rocketmq/connect/redis/parser/RPushParser.java
new file mode 100644
index 0000000..8824d0b
--- /dev/null
+++ b/src/main/java/org/apache/rocketmq/connect/redis/parser/RPushParser.java
@@ -0,0 +1,41 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.rocketmq.connect.redis.parser;
+
+import java.util.ArrayList;
+import java.util.Arrays;
+import java.util.stream.Collectors;
+
+import io.openmessaging.connector.api.data.FieldType;
+import org.apache.rocketmq.connect.redis.pojo.KVEntry;
+import org.apache.rocketmq.connect.redis.pojo.RedisEntry;
+
+/**
+ * RPUSH key value [value ...]
+ */
+public class RPushParser extends AbstractCommandParser {
+    @Override
+    public KVEntry createBuilder() {
+        return RedisEntry.newEntry(FieldType.ARRAY);
+    }
+
+    @Override
+    public KVEntry handleValue(KVEntry builder, String[] args) {
+        return builder.value(Arrays.stream(args).collect(Collectors.toCollection(ArrayList::new)));
+    }
+}
diff --git a/src/main/java/org/apache/rocketmq/connect/redis/parser/RPushXParser.java b/src/main/java/org/apache/rocketmq/connect/redis/parser/RPushXParser.java
new file mode 100644
index 0000000..de2d21b
--- /dev/null
+++ b/src/main/java/org/apache/rocketmq/connect/redis/parser/RPushXParser.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.connect.redis.parser;
+
+import io.openmessaging.connector.api.data.FieldType;
+import org.apache.rocketmq.connect.redis.pojo.KVEntry;
+import org.apache.rocketmq.connect.redis.pojo.RedisEntry;
+
+/**
+ * RPUSHX key value
+ */
+public class RPushXParser extends AbstractCommandParser {
+    @Override
+    public KVEntry createBuilder() {
+        return RedisEntry.newEntry(FieldType.STRING);
+    }
+
+    @Override
+    public KVEntry handleValue(KVEntry builder, String[] args) {
+        return builder.value(args[0]);
+    }
+}
diff --git a/src/main/java/org/apache/rocketmq/connect/redis/parser/RedisRdbParser.java b/src/main/java/org/apache/rocketmq/connect/redis/parser/RedisRdbParser.java
new file mode 100644
index 0000000..fa4e46a
--- /dev/null
+++ b/src/main/java/org/apache/rocketmq/connect/redis/parser/RedisRdbParser.java
@@ -0,0 +1,67 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.rocketmq.connect.redis.parser;
+
+import java.io.IOException;
+import java.util.List;
+import java.util.Map;
+import java.util.Set;
+
+import com.moilioncircle.redis.replicator.rdb.datatype.KeyValuePair;
+import com.moilioncircle.redis.replicator.rdb.datatype.Module;
+import com.moilioncircle.redis.replicator.rdb.datatype.Stream;
+import com.moilioncircle.redis.replicator.rdb.datatype.ZSetEntry;
+
+/**
+ * 处理RDB文件接口
+ *
+ * @param <T>
+ */
+public interface RedisRdbParser<T> {
+
+    T applyString(KeyValuePair<byte[], byte[]> keyValuePair) throws IOException;
+
+    T applyList(KeyValuePair<byte[], List<byte[]>> keyValuePair) throws IOException;
+
+    T applySet(KeyValuePair<byte[], Set<byte[]>> keyValuePair) throws IOException;
+
+    T applyZSet(KeyValuePair<byte[], Set<ZSetEntry>> keyValuePair) throws IOException;
+
+    T applyZSet2(KeyValuePair<byte[], Set<ZSetEntry>> keyValuePair) throws IOException;
+
+    T applyHash(KeyValuePair<byte[], Map<byte[], byte[]>> keyValuePair) throws IOException;
+
+    T applyHashZipMap(KeyValuePair<byte[], Map<byte[], byte[]>> keyValuePair) throws IOException;
+
+    T applyListZipList(KeyValuePair<byte[], List<byte[]>> keyValuePair) throws IOException;
+
+    T applySetIntSet(KeyValuePair<byte[], Set<byte[]>> keyValuePair) throws IOException;
+
+    T applyZSetZipList(KeyValuePair<byte[], Set<ZSetEntry>> keyValuePair) throws IOException;
+
+    T applyHashZipList(KeyValuePair<byte[], Map<byte[], byte[]>> keyValuePair) throws IOException;
+
+    T applyListQuickList(KeyValuePair<byte[], List<byte[]>> keyValuePair) throws IOException;
+
+    T applyModule(KeyValuePair<byte[], Module> keyValuePair) throws IOException;
+
+    T applyModule2(KeyValuePair<byte[], Module> keyValuePair) throws IOException;
+
+    T applyStreamListPacks(KeyValuePair<byte[], Stream> keyValuePair) throws IOException;
+
+}
diff --git a/src/main/java/org/apache/rocketmq/connect/redis/parser/RenameNxParser.java b/src/main/java/org/apache/rocketmq/connect/redis/parser/RenameNxParser.java
new file mode 100644
index 0000000..14f4c4f
--- /dev/null
+++ b/src/main/java/org/apache/rocketmq/connect/redis/parser/RenameNxParser.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.connect.redis.parser;
+
+import io.openmessaging.connector.api.data.FieldType;
+import org.apache.rocketmq.connect.redis.pojo.KVEntry;
+import org.apache.rocketmq.connect.redis.pojo.RedisEntry;
+
+/**
+ * RENAMENX key newkey
+ */
+public class RenameNxParser extends AbstractCommandParser {
+    @Override
+    public KVEntry createBuilder() {
+        return RedisEntry.newEntry(FieldType.STRING);
+    }
+
+    @Override
+    public KVEntry handleValue(KVEntry builder, String[] args) {
+        return builder.value(args[0]);
+    }
+}
diff --git a/src/main/java/org/apache/rocketmq/connect/redis/parser/RenameParser.java b/src/main/java/org/apache/rocketmq/connect/redis/parser/RenameParser.java
new file mode 100644
index 0000000..9de8fe3
--- /dev/null
+++ b/src/main/java/org/apache/rocketmq/connect/redis/parser/RenameParser.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.connect.redis.parser;
+
+import io.openmessaging.connector.api.data.FieldType;
+import org.apache.rocketmq.connect.redis.pojo.KVEntry;
+import org.apache.rocketmq.connect.redis.pojo.RedisEntry;
+
+/**
+ * RENAME key newkey
+ */
+public class RenameParser extends AbstractCommandParser {
+    @Override
+    public KVEntry createBuilder() {
+        return RedisEntry.newEntry(FieldType.STRING);
+    }
+
+    @Override
+    public KVEntry handleValue(KVEntry builder, String[] args) {
+        return builder.value(args[0]);
+    }
+}
diff --git a/src/main/java/org/apache/rocketmq/connect/redis/parser/RestoreParser.java b/src/main/java/org/apache/rocketmq/connect/redis/parser/RestoreParser.java
new file mode 100644
index 0000000..53a4569
--- /dev/null
+++ b/src/main/java/org/apache/rocketmq/connect/redis/parser/RestoreParser.java
@@ -0,0 +1,45 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.rocketmq.connect.redis.parser;
+
+import io.openmessaging.connector.api.data.FieldType;
+import org.apache.rocketmq.connect.redis.pojo.KVEntry;
+import org.apache.rocketmq.connect.redis.common.Options;
+import org.apache.rocketmq.connect.redis.pojo.RedisEntry;
+
+/**
+ * Restore key ttl serialized-value [REPLACE]
+ */
+public class RestoreParser extends AbstractCommandParser {
+    @Override
+    public KVEntry createBuilder() {
+        return RedisEntry.newEntry(FieldType.STRING);
+    }
+
+    @Override
+    public KVEntry handleValue(KVEntry builder, String[] args) {
+        if (args.length > 1) {
+            long ttl = Long.parseLong(args[0]);
+            builder.param(Options.REDIS_TTL, ttl).value(args[1]);
+            if (args.length > 2) {
+                builder.param(Options.REDIS_REPLACE, Boolean.TRUE);
+            }
+        }
+        return builder;
+    }
+}
diff --git a/src/main/java/org/apache/rocketmq/connect/redis/parser/SAddParser.java b/src/main/java/org/apache/rocketmq/connect/redis/parser/SAddParser.java
new file mode 100644
index 0000000..bb89eff
--- /dev/null
+++ b/src/main/java/org/apache/rocketmq/connect/redis/parser/SAddParser.java
@@ -0,0 +1,41 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.rocketmq.connect.redis.parser;
+
+import java.util.ArrayList;
+import java.util.Arrays;
+import java.util.stream.Collectors;
+
+import io.openmessaging.connector.api.data.FieldType;
+import org.apache.rocketmq.connect.redis.pojo.KVEntry;
+import org.apache.rocketmq.connect.redis.pojo.RedisEntry;
+
+/**
+ * SADD key member [member ...]
+ */
+public class SAddParser extends AbstractCommandParser {
+    @Override
+    public KVEntry createBuilder() {
+        return RedisEntry.newEntry(FieldType.ARRAY);
+    }
+
+    @Override
+    public KVEntry handleValue(KVEntry builder, String[] args) {
+        return builder.value(Arrays.stream(args).collect(Collectors.toCollection(ArrayList::new)));
+    }
+}
diff --git a/src/main/java/org/apache/rocketmq/connect/redis/parser/SDiffStoreParser.java b/src/main/java/org/apache/rocketmq/connect/redis/parser/SDiffStoreParser.java
new file mode 100644
index 0000000..dc5ca3e
--- /dev/null
+++ b/src/main/java/org/apache/rocketmq/connect/redis/parser/SDiffStoreParser.java
@@ -0,0 +1,41 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.rocketmq.connect.redis.parser;
+
+import java.util.ArrayList;
+import java.util.Arrays;
+import java.util.stream.Collectors;
+
+import io.openmessaging.connector.api.data.FieldType;
+import org.apache.rocketmq.connect.redis.pojo.KVEntry;
+import org.apache.rocketmq.connect.redis.pojo.RedisEntry;
+
+/**
+ * SDIFFSTORE destination key [key ...]
+ */
+public class SDiffStoreParser extends AbstractCommandParser {
+    @Override
+    public KVEntry createBuilder() {
+        return RedisEntry.newEntry(FieldType.ARRAY);
+    }
+
+    @Override
+    public KVEntry handleValue(KVEntry builder, String[] args) {
+        return builder.value(Arrays.stream(args).collect(Collectors.toCollection(ArrayList::new)));
+    }
+}
diff --git a/src/main/java/org/apache/rocketmq/connect/redis/parser/SInterStoreParser.java b/src/main/java/org/apache/rocketmq/connect/redis/parser/SInterStoreParser.java
new file mode 100644
index 0000000..be52db7
--- /dev/null
+++ b/src/main/java/org/apache/rocketmq/connect/redis/parser/SInterStoreParser.java
@@ -0,0 +1,41 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.rocketmq.connect.redis.parser;
+
+import java.util.ArrayList;
+import java.util.Arrays;
+import java.util.stream.Collectors;
+
+import io.openmessaging.connector.api.data.FieldType;
+import org.apache.rocketmq.connect.redis.pojo.KVEntry;
+import org.apache.rocketmq.connect.redis.pojo.RedisEntry;
+
+/**
+ * SINTERSTORE destination key [key ...]
+ */
+public class SInterStoreParser extends AbstractCommandParser {
+    @Override
+    public KVEntry createBuilder() {
+        return RedisEntry.newEntry(FieldType.ARRAY);
+    }
+
+    @Override
+    public KVEntry handleValue(KVEntry builder, String[] args) {
+        return builder.value(Arrays.stream(args).collect(Collectors.toCollection(ArrayList::new)));
+    }
+}
diff --git a/src/main/java/org/apache/rocketmq/connect/redis/parser/SMoveParser.java b/src/main/java/org/apache/rocketmq/connect/redis/parser/SMoveParser.java
new file mode 100644
index 0000000..62007bb
--- /dev/null
+++ b/src/main/java/org/apache/rocketmq/connect/redis/parser/SMoveParser.java
@@ -0,0 +1,41 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.rocketmq.connect.redis.parser;
+
+import java.util.ArrayList;
+import java.util.Arrays;
+import java.util.stream.Collectors;
+
+import io.openmessaging.connector.api.data.FieldType;
+import org.apache.rocketmq.connect.redis.pojo.KVEntry;
+import org.apache.rocketmq.connect.redis.pojo.RedisEntry;
+
+/**
+ * SMOVE source destination member
+ */
+public class SMoveParser extends AbstractCommandParser {
+    @Override
+    public KVEntry createBuilder() {
+        return RedisEntry.newEntry(FieldType.ARRAY);
+    }
+
+    @Override
+    public KVEntry handleValue(KVEntry builder, String[] args) {
+        return builder.value(Arrays.stream(args).collect(Collectors.toCollection(ArrayList::new)));
+    }
+}
diff --git a/src/main/java/org/apache/rocketmq/connect/redis/parser/SRemParser.java b/src/main/java/org/apache/rocketmq/connect/redis/parser/SRemParser.java
new file mode 100644
index 0000000..f51ad00
--- /dev/null
+++ b/src/main/java/org/apache/rocketmq/connect/redis/parser/SRemParser.java
@@ -0,0 +1,41 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.rocketmq.connect.redis.parser;
+
+import java.util.ArrayList;
+import java.util.Arrays;
+import java.util.stream.Collectors;
+
+import io.openmessaging.connector.api.data.FieldType;
+import org.apache.rocketmq.connect.redis.pojo.KVEntry;
+import org.apache.rocketmq.connect.redis.pojo.RedisEntry;
+
+/**
+ * SREM key member [member ...]
+ */
+public class SRemParser extends AbstractCommandParser {
+    @Override
+    public KVEntry createBuilder() {
+        return RedisEntry.newEntry(FieldType.ARRAY);
+    }
+
+    @Override
+    public KVEntry handleValue(KVEntry builder, String[] args) {
+        return builder.value(Arrays.stream(args).collect(Collectors.toCollection(ArrayList::new)));
+    }
+}
diff --git a/src/main/java/org/apache/rocketmq/connect/redis/parser/SUnionStoreParser.java b/src/main/java/org/apache/rocketmq/connect/redis/parser/SUnionStoreParser.java
new file mode 100644
index 0000000..d821aba
--- /dev/null
+++ b/src/main/java/org/apache/rocketmq/connect/redis/parser/SUnionStoreParser.java
@@ -0,0 +1,41 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.rocketmq.connect.redis.parser;
+
+import java.util.ArrayList;
+import java.util.Arrays;
+import java.util.stream.Collectors;
+
+import io.openmessaging.connector.api.data.FieldType;
+import org.apache.rocketmq.connect.redis.pojo.KVEntry;
+import org.apache.rocketmq.connect.redis.pojo.RedisEntry;
+
+/**
+ * SUNIONSTORE destination key [key ...]
+ */
+public class SUnionStoreParser extends AbstractCommandParser {
+    @Override
+    public KVEntry createBuilder() {
+        return RedisEntry.newEntry(FieldType.ARRAY);
+    }
+
+    @Override
+    public KVEntry handleValue(KVEntry builder, String[] args) {
+        return builder.value(Arrays.stream(args).collect(Collectors.toCollection(ArrayList::new)));
+    }
+}
diff --git a/src/main/java/org/apache/rocketmq/connect/redis/parser/ScriptParser.java b/src/main/java/org/apache/rocketmq/connect/redis/parser/ScriptParser.java
new file mode 100644
index 0000000..746edb2
--- /dev/null
+++ b/src/main/java/org/apache/rocketmq/connect/redis/parser/ScriptParser.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
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.rocketmq.connect.redis.parser;
+
+import io.openmessaging.connector.api.data.FieldType;
+import org.apache.rocketmq.connect.redis.pojo.KVEntry;
+import org.apache.rocketmq.connect.redis.pojo.RedisEntry;
+
+/**
+ *
+ */
+public class ScriptParser extends AbstractCommandParser {
+    @Override
+    public KVEntry createBuilder() {
+        return RedisEntry.newEntry(FieldType.STRING);
+    }
+}
diff --git a/src/main/java/org/apache/rocketmq/connect/redis/parser/SelectParser.java b/src/main/java/org/apache/rocketmq/connect/redis/parser/SelectParser.java
new file mode 100644
index 0000000..a6ba72f
--- /dev/null
+++ b/src/main/java/org/apache/rocketmq/connect/redis/parser/SelectParser.java
@@ -0,0 +1,38 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.rocketmq.connect.redis.parser;
+
+import io.openmessaging.connector.api.data.FieldType;
+import org.apache.rocketmq.connect.redis.common.Options;
+import org.apache.rocketmq.connect.redis.pojo.KVEntry;
+import org.apache.rocketmq.connect.redis.pojo.RedisEntry;
+
+/**
+ * Select index
+ */
+public class SelectParser extends AbstractCommandParser {
+    @Override
+    public KVEntry createBuilder() {
+        return RedisEntry.newEntry(FieldType.STRING);
+    }
+
+    @Override
+    public KVEntry handleNoArgValue(KVEntry builder) {
+        return builder.param(Options.REDIS_DB_INDEX, Integer.parseInt(builder.getKey()));
+    }
+}
diff --git a/src/main/java/org/apache/rocketmq/connect/redis/parser/SetBitParser.java b/src/main/java/org/apache/rocketmq/connect/redis/parser/SetBitParser.java
new file mode 100644
index 0000000..b7b9495
--- /dev/null
+++ b/src/main/java/org/apache/rocketmq/connect/redis/parser/SetBitParser.java
@@ -0,0 +1,41 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.rocketmq.connect.redis.parser;
+
+import io.openmessaging.connector.api.data.FieldType;
+import org.apache.rocketmq.connect.redis.pojo.KVEntry;
+import org.apache.rocketmq.connect.redis.common.Options;
+import org.apache.rocketmq.connect.redis.pojo.RedisEntry;
+
+/**
+ * SetBit key offset value
+ */
+public class SetBitParser extends AbstractCommandParser {
+    @Override
+    public KVEntry createBuilder() {
+        return RedisEntry.newEntry(FieldType.STRING);
+    }
+
+    @Override
+    public KVEntry handleValue(KVEntry builder, String[] args) {
+        if (args.length == 2) {
+            builder.value(args[1]).param(Options.REDIS_OFFSET, Long.parseLong(args[0]));
+        }
+        return builder;
+    }
+}
diff --git a/src/main/java/org/apache/rocketmq/connect/redis/parser/SetExParser.java b/src/main/java/org/apache/rocketmq/connect/redis/parser/SetExParser.java
new file mode 100644
index 0000000..4c41260
--- /dev/null
+++ b/src/main/java/org/apache/rocketmq/connect/redis/parser/SetExParser.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.connect.redis.parser;
+
+import io.openmessaging.connector.api.data.FieldType;
+import org.apache.rocketmq.connect.redis.pojo.KVEntry;
+import org.apache.rocketmq.connect.redis.common.Options;
+import org.apache.rocketmq.connect.redis.pojo.RedisEntry;
+
+/**
+ * setex key seconds value
+ */
+public class SetExParser extends AbstractCommandParser {
+    @Override
+    public KVEntry createBuilder() {
+        return RedisEntry.newEntry(FieldType.STRING);
+    }
+
+    @Override
+    public KVEntry handleValue(KVEntry builder, String[] args) {
+        if (args.length == 2) {
+            builder.value(args[1]);
+            builder.param(Options.REDIS_EX, Integer.parseInt(args[0]));
+        }
+        return builder;
+    }
+}
diff --git a/src/main/java/org/apache/rocketmq/connect/redis/parser/SetNxParser.java b/src/main/java/org/apache/rocketmq/connect/redis/parser/SetNxParser.java
new file mode 100644
index 0000000..ba784b0
--- /dev/null
+++ b/src/main/java/org/apache/rocketmq/connect/redis/parser/SetNxParser.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.connect.redis.parser;
+
+import io.openmessaging.connector.api.data.FieldType;
+import org.apache.rocketmq.connect.redis.pojo.KVEntry;
+import org.apache.rocketmq.connect.redis.pojo.RedisEntry;
+
+/**
+ * SETNX a b
+ */
+public class SetNxParser extends AbstractCommandParser {
+    @Override
+    public KVEntry createBuilder() {
+        return RedisEntry.newEntry(FieldType.STRING);
+    }
+
+    @Override
+    public KVEntry handleValue(KVEntry builder, String[] args) {
+        return  builder.value(args[0]);
+    }
+}
diff --git a/src/main/java/org/apache/rocketmq/connect/redis/parser/SetParser.java b/src/main/java/org/apache/rocketmq/connect/redis/parser/SetParser.java
new file mode 100644
index 0000000..b21fa1d
--- /dev/null
+++ b/src/main/java/org/apache/rocketmq/connect/redis/parser/SetParser.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.connect.redis.parser;
+
+import io.openmessaging.connector.api.data.FieldType;
+import org.apache.rocketmq.connect.redis.common.RedisConstants;
+import org.apache.rocketmq.connect.redis.pojo.KVEntry;
+import org.apache.rocketmq.connect.redis.common.Options;
+import org.apache.rocketmq.connect.redis.pojo.RedisEntry;
+
+/**
+ * set key value [expiration EX seconds|PX milliseconds] [NX|XX]
+ */
+public class SetParser extends AbstractCommandParser {
+    @Override
+    public KVEntry createBuilder() {
+        return RedisEntry.newEntry(FieldType.STRING);
+    }
+
+    @Override
+    public KVEntry handleValue(KVEntry builder, String[] args) {
+        builder.value(args[0]);
+        int idx = 1;
+        while (args.length > idx){
+            String expiration = args[idx];
+            switch (expiration) {
+                case RedisConstants
+                    .EX:
+                    if (args.length > 2) {
+                        builder.param(Options.REDIS_EX, Integer.parseInt(args[2]));
+                        idx += 2;
+                    }
+                    break;
+                case RedisConstants
+                    .PX:
+                    if (args.length > 2) {
+                        builder.param(Options.REDIS_PX, Long.parseLong(args[2]));
+                        idx += 2;
+                    }
+                    break;
+                case RedisConstants
+                    .NX:
+                    builder.param(Options.REDIS_NX, Boolean.TRUE);
+                    idx += 2;
+                    break;
+                case RedisConstants
+                    .XX:
+                    builder.param(Options.REDIS_XX, Boolean.TRUE);
+                    idx += 2;
+                    break;
+            }
+        }
+        return builder;
+    }
+}
diff --git a/src/main/java/org/apache/rocketmq/connect/redis/parser/SetRangeParser.java b/src/main/java/org/apache/rocketmq/connect/redis/parser/SetRangeParser.java
new file mode 100644
index 0000000..61b3c39
--- /dev/null
+++ b/src/main/java/org/apache/rocketmq/connect/redis/parser/SetRangeParser.java
@@ -0,0 +1,41 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.rocketmq.connect.redis.parser;
+
+import io.openmessaging.connector.api.data.FieldType;
+import org.apache.rocketmq.connect.redis.pojo.KVEntry;
+import org.apache.rocketmq.connect.redis.common.Options;
+import org.apache.rocketmq.connect.redis.pojo.RedisEntry;
+
+/**
+ * SETRANGE key offset value
+ */
+public class SetRangeParser extends AbstractCommandParser {
+    @Override
+    public KVEntry createBuilder() {
+        return RedisEntry.newEntry(FieldType.STRING);
+    }
+
+    @Override
+    public KVEntry handleValue(KVEntry builder, String[] args) {
+        if (args.length == 2) {
+            builder.value(args[1]).param(Options.REDIS_OFFSET, Long.parseLong(args[0]));
+        }
+        return builder;
+    }
+}
diff --git a/src/main/java/org/apache/rocketmq/connect/redis/parser/SortParser.java b/src/main/java/org/apache/rocketmq/connect/redis/parser/SortParser.java
new file mode 100644
index 0000000..0888917
--- /dev/null
+++ b/src/main/java/org/apache/rocketmq/connect/redis/parser/SortParser.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
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.rocketmq.connect.redis.parser;
+
+import io.openmessaging.connector.api.data.FieldType;
+import org.apache.rocketmq.connect.redis.pojo.KVEntry;
+import org.apache.rocketmq.connect.redis.pojo.RedisEntry;
+
+/**
+ * SORT key [BY pattern] [LIMIT offset count] [GET pattern [GET pattern ...]] [ASC|DESC] [ALPHA] [STORE destination]
+ */
+public class SortParser extends AbstractCommandParser {
+    @Override
+    public KVEntry createBuilder() {
+        return RedisEntry.newEntry(FieldType.STRING);
+    }
+}
diff --git a/src/main/java/org/apache/rocketmq/connect/redis/parser/SwapDbParser.java b/src/main/java/org/apache/rocketmq/connect/redis/parser/SwapDbParser.java
new file mode 100644
index 0000000..27d7a33
--- /dev/null
+++ b/src/main/java/org/apache/rocketmq/connect/redis/parser/SwapDbParser.java
@@ -0,0 +1,38 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.rocketmq.connect.redis.parser;
+
+
+import io.openmessaging.connector.api.data.FieldType;
+import org.apache.rocketmq.connect.redis.pojo.KVEntry;
+import org.apache.rocketmq.connect.redis.pojo.RedisEntry;
+
+/**
+ * SWAPDB index index
+ */
+public class SwapDbParser extends AbstractCommandParser {
+    @Override
+    public KVEntry createBuilder() {
+        return RedisEntry.newEntry(FieldType.STRING);
+    }
+
+    @Override
+    public KVEntry handleValue(KVEntry builder, String[] args) {
+        return builder.value(args[0]);
+    }
+}
diff --git a/src/main/java/org/apache/rocketmq/connect/redis/parser/UnLinkParser.java b/src/main/java/org/apache/rocketmq/connect/redis/parser/UnLinkParser.java
new file mode 100644
index 0000000..6ee98ce
--- /dev/null
+++ b/src/main/java/org/apache/rocketmq/connect/redis/parser/UnLinkParser.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.connect.redis.parser;
+
+import io.openmessaging.connector.api.data.FieldType;
+import org.apache.rocketmq.connect.redis.pojo.KVEntry;
+import org.apache.rocketmq.connect.redis.pojo.RedisEntry;
+
+/**
+ *  UnLink key [key ...]
+ */
+public class UnLinkParser extends AbstractCommandParser {
+    @Override
+    public KVEntry createBuilder() {
+        return RedisEntry.newEntry(FieldType.ARRAY);
+    }
+
+    @Override
+    public KVEntry handleValue(KVEntry builder, String[] args) {
+        return CommonParser.noCommandKeyArrayParser(builder, args);
+    }
+}
diff --git a/src/main/java/org/apache/rocketmq/connect/redis/parser/XAckParser.java b/src/main/java/org/apache/rocketmq/connect/redis/parser/XAckParser.java
new file mode 100644
index 0000000..7c2e2c9
--- /dev/null
+++ b/src/main/java/org/apache/rocketmq/connect/redis/parser/XAckParser.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.connect.redis.parser;
+
+import java.util.ArrayList;
+import java.util.List;
+
+import io.openmessaging.connector.api.data.FieldType;
+import org.apache.rocketmq.connect.redis.pojo.KVEntry;
+import org.apache.rocketmq.connect.redis.common.Options;
+import org.apache.rocketmq.connect.redis.pojo.RedisEntry;
+
+/**
+ * XAck key group ID [ID ...]
+ */
+public class XAckParser extends AbstractCommandParser {
+    @Override
+    public KVEntry createBuilder() {
+        return RedisEntry.newEntry(FieldType.ARRAY);
+    }
+
+    @Override
+    public KVEntry handleValue(KVEntry builder, String[] args) {
+        if (args.length > 1) {
+            builder.param(Options.REDIS_GROUP, args[0]);
+            List<String> ids = new ArrayList<>(args.length - 1);
+            for (int i = 1; i < args.length; i++) {
+                ids.add(args[i]);
+            }
+            builder.value(ids);
+        }
+        return builder;
+    }
+}
diff --git a/src/main/java/org/apache/rocketmq/connect/redis/parser/XAddParser.java b/src/main/java/org/apache/rocketmq/connect/redis/parser/XAddParser.java
new file mode 100644
index 0000000..755c267
--- /dev/null
+++ b/src/main/java/org/apache/rocketmq/connect/redis/parser/XAddParser.java
@@ -0,0 +1,46 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.rocketmq.connect.redis.parser;
+
+import java.util.HashMap;
+import java.util.Map;
+
+import io.openmessaging.connector.api.data.FieldType;
+import org.apache.rocketmq.connect.redis.pojo.KVEntry;
+import org.apache.rocketmq.connect.redis.common.Options;
+import org.apache.rocketmq.connect.redis.pojo.RedisEntry;
+
+/**
+ * XAdd key ID field string [field string ...]
+ */
+public class XAddParser extends AbstractCommandParser {
+    @Override
+    public KVEntry createBuilder() {
+        return RedisEntry.newEntry(FieldType.MAP);
+    }
+
+    @Override
+    public KVEntry handleValue(KVEntry builder, String[] args) {
+        Map<String, String> kvMap = new HashMap<>(args.length / 2);
+        for (int i = 1, j = i + 1; j < args.length; i++, i++, j++, j++) {
+            kvMap.put(args[i], args[j]);
+        }
+        builder.param(Options.REDIS_ID, args[0]).value(kvMap);
+        return builder;
+    }
+}
diff --git a/src/main/java/org/apache/rocketmq/connect/redis/parser/XClaimParser.java b/src/main/java/org/apache/rocketmq/connect/redis/parser/XClaimParser.java
new file mode 100644
index 0000000..d3a2aa0
--- /dev/null
+++ b/src/main/java/org/apache/rocketmq/connect/redis/parser/XClaimParser.java
@@ -0,0 +1,88 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.rocketmq.connect.redis.parser;
+
+import java.util.Arrays;
+
+import io.openmessaging.connector.api.data.FieldType;
+import org.apache.rocketmq.connect.redis.common.RedisConstants;
+import org.apache.rocketmq.connect.redis.pojo.KVEntry;
+import org.apache.rocketmq.connect.redis.common.Options;
+import org.apache.rocketmq.connect.redis.pojo.RedisEntry;
+
+/**
+ * XCLAIM key group consumer min-idle-time ID [ID ...] [IDLE ms] [TIME ms-unix-time] [RETRYCOUNT count] [force]
+ */
+public class XClaimParser extends AbstractCommandParser {
+    @Override
+    public KVEntry createBuilder() {
+        return RedisEntry.newEntry(FieldType.ARRAY);
+    }
+
+    @Override
+    public KVEntry handleValue(KVEntry builder, String[] args) {
+        builder.param(Options.REDIS_GROUP, args[0]);
+        builder.param(Options.REDIS_CONSUMER, args[1]);
+        builder.param(Options.REDIS_MIN_IDLE_TIME, Long.parseLong(args[2]));
+        int idx = args.length;
+        for (int i = 3; i < args.length; i++) {
+            if (RedisConstants.IDLE.equals(args[i].toUpperCase()) ||
+                RedisConstants.TIME.equals(args[i].toUpperCase()) ||
+                RedisConstants.RETRYCOUNT.equals(args[i].toUpperCase()) ||
+                RedisConstants.FORCE.equals(args[i].toUpperCase()) ||
+                RedisConstants.JUSTID.equals(args[i].toUpperCase())
+            ) {
+                idx = i;
+                break;
+            }
+        }
+        String[] ids = new String[idx - 3];
+        System.arraycopy(args, 3, ids, 0, idx - 3);
+        builder.value(Arrays.asList(ids));
+
+        while (idx < args.length) {
+            switch (args[idx].toUpperCase()) {
+                case RedisConstants.IDLE:
+                    builder.param(Options.REDIS_IDLE, Long.parseLong(args[idx + 1]));
+                    idx += 2;
+                    break;
+                case RedisConstants.TIME:
+                    builder.param(Options.REDIS_TIME, Long.parseLong(args[idx + 1]));
+                    idx += 2;
+                    break;
+                case RedisConstants.RETRYCOUNT:
+                    builder.param(Options.REDIS_RETRYCOUNT, Integer.parseInt(args[idx + 1]));
+                    idx += 2;
+                    break;
+                case RedisConstants.FORCE:
+                    builder.param(Options.REDIS_FORCE, Boolean.TRUE);
+                    idx += 1;
+                    break;
+                case RedisConstants.JUSTID:
+                    builder.param(Options.REDIS_JUSTID, Boolean.TRUE);
+                    idx += 1;
+                    break;
+                default:
+                    idx += 1;
+                    break;
+            }
+        }
+
+        return builder;
+    }
+}
diff --git a/src/main/java/org/apache/rocketmq/connect/redis/parser/XDelParser.java b/src/main/java/org/apache/rocketmq/connect/redis/parser/XDelParser.java
new file mode 100644
index 0000000..f1dcefa
--- /dev/null
+++ b/src/main/java/org/apache/rocketmq/connect/redis/parser/XDelParser.java
@@ -0,0 +1,41 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.rocketmq.connect.redis.parser;
+
+import java.util.ArrayList;
+import java.util.Arrays;
+import java.util.stream.Collectors;
+
+import io.openmessaging.connector.api.data.FieldType;
+import org.apache.rocketmq.connect.redis.pojo.KVEntry;
+import org.apache.rocketmq.connect.redis.pojo.RedisEntry;
+
+/**
+ * XDel key ID [ID ...]
+ */
+public class XDelParser extends AbstractCommandParser {
+    @Override
+    public KVEntry createBuilder() {
+        return RedisEntry.newEntry(FieldType.ARRAY);
+    }
+
+    @Override
+    public KVEntry handleValue(KVEntry builder, String[] args) {
+        return builder.value(Arrays.stream(args).collect(Collectors.toCollection(ArrayList::new)));
+    }
+}
diff --git a/src/main/java/org/apache/rocketmq/connect/redis/parser/XGroupParser.java b/src/main/java/org/apache/rocketmq/connect/redis/parser/XGroupParser.java
new file mode 100644
index 0000000..5495270
--- /dev/null
+++ b/src/main/java/org/apache/rocketmq/connect/redis/parser/XGroupParser.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
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.rocketmq.connect.redis.parser;
+
+import io.openmessaging.connector.api.data.FieldType;
+import org.apache.rocketmq.connect.redis.pojo.KVEntry;
+import org.apache.rocketmq.connect.redis.pojo.RedisEntry;
+
+/**
+ * XGroup [CREATE key groupname id-or-$] [SETID key id-or-$] [DESTROY key groupname] [DELCONSUMER key groupname consumername]
+ */
+public class XGroupParser extends AbstractCommandParser {
+    @Override
+    public KVEntry createBuilder() {
+        return RedisEntry.newEntry(FieldType.STRING);
+    }
+}
diff --git a/src/main/java/org/apache/rocketmq/connect/redis/parser/XSetIdParser.java b/src/main/java/org/apache/rocketmq/connect/redis/parser/XSetIdParser.java
new file mode 100644
index 0000000..2af95c3
--- /dev/null
+++ b/src/main/java/org/apache/rocketmq/connect/redis/parser/XSetIdParser.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.connect.redis.parser;
+
+import io.openmessaging.connector.api.data.FieldType;
+import org.apache.rocketmq.connect.redis.pojo.KVEntry;
+import org.apache.rocketmq.connect.redis.pojo.RedisEntry;
+
+/**
+ * XSetId key arg
+ */
+public class XSetIdParser extends AbstractCommandParser {
+    @Override
+    public KVEntry createBuilder() {
+        return RedisEntry.newEntry(FieldType.STRING);
+    }
+
+    @Override
+    public KVEntry handleValue(KVEntry builder, String[] args) {
+        return builder.value(args[0]);
+    }
+}
diff --git a/src/main/java/org/apache/rocketmq/connect/redis/parser/XTrimParser.java b/src/main/java/org/apache/rocketmq/connect/redis/parser/XTrimParser.java
new file mode 100644
index 0000000..86a85b6
--- /dev/null
+++ b/src/main/java/org/apache/rocketmq/connect/redis/parser/XTrimParser.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.connect.redis.parser;
+
+import io.openmessaging.connector.api.data.FieldType;
+import org.apache.rocketmq.connect.redis.pojo.KVEntry;
+import org.apache.rocketmq.connect.redis.common.Options;
+import org.apache.rocketmq.connect.redis.pojo.RedisEntry;
+
+/**
+ * XTrim key MAXLEN [~] count
+ */
+public class XTrimParser extends AbstractCommandParser {
+    @Override
+    public KVEntry createBuilder() {
+        return RedisEntry.newEntry(FieldType.STRING);
+    }
+
+    @Override
+    public KVEntry handleValue(KVEntry builder, String[] args) {
+        if (args.length == 3) {
+            builder.param(Options.REDIS_XTRIM, Boolean.TRUE).value(args[2]);
+        } else if (args.length == 2) {
+            builder.param(Options.REDIS_XTRIM, Boolean.FALSE).value(args[1]);
+        }
+        return builder;
+    }
+}
diff --git a/src/main/java/org/apache/rocketmq/connect/redis/parser/ZAddParser.java b/src/main/java/org/apache/rocketmq/connect/redis/parser/ZAddParser.java
new file mode 100644
index 0000000..555aaf5
--- /dev/null
+++ b/src/main/java/org/apache/rocketmq/connect/redis/parser/ZAddParser.java
@@ -0,0 +1,69 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.rocketmq.connect.redis.parser;
+
+import java.util.HashMap;
+import java.util.Map;
+
+import io.openmessaging.connector.api.data.FieldType;
+import org.apache.rocketmq.connect.redis.common.RedisConstants;
+import org.apache.rocketmq.connect.redis.pojo.KVEntry;
+import org.apache.rocketmq.connect.redis.common.Options;
+import org.apache.rocketmq.connect.redis.pojo.RedisEntry;
+
+/**
+ * ZADD key [NX|XX] [CH] [INCR] score member [score member ...]
+ */
+public class ZAddParser extends AbstractCommandParser {
+    @Override
+    public KVEntry createBuilder() {
+        return RedisEntry.newEntry(FieldType.MAP);
+    }
+
+    @Override
+    public KVEntry handleValue(KVEntry builder, String[] args) {
+        int idx = 0;
+        loop:
+        while (true) {
+            String param = args[idx];
+            switch (param) {
+                case RedisConstants.NX:
+                    builder.param(Options.REDIS_NX, Boolean.TRUE);
+                    break;
+                case RedisConstants.XX:
+                    builder.param(Options.REDIS_XX, Boolean.TRUE);
+                    break;
+                case RedisConstants.CH:
+                    builder.param(Options.REDIS_CH, Boolean.TRUE);
+                    break;
+                case RedisConstants.INCR:
+                    builder.param(Options.REDIS_INCR, Boolean.TRUE);
+                    break;
+                default:
+                    break loop;
+            }
+            idx++;
+        }
+        Map<String, String> kvMap = new HashMap<>((args.length - idx) / 2);
+        for (int i = idx, j = i + 1; i < args.length; i++, i++, j++, j++) {
+            kvMap.put(args[j], args[i]);
+        }
+        builder.value(kvMap);
+        return builder;
+    }
+}
diff --git a/src/main/java/org/apache/rocketmq/connect/redis/parser/ZIncrByParser.java b/src/main/java/org/apache/rocketmq/connect/redis/parser/ZIncrByParser.java
new file mode 100644
index 0000000..6bbf0d5
--- /dev/null
+++ b/src/main/java/org/apache/rocketmq/connect/redis/parser/ZIncrByParser.java
@@ -0,0 +1,41 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.rocketmq.connect.redis.parser;
+
+import io.openmessaging.connector.api.data.FieldType;
+import org.apache.rocketmq.connect.redis.pojo.KVEntry;
+import org.apache.rocketmq.connect.redis.common.Options;
+import org.apache.rocketmq.connect.redis.pojo.RedisEntry;
+
+/**
+ * ZINCRBY key increment member
+ */
+public class ZIncrByParser extends AbstractCommandParser {
+    @Override
+    public KVEntry createBuilder() {
+        return RedisEntry.newEntry(FieldType.STRING);
+    }
+
+    @Override
+    public KVEntry handleValue(KVEntry builder, String[] args) {
+        if (args.length == 2) {
+            builder.value(args[1]).param(Options.REDIS_INCREMENT, Long.parseLong(args[0]));
+        }
+        return builder;
+    }
+}
diff --git a/src/main/java/org/apache/rocketmq/connect/redis/parser/ZInterStoreParser.java b/src/main/java/org/apache/rocketmq/connect/redis/parser/ZInterStoreParser.java
new file mode 100644
index 0000000..9c8382e
--- /dev/null
+++ b/src/main/java/org/apache/rocketmq/connect/redis/parser/ZInterStoreParser.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.connect.redis.parser;
+
+import io.openmessaging.connector.api.data.FieldType;
+import org.apache.rocketmq.connect.redis.pojo.KVEntry;
+import org.apache.rocketmq.connect.redis.pojo.RedisEntry;
+
+/**
+ * ZINTERSTORE destination numkeys key [key ...] [weights weight] [aggregate SUM|MIN|MAX]
+ */
+public class ZInterStoreParser extends AbstractCommandParser {
+    @Override
+    public KVEntry createBuilder() {
+        return RedisEntry.newEntry(FieldType.MAP);
+    }
+
+    @Override
+    public KVEntry handleValue(KVEntry builder, String[] args) {
+        return CommonParser.commonZStoreHandler(builder, args);
+    }
+}
diff --git a/src/main/java/org/apache/rocketmq/connect/redis/parser/ZPopMaxParser.java b/src/main/java/org/apache/rocketmq/connect/redis/parser/ZPopMaxParser.java
new file mode 100644
index 0000000..d4b4ff0
--- /dev/null
+++ b/src/main/java/org/apache/rocketmq/connect/redis/parser/ZPopMaxParser.java
@@ -0,0 +1,38 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.rocketmq.connect.redis.parser;
+
+import io.openmessaging.connector.api.data.FieldType;
+import org.apache.rocketmq.connect.redis.pojo.KVEntry;
+import org.apache.rocketmq.connect.redis.common.Options;
+import org.apache.rocketmq.connect.redis.pojo.RedisEntry;
+
+/**
+ * ZPOPMAX key [count]
+ */
+public class ZPopMaxParser extends AbstractCommandParser {
+    @Override
+    public KVEntry createBuilder() {
+        return RedisEntry.newEntry(FieldType.STRING);
+    }
+
+    @Override
+    public KVEntry handleValue(KVEntry builder, String[] args) {
+        return builder.param(Options.REDIS_COUNT, Long.parseLong(args[0]));
+    }
+}
diff --git a/src/main/java/org/apache/rocketmq/connect/redis/parser/ZPopMinParser.java b/src/main/java/org/apache/rocketmq/connect/redis/parser/ZPopMinParser.java
new file mode 100644
index 0000000..070d2da
--- /dev/null
+++ b/src/main/java/org/apache/rocketmq/connect/redis/parser/ZPopMinParser.java
@@ -0,0 +1,38 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.rocketmq.connect.redis.parser;
+
+import io.openmessaging.connector.api.data.FieldType;
+import org.apache.rocketmq.connect.redis.pojo.KVEntry;
+import org.apache.rocketmq.connect.redis.common.Options;
+import org.apache.rocketmq.connect.redis.pojo.RedisEntry;
+
+/**
+ * ZPOPMIN key [count]
+ */
+public class ZPopMinParser extends AbstractCommandParser {
+    @Override
+    public KVEntry createBuilder() {
+        return RedisEntry.newEntry(FieldType.STRING);
+    }
+
+    @Override
+    public KVEntry handleValue(KVEntry builder, String[] args) {
+        return builder.param(Options.REDIS_COUNT, Long.parseLong(args[0]));
+    }
+}
diff --git a/src/main/java/org/apache/rocketmq/connect/redis/parser/ZRemParser.java b/src/main/java/org/apache/rocketmq/connect/redis/parser/ZRemParser.java
new file mode 100644
index 0000000..775a646
--- /dev/null
+++ b/src/main/java/org/apache/rocketmq/connect/redis/parser/ZRemParser.java
@@ -0,0 +1,41 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.rocketmq.connect.redis.parser;
+
+import java.util.ArrayList;
+import java.util.Arrays;
+import java.util.stream.Collectors;
+
+import io.openmessaging.connector.api.data.FieldType;
+import org.apache.rocketmq.connect.redis.pojo.KVEntry;
+import org.apache.rocketmq.connect.redis.pojo.RedisEntry;
+
+/**
+ * ZRem key member [member ...]
+ */
+public class ZRemParser extends AbstractCommandParser {
+    @Override
+    public KVEntry createBuilder() {
+        return RedisEntry.newEntry(FieldType.ARRAY);
+    }
+
+    @Override
+    public KVEntry handleValue(KVEntry builder, String[] args) {
+        return builder.value(Arrays.stream(args).collect(Collectors.toCollection(ArrayList::new)));
+    }
+}
diff --git a/src/main/java/org/apache/rocketmq/connect/redis/parser/ZRemRangeByLexParser.java b/src/main/java/org/apache/rocketmq/connect/redis/parser/ZRemRangeByLexParser.java
new file mode 100644
index 0000000..cbdaf91
--- /dev/null
+++ b/src/main/java/org/apache/rocketmq/connect/redis/parser/ZRemRangeByLexParser.java
@@ -0,0 +1,41 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.rocketmq.connect.redis.parser;
+
+import java.util.ArrayList;
+import java.util.Arrays;
+import java.util.stream.Collectors;
+
+import io.openmessaging.connector.api.data.FieldType;
+import org.apache.rocketmq.connect.redis.pojo.KVEntry;
+import org.apache.rocketmq.connect.redis.pojo.RedisEntry;
+
+/**
+ * ZRemRangeByLex key min max
+ */
+public class ZRemRangeByLexParser extends AbstractCommandParser {
+    @Override
+    public KVEntry createBuilder() {
+        return RedisEntry.newEntry(FieldType.ARRAY);
+    }
+
+    @Override
+    public KVEntry handleValue(KVEntry builder, String[] args) {
+        return builder.value(Arrays.stream(args).collect(Collectors.toCollection(ArrayList::new)));
+    }
+}
diff --git a/src/main/java/org/apache/rocketmq/connect/redis/parser/ZRemRangeByRankParser.java b/src/main/java/org/apache/rocketmq/connect/redis/parser/ZRemRangeByRankParser.java
new file mode 100644
index 0000000..3827a41
--- /dev/null
+++ b/src/main/java/org/apache/rocketmq/connect/redis/parser/ZRemRangeByRankParser.java
@@ -0,0 +1,41 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.rocketmq.connect.redis.parser;
+
+import java.util.ArrayList;
+import java.util.Arrays;
+import java.util.stream.Collectors;
+
+import io.openmessaging.connector.api.data.FieldType;
+import org.apache.rocketmq.connect.redis.pojo.KVEntry;
+import org.apache.rocketmq.connect.redis.pojo.RedisEntry;
+
+/**
+ * ZREMRANGEBYRANK key start stop
+ */
+public class ZRemRangeByRankParser extends AbstractCommandParser {
+    @Override
+    public KVEntry createBuilder() {
+        return RedisEntry.newEntry(FieldType.ARRAY);
+    }
+
+    @Override
+    public KVEntry handleValue(KVEntry builder, String[] args) {
+        return builder.value(Arrays.stream(args).collect(Collectors.toCollection(ArrayList::new)));
+    }
+}
diff --git a/src/main/java/org/apache/rocketmq/connect/redis/parser/ZRemRangeByScoreParser.java b/src/main/java/org/apache/rocketmq/connect/redis/parser/ZRemRangeByScoreParser.java
new file mode 100644
index 0000000..8693382
--- /dev/null
+++ b/src/main/java/org/apache/rocketmq/connect/redis/parser/ZRemRangeByScoreParser.java
@@ -0,0 +1,41 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.rocketmq.connect.redis.parser;
+
+import java.util.ArrayList;
+import java.util.Arrays;
+import java.util.stream.Collectors;
+
+import io.openmessaging.connector.api.data.FieldType;
+import org.apache.rocketmq.connect.redis.pojo.KVEntry;
+import org.apache.rocketmq.connect.redis.pojo.RedisEntry;
+
+/**
+ * ZREMRANGEBYSCORE key min max
+ */
+public class ZRemRangeByScoreParser extends AbstractCommandParser {
+    @Override
+    public KVEntry createBuilder() {
+        return RedisEntry.newEntry(FieldType.ARRAY);
+    }
+
+    @Override
+    public KVEntry handleValue(KVEntry builder, String[] args) {
+        return builder.value(Arrays.stream(args).collect(Collectors.toCollection(ArrayList::new)));
+    }
+}
diff --git a/src/main/java/org/apache/rocketmq/connect/redis/parser/ZUnionStoreParser.java b/src/main/java/org/apache/rocketmq/connect/redis/parser/ZUnionStoreParser.java
new file mode 100644
index 0000000..04c0ef4
--- /dev/null
+++ b/src/main/java/org/apache/rocketmq/connect/redis/parser/ZUnionStoreParser.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.connect.redis.parser;
+
+import io.openmessaging.connector.api.data.FieldType;
+import org.apache.rocketmq.connect.redis.pojo.KVEntry;
+import org.apache.rocketmq.connect.redis.pojo.RedisEntry;
+
+/**
+ * ZUNIONSTORE destination numkeys key [key ...] [weights weight] [aggregate SUM|MIN|MAX]
+ */
+public class ZUnionStoreParser extends AbstractCommandParser {
+    @Override
+    public KVEntry createBuilder() {
+        return RedisEntry.newEntry(FieldType.MAP);
+    }
+
+    @Override
+    public KVEntry handleValue(KVEntry builder, String[] args) {
+        return CommonParser.commonZStoreHandler(builder, args);
+    }
+}
diff --git a/src/main/java/org/apache/rocketmq/connect/redis/pojo/Geo.java b/src/main/java/org/apache/rocketmq/connect/redis/pojo/Geo.java
new file mode 100644
index 0000000..cf539db
--- /dev/null
+++ b/src/main/java/org/apache/rocketmq/connect/redis/pojo/Geo.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.connect.redis.pojo;
+
+public class Geo {
+    private long longitude;
+    private long latitude;
+    private String member;
+
+    public long getLongitude() {
+        return longitude;
+    }
+
+    public void setLongitude(long longitude) {
+        this.longitude = longitude;
+    }
+
+    public long getLatitude() {
+        return latitude;
+    }
+
+    public void setLatitude(long latitude) {
+        this.latitude = latitude;
+    }
+
+    public String getMember() {
+        return member;
+    }
+
+    public void setMember(String member) {
+        this.member = member;
+    }
+}
diff --git a/src/main/java/org/apache/rocketmq/connect/redis/pojo/KVEntry.java b/src/main/java/org/apache/rocketmq/connect/redis/pojo/KVEntry.java
new file mode 100644
index 0000000..ff8cb29
--- /dev/null
+++ b/src/main/java/org/apache/rocketmq/connect/redis/pojo/KVEntry.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.connect.redis.pojo;
+
+import io.openmessaging.connector.api.data.FieldType;
+import java.util.List;
+
+import com.moilioncircle.redis.replicator.cmd.Command;
+import io.openmessaging.connector.api.data.EntryType;
+import java.util.Map;
+import org.apache.rocketmq.connect.redis.common.Options;
+
+public interface KVEntry extends Command {
+
+    KVEntry partition(String partition);
+
+    String getPartition();
+
+    KVEntry queueName(String queueName);
+
+    String getQueueName();
+
+    KVEntry entryType(EntryType entryType);
+
+    EntryType getEntryType();
+
+    KVEntry sourceId(String id);
+
+    String getSourceId();
+
+    KVEntry offset(Long offset);
+
+    Long getOffset();
+
+    KVEntry command(String command);
+
+    String getCommand();
+
+    KVEntry key(String key);
+
+    String getKey();
+
+    KVEntry value(Object value);
+
+    Object getValue();
+
+    KVEntry valueType(FieldType valueType);
+
+    FieldType getValueType();
+
+    <T> KVEntry param(Options<T> k, T v);
+
+    <T> T getParam(Options<T> k);
+
+    Map<String, Object> getParams();
+}
diff --git a/src/main/java/org/apache/rocketmq/connect/redis/pojo/RedisEntry.java b/src/main/java/org/apache/rocketmq/connect/redis/pojo/RedisEntry.java
new file mode 100644
index 0000000..5cf2d86
--- /dev/null
+++ b/src/main/java/org/apache/rocketmq/connect/redis/pojo/RedisEntry.java
@@ -0,0 +1,171 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.rocketmq.connect.redis.pojo;
+
+import java.util.HashMap;
+import java.util.Map;
+import io.openmessaging.connector.api.data.EntryType;
+import io.openmessaging.connector.api.data.FieldType;
+import org.apache.rocketmq.connect.redis.common.Options;
+
+public class RedisEntry implements KVEntry {
+    private Long replOffset;
+    private String partition;
+    private Map<String, Object> params;
+    private String queueName;
+    private EntryType entryType;
+    private String command;
+    private String key;
+    private FieldType valueType;
+    private Object value;
+
+    public static RedisEntry newEntry(FieldType valueType) {
+        return new RedisEntry(valueType);
+    }
+
+    public static RedisEntry newEntry(String partition, FieldType valueType) {
+        return new RedisEntry(partition, valueType);
+    }
+
+    public RedisEntry(FieldType valueType) {
+        this(Options.REDIS_PARTITION.name(), valueType);
+    }
+
+    public RedisEntry(String partition, FieldType valueType) {
+        this.partition = partition;
+        this.valueType = valueType;
+        this.params = new HashMap<>();
+    }
+
+    @Override public KVEntry partition(String partition) {
+        this.partition = partition;
+        return this;
+    }
+
+    @Override public String getPartition() {
+        return this.partition;
+    }
+
+    @Override public KVEntry queueName(String queueName) {
+        this.queueName = queueName;
+        return this;
+    }
+
+    @Override public String getQueueName() {
+        return this.queueName;
+    }
+
+    @Override public KVEntry entryType(EntryType entryType) {
+        this.entryType = entryType;
+        return this;
+    }
+
+    @Override public EntryType getEntryType() {
+        return this.entryType;
+    }
+
+    @Override public RedisEntry sourceId(String id) {
+        this.params.put(Options.REDIS_REPLID.name(), id);
+        return this;
+    }
+
+    @Override public String getSourceId() {
+        Object ob = this.params.get(Options.REDIS_REPLID.name());
+        if (ob != null) {
+            return ob.toString();
+        }
+        return null;
+    }
+
+    @Override public RedisEntry offset(Long offset) {
+        this.replOffset = offset;
+        return this;
+    }
+
+    @Override public Long getOffset() {
+        return this.replOffset;
+    }
+
+    @Override public RedisEntry command(String command) {
+        this.command = command;
+        return this;
+    }
+
+    @Override public String getCommand() {
+        return this.command;
+    }
+
+    @Override public RedisEntry key(String key) {
+        this.key = key;
+        return this;
+    }
+
+    @Override public String getKey() {
+        return this.key;
+    }
+
+    @Override public RedisEntry value(Object value) {
+        this.value = value;
+        return this;
+    }
+
+    @Override public Object getValue() {
+        return this.value;
+    }
+
+    @Override public KVEntry valueType(FieldType valueType) {
+        this.valueType = valueType;
+        return this;
+    }
+
+    @Override public FieldType getValueType() {
+        return this.valueType;
+    }
+
+    @Override public <T> RedisEntry param(Options<T> k, T v) {
+        this.params.put(k.name(), v);
+        return this;
+    }
+
+    @Override public <T> T getParam(Options<T> k) {
+        if(k == null){
+            return null;
+        }
+        return (T) this.params.get(k.name());
+    }
+
+    @Override public Map<String, Object> getParams() {
+        return this.params;
+    }
+
+
+    @Override public String toString() {
+        StringBuilder sb = new StringBuilder();
+        sb.append("RedisEntry{")
+            .append("partition:'").append(this.partition).append("', ")
+            .append("queueName:'").append(this.queueName).append("', ")
+            .append("entryType:'").append(this.entryType).append("', ")
+            .append("command:'").append(this.command).append("', ")
+            .append("key:'").append(this.key).append("', ")
+            .append("valueType:").append(this.valueType).append("', ")
+            .append("value:'").append(this.value).append("', ")
+            .append("params:").append(this.params)
+            .append("}");
+        return sb.toString();
+    }
+}
diff --git a/src/main/java/org/apache/rocketmq/connect/redis/pojo/RedisEvent.java b/src/main/java/org/apache/rocketmq/connect/redis/pojo/RedisEvent.java
new file mode 100644
index 0000000..585df47
--- /dev/null
+++ b/src/main/java/org/apache/rocketmq/connect/redis/pojo/RedisEvent.java
@@ -0,0 +1,59 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.rocketmq.connect.redis.pojo;
+
+import com.moilioncircle.redis.replicator.event.Event;
+
+public class RedisEvent {
+    private Event event;
+    private String replId;
+    private Long replOffset;
+    private int streamDB;
+
+    public Event getEvent() {
+        return event;
+    }
+
+    public void setEvent(Event event) {
+        this.event = event;
+    }
+
+    public Long getReplOffset() {
+        return replOffset;
+    }
+
+    public void setReplOffset(Long replOffset) {
+        this.replOffset = replOffset;
+    }
+
+    public String getReplId() {
+        return replId;
+    }
+
+    public void setReplId(String replId) {
+        this.replId = replId;
+    }
+
+    public int getStreamDB() {
+        return streamDB;
+    }
+
+    public void setStreamDB(int streamDB) {
+        this.streamDB = streamDB;
+    }
+}
diff --git a/src/main/java/org/apache/rocketmq/connect/redis/processor/DefaultRedisEventProcessor.java b/src/main/java/org/apache/rocketmq/connect/redis/processor/DefaultRedisEventProcessor.java
new file mode 100644
index 0000000..cde4367
--- /dev/null
+++ b/src/main/java/org/apache/rocketmq/connect/redis/processor/DefaultRedisEventProcessor.java
@@ -0,0 +1,453 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.rocketmq.connect.redis.processor;
+
+import com.moilioncircle.redis.replicator.RedisURI;
+import com.moilioncircle.redis.replicator.cmd.Command;
+import com.moilioncircle.redis.replicator.cmd.CommandName;
+import com.moilioncircle.redis.replicator.cmd.CommandParser;
+import com.moilioncircle.redis.replicator.cmd.parser.PingParser;
+import com.moilioncircle.redis.replicator.cmd.parser.ReplConfParser;
+import com.moilioncircle.redis.replicator.rdb.datatype.KeyValuePair;
+import com.moilioncircle.redis.replicator.rdb.iterable.datatype.BatchedKeyValuePair;
+import java.io.IOException;
+import java.util.HashMap;
+import java.util.List;
+import java.util.Map;
+import java.util.concurrent.BlockingQueue;
+import java.util.concurrent.CopyOnWriteArrayList;
+import java.util.concurrent.LinkedBlockingQueue;
+import java.util.concurrent.TimeUnit;
+import java.util.concurrent.atomic.AtomicBoolean;
+
+import com.moilioncircle.redis.replicator.CloseListener;
+import com.moilioncircle.redis.replicator.ExceptionListener;
+import com.moilioncircle.redis.replicator.Replicator;
+import com.moilioncircle.redis.replicator.event.EventListener;
+import org.apache.commons.lang.StringUtils;
+import org.apache.rocketmq.connect.redis.common.Config;
+import org.apache.rocketmq.connect.redis.common.RedisConstants;
+import org.apache.rocketmq.connect.redis.common.SyncMod;
+import org.apache.rocketmq.connect.redis.handler.RedisEventHandler;
+import org.apache.rocketmq.connect.redis.parser.AppendParser;
+import org.apache.rocketmq.connect.redis.parser.BitFieldParser;
+import org.apache.rocketmq.connect.redis.parser.BitOpParser;
+import org.apache.rocketmq.connect.redis.parser.BrPopLPushParser;
+import org.apache.rocketmq.connect.redis.parser.DecrByParser;
+import org.apache.rocketmq.connect.redis.parser.DecrParser;
+import org.apache.rocketmq.connect.redis.parser.DelParser;
+import org.apache.rocketmq.connect.redis.parser.EvalParser;
+import org.apache.rocketmq.connect.redis.parser.EvalShaParser;
+import org.apache.rocketmq.connect.redis.parser.ExecParser;
+import org.apache.rocketmq.connect.redis.parser.ExpireAtParser;
+import org.apache.rocketmq.connect.redis.parser.ExpireParser;
+import org.apache.rocketmq.connect.redis.parser.FlushAllParser;
+import org.apache.rocketmq.connect.redis.parser.FlushDbParser;
+import org.apache.rocketmq.connect.redis.parser.GeoAddParser;
+import org.apache.rocketmq.connect.redis.parser.GetsetParser;
+import org.apache.rocketmq.connect.redis.parser.HDelParser;
+import org.apache.rocketmq.connect.redis.parser.HIncrByParser;
+import org.apache.rocketmq.connect.redis.parser.HSetNxParser;
+import org.apache.rocketmq.connect.redis.parser.HSetParser;
+import org.apache.rocketmq.connect.redis.parser.HmSetParser;
+import org.apache.rocketmq.connect.redis.parser.IncrByParser;
+import org.apache.rocketmq.connect.redis.parser.IncrParser;
+import org.apache.rocketmq.connect.redis.parser.LPopParser;
+import org.apache.rocketmq.connect.redis.parser.LPushParser;
+import org.apache.rocketmq.connect.redis.parser.LPushXParser;
+import org.apache.rocketmq.connect.redis.parser.LRemParser;
+import org.apache.rocketmq.connect.redis.parser.LSetParser;
+import org.apache.rocketmq.connect.redis.parser.LTrimParser;
+import org.apache.rocketmq.connect.redis.parser.LinsertParser;
+import org.apache.rocketmq.connect.redis.parser.MSetNxParser;
+import org.apache.rocketmq.connect.redis.parser.MSetParser;
+import org.apache.rocketmq.connect.redis.parser.MoveParser;
+import org.apache.rocketmq.connect.redis.parser.MultiParser;
+import org.apache.rocketmq.connect.redis.parser.PExpireAtParser;
+import org.apache.rocketmq.connect.redis.parser.PExpireParser;
+import org.apache.rocketmq.connect.redis.parser.PSetExParser;
+import org.apache.rocketmq.connect.redis.parser.PersistParser;
+import org.apache.rocketmq.connect.redis.parser.PfAddParser;
+import org.apache.rocketmq.connect.redis.parser.PfCountParser;
+import org.apache.rocketmq.connect.redis.parser.PfMergeParser;
+import org.apache.rocketmq.connect.redis.parser.PublishParser;
+import org.apache.rocketmq.connect.redis.parser.RPopLPushParser;
+import org.apache.rocketmq.connect.redis.parser.RPopParser;
+import org.apache.rocketmq.connect.redis.parser.RPushParser;
+import org.apache.rocketmq.connect.redis.parser.RPushXParser;
+import org.apache.rocketmq.connect.redis.parser.RenameNxParser;
+import org.apache.rocketmq.connect.redis.parser.RenameParser;
+import org.apache.rocketmq.connect.redis.parser.RestoreParser;
+import org.apache.rocketmq.connect.redis.parser.SAddParser;
+import org.apache.rocketmq.connect.redis.parser.SDiffStoreParser;
+import org.apache.rocketmq.connect.redis.parser.SInterStoreParser;
+import org.apache.rocketmq.connect.redis.parser.SMoveParser;
+import org.apache.rocketmq.connect.redis.parser.SRemParser;
+import org.apache.rocketmq.connect.redis.parser.SUnionStoreParser;
+import org.apache.rocketmq.connect.redis.parser.ScriptParser;
+import org.apache.rocketmq.connect.redis.parser.SelectParser;
+import org.apache.rocketmq.connect.redis.parser.SetBitParser;
+import org.apache.rocketmq.connect.redis.parser.SetExParser;
+import org.apache.rocketmq.connect.redis.parser.SetNxParser;
+import org.apache.rocketmq.connect.redis.parser.SetParser;
+import org.apache.rocketmq.connect.redis.parser.SetRangeParser;
+import org.apache.rocketmq.connect.redis.parser.SortParser;
+import org.apache.rocketmq.connect.redis.parser.SwapDbParser;
+import org.apache.rocketmq.connect.redis.parser.UnLinkParser;
+import org.apache.rocketmq.connect.redis.parser.XAckParser;
+import org.apache.rocketmq.connect.redis.parser.XAddParser;
+import org.apache.rocketmq.connect.redis.parser.XClaimParser;
+import org.apache.rocketmq.connect.redis.parser.XDelParser;
+import org.apache.rocketmq.connect.redis.parser.XGroupParser;
+import org.apache.rocketmq.connect.redis.parser.XSetIdParser;
+import org.apache.rocketmq.connect.redis.parser.XTrimParser;
+import org.apache.rocketmq.connect.redis.parser.ZAddParser;
+import org.apache.rocketmq.connect.redis.parser.ZIncrByParser;
+import org.apache.rocketmq.connect.redis.parser.ZInterStoreParser;
+import org.apache.rocketmq.connect.redis.parser.ZPopMaxParser;
+import org.apache.rocketmq.connect.redis.parser.ZPopMinParser;
+import org.apache.rocketmq.connect.redis.parser.ZRemParser;
+import org.apache.rocketmq.connect.redis.parser.ZRemRangeByLexParser;
+import org.apache.rocketmq.connect.redis.parser.ZRemRangeByRankParser;
+import org.apache.rocketmq.connect.redis.parser.ZRemRangeByScoreParser;
+import org.apache.rocketmq.connect.redis.parser.ZUnionStoreParser;
+import org.apache.rocketmq.connect.redis.pojo.KVEntry;
+import org.apache.rocketmq.connect.redis.pojo.RedisEvent;
+import org.apache.rocketmq.connect.redis.util.ParseStringUtils;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+import redis.clients.jedis.Jedis;
+import redis.clients.jedis.JedisPool;
+import redis.clients.jedis.JedisPoolConfig;
+
+/**
+ * listen redis event
+ */
+public class DefaultRedisEventProcessor implements RedisEventProcessor {
+    protected final Logger LOGGER = LoggerFactory.getLogger(DefaultRedisEventProcessor.class);
+    /**
+     * redis event cache.
+     */
+    protected BlockingQueue<RedisEvent> eventQueue = new LinkedBlockingQueue(50000);
+    protected Config config;
+    private volatile AtomicBoolean stop = new AtomicBoolean(true);
+    /**
+     * handle different kind of redis event.
+     */
+    private RedisEventHandler redisEventHandler;
+    private JedisPool pool;
+    /**
+     * redis info msg cache.
+     */
+    private Map<String, String> redisInfo;
+    /**
+     * redis event parsers cache.
+     */
+    private ParserCache parserCache;
+
+    /**
+     * redis replicator
+     */
+    private Replicator replicator;
+    private final EventListener eventListener;
+    private final ExceptionListener exceptionListener;
+    private final CloseListener closeListener;
+
+    private List<RedisEventProcessorCallback> redisEventProcessorCallbacks = new CopyOnWriteArrayList<>();
+
+    private final Integer pollTimeout = 1000;
+    private final Integer offerTimeout = 1000;
+    /**
+     * base construct
+     *
+     * @param config
+     */
+    public DefaultRedisEventProcessor(Config config) {
+        this.config = config;
+        this.pool = getJedisPool(config);
+
+        this.parserCache = new ParserCache();
+        this.closeListener = new RedisClosedListener(this);
+        this.exceptionListener = new RedisExceptionListener(this);
+        this.eventListener = new RedisEventListener(this.config,this);
+    }
+
+    @Override public void registEventHandler(RedisEventHandler eventHandler) {
+        this.redisEventHandler = eventHandler;
+    }
+
+    @Override public void registProcessorCallback(RedisEventProcessorCallback redisEventProcessorCallback){
+        redisEventProcessorCallbacks.add(redisEventProcessorCallback);
+    }
+
+    /**
+     * start redis replicator and asynchronous processing threads
+     */
+    @Override public void start() throws IllegalStateException, IOException {
+        if (this.stop.compareAndSet(true, false)) {
+            Jedis jedis = this.pool.getResource();
+            String jedisInfo = jedis.info(RedisConstants.REDIS_INFO_REPLICATION);
+            this.redisInfo = ParseStringUtils.parseRedisInfo2Map(jedisInfo);
+            String replId = this.redisInfo.get(RedisConstants.REDIS_INFO_REPLICATION_MASTER_REPLID);
+            if (StringUtils.isNotEmpty(replId)
+                && StringUtils.isEmpty(this.config.getReplId())) {
+                this.config.setReplId(replId);
+            }
+            String offset = this.redisInfo.get(RedisConstants.REDIS_INFO_REPLICATION_MASTER_REPL_OFFSET);
+            // 如果是LAST_OFFSET,则将offset设置为当前Redis最新的offset值。
+            // LAST_OFFSET、CUSTOM_OFFSET,优先使用connector runtime中的存储位点信息。
+            if (SyncMod.LAST_OFFSET.equals(this.config.getSyncMod())) {
+                if (this.config.getPosition() != null) {
+                    this.config.setOffset(this.config.getPosition());
+                } else if (StringUtils.isNotBlank(offset)) {
+                    this.config.setOffset(Long.parseLong(offset));
+                }
+            } else if(SyncMod.CUSTOM_OFFSET.equals(this.config.getSyncMod())){
+                if(this.config.getPosition() != null){
+                    this.config.setOffset(this.config.getPosition());
+                }
+            }
+
+            startReplicatorAsync(this.config.getReplId(), this.config.getOffset());
+            LOGGER.info("processor start from replId: {}, offset: {}", this.config.getReplId(), this.config.getOffset());
+        } else {
+            LOGGER.warn("processor is already started.");
+        }
+    }
+
+    @Override public void stop() throws IOException {
+        if (this.stop.compareAndSet(false, true)) {
+            if (this.replicator != null) {
+                this.replicator.close();
+            }
+            this.pool.close();
+            int size = redisEventProcessorCallbacks.size();
+            for (int i = 0; i < size; i++) {
+                redisEventProcessorCallbacks.get(i).onStop(this);
+            }
+            LOGGER.info("processor is stopped.");
+        } else {
+            LOGGER.info("processor is already stopped.");
+        }
+    }
+
+
+    @Override public boolean commit(RedisEvent event) throws Exception {
+        return this.eventQueue.offer(event, this.offerTimeout, TimeUnit.MILLISECONDS);
+    }
+
+
+    @Override public KVEntry poll() throws Exception {
+        RedisEvent event = this.eventQueue.poll(this.pollTimeout, TimeUnit.MILLISECONDS);
+        if (event == null) {
+            return null;
+        }
+        if (event.getEvent() instanceof KeyValuePair) {
+            if (event.getEvent() instanceof BatchedKeyValuePair) {
+                return redisEventHandler.handleBatchKVString(event.getReplId(), event.getReplOffset(),
+                    (BatchedKeyValuePair) event.getEvent());
+            } else {
+                return redisEventHandler.handleKVString(event.getReplId(), event.getReplOffset(),
+                    (KeyValuePair) event.getEvent());
+            }
+        } else if (event.getEvent() instanceof Command) {
+            return redisEventHandler.handleCommand(event.getReplId(), event.getReplOffset(), (Command) event.getEvent());
+        } else {
+            return redisEventHandler.handleOtherEvent(event.getReplId(), event.getReplOffset(), event.getEvent());
+        }
+    }
+
+
+    @Override public boolean isStopped() {
+        return this.stop.get();
+    }
+
+    /**
+     * start redis replicator async
+     *
+     * @throws IOException
+     */
+    private void startReplicatorAsync(String replId, Long offset) throws IllegalStateException, IOException {
+        RedisURI uri = this.config.getRedisUri();
+        if (uri == null) {
+            throw new IllegalStateException("redis uri error.");
+        }
+        this.replicator = new com.moilioncircle.redis.replicator.RedisReplicator(uri);
+        this.dress(this.replicator);
+        // set listeners
+        this.replicator.addEventListener(eventListener);
+        this.replicator.addExceptionListener(exceptionListener);
+        this.replicator.addCloseListener(closeListener);
+
+        if (this.config.getReplId() != null) {
+            this.replicator.getConfiguration().setReplId(replId);
+        }
+        if (this.config.getOffset() != null && this.config.getOffset() >= -1) {
+            this.replicator.getConfiguration().setReplOffset(offset);
+        }
+        new Thread(() -> {
+            try {
+                this.replicator.open();
+            } catch (IOException e) {
+                LOGGER.error("start replicator error. {}", e);
+                try {
+                    this.stop();
+                } catch (IOException ie) {
+                }
+            }
+        }).start();
+    }
+
+    private JedisPool getJedisPool(Config config) {
+        JedisPoolConfig jedisPoolConfig = new JedisPoolConfig();
+        jedisPoolConfig.setMaxTotal(100);
+        jedisPoolConfig.setMaxIdle(50);
+        jedisPoolConfig.setMaxWaitMillis(3000);
+        jedisPoolConfig.setTestOnBorrow(true);
+        jedisPoolConfig.setTestOnReturn(true);
+        String pwd = null;
+        if (StringUtils.isNotBlank(config.getRedisPassword())) {
+            pwd = config.getRedisPassword();
+        }
+
+        return new JedisPool(jedisPoolConfig,
+            config.getRedisAddr(),
+            config.getRedisPort(),
+            config.getTimeout(),
+            pwd);
+    }
+
+    /**
+     * 添加用户未指定的Redis指令对应的parser
+     *
+     * @param r
+     * @return
+     */
+    private Replicator dress(Replicator r) {
+        String commands = this.config.getCommands();
+        if (!StringUtils.isEmpty(commands) && !RedisConstants.ALL_COMMAND.equals(commands)) {
+            List<String> commandList = ParseStringUtils.parseCommands(commands);
+            if (commandList != null) {
+                int size = commandList.size();
+                for (int i = 0; i < size; i++) {
+                    CommandName commandName = CommandName.name(commandList.get(i));
+                    if (parserCache.containsKey(commandName)) {
+                        r.addCommandParser(commandName, parserCache.get(commandName));
+                    }
+                }
+            }
+        }
+        for (Map.Entry<CommandName, CommandParser> entry : parserCache.entrySet()) {
+            r.addCommandParser(entry.getKey(), entry.getValue());
+        }
+        return r;
+    }
+
+    private class ParserCache extends HashMap<CommandName, CommandParser> {
+        public ParserCache() {
+            init();
+        }
+
+        public void init() {
+            put(CommandName.name("PING"), new PingParser());
+            put(CommandName.name("REPLCONF"), new ReplConfParser());
+            put(CommandName.name("APPEND"), new AppendParser());
+            put(CommandName.name("SET"), new SetParser());
+            put(CommandName.name("SETEX"), new SetExParser());
+            put(CommandName.name("MSET"), new MSetParser());
+            put(CommandName.name("DEL"), new DelParser());
+            put(CommandName.name("SADD"), new SAddParser());
+            put(CommandName.name("HMSET"), new HmSetParser());
+            put(CommandName.name("HSET"), new HSetParser());
+            put(CommandName.name("LSET"), new LSetParser());
+            put(CommandName.name("EXPIRE"), new ExpireParser());
+            put(CommandName.name("EXPIREAT"), new ExpireAtParser());
+            put(CommandName.name("GETSET"), new GetsetParser());
+            put(CommandName.name("HSETNX"), new HSetNxParser());
+            put(CommandName.name("MSETNX"), new MSetNxParser());
+            put(CommandName.name("PSETEX"), new PSetExParser());
+            put(CommandName.name("SETNX"), new SetNxParser());
+            put(CommandName.name("SETRANGE"), new SetRangeParser());
+            put(CommandName.name("HDEL"), new HDelParser());
+            put(CommandName.name("LPOP"), new LPopParser());
+            put(CommandName.name("LPUSH"), new LPushParser());
+            put(CommandName.name("LPUSHX"), new LPushXParser());
+            put(CommandName.name("LRem"), new LRemParser());
+            put(CommandName.name("RPOP"), new RPopParser());
+            put(CommandName.name("RPUSH"), new RPushParser());
+            put(CommandName.name("RPUSHX"), new RPushXParser());
+            put(CommandName.name("ZREM"), new ZRemParser());
+            put(CommandName.name("RENAME"), new RenameParser());
+            put(CommandName.name("INCR"), new IncrParser());
+            put(CommandName.name("DECR"), new DecrParser());
+            put(CommandName.name("INCRBY"), new IncrByParser());
+            put(CommandName.name("DECRBY"), new DecrByParser());
+            put(CommandName.name("PERSIST"), new PersistParser());
+            put(CommandName.name("SELECT"), new SelectParser());
+            put(CommandName.name("FLUSHALL"), new FlushAllParser());
+            put(CommandName.name("FLUSHDB"), new FlushDbParser());
+            put(CommandName.name("HINCRBY"), new HIncrByParser());
+            put(CommandName.name("ZINCRBY"), new ZIncrByParser());
+            put(CommandName.name("MOVE"), new MoveParser());
+            put(CommandName.name("SMOVE"), new SMoveParser());
+            put(CommandName.name("PFADD"), new PfAddParser());
+            put(CommandName.name("PFCOUNT"), new PfCountParser());
+            put(CommandName.name("PFMERGE"), new PfMergeParser());
+            put(CommandName.name("SDIFFSTORE"), new SDiffStoreParser());
+            put(CommandName.name("SINTERSTORE"), new SInterStoreParser());
+            put(CommandName.name("SUNIONSTORE"), new SUnionStoreParser());
+            put(CommandName.name("ZADD"), new ZAddParser());
+            put(CommandName.name("ZINTERSTORE"), new ZInterStoreParser());
+            put(CommandName.name("ZUNIONSTORE"), new ZUnionStoreParser());
+            put(CommandName.name("BRPOPLPUSH"), new BrPopLPushParser());
+            put(CommandName.name("LINSERT"), new LinsertParser());
+            put(CommandName.name("RENAMENX"), new RenameNxParser());
+            put(CommandName.name("RESTORE"), new RestoreParser());
+            put(CommandName.name("PEXPIRE"), new PExpireParser());
+            put(CommandName.name("PEXPIREAT"), new PExpireAtParser());
+            put(CommandName.name("GEOADD"), new GeoAddParser());
+            put(CommandName.name("EVAL"), new EvalParser());
+            put(CommandName.name("EVALSHA"), new EvalShaParser());
+            put(CommandName.name("SCRIPT"), new ScriptParser());
+            put(CommandName.name("PUBLISH"), new PublishParser());
+            put(CommandName.name("BITOP"), new BitOpParser());
+            put(CommandName.name("BITFIELD"), new BitFieldParser());
+            put(CommandName.name("SETBIT"), new SetBitParser());
+            put(CommandName.name("SREM"), new SRemParser());
+            put(CommandName.name("UNLINK"), new UnLinkParser());
+            put(CommandName.name("SWAPDB"), new SwapDbParser());
+            put(CommandName.name("MULTI"), new MultiParser());
+            put(CommandName.name("EXEC"), new ExecParser());
+            put(CommandName.name("ZREMRANGEBYSCORE"), new ZRemRangeByScoreParser());
+            put(CommandName.name("ZREMRANGEBYRANK"), new ZRemRangeByRankParser());
+            put(CommandName.name("ZREMRANGEBYLEX"), new ZRemRangeByLexParser());
+            put(CommandName.name("LTRIM"), new LTrimParser());
+            put(CommandName.name("SORT"), new SortParser());
+            put(CommandName.name("RPOPLPUSH"), new RPopLPushParser());
+            put(CommandName.name("ZPOPMIN"), new ZPopMinParser());
+            put(CommandName.name("ZPOPMAX"), new ZPopMaxParser());
+            put(CommandName.name("XACK"), new XAckParser());
+            put(CommandName.name("XADD"), new XAddParser());
+            put(CommandName.name("XCLAIM"), new XClaimParser());
+            put(CommandName.name("XDEL"), new XDelParser());
+            put(CommandName.name("XGROUP"), new XGroupParser());
+            put(CommandName.name("XTRIM"), new XTrimParser());
+            put(CommandName.name("XSETID"), new XSetIdParser());
+        }
+    }
+}
diff --git a/src/main/java/org/apache/rocketmq/connect/redis/processor/RedisClosedListener.java b/src/main/java/org/apache/rocketmq/connect/redis/processor/RedisClosedListener.java
new file mode 100644
index 0000000..8a47fd8
--- /dev/null
+++ b/src/main/java/org/apache/rocketmq/connect/redis/processor/RedisClosedListener.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.connect.redis.processor;
+
+import com.moilioncircle.redis.replicator.CloseListener;
+import com.moilioncircle.redis.replicator.Replicator;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+public class RedisClosedListener implements CloseListener {
+    protected final Logger LOGGER = LoggerFactory.getLogger(RedisClosedListener.class);
+
+    private RedisEventProcessor processor;
+
+    public RedisClosedListener(RedisEventProcessor processor){
+        this.processor = processor;
+    }
+
+    @Override public void handle(Replicator replicator) {
+        LOGGER.error("replicator is stopped");
+    }
+}
diff --git a/src/main/java/org/apache/rocketmq/connect/redis/processor/RedisEventListener.java b/src/main/java/org/apache/rocketmq/connect/redis/processor/RedisEventListener.java
new file mode 100644
index 0000000..57d5aa8
--- /dev/null
+++ b/src/main/java/org/apache/rocketmq/connect/redis/processor/RedisEventListener.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.connect.redis.processor;
+
+import com.moilioncircle.redis.replicator.Replicator;
+import com.moilioncircle.redis.replicator.event.Event;
+import com.moilioncircle.redis.replicator.event.EventListener;
+import com.moilioncircle.redis.replicator.event.PostRdbSyncEvent;
+import com.moilioncircle.redis.replicator.event.PreCommandSyncEvent;
+import com.moilioncircle.redis.replicator.event.PreRdbSyncEvent;
+import com.moilioncircle.redis.replicator.rdb.datatype.AuxField;
+import java.io.IOException;
+import org.apache.rocketmq.connect.redis.common.Config;
+import org.apache.rocketmq.connect.redis.pojo.RedisEvent;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+/**
+ * Do simple event filtering to push event events to queues
+ */
+public class RedisEventListener implements EventListener {
+    protected final Logger LOGGER = LoggerFactory.getLogger(RedisEventListener.class);
+
+    private Config config;
+    private RedisEventProcessor processor;
+
+    public RedisEventListener(Config config, RedisEventProcessor processor) {
+        if (config == null) {
+            throw new IllegalArgumentException("config shouldn't be null.");
+        }
+        this.config = config;
+        this.processor = processor;
+    }
+
+    @Override public void onEvent(Replicator replicator, Event event) {
+        if (isUsefulEvent(event)) {
+            LOGGER.info("receive event: {}", event.getClass());
+            RedisEvent redisEvent = new RedisEvent();
+            redisEvent.setEvent(event);
+            if (replicator != null) {
+                redisEvent.setReplId(replicator.getConfiguration().getReplId());
+                redisEvent.setReplOffset(replicator.getConfiguration().getReplOffset());
+                redisEvent.setStreamDB(replicator.getConfiguration().getReplStreamDB());
+            }
+
+            boolean commitSuccess = commitWithRetry(redisEvent,
+                this.config.getEventCommitRetryTimes(),
+                this.config.getEventCommitRetryInterval());
+            if(!commitSuccess){
+                LOGGER.error("redis listener commit event error.");
+                try {
+                    this.processor.stop();
+                } catch (IOException e) {
+                    LOGGER.error("processor stop error. {}", e);
+                }
+            }
+        }
+    }
+
+    private boolean commitWithRetry(RedisEvent redisEvent, int retryTimes, long retryInterval) {
+        if (retryTimes < 0) {
+            return false;
+        }
+        retryTimes--;
+        try {
+            if (processor.commit(redisEvent)) {
+                return true;
+            }
+            if(retryInterval > 0){
+                Thread.sleep(retryInterval);
+            }
+        } catch (Exception e) {
+            if(retryInterval > 0){
+                try {
+                    Thread.sleep(retryInterval);
+                } catch (InterruptedException ie) {
+                }
+            }
+            LOGGER.error("processor commit redisEvent with retry({}) error: {}", retryTimes, e);
+        }
+        return commitWithRetry(redisEvent, retryTimes, retryInterval);
+    }
+
+    /**
+     * Check whether event is an event to be processed
+     */
+    private boolean isUsefulEvent(Event event) {
+        if (event instanceof AuxField) {
+            LOGGER.warn("skip AuxField event: {} - {}", ((AuxField) event).getAuxKey(), ((AuxField) event).getAuxValue());
+            return false;
+        }
+        if (event instanceof PreRdbSyncEvent) {
+            LOGGER.warn("skip PreRdbSync event: {}", event.getClass());
+            return false;
+        }
+        if (event instanceof PreCommandSyncEvent) {
+            LOGGER.warn("skip PreCommandSync event: {}", event.getClass());
+            return false;
+        }
+        if (event instanceof PostRdbSyncEvent) {
+            LOGGER.warn("skip PostRdbSync event: {}", event.getClass());
+            return false;
+        }
+        return true;
+    }
+}
diff --git a/src/main/java/org/apache/rocketmq/connect/redis/processor/RedisEventProcessor.java b/src/main/java/org/apache/rocketmq/connect/redis/processor/RedisEventProcessor.java
new file mode 100644
index 0000000..bdab382
--- /dev/null
+++ b/src/main/java/org/apache/rocketmq/connect/redis/processor/RedisEventProcessor.java
@@ -0,0 +1,40 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.rocketmq.connect.redis.processor;
+
+import java.io.IOException;
+import org.apache.rocketmq.connect.redis.handler.RedisEventHandler;
+import org.apache.rocketmq.connect.redis.pojo.KVEntry;
+import org.apache.rocketmq.connect.redis.pojo.RedisEvent;
+
+public interface RedisEventProcessor {
+
+    void registEventHandler(RedisEventHandler eventHandler);
+
+    void registProcessorCallback(RedisEventProcessorCallback redisEventProcessorCallback);
+
+    void start() throws IllegalStateException, IOException;
+
+    void stop() throws IOException;
+
+    boolean commit(RedisEvent event) throws Exception;
+
+    KVEntry poll() throws Exception;
+
+    boolean isStopped();
+}
diff --git a/src/main/java/org/apache/rocketmq/connect/redis/processor/RedisEventProcessorCallback.java b/src/main/java/org/apache/rocketmq/connect/redis/processor/RedisEventProcessorCallback.java
new file mode 100644
index 0000000..c44f460
--- /dev/null
+++ b/src/main/java/org/apache/rocketmq/connect/redis/processor/RedisEventProcessorCallback.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
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.rocketmq.connect.redis.processor;
+
+public interface RedisEventProcessorCallback {
+
+    void onStop(RedisEventProcessor eventProcessor);
+}
diff --git a/src/main/java/org/apache/rocketmq/connect/redis/processor/RedisExceptionListener.java b/src/main/java/org/apache/rocketmq/connect/redis/processor/RedisExceptionListener.java
new file mode 100644
index 0000000..bdbd139
--- /dev/null
+++ b/src/main/java/org/apache/rocketmq/connect/redis/processor/RedisExceptionListener.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.connect.redis.processor;
+
+import com.moilioncircle.redis.replicator.ExceptionListener;
+import com.moilioncircle.redis.replicator.Replicator;
+import com.moilioncircle.redis.replicator.event.Event;
+import java.io.IOException;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+public class RedisExceptionListener implements ExceptionListener {
+    protected final Logger LOGGER = LoggerFactory.getLogger(RedisExceptionListener.class);
+
+    private RedisEventProcessor processor;
+
+    public RedisExceptionListener(RedisEventProcessor processor){
+        this.processor = processor;
+    }
+
+
+    @Override public void handle(Replicator replicator, Throwable throwable, Event event) {
+        LOGGER.error("listen event error. {}", throwable);
+        try {
+            this.processor.stop();
+        } catch (IOException e) {
+        }
+    }
+}
diff --git a/src/main/java/org/apache/rocketmq/connect/redis/util/ParseStringUtils.java b/src/main/java/org/apache/rocketmq/connect/redis/util/ParseStringUtils.java
new file mode 100644
index 0000000..b4714e5
--- /dev/null
+++ b/src/main/java/org/apache/rocketmq/connect/redis/util/ParseStringUtils.java
@@ -0,0 +1,54 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.rocketmq.connect.redis.util;
+
+import java.util.ArrayList;
+import java.util.HashMap;
+import java.util.List;
+import java.util.Map;
+import org.apache.commons.lang.StringUtils;
+
+public class ParseStringUtils {
+
+    public static Map<String, String> parseRedisInfo2Map(String info) {
+        Map<String, String> res = new HashMap<>();
+        String[] kvs = info.split("\n");
+        for (int i = 0; i < kvs.length; i++) {
+            String kv = kvs[i];
+            if (kv != null && !kv.startsWith("#")){
+                String[] kvArr = kv.split(":");
+                if(kvArr.length == 2){
+                    res.put(StringUtils.trimToEmpty(kvArr[0]), StringUtils.trimToEmpty(kvArr[1]));
+                }
+            }
+        }
+        return res;
+    }
+
+    public static List<String> parseCommands(String commands){
+        if (StringUtils.isEmpty(StringUtils.trimToEmpty(commands))) {
+            return null;
+        }
+        List<String> res = new ArrayList<>();
+        String[] commandNameArr = commands.split(",");
+        for (int i = 0; i < commandNameArr.length; i++) {
+            res.add(commandNameArr[i]);
+        }
+        return res;
+    }
+}
diff --git a/src/main/java/org/apache/rocketmq/connect/redis/util/PropertyToObjectUtils.java b/src/main/java/org/apache/rocketmq/connect/redis/util/PropertyToObjectUtils.java
new file mode 100644
index 0000000..62f09ce
--- /dev/null
+++ b/src/main/java/org/apache/rocketmq/connect/redis/util/PropertyToObjectUtils.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.connect.redis.util;
+
+import java.lang.reflect.InvocationTargetException;
+import java.lang.reflect.Method;
+
+import io.openmessaging.KeyValue;
+
+public class PropertyToObjectUtils {
+
+    public static void properties2Object(final KeyValue p, final Object object)
+        throws InvocationTargetException, IllegalAccessException {
+        Method[] methods = object.getClass().getMethods();
+        for (Method method : methods) {
+            String mn = method.getName();
+            if (mn.startsWith("set")) {
+                String tmp = mn.substring(4);
+                String first = mn.substring(3, 4);
+
+                String key = first.toLowerCase() + tmp;
+                String property = p.getString(key);
+                if (property != null) {
+                    Class<?>[] pt = method.getParameterTypes();
+                    if (pt != null && pt.length > 0) {
+                        String cn = pt[0].getSimpleName();
+                        Object arg;
+                        if (cn.equals("int") || cn.equals("Integer")) {
+                            arg = Integer.parseInt(property);
+                        } else if (cn.equals("long") || cn.equals("Long")) {
+                            arg = Long.parseLong(property);
+                        } else if (cn.equals("double") || cn.equals("Double")) {
+                            arg = Double.parseDouble(property);
+                        } else if (cn.equals("boolean") || cn.equals("Boolean")) {
+                            arg = Boolean.parseBoolean(property);
+                        } else if (cn.equals("float") || cn.equals("Float")) {
+                            arg = Float.parseFloat(property);
+                        } else if (cn.equals("String")) {
+                            arg = property;
+                        } else {
+                            continue;
+                        }
+                        method.invoke(object, arg);
+                    }
+                }
+            }
+        }
+    }
+}
diff --git a/src/test/java/org/apache/rocketmq/redis/test/common/ConfigTest.java b/src/test/java/org/apache/rocketmq/redis/test/common/ConfigTest.java
new file mode 100644
index 0000000..e63f45f
--- /dev/null
+++ b/src/test/java/org/apache/rocketmq/redis/test/common/ConfigTest.java
@@ -0,0 +1,107 @@
+package org.apache.rocketmq.redis.test.common;
+
+import java.nio.ByteBuffer;
+import java.util.Set;
+
+import com.moilioncircle.redis.replicator.RedisURI;
+import io.openmessaging.KeyValue;
+import io.openmessaging.internal.DefaultKeyValue;
+import org.apache.rocketmq.connect.redis.common.Config;
+import org.apache.rocketmq.connect.redis.common.SyncMod;
+import org.junit.Assert;
+import org.junit.Test;
+
+public class ConfigTest {
+
+    @Test
+    public void testNull() {
+        Config config = new Config();
+        RedisURI redisURI = config.getRedisUri();
+        ByteBuffer po = config.getPositionPartitionKey();
+        Assert.assertNull(redisURI);
+        Assert.assertNull(po);
+
+        config.setRedisAddr("127.0.0.1");
+        redisURI = config.getRedisUri();
+        po = config.getPositionPartitionKey();
+        Assert.assertNull(redisURI);
+        Assert.assertNull(po);
+
+        config.setRedisPassword("123456");
+        redisURI = config.getRedisUri();
+        po = config.getPositionPartitionKey();
+        Assert.assertNull(redisURI);
+        Assert.assertNull(po);
+
+        config.setRedisPort(6379);
+        redisURI = config.getRedisUri();
+        po = config.getPositionPartitionKey();
+        Assert.assertNotNull(redisURI);
+        Assert.assertNotNull(po);
+    }
+
+    @Test
+    public void test() {
+        Config config = new Config();
+
+        config.setRedisAddr("127.0.0.1");
+        config.setRedisPort(6379);
+        config.setRedisPassword("123456");
+        config.setTimeout(500);
+        config.setSyncMod(SyncMod.CUSTOM_OFFSET.name());
+        config.setOffset(65535L);
+        config.setReplId("c18cece63c7b16851a6f387f52dbbb9eee07e46f");
+        config.setCommands("SET,GET");
+        config.setPosition(3926872L);
+        config.setEventCommitRetryInterval(1000L);
+        config.setEventCommitRetryTimes(10);
+
+
+        Assert.assertEquals("127.0.0.1", config.getRedisAddr());
+        Assert.assertEquals(6379, (int)config.getRedisPort());
+        Assert.assertEquals("123456", config.getRedisPassword());
+        Assert.assertEquals(500, (int)config.getTimeout());
+        Assert.assertEquals(SyncMod.CUSTOM_OFFSET, config.getSyncMod());
+        Assert.assertEquals(65535L, (long)config.getOffset());
+        Assert.assertEquals("c18cece63c7b16851a6f387f52dbbb9eee07e46f", config.getReplId());
+        Assert.assertEquals("SET,GET", config.getCommands());
+        Assert.assertEquals(3926872L, (long)config.getPosition());
+        Assert.assertEquals(1000L, (long)config.getEventCommitRetryInterval());
+
+        RedisURI redisURI = config.getRedisUri();
+        Assert.assertNotNull(redisURI);
+
+        ByteBuffer byteBuffer = config.getPositionPartitionKey();
+        Assert.assertNotNull(byteBuffer);
+
+        Set<String> set = Config.getRequestConfig();
+        Assert.assertNotNull(set);
+
+        KeyValue keyValue = new DefaultKeyValue();
+        keyValue.put("a", "B");
+        String bo = Config.checkConfig(keyValue);
+        Assert.assertNotNull(bo);
+
+        keyValue.put("redisAddr", "127.0.0.1");
+        keyValue.put("redisPort", "6379");
+
+        bo = Config.checkConfig(keyValue);
+        Assert.assertNotNull(bo);
+
+        keyValue.put("redisPassword", "1234567");
+        bo = Config.checkConfig(keyValue);
+        Assert.assertNull(bo);
+
+        keyValue.put("syncMod", SyncMod.LAST_OFFSET.name());
+
+        Config config1 = new Config();
+        config1.load(keyValue);
+        Assert.assertEquals("127.0.0.1", config.getRedisAddr());
+        Assert.assertTrue(6379 == config1.getRedisPort());
+        Assert.assertEquals("1234567", config1.getRedisPassword());
+        Assert.assertEquals(SyncMod.LAST_OFFSET, config1.getSyncMod());
+
+        String err = config1.load(null);
+        Assert.assertNotNull(err);
+    }
+}
diff --git a/src/test/java/org/apache/rocketmq/redis/test/common/OptionsTest.java b/src/test/java/org/apache/rocketmq/redis/test/common/OptionsTest.java
new file mode 100644
index 0000000..a7ac8ae
--- /dev/null
+++ b/src/test/java/org/apache/rocketmq/redis/test/common/OptionsTest.java
@@ -0,0 +1,34 @@
+package org.apache.rocketmq.redis.test.common;
+
+import org.apache.rocketmq.connect.redis.common.Options;
+import org.junit.Assert;
+import org.junit.Test;
+
+
+public class OptionsTest {
+
+    @Test
+    public void test(){
+        Options options1 = Options.valueOf("TEST");
+        Assert.assertNull(options1);
+
+        Options options2 = Options.REDIS_PARTITION;
+        Assert.assertEquals(options2.name(), "DEFAULT_PARTITION");
+
+        Assert.assertNotNull(options2.toString());
+
+        Options options3 = Options.valueOf("DEFAULT_PARTITION");
+        Assert.assertTrue(options3.equals(options2));
+    }
+
+    @Test
+    public void testNull(){
+        Exception ex = null;
+        try {
+            Options.valueOf("");
+        }catch (Exception e){
+            ex = e;
+        }
+        Assert.assertNotNull(ex);
+    }
+}
diff --git a/src/test/java/org/apache/rocketmq/redis/test/connector/RedisSourceConnectorTest.java b/src/test/java/org/apache/rocketmq/redis/test/connector/RedisSourceConnectorTest.java
new file mode 100644
index 0000000..355351b
--- /dev/null
+++ b/src/test/java/org/apache/rocketmq/redis/test/connector/RedisSourceConnectorTest.java
@@ -0,0 +1,35 @@
+package org.apache.rocketmq.redis.test.connector;
+
+import java.util.List;
+
+import io.openmessaging.KeyValue;
+import io.openmessaging.internal.DefaultKeyValue;
+import org.apache.rocketmq.connect.redis.connector.RedisSourceConnector;
+import org.junit.Assert;
+import org.junit.Before;
+import org.junit.Test;
+
+public class RedisSourceConnectorTest {
+    private KeyValue keyValue;
+
+    @Before
+    public void initKeyValue(){
+        this.keyValue = new DefaultKeyValue();
+        this.keyValue.put("redisAddr", "127.0.0.1");
+        this.keyValue.put("redisPort", "6379");
+        this.keyValue.put("redisPassword", "");
+    }
+    @Test
+    public void testConnector(){
+        RedisSourceConnector connector = new RedisSourceConnector();
+        connector.verifyAndSetConfig(this.keyValue);
+        Class cl = connector.taskClass();
+        Assert.assertNotNull(cl);
+        List<KeyValue> keyValues = connector.taskConfigs();
+        Assert.assertNotNull(keyValues);
+        connector.start();
+        connector.pause();
+        connector.resume();
+        connector.stop();
+    }
+}
diff --git a/src/test/java/org/apache/rocketmq/redis/test/connector/RedisSourceTaskTest.java b/src/test/java/org/apache/rocketmq/redis/test/connector/RedisSourceTaskTest.java
new file mode 100644
index 0000000..4dd4397
--- /dev/null
+++ b/src/test/java/org/apache/rocketmq/redis/test/connector/RedisSourceTaskTest.java
@@ -0,0 +1,150 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.rocketmq.redis.test.connector;
+
+import com.moilioncircle.redis.replicator.rdb.datatype.KeyStringValueString;
+import io.openmessaging.KeyValue;
+import io.openmessaging.connector.api.PositionStorageReader;
+import io.openmessaging.connector.api.data.FieldType;
+import io.openmessaging.connector.api.data.SourceDataEntry;
+import io.openmessaging.connector.api.source.SourceTaskContext;
+import io.openmessaging.internal.DefaultKeyValue;
+import java.nio.ByteBuffer;
+import java.util.Collection;
+import java.util.Map;
+import org.apache.rocketmq.connect.redis.connector.RedisSourceTask;
+import org.apache.rocketmq.connect.redis.pojo.RedisEvent;
+import org.apache.rocketmq.connect.redis.processor.DefaultRedisEventProcessor;
+import org.apache.rocketmq.connect.redis.processor.RedisEventProcessor;
+import org.junit.After;
+import org.junit.Assert;
+import org.junit.Before;
+import org.junit.Test;
+import redis.clients.jedis.exceptions.JedisConnectionException;
+
+import static com.moilioncircle.redis.replicator.Constants.RDB_TYPE_STRING;
+import static org.mockito.Mockito.mock;
+import static org.mockito.Mockito.when;
+
+public class RedisSourceTaskTest {
+    private KeyValue keyValue;
+    private RedisSourceTask task;
+
+    @Before
+    public void initAndStartTask() {
+        try {
+            initKeyValue();
+            this.task = new RedisSourceTask();
+            this.task.initialize(new SourceTaskContext() {
+                @Override
+                public PositionStorageReader positionStorageReader() {
+                    return new PositionStorageReader() {
+                        @Override
+                        public ByteBuffer getPosition(ByteBuffer byteBuffer) {
+                            return null;
+                        }
+
+                        @Override
+                        public Map<ByteBuffer, ByteBuffer> getPositions(Collection<ByteBuffer> collection) {
+                            return null;
+                        }
+                    };
+                }
+
+                @Override
+                public KeyValue configs() {
+                    return keyValue;
+                }
+            });
+            this.task.start(this.keyValue);
+        } catch (JedisConnectionException e) {
+
+        }
+    }
+
+    @Test
+    public void testTask() throws Exception {
+        if (this.task != null) {
+            RedisEvent redisEvent = getRedisEvent();
+            this.task.getEventProcessor().commit(redisEvent);
+            Collection<SourceDataEntry> col = this.task.poll();
+            Assert.assertNotNull(col);
+            Assert.assertEquals(1, col.size());
+            Assert.assertNotNull(this.task.getConfig());
+        }
+    }
+
+    @Test
+    public void testException() {
+        RedisEventProcessor processor = mock(DefaultRedisEventProcessor.class);
+        try {
+            when(processor.poll()).thenThrow(new InterruptedException());
+        } catch (Exception e) {
+            e.printStackTrace();
+        }
+
+        RedisSourceTask redisSourceTask = new RedisSourceTask();
+        redisSourceTask.setEventProcessor(processor);
+        redisSourceTask.poll();
+
+
+
+        RedisEventProcessor processor2 = mock(DefaultRedisEventProcessor.class);
+        try {
+            when(processor2.poll()).thenThrow(new Exception());
+        } catch (Exception e) {
+            e.printStackTrace();
+        }
+
+        RedisSourceTask redisSourceTask2 = new RedisSourceTask();
+        redisSourceTask2.setEventProcessor(processor2);
+        redisSourceTask2.poll();
+    }
+
+    @After
+    public void stopTask() {
+        if (this.task != null) {
+            this.task.pause();
+            this.task.resume();
+            this.task.stop();
+        }
+    }
+
+    private RedisEvent getRedisEvent() {
+        RedisEvent redisEvent = new RedisEvent();
+
+        KeyStringValueString event = new KeyStringValueString();
+        event.setKey("key".getBytes());
+        event.setValue("value".getBytes());
+        event.setValueRdbType(RDB_TYPE_STRING);
+
+        redisEvent.setEvent(event);
+        redisEvent.setReplId("replId");
+        redisEvent.setReplOffset(6000L);
+        redisEvent.setStreamDB(0);
+        return redisEvent;
+    }
+
+    private void initKeyValue() {
+        this.keyValue = new DefaultKeyValue();
+        this.keyValue.put("redisAddr", "127.0.0.1");
+        this.keyValue.put("redisPort", "6379");
+        this.keyValue.put("redisPassword", "");
+    }
+
+}
diff --git a/src/test/java/org/apache/rocketmq/redis/test/converter/RedisEntryConverterTest.java b/src/test/java/org/apache/rocketmq/redis/test/converter/RedisEntryConverterTest.java
new file mode 100644
index 0000000..9b4d91c
--- /dev/null
+++ b/src/test/java/org/apache/rocketmq/redis/test/converter/RedisEntryConverterTest.java
@@ -0,0 +1,118 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this 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.redis.test.converter;
+
+import io.openmessaging.connector.api.data.EntryType;
+import io.openmessaging.connector.api.data.FieldType;
+import io.openmessaging.connector.api.data.SourceDataEntry;
+import java.util.ArrayList;
+import java.util.Collection;
+import java.util.HashMap;
+import java.util.List;
+import java.util.Map;
+import org.apache.rocketmq.connect.redis.common.Options;
+import org.apache.rocketmq.connect.redis.converter.KVEntryConverter;
+import org.apache.rocketmq.connect.redis.converter.RedisEntryConverter;
+import org.apache.rocketmq.connect.redis.pojo.KVEntry;
+import org.apache.rocketmq.connect.redis.pojo.RedisEntry;
+import org.junit.Assert;
+import org.junit.Test;
+
+public class RedisEntryConverterTest {
+
+    @Test
+    public void test(){
+        KVEntry entry = getKVEntry();
+        KVEntryConverter converter = new RedisEntryConverter();
+        Collection<SourceDataEntry> res = converter.kVEntryToDataEntries(entry);
+        Assert.assertNotNull(res);
+        Assert.assertEquals(1, res.size());
+        Assert.assertEquals("key", ((List<SourceDataEntry>) res).get(0).getPayload()[1]);
+        Assert.assertEquals("value", ((List<SourceDataEntry>) res).get(0).getPayload()[2]);
+        Assert.assertEquals("set", ((List<SourceDataEntry>) res).get(0).getPayload()[0]);
+        Map<String, Object> params = (Map<String, Object>) ((List<SourceDataEntry>) res).get(0).getPayload()[3];
+        Assert.assertEquals("replId", params.get(Options.REDIS_REPLID.name()));
+    }
+
+    @Test
+    public void testListSplit(){
+        KVEntryConverter converter = new RedisEntryConverter();
+
+
+        KVEntry entry = getArrayKVEntry(999);
+        Collection res = converter.kVEntryToDataEntries(entry);
+        Assert.assertEquals(2, res.size());
+
+        KVEntry entry1 = getArrayKVEntry(1001);
+        Collection res1 = converter.kVEntryToDataEntries(entry1);
+        Assert.assertEquals(3, res1.size());
+
+        KVEntry entry2 = getArrayKVEntry(1000);
+        Collection res2 = converter.kVEntryToDataEntries(entry2);
+        Assert.assertEquals(2, res2.size());
+    }
+
+
+    @Test
+    public void testMapSplit(){
+        KVEntryConverter converter = new RedisEntryConverter();
+
+        RedisEntry entry = RedisEntry.newEntry(FieldType.MAP);
+        entry.queueName("queue1");
+        entry.key("key");
+        entry.entryType(EntryType.UPDATE);
+        entry.offset(65535L);
+        entry.param(Options.REDIS_INCREMENT, 15L);
+        entry.sourceId("123");
+        entry.command("set");
+
+        Map<String, String> values = new HashMap<>();
+        int num = 10001;
+        for (int i = 0; i < num; i++) {
+            values.put("a" + i, Integer.toString(i));
+        }
+        entry.value(values);
+
+        List<SourceDataEntry> entryList = converter.kVEntryToDataEntries(entry);
+        Assert.assertNotNull(entryList);
+        Assert.assertEquals(21, entryList.size());
+        Assert.assertEquals("set", entryList.get(0).getPayload()[0]);
+    }
+
+    private KVEntry getKVEntry(){
+        KVEntry entry = new RedisEntry(FieldType.STRING);
+        return entry.key("key")
+                    .value("value")
+                    .command("set")
+                    .sourceId("replId")
+                    .offset(6000L);
+    }
+
+    private KVEntry getArrayKVEntry(int size){
+        KVEntry entry = new RedisEntry(FieldType.ARRAY);
+        List<String> values = new ArrayList<>(size);
+        for (int i = 0; i < size; i++) {
+            values.add(Integer.toString(i));
+        }
+        return entry.key("key")
+            .value(values)
+            .command("set")
+            .sourceId("replId")
+            .offset(6000L);
+    }
+}
diff --git a/src/test/java/org/apache/rocketmq/redis/test/converter/RedisPositionConverterTest.java b/src/test/java/org/apache/rocketmq/redis/test/converter/RedisPositionConverterTest.java
new file mode 100644
index 0000000..03986d2
--- /dev/null
+++ b/src/test/java/org/apache/rocketmq/redis/test/converter/RedisPositionConverterTest.java
@@ -0,0 +1,51 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.rocketmq.redis.test.converter;
+
+import com.alibaba.fastjson.JSONObject;
+import java.nio.ByteBuffer;
+import org.apache.rocketmq.connect.redis.common.Config;
+import org.apache.rocketmq.connect.redis.common.RedisConstants;
+import org.apache.rocketmq.connect.redis.converter.RedisPositionConverter;
+import org.junit.Assert;
+import org.junit.Test;
+
+public class RedisPositionConverterTest {
+    @Test
+    public void test(){
+        JSONObject jsonObject = RedisPositionConverter.longToJson(10000L);
+        long offset = jsonObject.getLong(RedisConstants.POSITION);
+        Assert.assertEquals(10000L, offset);
+
+        Config config = getConfig();
+        ByteBuffer byteBuffer = ByteBuffer.wrap(jsonObject.toJSONString().getBytes());
+        Long position = RedisPositionConverter.jsonToLong(byteBuffer);
+        config.setPosition(position);
+        Assert.assertEquals(10000L, (long)config.getPosition());
+    }
+
+    private Config getConfig(){
+        Config config = new Config();
+
+        config.setRedisAddr("127.0.0.1");
+        config.setRedisPort(6379);
+        config.setRedisPassword("");
+        config.setEventCommitRetryTimes(10);
+        return config;
+    }
+}
diff --git a/src/test/java/org/apache/rocketmq/redis/test/handler/RedisEventHandlerTest.java b/src/test/java/org/apache/rocketmq/redis/test/handler/RedisEventHandlerTest.java
new file mode 100644
index 0000000..068187f
--- /dev/null
+++ b/src/test/java/org/apache/rocketmq/redis/test/handler/RedisEventHandlerTest.java
@@ -0,0 +1,468 @@
+package org.apache.rocketmq.redis.test.handler;
+
+import com.moilioncircle.redis.replicator.cmd.impl.GetSetCommand;
+import java.util.ArrayList;
+import java.util.HashMap;
+import java.util.HashSet;
+import java.util.List;
+import java.util.Map;
+import java.util.Set;
+
+import com.moilioncircle.redis.replicator.rdb.datatype.AuxField;
+import com.moilioncircle.redis.replicator.rdb.datatype.KeyStringValueList;
+import com.moilioncircle.redis.replicator.rdb.datatype.KeyStringValueString;
+import com.moilioncircle.redis.replicator.rdb.datatype.KeyValuePair;
+import com.moilioncircle.redis.replicator.rdb.datatype.Module;
+import com.moilioncircle.redis.replicator.rdb.datatype.Stream;
+import com.moilioncircle.redis.replicator.rdb.datatype.ZSetEntry;
+import com.moilioncircle.redis.replicator.rdb.iterable.datatype.BatchedKeyStringValueString;
+import com.moilioncircle.redis.replicator.rdb.iterable.datatype.BatchedKeyValuePair;
+import io.openmessaging.connector.api.data.FieldType;
+import org.apache.rocketmq.connect.redis.common.Config;
+import org.apache.rocketmq.connect.redis.common.SyncMod;
+import org.apache.rocketmq.connect.redis.handler.DefaultRedisEventHandler;
+import org.apache.rocketmq.connect.redis.handler.RedisEventHandler;
+import org.apache.rocketmq.connect.redis.pojo.KVEntry;
+import org.apache.rocketmq.connect.redis.pojo.RedisEntry;
+import org.junit.Assert;
+import org.junit.Test;
+
+import static com.moilioncircle.redis.replicator.Constants.RDB_TYPE_HASH;
+import static com.moilioncircle.redis.replicator.Constants.RDB_TYPE_HASH_ZIPLIST;
+import static com.moilioncircle.redis.replicator.Constants.RDB_TYPE_HASH_ZIPMAP;
+import static com.moilioncircle.redis.replicator.Constants.RDB_TYPE_LIST;
+import static com.moilioncircle.redis.replicator.Constants.RDB_TYPE_LIST_QUICKLIST;
+import static com.moilioncircle.redis.replicator.Constants.RDB_TYPE_LIST_ZIPLIST;
+import static com.moilioncircle.redis.replicator.Constants.RDB_TYPE_MODULE;
+import static com.moilioncircle.redis.replicator.Constants.RDB_TYPE_MODULE_2;
+import static com.moilioncircle.redis.replicator.Constants.RDB_TYPE_SET;
+import static com.moilioncircle.redis.replicator.Constants.RDB_TYPE_SET_INTSET;
+import static com.moilioncircle.redis.replicator.Constants.RDB_TYPE_STREAM_LISTPACKS;
+import static com.moilioncircle.redis.replicator.Constants.RDB_TYPE_STRING;
+import static com.moilioncircle.redis.replicator.Constants.RDB_TYPE_ZSET;
+import static com.moilioncircle.redis.replicator.Constants.RDB_TYPE_ZSET_2;
+import static com.moilioncircle.redis.replicator.Constants.RDB_TYPE_ZSET_ZIPLIST;
+
+public class RedisEventHandlerTest {
+    private String replId = "c18cece63c7b16851a6f387f52dbbb9eee07e46f";
+    private Long offset = 3926872L;
+
+    @Test
+    public void testNull() {
+        Config config = getConfig();
+        RedisEventHandler handler = new DefaultRedisEventHandler(config);
+        KVEntry res = null;
+        Exception ex = null;
+
+        // 测试increment下的rdb数据处理
+        config.setSyncMod(SyncMod.LAST_OFFSET.name());
+        KeyValuePair keyValuePair = new KeyStringValueString();
+        keyValuePair.setKey("key".getBytes());
+        keyValuePair.setValue("value".getBytes());
+        try {
+            res = handler.handleKVString(replId, offset, keyValuePair);
+        } catch (Exception e) {
+            e.printStackTrace();
+            ex = e;
+        }
+        Assert.assertNull(res);
+        Assert.assertNull(ex);
+
+        // 测试未指定的command
+        GetSetCommand command = new GetSetCommand();
+        command.setKey("key".getBytes());
+        command.setValue("value".getBytes());
+        try {
+            res = handler.handleCommand(replId, offset, command);
+        } catch (Exception e) {
+            ex = e;
+        }
+        Assert.assertNull(res);
+        Assert.assertNull(ex);
+
+
+
+    }
+
+    @Test
+    public void test() {
+        Config config = getConfig();
+        RedisEventHandler handler = new DefaultRedisEventHandler(config);
+        KVEntry entry = getBuilder();
+
+        KVEntry res = null;
+        Exception ex = null;
+        try {
+            res = handler.handleCommand(replId, offset, entry);
+        } catch (Exception e) {
+            e.printStackTrace();
+            ex = e;
+        }
+        Assert.assertEquals("c18cece63c7b16851a6f387f52dbbb9eee07e46f", res.getSourceId());
+        Assert.assertTrue(res.getOffset() == 3926872L);
+        Assert.assertNull(ex);
+        AuxField auxField = new AuxField("a", "b");
+        try {
+            res = handler.handleOtherEvent(replId, offset, auxField);
+        } catch (Exception e) {
+            e.printStackTrace();
+            ex = e;
+        }
+        Assert.assertNull(res);
+        Assert.assertNull(ex);
+
+        BatchedKeyValuePair pair = new BatchedKeyStringValueString();
+        pair.setKey("A".getBytes());
+        pair.setValue("B".getBytes());
+        try {
+            res = handler.handleBatchKVString(replId, offset, pair);
+        } catch (Exception e) {
+            e.printStackTrace();
+            ex = e;
+        }
+        Assert.assertNull(res);
+        Assert.assertNull(ex);
+
+    }
+
+    @Test
+    public void testKeyValuePair() {
+        KVEntry kvEntry =
+            handlerTest(RDB_TYPE_STRING, () ->
+                "value".getBytes()
+            );
+        Assert.assertEquals(replId, kvEntry.getSourceId());
+        Assert.assertTrue(kvEntry.getOffset() == 3926872L);
+        Assert.assertEquals("key", kvEntry.getKey());
+        Assert.assertEquals("value", kvEntry.getValue());
+
+    }
+
+    @Test
+    public void testList() {
+        KVEntry builder =
+            handlerTest(RDB_TYPE_LIST, () -> {
+                List<byte[]> values = new ArrayList<>();
+                values.add("a".getBytes());
+                return values;
+            });
+        List<String> va = (List)builder.getValue();
+        Assert.assertNotNull(va);
+        Assert.assertEquals(1, va.size());
+        Assert.assertEquals("a", va.get(0));
+    }
+
+    @Test
+    public void testSet() {
+        KVEntry builder =
+            handlerTest(RDB_TYPE_SET, () -> {
+                Set<byte[]> values = new HashSet<>();
+                values.add("myValue".getBytes());
+                values.add("myValue2".getBytes());
+                return values;
+            });
+
+        List<String> va = (List<String>)builder.getValue();
+        Assert.assertNotNull(va);
+        Assert.assertEquals(2, va.size());
+        Assert.assertTrue(va.contains("myValue"));
+        Assert.assertTrue(va.contains("myValue2"));
+    }
+
+
+    @Test
+    public void testZSET() {
+        KVEntry builder =
+            handlerTest(RDB_TYPE_ZSET, () -> {
+                Set<ZSetEntry> values = new HashSet<>();
+                values.add(new ZSetEntry("key1".getBytes(), 100));
+                values.add(new ZSetEntry("key2".getBytes(), 80));
+                return values;
+            });
+
+        Map<String, Double> va = (Map<String, Double>)builder.getValue();
+        Assert.assertNotNull(va);
+        Assert.assertEquals(2, va.size());
+        Assert.assertTrue(100 == va.get("key1"));
+        Assert.assertTrue(80 ==  va.get("key2"));
+    }
+
+
+    @Test
+    public void testZSet2() {
+        KVEntry builder =
+            handlerTest(RDB_TYPE_ZSET_2, () -> {
+                Set<ZSetEntry> values = new HashSet<>();
+                values.add(new ZSetEntry("key1".getBytes(), 100));
+                values.add(new ZSetEntry("key2".getBytes(), 80));
+                return values;
+            });
+
+        Map<String, Double> va = (Map<String, Double>)builder.getValue();
+        Assert.assertNotNull(va);
+        Assert.assertEquals(2, va.size());
+        Assert.assertTrue(100 == va.get("key1"));
+        Assert.assertTrue(80 ==  va.get("key2"));
+    }
+
+
+    @Test
+    public void testHash() {
+        KVEntry builder =
+            handlerTest(RDB_TYPE_HASH, () -> {
+                Map<byte[], byte[]> values = new HashMap<>();
+                values.put("key1".getBytes(), "value1".getBytes());
+                values.put("key2".getBytes(), "value2".getBytes());
+                return values;
+            });
+
+        Map<String, String> va = (Map<String, String>)builder.getValue();
+        Assert.assertNotNull(va);
+        Assert.assertEquals(2, va.size());
+        Assert.assertEquals("value1", va.get("key1"));
+        Assert.assertEquals("value2", va.get("key2"));
+    }
+
+
+    @Test
+    public void testHashZipMap() {
+        KVEntry builder =
+            handlerTest(RDB_TYPE_HASH_ZIPMAP, () -> {
+                Map<byte[], byte[]> values = new HashMap<>();
+                values.put("key1".getBytes(), "value1".getBytes());
+                values.put("key2".getBytes(), "value2".getBytes());
+                return values;
+            });
+
+        Map<String, String> va = (Map<String, String>)builder.getValue();
+        Assert.assertNotNull(va);
+        Assert.assertEquals(2, va.size());
+        Assert.assertEquals("value1", va.get("key1"));
+        Assert.assertEquals("value2", va.get("key2"));
+    }
+
+    @Test
+    public void testListZipList() {
+        KVEntry builder =
+            handlerTest(RDB_TYPE_LIST_ZIPLIST, () -> {
+                List<byte[]> values = new ArrayList<>();
+                values.add("v1".getBytes());
+                values.add("v2".getBytes());
+                return values;
+            });
+
+        List<String> va = (List<String>)builder.getValue();
+        Assert.assertNotNull(va);
+        Assert.assertEquals(2, va.size());
+        Assert.assertEquals("v1", va.get(0));
+        Assert.assertEquals("v2", va.get(1));
+    }
+
+    @Test
+    public void testSetIntSet() {
+        KVEntry builder =
+            handlerTest(RDB_TYPE_SET_INTSET, () -> {
+                Set<byte[]> values = new HashSet<>();
+                values.add("v1".getBytes());
+                values.add("v2".getBytes());
+                return values;
+            });
+
+        List<String> va = (List<String>)builder.getValue();
+        Assert.assertNotNull(va);
+        Assert.assertEquals(2, va.size());
+        Assert.assertTrue(va.contains("v1"));
+        Assert.assertTrue(va.contains("v2"));
+    }
+
+
+    @Test
+    public void testZSetZipList() {
+        KVEntry builder =
+            handlerTest(RDB_TYPE_ZSET_ZIPLIST, () -> {
+                Set<ZSetEntry> values = new HashSet<>();
+                values.add(new ZSetEntry("v1".getBytes(), 100));
+                values.add(new ZSetEntry("v2".getBytes(), 80));
+                return values;
+            });
+
+        Map<String, Double> va = (Map<String, Double>)builder.getValue();
+        Assert.assertNotNull(va);
+        Assert.assertEquals(2, va.size());
+        Assert.assertTrue(100 == va.get("v1"));
+        Assert.assertTrue(80 == va.get("v2"));
+    }
+
+    @Test
+    public void testHashZipList() {
+        KVEntry builder =
+            handlerTest(RDB_TYPE_HASH_ZIPLIST, () -> {
+                Map<byte[], byte[]> values = new HashMap<>();
+                values.put("k1".getBytes(), "v1".getBytes());
+                values.put("k2".getBytes(), "v2".getBytes());
+                return values;
+            });
+
+        Map<String, String> va = (Map<String, String>)builder.getValue();
+        Assert.assertNotNull(va);
+        Assert.assertEquals(2, va.size());
+        Assert.assertEquals("v1", va.get("k1"));
+        Assert.assertEquals("v2", va.get("k2"));
+    }
+
+    @Test
+    public void testListQuickList() {
+        KVEntry builder =
+            handlerTest(RDB_TYPE_LIST_QUICKLIST, () -> {
+                List<byte[]> values = new ArrayList<>();
+                values.add("v1".getBytes());
+                values.add("v2".getBytes());
+                return values;
+            });
+
+        List<String> va = (List<String>)builder.getValue();
+        Assert.assertNotNull(va);
+        Assert.assertEquals(2, va.size());
+        Assert.assertEquals("v1", va.get(0));
+        Assert.assertEquals("v2", va.get(1));
+    }
+
+    @Test
+    public void testModule() {
+        KVEntry builder =
+            handlerTest(RDB_TYPE_MODULE, () -> {
+                Module module = new Module() {
+                    @Override
+                    public int hashCode() {
+                        return super.hashCode();
+                    }
+
+                    @Override
+                    public boolean equals(Object obj) {
+                        return super.equals(obj);
+                    }
+
+                    @Override
+                    protected Object clone() throws CloneNotSupportedException {
+                        return super.clone();
+                    }
+
+                    @Override
+                    public String toString() {
+                        return super.toString();
+                    }
+
+                    @Override
+                    protected void finalize() throws Throwable {
+                        super.finalize();
+                    }
+                };
+                return module;
+            });
+
+        Module va = (Module)builder.getValue();
+        Assert.assertNotNull(va);
+    }
+
+    @Test
+    public void testModule2() {
+        KVEntry builder =
+            handlerTest(RDB_TYPE_MODULE_2, () -> {
+                Module module = new Module() {
+                    @Override
+                    public int hashCode() {
+                        return super.hashCode();
+                    }
+
+                    @Override
+                    public boolean equals(Object obj) {
+                        return super.equals(obj);
+                    }
+
+                    @Override
+                    protected Object clone() throws CloneNotSupportedException {
+                        return super.clone();
+                    }
+
+                    @Override
+                    public String toString() {
+                        return super.toString();
+                    }
+
+                    @Override
+                    protected void finalize() throws Throwable {
+                        super.finalize();
+                    }
+                };
+                return module;
+            });
+
+        Module va = (Module)builder.getValue();
+        Assert.assertNotNull(va);
+    }
+
+    @Test
+    public void testStreamListPacks() {
+        KVEntry builder =
+            handlerTest(RDB_TYPE_STREAM_LISTPACKS, () -> {
+                Stream stream = new Stream();
+                return stream;
+            });
+
+        Stream va = (Stream)builder.getValue();
+        Assert.assertNotNull(va);
+    }
+
+    @Test
+    public void testException() {
+        Error ex = null;
+        try{
+            handlerTest(999, () -> new Object());
+        }catch (AssertionError e){
+            ex = e;
+        }
+        Assert.assertNotNull(ex);
+    }
+
+    private interface ValueSetter<T> {
+        T getValue();
+    }
+
+
+    private <T> KVEntry handlerTest(int rdbType, ValueSetter<T> setter) {
+        KVEntry res = null;
+        Exception ex = null;
+        Config config = getConfig();
+        RedisEventHandler handler = new DefaultRedisEventHandler(config);
+
+        KeyValuePair keyValuePair = new KeyStringValueList();
+        keyValuePair.setValueRdbType(rdbType);
+        keyValuePair.setKey("key".getBytes());
+
+        Object value = setter.getValue();
+
+        keyValuePair.setValue(value);
+        try {
+            res = handler.handleKVString(replId, offset, keyValuePair);
+        } catch (Exception e) {
+            e.printStackTrace();
+            ex = e;
+        }
+        Assert.assertNull(ex);
+        return res;
+    }
+
+    private Config getConfig() {
+        Config config = new Config();
+
+        config.setRedisAddr("127.0.0.1");
+        config.setRedisPort(6379);
+        config.setRedisPassword("123456");
+        return config;
+    }
+
+    private KVEntry getBuilder() {
+        return RedisEntry.newEntry(FieldType.STRING);
+    }
+
+}
diff --git a/src/test/java/org/apache/rocketmq/redis/test/parser/ParserTest.java b/src/test/java/org/apache/rocketmq/redis/test/parser/ParserTest.java
new file mode 100644
index 0000000..b2b7960
--- /dev/null
+++ b/src/test/java/org/apache/rocketmq/redis/test/parser/ParserTest.java
@@ -0,0 +1,1108 @@
+package org.apache.rocketmq.redis.test.parser;
+
+import java.util.List;
+import java.util.Map;
+
+import com.moilioncircle.redis.replicator.rdb.datatype.ExpiredType;
+import org.apache.rocketmq.connect.redis.common.Options;
+import org.apache.rocketmq.connect.redis.parser.AppendParser;
+import org.apache.rocketmq.connect.redis.parser.BitFieldParser;
+import org.apache.rocketmq.connect.redis.parser.BitOpParser;
+import org.apache.rocketmq.connect.redis.parser.BrPopLPushParser;
+import org.apache.rocketmq.connect.redis.parser.DecrByParser;
+import org.apache.rocketmq.connect.redis.parser.DecrParser;
+import org.apache.rocketmq.connect.redis.parser.DelParser;
+import org.apache.rocketmq.connect.redis.parser.EvalParser;
+import org.apache.rocketmq.connect.redis.parser.EvalShaParser;
+import org.apache.rocketmq.connect.redis.parser.ExecParser;
+import org.apache.rocketmq.connect.redis.parser.ExpireAtParser;
+import org.apache.rocketmq.connect.redis.parser.ExpireParser;
+import org.apache.rocketmq.connect.redis.parser.FlushAllParser;
+import org.apache.rocketmq.connect.redis.parser.FlushDbParser;
+import org.apache.rocketmq.connect.redis.parser.GeoAddParser;
+import org.apache.rocketmq.connect.redis.parser.GetsetParser;
+import org.apache.rocketmq.connect.redis.parser.HDelParser;
+import org.apache.rocketmq.connect.redis.parser.HIncrByParser;
+import org.apache.rocketmq.connect.redis.parser.HSetNxParser;
+import org.apache.rocketmq.connect.redis.parser.HSetParser;
+import org.apache.rocketmq.connect.redis.parser.HmSetParser;
+import org.apache.rocketmq.connect.redis.parser.IncrByParser;
+import org.apache.rocketmq.connect.redis.parser.IncrParser;
+import org.apache.rocketmq.connect.redis.parser.LPopParser;
+import org.apache.rocketmq.connect.redis.parser.LPushParser;
+import org.apache.rocketmq.connect.redis.parser.LPushXParser;
+import org.apache.rocketmq.connect.redis.parser.LRemParser;
+import org.apache.rocketmq.connect.redis.parser.LSetParser;
+import org.apache.rocketmq.connect.redis.parser.LTrimParser;
+import org.apache.rocketmq.connect.redis.parser.LinsertParser;
+import org.apache.rocketmq.connect.redis.parser.MSetNxParser;
+import org.apache.rocketmq.connect.redis.parser.MSetParser;
+import org.apache.rocketmq.connect.redis.parser.MoveParser;
+import org.apache.rocketmq.connect.redis.parser.MultiParser;
+import org.apache.rocketmq.connect.redis.parser.PExpireAtParser;
+import org.apache.rocketmq.connect.redis.parser.PExpireParser;
+import org.apache.rocketmq.connect.redis.parser.PSetExParser;
+import org.apache.rocketmq.connect.redis.parser.PersistParser;
+import org.apache.rocketmq.connect.redis.parser.PfAddParser;
+import org.apache.rocketmq.connect.redis.parser.PfCountParser;
+import org.apache.rocketmq.connect.redis.parser.PfMergeParser;
+import org.apache.rocketmq.connect.redis.parser.PublishParser;
+import org.apache.rocketmq.connect.redis.parser.RPopLPushParser;
+import org.apache.rocketmq.connect.redis.parser.RPopParser;
+import org.apache.rocketmq.connect.redis.parser.RPushParser;
+import org.apache.rocketmq.connect.redis.parser.RPushXParser;
+import org.apache.rocketmq.connect.redis.parser.RenameNxParser;
+import org.apache.rocketmq.connect.redis.parser.RenameParser;
+import org.apache.rocketmq.connect.redis.parser.RestoreParser;
+import org.apache.rocketmq.connect.redis.parser.SAddParser;
+import org.apache.rocketmq.connect.redis.parser.SDiffStoreParser;
+import org.apache.rocketmq.connect.redis.parser.SInterStoreParser;
+import org.apache.rocketmq.connect.redis.parser.SMoveParser;
+import org.apache.rocketmq.connect.redis.parser.SRemParser;
+import org.apache.rocketmq.connect.redis.parser.SUnionStoreParser;
+import org.apache.rocketmq.connect.redis.parser.ScriptParser;
+import org.apache.rocketmq.connect.redis.parser.SelectParser;
+import org.apache.rocketmq.connect.redis.parser.SetBitParser;
+import org.apache.rocketmq.connect.redis.parser.SetExParser;
+import org.apache.rocketmq.connect.redis.parser.SetNxParser;
+import org.apache.rocketmq.connect.redis.parser.SetParser;
+import org.apache.rocketmq.connect.redis.parser.SetRangeParser;
+import org.apache.rocketmq.connect.redis.parser.SortParser;
+import org.apache.rocketmq.connect.redis.parser.SwapDbParser;
+import org.apache.rocketmq.connect.redis.parser.UnLinkParser;
+import org.apache.rocketmq.connect.redis.parser.XAckParser;
+import org.apache.rocketmq.connect.redis.parser.XAddParser;
+import org.apache.rocketmq.connect.redis.parser.XClaimParser;
+import org.apache.rocketmq.connect.redis.parser.XDelParser;
+import org.apache.rocketmq.connect.redis.parser.XGroupParser;
+import org.apache.rocketmq.connect.redis.parser.XSetIdParser;
+import org.apache.rocketmq.connect.redis.parser.XTrimParser;
+import org.apache.rocketmq.connect.redis.parser.ZAddParser;
+import org.apache.rocketmq.connect.redis.parser.ZIncrByParser;
+import org.apache.rocketmq.connect.redis.parser.ZInterStoreParser;
+import org.apache.rocketmq.connect.redis.parser.ZPopMaxParser;
+import org.apache.rocketmq.connect.redis.parser.ZPopMinParser;
+import org.apache.rocketmq.connect.redis.parser.ZRemParser;
+import org.apache.rocketmq.connect.redis.parser.ZRemRangeByLexParser;
+import org.apache.rocketmq.connect.redis.parser.ZRemRangeByRankParser;
+import org.apache.rocketmq.connect.redis.parser.ZRemRangeByScoreParser;
+import org.apache.rocketmq.connect.redis.parser.ZUnionStoreParser;
+import org.apache.rocketmq.connect.redis.pojo.Geo;
+import org.apache.rocketmq.connect.redis.pojo.KVEntry;
+import org.junit.Assert;
+import org.junit.Test;
+
+public class ParserTest {
+
+    private Object[] parseCommand(String command){
+        String[] commandArr = command.split(" ");
+        Object[] res = new Object[commandArr.length];
+        for (int i = 0; i < commandArr.length; i++) {
+            String sg = commandArr[i];
+            res[i] = sg.getBytes();
+        }
+        return res;
+    }
+
+    @Test
+    public void testAppendParser(){
+        String command = "APPEND k1 v1";
+        KVEntry builder = new AppendParser().parse(parseCommand(command));
+        Assert.assertEquals("APPEND", builder.getCommand());
+        Assert.assertEquals("k1", builder.getKey());
+        Assert.assertEquals("v1", builder.getValue());
+    }
+
+    @Test
+    public void testBitFieldParser(){
+        String command = "BITFIELD k1";
+        KVEntry builder = new BitFieldParser().parse(parseCommand(command));
+        Assert.assertEquals("BITFIELD", builder.getCommand());
+        Assert.assertEquals("k1", builder.getKey());
+    }
+
+    @Test
+    public void testBitOpParser(){
+        String command = "BITOP operation destkey k1 k2";
+        KVEntry builder = new BitOpParser().parse(parseCommand(command));
+        Assert.assertEquals("BITOP", builder.getCommand());
+        Assert.assertEquals("operation", builder.getKey());
+        List<String> res = (List<String>)builder.getValue();
+        Assert.assertEquals(3, res.size());
+        Assert.assertEquals("destkey", res.get(0));
+        Assert.assertEquals("k1", res.get(1));
+        Assert.assertEquals("k2", res.get(2));
+    }
+
+
+    @Test
+    public void testBrPopLPushParser(){
+        String command = "BRPOPLPUSH source destination 500";
+        KVEntry builder = new BrPopLPushParser().parse(parseCommand(command));
+        Assert.assertEquals("BRPOPLPUSH", builder.getCommand());
+        Assert.assertEquals("source", builder.getKey());
+        Assert.assertEquals("destination", builder.getValue());
+        Assert.assertTrue(500L == builder.getParam(Options.REDIS_TIMEOUT));
+    }
+
+
+    @Test
+    public void testDecrByParser(){
+        String command = "DECRBY key 100";
+        KVEntry builder = new DecrByParser().parse(parseCommand(command));
+        Assert.assertEquals("DECRBY", builder.getCommand());
+        Assert.assertEquals("key", builder.getKey());
+        Assert.assertTrue(100L == builder.getParam(Options.REDIS_INCREMENT));
+    }
+
+    @Test
+    public void testDecrParser(){
+        String command = "DECR key";
+        KVEntry builder = new DecrParser().parse(parseCommand(command));
+        Assert.assertEquals("DECR", builder.getCommand());
+        Assert.assertEquals("key", builder.getKey());
+    }
+
+    @Test
+    public void testDelParser(){
+        String command = "del k1 k2 k3";
+        KVEntry builder = new DelParser().parse(parseCommand(command));
+        Assert.assertEquals("del", builder.getCommand());
+        Assert.assertEquals("k1", builder.getKey());
+        List<String> res = (List<String>)builder.getValue();
+        Assert.assertEquals(3, res.size());
+        Assert.assertEquals("k1", res.get(0));
+        Assert.assertEquals("k2", res.get(1));
+        Assert.assertEquals("k3", res.get(2));
+    }
+
+    @Test
+    public void testEvalParser(){
+        String command = "EVAL script 2 k1 k2 arg1 arg2";
+        KVEntry builder = new EvalParser().parse(parseCommand(command));
+        Assert.assertEquals("EVAL", builder.getCommand());
+        Assert.assertEquals("script", builder.getKey());
+        List<List<String>> res = (List<List<String>>)builder.getValue();
+        Assert.assertEquals(2, res.size());
+        Assert.assertEquals(2, res.get(0).size());
+        Assert.assertEquals(2, res.get(1).size());
+        Assert.assertEquals("k1", res.get(0).get(0));
+        Assert.assertEquals("k2", res.get(0).get(1));
+        Assert.assertEquals("arg1", res.get(1).get(0));
+        Assert.assertEquals("arg2", res.get(1).get(1));
+    }
+
+    @Test
+    public void testEvalShaParser(){
+        String command = "EVAL sha1 2 k1 k2 arg1 arg2";
+        KVEntry builder = new EvalShaParser().parse(parseCommand(command));
+        Assert.assertEquals("EVAL", builder.getCommand());
+        Assert.assertEquals("sha1", builder.getKey());
+        List<List<String>> res = (List<List<String>>)builder.getValue();
+        Assert.assertEquals(2, res.size());
+        Assert.assertEquals(2, res.get(0).size());
+        Assert.assertEquals(2, res.get(1).size());
+        Assert.assertEquals("k1", res.get(0).get(0));
+        Assert.assertEquals("k2", res.get(0).get(1));
+        Assert.assertEquals("arg1", res.get(1).get(0));
+        Assert.assertEquals("arg2", res.get(1).get(1));
+    }
+
+    @Test
+    public void testExecParser(){
+        String command = "exec";
+        KVEntry builder = new ExecParser().parse(parseCommand(command));
+        Assert.assertEquals("exec", builder.getCommand());
+    }
+
+    @Test
+    public void testExpireAtParser(){
+        String command = "expireat 500";
+        KVEntry builder = new ExpireAtParser().parse(parseCommand(command));
+        Assert.assertEquals("expireat", builder.getCommand());
+        Assert.assertEquals("500", builder.getKey());
+        Assert.assertTrue(500 == builder.getParam(Options.REDIS_EX_TIMESTAMP));
+    }
+
+    @Test
+    public void testExpireParser(){
+        String command = "expire key 500";
+        KVEntry builder = new ExpireParser().parse(parseCommand(command));
+        Assert.assertEquals("expire", builder.getCommand());
+        Assert.assertEquals("key", builder.getKey());
+        Assert.assertTrue(500L == builder.getParam(Options.EXPIRED_TIME));
+        Assert.assertEquals(ExpiredType.SECOND, builder.getParam(Options.EXPIRED_TYPE));
+    }
+
+    @Test
+    public void testFlushAllParser(){
+        String command = "FLUSHALL ASYNC";
+        KVEntry builder = new FlushAllParser().parse(parseCommand(command));
+        Assert.assertEquals("FLUSHALL", builder.getCommand());
+        Assert.assertEquals("ASYNC", builder.getKey());
+    }
+
+    @Test
+    public void testFlushDbParser(){
+        String command = "FLUSHDB ASYNC";
+        KVEntry builder = new FlushDbParser().parse(parseCommand(command));
+        Assert.assertEquals("FLUSHDB", builder.getCommand());
+        Assert.assertEquals("ASYNC", builder.getKey());
+    }
+
+    @Test
+    public void testGeoAddParser(){
+        String command = "GEOADD K1 100 20 M1 300 90 M2";
+        KVEntry builder = new GeoAddParser().parse(parseCommand(command));
+        Assert.assertEquals("GEOADD", builder.getCommand());
+        Assert.assertEquals("K1", builder.getKey());
+        List<Geo> res = (List<Geo>)builder.getValue();
+        Assert.assertEquals(2, res.size());
+        Assert.assertEquals(100, res.get(0).getLongitude());
+        Assert.assertEquals(20, res.get(0).getLatitude());
+        Assert.assertEquals("M1", res.get(0).getMember());
+        Assert.assertEquals(300, res.get(1).getLongitude());
+        Assert.assertEquals(90, res.get(1).getLatitude());
+        Assert.assertEquals("M2", res.get(1).getMember());
+
+    }
+
+    @Test
+    public void testGetsetParser(){
+        String command = "Getset key value";
+        KVEntry builder = new GetsetParser().parse(parseCommand(command));
+        Assert.assertEquals("Getset", builder.getCommand());
+        Assert.assertEquals("key", builder.getKey());
+        Assert.assertEquals("value", builder.getValue());
+    }
+
+    @Test
+    public void testHDelParser(){
+        String command = "HDEL key f1 f2 f3";
+        KVEntry builder = new HDelParser().parse(parseCommand(command));
+        Assert.assertEquals("HDEL", builder.getCommand());
+        Assert.assertEquals("key", builder.getKey());
+        List<String> res = (List<String>)builder.getValue();
+        Assert.assertEquals(3, res.size());
+        Assert.assertEquals("f1", res.get(0));
+        Assert.assertEquals("f2", res.get(1));
+        Assert.assertEquals("f3", res.get(2));
+    }
+
+    @Test
+    public void testHIncrByParser(){
+        String command = "HINCRBY key field 100";
+        KVEntry builder = new HIncrByParser().parse(parseCommand(command));
+        Assert.assertEquals("HINCRBY", builder.getCommand());
+        Assert.assertEquals("key", builder.getKey());
+        Assert.assertEquals("field", builder.getValue());
+        Assert.assertTrue(100L == builder.getParam(Options.REDIS_INCREMENT));
+    }
+
+    @Test
+    public void testHmSetParser(){
+        String command = "hmset key f1 v1 f2 v2";
+        KVEntry builder = new HmSetParser().parse(parseCommand(command));
+        Assert.assertEquals("hmset", builder.getCommand());
+        Assert.assertEquals("key", builder.getKey());
+        Map<String, String> res = (Map<String, String>)builder.getValue();
+        Assert.assertEquals(2, res.size());
+        Assert.assertEquals("v1", res.get("f1"));
+        Assert.assertEquals("v2", res.get("f2"));
+    }
+
+    @Test
+    public void testHSetNxParser(){
+        String command = "hsetnx key field value";
+        KVEntry builder = new HSetNxParser().parse(parseCommand(command));
+        Assert.assertEquals("hsetnx", builder.getCommand());
+        Assert.assertEquals("key", builder.getKey());
+        Map<String, String> res = (Map<String, String>)builder.getValue();
+        Assert.assertEquals(1, res.size());
+        Assert.assertEquals("value", res.get("field"));
+    }
+
+    @Test
+    public void testHSetParser(){
+        String command = "hset key field value";
+        KVEntry builder = new HSetParser().parse(parseCommand(command));
+        Assert.assertEquals("hset", builder.getCommand());
+        Assert.assertEquals("key", builder.getKey());
+        Map<String, String> res = (Map<String, String>)builder.getValue();
+        Assert.assertEquals(1, res.size());
+        Assert.assertEquals("value", res.get("field"));
+    }
+
+    @Test
+    public void testIncrByParser(){
+        String command = "INCRBY key 100";
+        KVEntry builder = new IncrByParser().parse(parseCommand(command));
+        Assert.assertEquals("INCRBY", builder.getCommand());
+        Assert.assertEquals("key", builder.getKey());
+        Assert.assertTrue(100L == builder.getParam(Options.REDIS_INCREMENT));
+    }
+
+    @Test
+    public void testIncrParser(){
+        String command = "INCR key";
+        KVEntry builder = new IncrParser().parse(parseCommand(command));
+        Assert.assertEquals("INCR", builder.getCommand());
+        Assert.assertEquals("key", builder.getKey());
+    }
+
+    @Test
+    public void testLinsertParserBefore(){
+        String command = "Linsert key BEFORE pivot value";
+        KVEntry builder = new LinsertParser().parse(parseCommand(command));
+        Assert.assertEquals("Linsert", builder.getCommand());
+        Assert.assertEquals("key", builder.getKey());
+        List<String> res = (List<String>)builder.getValue();
+        Assert.assertEquals(2, res.size());
+        Assert.assertEquals("pivot", res.get(0));
+        Assert.assertEquals("value", res.get(1));
+        Assert.assertEquals(true, builder.getParam(Options.REDIS_BEFORE));
+    }
+
+    @Test
+    public void testLinsertParserAfter(){
+        String command = "Linsert key AFTER pivot value";
+        KVEntry builder = new LinsertParser().parse(parseCommand(command));
+        Assert.assertEquals("Linsert", builder.getCommand());
+        Assert.assertEquals("key", builder.getKey());
+        List<String> res = (List<String>)builder.getValue();
+        Assert.assertEquals(2, res.size());
+        Assert.assertEquals("pivot", res.get(0));
+        Assert.assertEquals("value", res.get(1));
+        Assert.assertEquals(true, builder.getParam(Options.REDIS_AFTER));
+    }
+
+    @Test
+    public void testLPopParser(){
+        String command = "LPOP key";
+        KVEntry builder = new LPopParser().parse(parseCommand(command));
+        Assert.assertEquals("LPOP", builder.getCommand());
+        Assert.assertEquals("key", builder.getKey());
+    }
+
+    @Test
+    public void testLPushParser(){
+        String command = "LPUSH key v1 v2";
+        KVEntry builder = new LPushParser().parse(parseCommand(command));
+        Assert.assertEquals("LPUSH", builder.getCommand());
+        Assert.assertEquals("key", builder.getKey());
+        List<String> res = (List<String>)builder.getValue();
+        Assert.assertEquals(2, res.size());
+        Assert.assertEquals("v1", res.get(0));
+        Assert.assertEquals("v2", res.get(1));
+    }
+
+    @Test
+    public void testLPushXParser(){
+        String command = "LPUSHX key value";
+        KVEntry builder = new LPushXParser().parse(parseCommand(command));
+        Assert.assertEquals("LPUSHX", builder.getCommand());
+        Assert.assertEquals("key", builder.getKey());
+        Assert.assertEquals("value", builder.getValue());
+    }
+
+    @Test
+    public void testLRemParser(){
+        String command = "LRem key 100 value";
+        KVEntry builder = new LRemParser().parse(parseCommand(command));
+        Assert.assertEquals("LRem", builder.getCommand());
+        Assert.assertEquals("key", builder.getKey());
+        Assert.assertEquals("value", builder.getValue());
+        Assert.assertTrue(100L == builder.getParam(Options.REDIS_COUNT));
+    }
+
+    @Test
+    public void testLSetParser(){
+        String command = "LSet key 10 value";
+        KVEntry builder = new LSetParser().parse(parseCommand(command));
+        Assert.assertEquals("LSet", builder.getCommand());
+        Assert.assertEquals("key", builder.getKey());
+        Assert.assertEquals("value", builder.getValue());
+        Assert.assertTrue(10L == builder.getParam(Options.REDIS_INDEX));
+    }
+
+    @Test
+    public void testLTrimParser(){
+        String command = "LTrim key 0 -1";
+        KVEntry builder = new LTrimParser().parse(parseCommand(command));
+        Assert.assertEquals("LTrim", builder.getCommand());
+        Assert.assertEquals("key", builder.getKey());
+        List<String> res = (List<String>)builder.getValue();
+        Assert.assertEquals(2, res.size());
+        Assert.assertEquals("0", res.get(0));
+        Assert.assertEquals("-1", res.get(1));
+    }
+
+    @Test
+    public void testMoveParser(){
+        String command = "move key 1";
+        KVEntry builder = new MoveParser().parse(parseCommand(command));
+        Assert.assertEquals("move", builder.getCommand());
+        Assert.assertEquals("key", builder.getKey());
+        Assert.assertTrue(1 == builder.getParam(Options.REDIS_DB_INDEX));
+    }
+
+    @Test
+    public void testMSetNxParser(){
+        String command = "msetnx k1 v1 k2 v2";
+        KVEntry builder = new MSetNxParser().parse(parseCommand(command));
+        Assert.assertEquals("msetnx", builder.getCommand());
+        Assert.assertEquals("k1", builder.getKey());
+        Map<String, String> res = (Map<String, String>)builder.getValue();
+        Assert.assertEquals("v1", res.get("k1"));
+        Assert.assertEquals("v2", res.get("k2"));
+    }
+
+    @Test
+    public void testMSetParser(){
+        String command = "mset k1 v1 k2 v2";
+        KVEntry builder = new MSetParser().parse(parseCommand(command));
+        Assert.assertEquals("mset", builder.getCommand());
+        Assert.assertEquals("k1", builder.getKey());
+        Map<String, String> res = (Map<String, String>)builder.getValue();
+        Assert.assertEquals("v1", res.get("k1"));
+        Assert.assertEquals("v2", res.get("k2"));
+    }
+
+    @Test
+    public void testMultiParser(){
+        String command = "Multi";
+        KVEntry builder = new MultiParser().parse(parseCommand(command));
+        Assert.assertEquals("Multi", builder.getCommand());
+    }
+
+    @Test
+    public void testPersistParser(){
+        String command = "PERSIST key";
+        KVEntry builder = new PersistParser().parse(parseCommand(command));
+        Assert.assertEquals("PERSIST", builder.getCommand());
+        Assert.assertEquals("key", builder.getKey());
+    }
+
+    @Test
+    public void testPExpireAtParser(){
+        String command = "PExpireAt key 3600";
+        KVEntry builder = new PExpireAtParser().parse(parseCommand(command));
+        Assert.assertEquals("PExpireAt", builder.getCommand());
+        Assert.assertEquals("key", builder.getKey());
+        Assert.assertTrue(3600L == builder.getParam(Options.REDIS_PX_TIMESTAMP));
+    }
+
+    @Test
+    public void testPExpireParser(){
+        String command = "PExpire key 3600";
+        KVEntry builder = new PExpireParser().parse(parseCommand(command));
+        Assert.assertEquals("PExpire", builder.getCommand());
+        Assert.assertEquals("key", builder.getKey());
+        Assert.assertTrue(3600L == builder.getParam(Options.REDIS_PX));
+    }
+
+    @Test
+    public void testPfAddParser(){
+        String command = "PFADD key e1 e2";
+        KVEntry builder = new PfAddParser().parse(parseCommand(command));
+        Assert.assertEquals("PFADD", builder.getCommand());
+        Assert.assertEquals("key", builder.getKey());
+        List<String> res = (List<String>)builder.getValue();
+        Assert.assertEquals(2, res.size());
+        Assert.assertEquals("e1", res.get(0));
+        Assert.assertEquals("e2", res.get(1));
+    }
+
+    @Test
+    public void testPfCountParser(){
+        String command = "PFCOUNT k1 k2";
+        KVEntry builder = new PfCountParser().parse(parseCommand(command));
+        Assert.assertEquals("PFCOUNT", builder.getCommand());
+        Assert.assertEquals("k1", builder.getKey());
+        List<String> res = (List<String>)builder.getValue();
+        Assert.assertEquals(2, res.size());
+        Assert.assertEquals("k1", res.get(0));
+        Assert.assertEquals("k2", res.get(1));
+    }
+
+    @Test
+    public void testPfMergeParser(){
+        String command = "PFMERGE destkey sourcekey1 sourcekey2";
+        KVEntry builder = new PfMergeParser().parse(parseCommand(command));
+        Assert.assertEquals("PFMERGE", builder.getCommand());
+        Assert.assertEquals("destkey", builder.getKey());
+        List<String> res = (List<String>)builder.getValue();
+        Assert.assertEquals(2, res.size());
+        Assert.assertEquals("sourcekey1", res.get(0));
+        Assert.assertEquals("sourcekey2", res.get(1));
+    }
+
+    @Test
+    public void testPSetExParser(){
+        String command = "PSETEX key 1000 value";
+        KVEntry builder = new PSetExParser().parse(parseCommand(command));
+        Assert.assertEquals("PSETEX", builder.getCommand());
+        Assert.assertEquals("key", builder.getKey());
+        Assert.assertEquals("value", builder.getValue());
+        Assert.assertTrue(1000L == builder.getParam(Options.REDIS_PX));
+    }
+
+    @Test
+    public void testPublishParser(){
+        String command = "Publish channel message";
+        KVEntry builder = new PublishParser().parse(parseCommand(command));
+        Assert.assertEquals("Publish", builder.getCommand());
+        Assert.assertEquals("channel", builder.getKey());
+        Assert.assertEquals("message", builder.getValue());
+    }
+
+    @Test
+    public void testRenameNxParser(){
+        String command = "RENAMENX key newkey";
+        KVEntry builder = new RenameNxParser().parse(parseCommand(command));
+        Assert.assertEquals("RENAMENX", builder.getCommand());
+        Assert.assertEquals("key", builder.getKey());
+        Assert.assertEquals("newkey", builder.getValue());
+    }
+
+    @Test
+    public void testRenameParser(){
+        String command = "RENAME key newkey";
+        KVEntry builder = new RenameParser().parse(parseCommand(command));
+        Assert.assertEquals("RENAME", builder.getCommand());
+        Assert.assertEquals("key", builder.getKey());
+        Assert.assertEquals("newkey", builder.getValue());
+    }
+
+    @Test
+    public void testRestoreParser(){
+        String command = "Restore key 300 serialized-value REPLACE";
+        KVEntry builder = new RestoreParser().parse(parseCommand(command));
+        Assert.assertEquals("Restore", builder.getCommand());
+        Assert.assertEquals("key", builder.getKey());
+        Assert.assertEquals("serialized-value", builder.getValue());
+        Assert.assertTrue(300L == builder.getParam(Options.REDIS_TTL));
+        Assert.assertEquals(true, builder.getParam(Options.REDIS_REPLACE));
+    }
+
+    @Test
+    public void testRPopLPushParser(){
+        String command = "RPOPLPUSH source destination";
+        KVEntry builder = new RPopLPushParser().parse(parseCommand(command));
+        Assert.assertEquals("RPOPLPUSH", builder.getCommand());
+        Assert.assertEquals("source", builder.getKey());
+        Assert.assertEquals("destination", builder.getValue());
+    }
+
+    @Test
+    public void testRPopParser(){
+        String command = "RPOP key";
+        KVEntry builder = new RPopParser().parse(parseCommand(command));
+        Assert.assertEquals("RPOP", builder.getCommand());
+        Assert.assertEquals("key", builder.getKey());
+    }
+
+    @Test
+    public void testRPushParser(){
+        String command = "RPUSH key v1 v2";
+        KVEntry builder = new RPushParser().parse(parseCommand(command));
+        Assert.assertEquals("RPUSH", builder.getCommand());
+        Assert.assertEquals("key", builder.getKey());
+        List<String> res = (List<String>)builder.getValue();
+        Assert.assertEquals(2, res.size());
+        Assert.assertEquals("v1", res.get(0));
+        Assert.assertEquals("v2", res.get(1));
+    }
+
+    @Test
+    public void testRPushXParser(){
+        String command = "RPUSHX key value";
+        KVEntry builder = new RPushXParser().parse(parseCommand(command));
+        Assert.assertEquals("RPUSHX", builder.getCommand());
+        Assert.assertEquals("key", builder.getKey());
+        Assert.assertEquals("value", builder.getValue());
+    }
+
+    @Test
+    public void testSAddParser(){
+        String command = "SADD key m1 m2";
+        KVEntry builder = new SAddParser().parse(parseCommand(command));
+        Assert.assertEquals("SADD", builder.getCommand());
+        Assert.assertEquals("key", builder.getKey());
+        List<String> res = (List<String>)builder.getValue();
+        Assert.assertEquals(2, res.size());
+        Assert.assertEquals("m1", res.get(0));
+        Assert.assertEquals("m2", res.get(1));
+    }
+
+    @Test
+    public void testScriptParser(){
+        String command = "Script";
+        KVEntry builder = new ScriptParser().parse(parseCommand(command));
+        Assert.assertEquals("Script", builder.getCommand());
+    }
+
+    @Test
+    public void testSDiffStoreParser(){
+        String command = "SDIFFSTORE destination k1 k2";
+        KVEntry builder = new SDiffStoreParser().parse(parseCommand(command));
+        Assert.assertEquals("SDIFFSTORE", builder.getCommand());
+        Assert.assertEquals("destination", builder.getKey());
+        List<String> res = (List<String>)builder.getValue();
+        Assert.assertEquals(2, res.size());
+        Assert.assertEquals("k1", res.get(0));
+        Assert.assertEquals("k2", res.get(1));
+    }
+
+    @Test
+    public void testSelectParser(){
+        String command = "Select 1";
+        KVEntry builder = new SelectParser().parse(parseCommand(command));
+        Assert.assertEquals("Select", builder.getCommand());
+        Assert.assertEquals("1", builder.getKey());
+        Assert.assertTrue(1 == builder.getParam(Options.REDIS_DB_INDEX));
+    }
+
+    @Test
+    public void testSetBitParser(){
+        String command = "SetBit key 100 value";
+        KVEntry builder = new SetBitParser().parse(parseCommand(command));
+        Assert.assertEquals("SetBit", builder.getCommand());
+        Assert.assertEquals("key", builder.getKey());
+        Assert.assertEquals("value", builder.getValue());
+        Assert.assertTrue(100L == builder.getParam(Options.REDIS_OFFSET));
+    }
+
+    @Test
+    public void testSetExParser(){
+        String command = "setex key 100 value";
+        KVEntry builder = new SetExParser().parse(parseCommand(command));
+        Assert.assertEquals("setex", builder.getCommand());
+        Assert.assertEquals("key", builder.getKey());
+        Assert.assertEquals("value", builder.getValue());
+        Assert.assertTrue(100 == builder.getParam(Options.REDIS_EX));
+    }
+
+    @Test
+    public void testSetNxParser(){
+        String command = "SETNX a b";
+        KVEntry builder = new SetNxParser().parse(parseCommand(command));
+        Assert.assertEquals("SETNX", builder.getCommand());
+        Assert.assertEquals("a", builder.getKey());
+        Assert.assertEquals("b", builder.getValue());
+    }
+
+    @Test
+    public void testSetParser1(){
+        String command = "set key value EX 100 NX";
+        KVEntry builder = new SetParser().parse(parseCommand(command));
+        Assert.assertEquals("set", builder.getCommand());
+        Assert.assertEquals("key", builder.getKey());
+        Assert.assertEquals("value", builder.getValue());
+        Assert.assertTrue(100 == builder.getParam(Options.REDIS_EX));
+        Assert.assertEquals(true, builder.getParam(Options.REDIS_NX));
+    }
+
+    @Test
+    public void testSetParser2(){
+        String command = "set key value EX 100";
+        KVEntry builder = new SetParser().parse(parseCommand(command));
+        Assert.assertEquals("set", builder.getCommand());
+        Assert.assertEquals("key", builder.getKey());
+        Assert.assertEquals("value", builder.getValue());
+        Assert.assertTrue(100 == builder.getParam(Options.REDIS_EX));
+    }
+
+    @Test
+    public void testSetParser3(){
+        String command = "set key value NX";
+        KVEntry builder = new SetParser().parse(parseCommand(command));
+        Assert.assertEquals("set", builder.getCommand());
+        Assert.assertEquals("key", builder.getKey());
+        Assert.assertEquals("value", builder.getValue());
+        Assert.assertEquals(true, builder.getParam(Options.REDIS_NX));
+    }
+
+    @Test
+    public void testSetParser4(){
+        String command = "set key value PX 100 XX";
+        KVEntry builder = new SetParser().parse(parseCommand(command));
+        Assert.assertEquals("set", builder.getCommand());
+        Assert.assertEquals("key", builder.getKey());
+        Assert.assertEquals("value", builder.getValue());
+        Assert.assertTrue(100L == builder.getParam(Options.REDIS_PX));
+        Assert.assertEquals(true, builder.getParam(Options.REDIS_XX));
+    }
+
+    @Test
+    public void testSetRangeParser(){
+        String command = "SETRANGE key 100 value";
+        KVEntry builder = new SetRangeParser().parse(parseCommand(command));
+        Assert.assertEquals("SETRANGE", builder.getCommand());
+        Assert.assertEquals("key", builder.getKey());
+        Assert.assertEquals("value", builder.getValue());
+        Assert.assertTrue(100L == builder.getParam(Options.REDIS_OFFSET));
+    }
+
+    @Test
+    public void testSInterStoreParser(){
+        String command = "SINTERSTORE destination k1 k2";
+        KVEntry builder = new SInterStoreParser().parse(parseCommand(command));
+        Assert.assertEquals("SINTERSTORE", builder.getCommand());
+        Assert.assertEquals("destination", builder.getKey());
+        List<String> res = (List<String>)builder.getValue();
+        Assert.assertEquals(2, res.size());
+        Assert.assertEquals("k1", res.get(0));
+        Assert.assertEquals("k2", res.get(1));
+    }
+
+    @Test
+    public void testSMoveParser(){
+        String command = "SMOVE source destination member";
+        KVEntry builder = new SMoveParser().parse(parseCommand(command));
+        Assert.assertEquals("SMOVE", builder.getCommand());
+        Assert.assertEquals("source", builder.getKey());
+        List<String> res = (List<String>)builder.getValue();
+        Assert.assertEquals(2, res.size());
+        Assert.assertEquals("destination", res.get(0));
+        Assert.assertEquals("member", res.get(1));
+    }
+
+    @Test
+    public void testSortParser(){
+        String command = "sort key";
+        KVEntry builder = new SortParser().parse(parseCommand(command));
+        Assert.assertEquals("sort", builder.getCommand());
+        Assert.assertEquals("key", builder.getKey());
+    }
+
+    @Test
+    public void testSRemParser(){
+        String command = "SREM key m1 m2";
+        KVEntry builder = new SRemParser().parse(parseCommand(command));
+        Assert.assertEquals("SREM", builder.getCommand());
+        Assert.assertEquals("key", builder.getKey());
+        List<String> res = (List<String>)builder.getValue();
+        Assert.assertEquals(2, res.size());
+        Assert.assertEquals("m1", res.get(0));
+        Assert.assertEquals("m2", res.get(1));
+    }
+
+    @Test
+    public void testSUnionStoreParser(){
+        String command = "SUNIONSTORE destination k1 k2";
+        KVEntry builder = new SUnionStoreParser().parse(parseCommand(command));
+        Assert.assertEquals("SUNIONSTORE", builder.getCommand());
+        Assert.assertEquals("destination", builder.getKey());
+        List<String> res = (List<String>)builder.getValue();
+        Assert.assertEquals(2, res.size());
+        Assert.assertEquals("k1", res.get(0));
+        Assert.assertEquals("k2", res.get(1));
+    }
+
+    @Test
+    public void testSwapDbParser(){
+        String command = "SWAPDB 0 1";
+        KVEntry builder = new SwapDbParser().parse(parseCommand(command));
+        Assert.assertEquals("SWAPDB", builder.getCommand());
+        Assert.assertEquals("0", builder.getKey());
+        Assert.assertEquals("1", builder.getValue());
+    }
+
+    @Test
+    public void testUnLinkParser(){
+        String command = "UnLink k1 k2";
+        KVEntry builder = new UnLinkParser().parse(parseCommand(command));
+        Assert.assertEquals("UnLink", builder.getCommand());
+        Assert.assertEquals("k1", builder.getKey());
+        List<String> res = (List<String>)builder.getValue();
+        Assert.assertEquals(2, res.size());
+        Assert.assertEquals("k1", res.get(0));
+        Assert.assertEquals("k2", res.get(1));
+    }
+
+    @Test
+    public void testXAckParser(){
+        String command = "XAck key group 1526569498055-0 1526569498055-1";
+        KVEntry builder = new XAckParser().parse(parseCommand(command));
+        Assert.assertEquals("XAck", builder.getCommand());
+        Assert.assertEquals("key", builder.getKey());
+        Assert.assertEquals("group", builder.getParam(Options.REDIS_GROUP));
+        List<String> res = (List<String>)builder.getValue();
+        Assert.assertEquals(2, res.size());
+        Assert.assertEquals("1526569498055-0", res.get(0));
+        Assert.assertEquals("1526569498055-1", res.get(1));
+    }
+
+    @Test
+    public void testXAddParser(){
+        String command = "XAdd key ID f1 v1 f2 v2";
+        KVEntry builder = new XAddParser().parse(parseCommand(command));
+        Assert.assertEquals("XAdd", builder.getCommand());
+        Assert.assertEquals("key", builder.getKey());
+        Assert.assertEquals("ID", builder.getParam(Options.REDIS_ID));
+        Map<String, String> res = (Map<String, String>)builder.getValue();
+        Assert.assertEquals(2, res.size());
+        Assert.assertEquals("v1", res.get("f1"));
+        Assert.assertEquals("v2", res.get("f2"));
+    }
+
+    @Test
+    public void testXClaimParser1(){
+        // XCLAIM key group consumer min-idle-time ID [ID ...] [IDLE ms] [TIME ms-unix-time] [RETRYCOUNT count] [force]
+        String command = "XCLAIM mystream mygroup Alice 3600000 1526569498055-0 1526569498055-1";
+        KVEntry builder = new XClaimParser().parse(parseCommand(command));
+        Assert.assertEquals("XCLAIM", builder.getCommand());
+        Assert.assertEquals("mystream", builder.getKey());
+        Assert.assertEquals("mygroup", builder.getParam(Options.REDIS_GROUP));
+        Assert.assertEquals("Alice", builder.getParam(Options.REDIS_CONSUMER));
+        Assert.assertTrue(3600000L == builder.getParam(Options.REDIS_MIN_IDLE_TIME));
+        List<String> res = (List<String>)builder.getValue();
+        Assert.assertEquals(2, res.size());
+        Assert.assertEquals("1526569498055-0", res.get(0));
+        Assert.assertEquals("1526569498055-1", res.get(1));
+    }
+
+    @Test
+    public void testXClaimParser2(){
+        // XCLAIM key group consumer min-idle-time ID [ID ...] [IDLE ms] [TIME ms-unix-time] [RETRYCOUNT count] [force]
+        String command = "XCLAIM mystream mygroup Alice 3600000 1526569498055-0 1526569498055-1 IDLE 100 TIME 3000 RETRYCOUNT 10 force";
+        KVEntry builder = new XClaimParser().parse(parseCommand(command));
+        Assert.assertEquals("XCLAIM", builder.getCommand());
+        Assert.assertEquals("mystream", builder.getKey());
+        Assert.assertEquals("mygroup", builder.getParam(Options.REDIS_GROUP));
+        Assert.assertEquals("Alice", builder.getParam(Options.REDIS_CONSUMER));
+        Assert.assertTrue(3600000L == builder.getParam(Options.REDIS_MIN_IDLE_TIME));
+        Assert.assertTrue(100L == builder.getParam(Options.REDIS_IDLE));
+        Assert.assertTrue(3000L == builder.getParam(Options.REDIS_TIME));
+        Assert.assertTrue(10 == builder.getParam(Options.REDIS_RETRYCOUNT));
+        Assert.assertEquals(true, builder.getParam(Options.REDIS_FORCE));
+        List<String> res = (List<String>)builder.getValue();
+        Assert.assertEquals(2, res.size());
+        Assert.assertEquals("1526569498055-0", res.get(0));
+        Assert.assertEquals("1526569498055-1", res.get(1));
+    }
+
+    @Test
+    public void testXDelParser(){
+        String command = "XDel key 1526569498055-0 1526569498055-1";
+        KVEntry builder = new XDelParser().parse(parseCommand(command));
+        Assert.assertEquals("XDel", builder.getCommand());
+        Assert.assertEquals("key", builder.getKey());
+        List<String> res = (List<String>)builder.getValue();
+        Assert.assertEquals(2, res.size());
+        Assert.assertEquals("1526569498055-0", res.get(0));
+        Assert.assertEquals("1526569498055-1", res.get(1));
+    }
+
+    @Test
+    public void testXGroupParser(){
+        String command = "XGroup [CREATE key groupname id-or-$] [SETID key id-or-$] [DESTROY key groupname] [DELCONSUMER key groupname consumername]";
+        KVEntry builder = new XGroupParser().parse(parseCommand(command));
+        Assert.assertEquals("XGroup", builder.getCommand());
+    }
+
+    @Test
+    public void testXSetIdParser(){
+        String command = "XSetId key arg";
+        KVEntry builder = new XSetIdParser().parse(parseCommand(command));
+        Assert.assertEquals("XSetId", builder.getCommand());
+        Assert.assertEquals("key", builder.getKey());
+        Assert.assertEquals("arg", builder.getValue());
+    }
+
+    @Test
+    public void testXTrimParser(){
+        String command = "XTrim key MAXLEN ~ 100";
+        KVEntry builder = new XTrimParser().parse(parseCommand(command));
+        Assert.assertEquals("XTrim", builder.getCommand());
+        Assert.assertEquals("key", builder.getKey());
+        Assert.assertEquals(true, builder.getParam(Options.REDIS_XTRIM));
+    }
+
+    @Test
+    public void testXTrimParser2(){
+        String command = "XTrim key MAXLEN 100";
+        KVEntry builder = new XTrimParser().parse(parseCommand(command));
+        Assert.assertEquals("XTrim", builder.getCommand());
+        Assert.assertEquals("key", builder.getKey());
+        Assert.assertEquals(false, builder.getParam(Options.REDIS_XTRIM));
+    }
+
+    @Test
+    public void testZAddParser1(){
+        // ZADD key [NX|XX] [CH] [INCR] score member [score member ...]
+        String command = "ZADD key 100 m1 80 m2";
+        KVEntry builder = new ZAddParser().parse(parseCommand(command));
+        Assert.assertEquals("ZADD", builder.getCommand());
+        Assert.assertEquals("key", builder.getKey());
+        Map<String, String> res = (Map<String, String>)builder.getValue();
+        Assert.assertEquals(2, res.size());
+        Assert.assertEquals("100", res.get("m1"));
+        Assert.assertEquals("80", res.get("m2"));
+    }
+
+    @Test
+    public void testZAddParser2(){
+        // ZADD key [NX|XX] [CH] [INCR] score member [score member ...]
+        String command = "ZADD key NX CH INCR 100 m1 80 m2";
+        KVEntry builder = new ZAddParser().parse(parseCommand(command));
+        Assert.assertEquals("ZADD", builder.getCommand());
+        Assert.assertEquals("key", builder.getKey());
+        Map<String, String> res = (Map<String, String>)builder.getValue();
+        Assert.assertEquals(2, res.size());
+        Assert.assertEquals("100", res.get("m1"));
+        Assert.assertEquals("80", res.get("m2"));
+        Assert.assertEquals(true, builder.getParam(Options.REDIS_NX));
+        Assert.assertEquals(true, builder.getParam(Options.REDIS_CH));
+        Assert.assertEquals(true, builder.getParam(Options.REDIS_INCR));
+    }
+
+    @Test
+    public void testZAddParser3(){
+        // ZADD key [NX|XX] [CH] [INCR] score member [score member ...]
+        String command = "ZADD key XX CH INCR 100 m1 80 m2";
+        KVEntry builder = new ZAddParser().parse(parseCommand(command));
+        Assert.assertEquals("ZADD", builder.getCommand());
+        Assert.assertEquals("key", builder.getKey());
+        Map<String, String> res = (Map<String, String>)builder.getValue();
+        Assert.assertEquals(2, res.size());
+        Assert.assertEquals("100", res.get("m1"));
+        Assert.assertEquals("80", res.get("m2"));
+        Assert.assertEquals(true, builder.getParam(Options.REDIS_XX));
+        Assert.assertEquals(true, builder.getParam(Options.REDIS_CH));
+        Assert.assertEquals(true, builder.getParam(Options.REDIS_INCR));
+    }
+
+    @Test
+    public void testZIncrByParser(){
+        String command = "ZINCRBY key 100 member";
+        KVEntry builder = new ZIncrByParser().parse(parseCommand(command));
+        Assert.assertEquals("ZINCRBY", builder.getCommand());
+        Assert.assertEquals("key", builder.getKey());
+        Assert.assertEquals("member", builder.getValue());
+        Assert.assertTrue(100L == builder.getParam(Options.REDIS_INCREMENT));
+    }
+
+    @Test
+    public void testZInterStoreParser1(){
+        // ZINTERSTORE destination numkeys key [key ...] [weights weight] [aggregate SUM|MIN|MAX]
+        String command = "ZINTERSTORE destination 2 k1 k2 weights 100 80";
+        KVEntry builder = new ZInterStoreParser().parse(parseCommand(command));
+        Assert.assertEquals("ZINTERSTORE", builder.getCommand());
+        Assert.assertEquals("destination", builder.getKey());
+        Map<String,String> res = (Map<String, String>)builder.getValue();
+        Assert.assertEquals(2, res.size());
+        Assert.assertEquals("100", res.get("k1"));
+        Assert.assertEquals("80", res.get("k2"));
+    }
+
+    @Test
+    public void testZInterStoreParser2(){
+        // ZINTERSTORE destination numkeys key [key ...] [weights weight] [aggregate SUM|MIN|MAX]
+        String command = "ZINTERSTORE destination 2 k1 k2 weights 100 80 aggregate SUM";
+        KVEntry builder = new ZInterStoreParser().parse(parseCommand(command));
+        Assert.assertEquals("ZINTERSTORE", builder.getCommand());
+        Assert.assertEquals("destination", builder.getKey());
+        Map<String,String> res = (Map<String, String>)builder.getValue();
+        Assert.assertEquals(2, res.size());
+        Assert.assertEquals("100", res.get("k1"));
+        Assert.assertEquals("80", res.get("k2"));
+        Assert.assertEquals("SUM", builder.getParam(Options.REDIS_AGGREGATE));
+    }
+
+    @Test
+    public void testZPopMaxParser(){
+        String command = "ZPOPMAX key 100";
+        KVEntry builder = new ZPopMaxParser().parse(parseCommand(command));
+        Assert.assertEquals("ZPOPMAX", builder.getCommand());
+        Assert.assertEquals("key", builder.getKey());
+        Assert.assertTrue(100L == builder.getParam(Options.REDIS_COUNT));
+    }
+
+    @Test
+    public void testZPopMinParser(){
+        String command = "ZPOPMIN key 100";
+        KVEntry builder = new ZPopMinParser().parse(parseCommand(command));
+        Assert.assertEquals("ZPOPMIN", builder.getCommand());
+        Assert.assertEquals("key", builder.getKey());
+        Assert.assertTrue(100L == builder.getParam(Options.REDIS_COUNT));
+    }
+
+    @Test
+    public void testZRemParser(){
+        String command = "ZRem key m1 m2";
+        KVEntry builder = new ZRemParser().parse(parseCommand(command));
+        Assert.assertEquals("ZRem", builder.getCommand());
+        Assert.assertEquals("key", builder.getKey());
+        List<String> res = (List<String>)builder.getValue();
+        Assert.assertEquals(2, res.size());
+        Assert.assertEquals("m1", res.get(0));
+        Assert.assertEquals("m2", res.get(1));
+    }
+
+
+    @Test
+    public void testZRemRangeByLexParser(){
+        String command = "ZRemRangeByLex key 0 10";
+        KVEntry builder = new ZRemRangeByLexParser().parse(parseCommand(command));
+        Assert.assertEquals("ZRemRangeByLex", builder.getCommand());
+        Assert.assertEquals("key", builder.getKey());
+        List<String> res = (List<String>)builder.getValue();
+        Assert.assertEquals(2, res.size());
+        Assert.assertEquals("0", res.get(0));
+        Assert.assertEquals("10", res.get(1));
+    }
+
+    @Test
+    public void testZRemRangeByRankParser(){
+        String command = "ZREMRANGEBYRANK key 0 10";
+        KVEntry builder = new ZRemRangeByRankParser().parse(parseCommand(command));
+        Assert.assertEquals("ZREMRANGEBYRANK", builder.getCommand());
+        Assert.assertEquals("key", builder.getKey());
+        List<String> res = (List<String>)builder.getValue();
+        Assert.assertEquals(2, res.size());
+        Assert.assertEquals("0", res.get(0));
+        Assert.assertEquals("10", res.get(1));
+    }
+
+    @Test
+    public void testZRemRangeByScoreParser(){
+        String command = "ZREMRANGEBYSCORE key 0 10";
+        KVEntry builder = new ZRemRangeByScoreParser().parse(parseCommand(command));
+        Assert.assertEquals("ZREMRANGEBYSCORE", builder.getCommand());
+        Assert.assertEquals("key", builder.getKey());
+        List<String> res = (List<String>)builder.getValue();
+        Assert.assertEquals(2, res.size());
+        Assert.assertEquals("0", res.get(0));
+        Assert.assertEquals("10", res.get(1));
+    }
+
+    @Test
+    public void testZUnionStoreParser1(){
+        // ZUNIONSTORE destination numkeys key [key ...] [weights weight] [aggregate SUM|MIN|MAX]
+        String command = "ZUNIONSTORE destination 2 k1 k2 weights 100 80";
+        KVEntry builder = new ZUnionStoreParser().parse(parseCommand(command));
+        Assert.assertEquals("ZUNIONSTORE", builder.getCommand());
+        Assert.assertEquals("destination", builder.getKey());
+        Map<String,String> res = (Map<String, String>)builder.getValue();
+        Assert.assertEquals(2, res.size());
+        Assert.assertEquals("100", res.get("k1"));
+        Assert.assertEquals("80", res.get("k2"));
+    }
+
+    @Test
+    public void testZUnionStoreParser2(){
+        // ZUNIONSTORE destination numkeys key [key ...] [weights weight] [aggregate SUM|MIN|MAX]
+        String command = "ZUNIONSTORE destination 2 k1 k2 weights 100 80 aggregate SUM";
+        KVEntry builder = new ZUnionStoreParser().parse(parseCommand(command));
+        Assert.assertEquals("ZUNIONSTORE", builder.getCommand());
+        Assert.assertEquals("destination", builder.getKey());
+        Map<String,String> res = (Map<String, String>)builder.getValue();
+        Assert.assertEquals(2, res.size());
+        Assert.assertEquals("100", res.get("k1"));
+        Assert.assertEquals("80", res.get("k2"));
+        Assert.assertEquals("SUM", builder.getParam(Options.REDIS_AGGREGATE));
+    }
+
+}
diff --git a/src/test/java/org/apache/rocketmq/redis/test/pojo/GeoTest.java b/src/test/java/org/apache/rocketmq/redis/test/pojo/GeoTest.java
new file mode 100644
index 0000000..c905140
--- /dev/null
+++ b/src/test/java/org/apache/rocketmq/redis/test/pojo/GeoTest.java
@@ -0,0 +1,19 @@
+package org.apache.rocketmq.redis.test.pojo;
+
+import org.apache.rocketmq.connect.redis.pojo.Geo;
+import org.junit.Assert;
+import org.junit.Test;
+
+public class GeoTest {
+    @Test
+    public void test(){
+        Geo geo = new Geo();
+        geo.setLatitude(1L);
+        geo.setLongitude(2L);
+        geo.setMember("hello");
+
+        Assert.assertTrue(1L == geo.getLatitude());
+        Assert.assertTrue(2L == geo.getLongitude());
+        Assert.assertEquals("hello", geo.getMember());
+    }
+}
diff --git a/src/test/java/org/apache/rocketmq/redis/test/pojo/KVEntryTest.java b/src/test/java/org/apache/rocketmq/redis/test/pojo/KVEntryTest.java
new file mode 100644
index 0000000..65d8bcf
--- /dev/null
+++ b/src/test/java/org/apache/rocketmq/redis/test/pojo/KVEntryTest.java
@@ -0,0 +1,85 @@
+package org.apache.rocketmq.redis.test.pojo;
+
+import java.util.ArrayList;
+import java.util.HashMap;
+import java.util.List;
+import java.util.Map;
+
+import io.openmessaging.connector.api.data.DataEntry;
+import io.openmessaging.connector.api.data.EntryType;
+import io.openmessaging.connector.api.data.FieldType;
+import io.openmessaging.connector.api.data.SourceDataEntry;
+import org.apache.rocketmq.connect.redis.common.Options;
+import org.apache.rocketmq.connect.redis.converter.KVEntryConverter;
+import org.apache.rocketmq.connect.redis.converter.RedisEntryConverter;
+import org.apache.rocketmq.connect.redis.pojo.KVEntry;
+import org.apache.rocketmq.connect.redis.pojo.RedisEntry;
+import org.junit.Assert;
+import org.junit.Test;
+
+public class KVEntryTest {
+
+    @Test
+    public void testConstruct(){
+        KVEntry entry = new RedisEntry(FieldType.STRING);
+        entry.value("value");
+        Assert.assertEquals(String.class, entry.getValue().getClass());
+
+
+        entry = new RedisEntry("partition", FieldType.ARRAY);
+        entry.value(new ArrayList<>());
+        Assert.assertEquals(ArrayList.class, entry.getValue().getClass());
+        Assert.assertEquals("partition", entry.getPartition());
+
+
+        entry = RedisEntry.newEntry(FieldType.MAP);
+        entry.value(new HashMap());
+        Assert.assertEquals(HashMap.class, entry.getValue().getClass());
+        entry = RedisEntry.newEntry("partition", FieldType.INT64);
+
+
+        entry.value(123L);
+        Assert.assertEquals(Long.class, entry.getValue().getClass());
+
+
+        Assert.assertTrue(123L == (long)entry.getValue());
+        Assert.assertEquals("partition", entry.getPartition());
+        Assert.assertNotNull(entry.toString());
+
+
+        List<SourceDataEntry> entries = getConverter().kVEntryToDataEntries(entry);
+        Assert.assertNotNull(entries);
+    }
+
+    @Test
+    public void test(){
+        RedisEntry entry = RedisEntry.newEntry(FieldType.STRING);
+        entry.partition("partition");
+        entry.queueName("queue1");
+        entry.entryType(EntryType.UPDATE);
+        entry.sourceId("replId");
+        entry.offset(65535L);
+        entry.command("set");
+        entry.key("key");
+        entry.valueType(FieldType.STRING);
+        entry.value("value");
+        entry.param(Options.REDIS_INCREMENT, 15L);
+
+        Assert.assertEquals("partition", entry.getPartition());
+        Assert.assertEquals("queue1", entry.getQueueName());
+        Assert.assertEquals(EntryType.UPDATE, entry.getEntryType());
+        Assert.assertEquals("replId", entry.getSourceId());
+        Assert.assertEquals(65535L, (long)entry.getOffset());
+        Assert.assertEquals("set", entry.getCommand());
+        Assert.assertEquals("key", entry.getKey());
+        Assert.assertEquals(FieldType.STRING, entry.getValueType());
+        Assert.assertEquals("value", entry.getValue());
+        Assert.assertEquals(15L, (long)entry.getParam(Options.REDIS_INCREMENT));
+        Assert.assertEquals(15L, (long)entry.getParams().get(Options.REDIS_INCREMENT.name()));
+    }
+
+
+    private KVEntryConverter getConverter(){
+        return new RedisEntryConverter();
+    }
+}
diff --git a/src/test/java/org/apache/rocketmq/redis/test/pojo/RedisEventTest.java b/src/test/java/org/apache/rocketmq/redis/test/pojo/RedisEventTest.java
new file mode 100644
index 0000000..a412e6d
--- /dev/null
+++ b/src/test/java/org/apache/rocketmq/redis/test/pojo/RedisEventTest.java
@@ -0,0 +1,33 @@
+package org.apache.rocketmq.redis.test.pojo;
+
+import com.moilioncircle.redis.replicator.rdb.datatype.KeyStringValueString;
+import com.moilioncircle.redis.replicator.rdb.datatype.KeyValuePair;
+import org.apache.rocketmq.connect.redis.pojo.RedisEvent;
+import org.junit.Assert;
+import org.junit.Test;
+
+public class RedisEventTest {
+    @Test
+    public void test(){
+        RedisEvent redisEvent = new RedisEvent();
+        redisEvent.setEvent(getKeyValuePair());
+        redisEvent.setReplOffset(3926872L);
+        redisEvent.setReplId("c18cece63c7b16851a6f387f52dbbb9eee07e46f");
+        redisEvent.setStreamDB(0);
+
+        Assert.assertEquals("c18cece63c7b16851a6f387f52dbbb9eee07e46f", redisEvent.getReplId());
+        Assert.assertTrue(3926872L == redisEvent.getReplOffset());
+        Assert.assertTrue(0 == redisEvent.getStreamDB());
+        Assert.assertNotNull(redisEvent.getEvent());
+        Assert.assertEquals(KeyStringValueString.class, redisEvent.getEvent().getClass());
+        Assert.assertEquals("key", new String(((KeyStringValueString)redisEvent.getEvent()).getKey()));
+        Assert.assertEquals("value", new String(((KeyStringValueString)redisEvent.getEvent()).getValue()));
+    }
+
+    private KeyValuePair getKeyValuePair(){
+        KeyValuePair pair = new KeyStringValueString();
+        pair.setKey("key".getBytes());
+        pair.setValue("value".getBytes());
+        return pair;
+    }
+}
diff --git a/src/test/java/org/apache/rocketmq/redis/test/processor/ListenerTest.java b/src/test/java/org/apache/rocketmq/redis/test/processor/ListenerTest.java
new file mode 100644
index 0000000..6500eba
--- /dev/null
+++ b/src/test/java/org/apache/rocketmq/redis/test/processor/ListenerTest.java
@@ -0,0 +1,135 @@
+package org.apache.rocketmq.redis.test.processor;
+
+import com.moilioncircle.redis.replicator.CloseListener;
+import com.moilioncircle.redis.replicator.ExceptionListener;
+import com.moilioncircle.redis.replicator.event.EventListener;
+import com.moilioncircle.redis.replicator.rdb.datatype.KeyStringValueString;
+import com.moilioncircle.redis.replicator.rdb.datatype.KeyValuePair;
+import java.io.IOException;
+import org.apache.rocketmq.connect.redis.common.Config;
+import org.apache.rocketmq.connect.redis.handler.DefaultRedisEventHandler;
+import org.apache.rocketmq.connect.redis.handler.RedisEventHandler;
+import org.apache.rocketmq.connect.redis.processor.DefaultRedisEventProcessor;
+import org.apache.rocketmq.connect.redis.processor.RedisClosedListener;
+import org.apache.rocketmq.connect.redis.processor.RedisEventListener;
+import org.apache.rocketmq.connect.redis.processor.RedisEventProcessor;
+import org.apache.rocketmq.connect.redis.processor.RedisExceptionListener;
+import org.junit.After;
+import org.junit.Assert;
+import org.junit.Before;
+import org.junit.Test;
+
+import static org.mockito.ArgumentMatchers.any;
+import static org.mockito.Mockito.mock;
+import static org.mockito.Mockito.when;
+
+public class ListenerTest {
+    private Config config;
+    private RedisEventProcessor processor;
+
+    @Before
+    public void init(){
+        this.config = getConfig();
+        this.processor = getProcessor();
+        Assert.assertTrue(this.processor.isStopped());
+    }
+
+    @Test
+    public void testEventListener(){
+        EventListener eventListener = new RedisEventListener(this.config, processor);
+        KeyValuePair pair = new KeyStringValueString();
+        eventListener.onEvent(null, pair);
+    }
+
+    @Test
+    public void testEventListenerConfigNull(){
+        Exception ex = null;
+        try {
+            new RedisEventListener(null, processor);
+        }catch (IllegalArgumentException e){
+            ex = e;
+        }
+        Assert.assertNotNull(ex);
+    }
+
+    @Test
+    public void testEventListenerRetry1(){
+        Config config = getConfig();
+        RedisEventProcessor processor = getFailedProcessor(config);
+
+        EventListener eventListener = new RedisEventListener(config, processor);
+        KeyValuePair pair = new KeyStringValueString();
+        eventListener.onEvent(null, pair);
+
+    }
+
+    @Test
+    public void testEventListenerRetry2(){
+        Config config = getConfig();
+        RedisEventProcessor processor = getExceptionProcessor(config);
+
+        EventListener eventListener = new RedisEventListener(config, processor);
+        KeyValuePair pair = new KeyStringValueString();
+        eventListener.onEvent(null, pair);
+
+    }
+
+    @Test
+    public void closeListenerTest(){
+        CloseListener closeListener = new RedisClosedListener(processor);
+        closeListener.handle(null);
+    }
+
+    @Test
+    public void exceptionListenerTest(){
+        ExceptionListener exceptionListener = new RedisExceptionListener(processor);
+        KeyValuePair pair = new KeyStringValueString();
+        exceptionListener.handle(null, new NullPointerException("adsf"), pair);
+    }
+
+    @After
+    public void stop() throws IOException {
+        this.processor.stop();
+    }
+
+
+    private RedisEventProcessor getProcessor(){
+        Config config = getConfig();
+        RedisEventHandler eventHandler = new DefaultRedisEventHandler(config);
+        RedisEventProcessor processor = new DefaultRedisEventProcessor(config);
+        processor.registEventHandler(eventHandler);
+        return processor;
+    }
+
+    private RedisEventProcessor getFailedProcessor(Config config){
+        RedisEventHandler eventHandler = new DefaultRedisEventHandler(config);
+        RedisEventProcessor processor = mock(DefaultRedisEventProcessor.class);
+        processor.registEventHandler(eventHandler);
+        try {
+            when(processor.commit(any())).thenReturn(false);
+        } catch (Exception e) {
+            e.printStackTrace();
+        }
+        return processor;
+    }
+
+    private RedisEventProcessor getExceptionProcessor(Config config){
+        RedisEventHandler eventHandler = new DefaultRedisEventHandler(config);
+        RedisEventProcessor processor = mock(DefaultRedisEventProcessor.class);
+        processor.registEventHandler(eventHandler);
+        try {
+            when(processor.commit(any())).thenThrow(new IllegalStateException("wrong number."));
+        } catch (Exception e) {
+            e.printStackTrace();
+        }
+        return processor;
+    }
+
+    private Config getConfig(){
+        Config config = new Config();
+
+        config.setRedisAddr("127.0.0.1");
+        config.setRedisPort(6379);
+        return config;
+    }
+}
diff --git a/src/test/java/org/apache/rocketmq/redis/test/processor/ProcessorTest.java b/src/test/java/org/apache/rocketmq/redis/test/processor/ProcessorTest.java
new file mode 100644
index 0000000..1410b2a
--- /dev/null
+++ b/src/test/java/org/apache/rocketmq/redis/test/processor/ProcessorTest.java
@@ -0,0 +1,193 @@
+package org.apache.rocketmq.redis.test.processor;
+
+import java.io.IOException;
+import java.util.HashSet;
+import java.util.Set;
+
+import com.moilioncircle.redis.replicator.rdb.datatype.KeyStringValueSet;
+import com.moilioncircle.redis.replicator.rdb.datatype.KeyStringValueString;
+import com.moilioncircle.redis.replicator.rdb.datatype.KeyValuePair;
+import org.apache.rocketmq.connect.redis.common.Config;
+import org.apache.rocketmq.connect.redis.common.SyncMod;
+import org.apache.rocketmq.connect.redis.handler.DefaultRedisEventHandler;
+import org.apache.rocketmq.connect.redis.handler.RedisEventHandler;
+import org.apache.rocketmq.connect.redis.pojo.KVEntry;
+import org.apache.rocketmq.connect.redis.pojo.RedisEvent;
+import org.apache.rocketmq.connect.redis.processor.DefaultRedisEventProcessor;
+import org.apache.rocketmq.connect.redis.processor.RedisEventProcessor;
+import org.junit.After;
+import org.junit.Assert;
+import org.junit.Test;
+import redis.clients.jedis.exceptions.JedisConnectionException;
+
+import static com.moilioncircle.redis.replicator.Constants.RDB_TYPE_SET;
+
+public class ProcessorTest {
+    private RedisEventProcessor processor;
+
+    @Test
+    public void testHandler() throws Exception {
+        processor = getProcessor();
+        DefaultRedisEventProcessor defaultRedisEventProcessor = (DefaultRedisEventProcessor)processor;
+
+
+        RedisEvent event = new RedisEvent();
+        event.setEvent(getKeyValuePair());
+        event.setReplOffset(123L);
+        event.setStreamDB(0);
+        event.setReplId("asdfsdfa");
+        defaultRedisEventProcessor.commit(event);
+
+        KVEntry res = null;
+        try {
+            res = defaultRedisEventProcessor.poll();
+        } catch (Exception e) {
+            e.printStackTrace();
+        }
+        Assert.assertNotNull(res);
+
+
+        event.setEvent(getKVCommandPair());
+        defaultRedisEventProcessor.commit(event);
+        try {
+            res = defaultRedisEventProcessor.poll();
+        } catch (Exception e) {
+            e.printStackTrace();
+        }
+        Assert.assertNotNull(res);
+    }
+
+    private KeyValuePair getKVCommandPair(){
+        KeyValuePair event = new KeyStringValueSet();
+        event.setValueRdbType(RDB_TYPE_SET);
+        event.setKey("mySet".getBytes());
+        Set<byte[]> values= new HashSet<>();
+        values.add("myValue".getBytes());
+        values.add("myValue2".getBytes());
+        event.setValue(values);
+        return event;
+    }
+
+    @Test
+    public void testRepeatStart() throws IOException {
+        try{
+            processor = getProcessor();
+            processor.start();
+            Assert.assertFalse(processor.isStopped());
+            processor.start();
+            Assert.assertFalse(processor.isStopped());
+
+            processor.stop();
+            Assert.assertTrue(processor.isStopped());
+            processor.stop();
+            Assert.assertTrue(processor.isStopped());
+        }catch (JedisConnectionException e){
+
+        }
+    }
+
+    @Test
+    public void testSyncMod() throws IOException {
+        RedisEventProcessor processor = null;
+        try{
+            processor = getProcessor(SyncMod.LAST_OFFSET_FORCE);
+            processor.start();
+
+            processor = getProcessor(SyncMod.CUSTOM_OFFSET_FORCE);
+            processor.start();
+
+            processor = getProcessor(SyncMod.LAST_OFFSET);
+            processor.start();
+
+            processor = getProcessor(SyncMod.CUSTOM_OFFSET);
+            processor.start();
+        }catch (JedisConnectionException e){
+
+        }
+    }
+
+    @Test
+    public void test(){
+        processor = getProcessor();
+        RedisEvent event = getRedisEvent();
+        Exception ex = null;
+        try {
+            processor.commit(event);
+        } catch (Exception e) {
+            e.printStackTrace();
+            ex = e;
+        }
+        Assert.assertNull(ex);
+
+        try {
+            KVEntry entry = processor.poll();
+            Assert.assertEquals("key", entry.getKey());
+            Assert.assertEquals("value", entry.getValue());
+            Assert.assertEquals("c18cece63c7b16851a6f387f52dbbb9eee07e46f", entry.getSourceId());
+        } catch (InterruptedException e) {
+            e.printStackTrace();
+            ex = e;
+        } catch (Exception e) {
+            e.printStackTrace();
+        }
+        Assert.assertNull(ex);
+    }
+
+    @After
+    public void stop() throws IOException {
+        if(this.processor != null){
+            this.processor.stop();
+        }
+    }
+
+    private RedisEvent getRedisEvent(){
+        RedisEvent redisEvent = new RedisEvent();
+        redisEvent.setEvent(getKeyValuePair());
+        redisEvent.setReplOffset(3926872L);
+        redisEvent.setReplId("c18cece63c7b16851a6f387f52dbbb9eee07e46f");
+        redisEvent.setStreamDB(0);
+        return redisEvent;
+    }
+
+    private KeyValuePair getKeyValuePair(){
+        KeyValuePair pair = new KeyStringValueString();
+        pair.setKey("key".getBytes());
+        pair.setValue("value".getBytes());
+        return pair;
+    }
+
+    private RedisEventProcessor getProcessor(){
+        return getProcessor(getConfig());
+    }
+
+    private RedisEventProcessor getProcessor(SyncMod syncMod){
+        return getProcessor(getConfig(syncMod));
+    }
+
+    private RedisEventProcessor getProcessor(Config config){
+        RedisEventHandler eventHandler = new DefaultRedisEventHandler(config);
+        RedisEventProcessor processor = new DefaultRedisEventProcessor(config);
+        processor.registEventHandler(eventHandler);
+        return processor;
+    }
+
+    private Config getConfig(){
+        return getConfig(SyncMod.CUSTOM_OFFSET);
+    }
+
+    private Config getConfig(SyncMod syncMod){
+        Config config = new Config();
+
+        config.setRedisAddr("127.0.0.1");
+        config.setRedisPort(6379);
+        config.setOffset(100L);
+        config.setPosition(200L);
+        config.setCommands("SET,HSET");
+
+        if(syncMod != null){
+            config.setSyncMod(syncMod.name());
+        }
+        return config;
+    }
+
+}
diff --git a/src/test/java/org/apache/rocketmq/redis/test/util/ParseStringUtilsTest.java b/src/test/java/org/apache/rocketmq/redis/test/util/ParseStringUtilsTest.java
new file mode 100644
index 0000000..2a67959
--- /dev/null
+++ b/src/test/java/org/apache/rocketmq/redis/test/util/ParseStringUtilsTest.java
@@ -0,0 +1,45 @@
+package org.apache.rocketmq.redis.test.util;
+
+import java.util.List;
+import java.util.Map;
+
+import org.apache.rocketmq.connect.redis.util.ParseStringUtils;
+import org.junit.Assert;
+import org.junit.Test;
+
+public class ParseStringUtilsTest {
+    private String info = "# Replication\n"
+        + "role:master\n"
+        + "connected_slaves:2\n"
+        + "slave0:ip=127.0.0.1,port=64690,state=online,offset=3926872,lag=1\n"
+        + "slave1:ip=127.0.0.1,port=64691,state=online,offset=3926872,lag=1\n"
+        + "master_replid:c18cece63c7b16851a6f387f52dbbb9eee07e46f\n"
+        + "master_replid2:0000000000000000000000000000000000000000\n"
+        + "master_repl_offset:3926872\n"
+        + "second_repl_offset:-1\n"
+        + "repl_backlog_active:1\n"
+        + "repl_backlog_size:1048576\n"
+        + "repl_backlog_first_byte_offset:3862270\n"
+        + "repl_backlog_histlen:64603";
+    @Test
+    public void test(){
+        Map<String, String> map = ParseStringUtils.parseRedisInfo2Map(info);
+        Assert.assertTrue(map.size() == 12);
+        Assert.assertEquals("c18cece63c7b16851a6f387f52dbbb9eee07e46f", map.get("master_replid"));
+        Assert.assertEquals("1048576", map.get("repl_backlog_size"));
+    }
+
+    private String commands = "SET,APPEND,HMSET";
+    @Test
+    public void testParseCommands(){
+        List<String> res = ParseStringUtils.parseCommands(commands);
+        Assert.assertNotNull(res);
+        Assert.assertEquals(3, res.size());
+        Assert.assertEquals("SET", res.get(0));
+        Assert.assertEquals("APPEND", res.get(1));
+        Assert.assertEquals("HMSET", res.get(2));
+
+        res = ParseStringUtils.parseCommands("  ");
+        Assert.assertNull(res);
+    }
+}
diff --git a/src/test/java/org/apache/rocketmq/redis/test/util/PropertyToObjectUtilsTest.java b/src/test/java/org/apache/rocketmq/redis/test/util/PropertyToObjectUtilsTest.java
new file mode 100644
index 0000000..ddcc59b
--- /dev/null
+++ b/src/test/java/org/apache/rocketmq/redis/test/util/PropertyToObjectUtilsTest.java
@@ -0,0 +1,118 @@
+package org.apache.rocketmq.redis.test.util;
+
+import java.lang.reflect.InvocationTargetException;
+import java.util.Objects;
+
+import io.openmessaging.KeyValue;
+import io.openmessaging.internal.DefaultKeyValue;
+import org.apache.rocketmq.connect.redis.util.PropertyToObjectUtils;
+import org.junit.Assert;
+import org.junit.Test;
+
+public class PropertyToObjectUtilsTest {
+
+    @Test
+    public void test(){
+        User user = new User();
+        user.setName("LLL");
+        user.setAge(100);
+        user.setScore(365);
+        user.setAverage(66.66);
+        user.setMath(78L);
+        user.setHigh(false);
+
+        KeyValue pair = new DefaultKeyValue();
+        pair.put("name", "LLL");
+        pair.put("age", "100");
+        pair.put("score", "365");
+        pair.put("average", "66.66");
+        pair.put("math", "78");
+        pair.put("high", "false");
+
+        User user1 = new User();
+        Exception ex = null;
+        try {
+            PropertyToObjectUtils.properties2Object(pair, user1);
+        } catch (InvocationTargetException e) {
+            ex = e;
+        } catch (IllegalAccessException e) {
+            ex = e;
+        }
+        Assert.assertEquals(user, user1);
+        Assert.assertNull(ex);
+    }
+
+
+    public class User{
+        private String name;
+        private Integer age;
+        private int score;
+        private Long math;
+        private Double average;
+        private boolean high;
+
+        public String getName() {
+            return name;
+        }
+
+        public void setName(String name) {
+            this.name = name;
+        }
+
+        public Integer getAge() {
+            return age;
+        }
+
+        public void setAge(Integer age) {
+            this.age = age;
+        }
+
+        public int getScore() {
+            return score;
+        }
+
+        public void setScore(int score) {
+            this.score = score;
+        }
+
+        public Long getMath() {
+            return math;
+        }
+
+        public void setMath(Long math) {
+            this.math = math;
+        }
+
+        public Double getAverage() {
+            return average;
+        }
+
+        public void setAverage(Double average) {
+            this.average = average;
+        }
+
+        public void setHigh(boolean high) {
+            this.high = high;
+        }
+
+        public boolean isHigh() {
+            return high;
+        }
+
+        @Override
+        public boolean equals(Object obj) {
+            if(this == obj){
+                return true;
+            }
+            if(obj == null || getClass() != obj.getClass()){
+                return false;
+            }
+            User other = (User)obj;
+            return Objects.equals(this.getName(), other.getName()) &&
+                Objects.equals(this.getAge(), other.getAge()) &&
+                Objects.equals(this.getAverage(), other.getAverage()) &&
+                Objects.equals(this.getMath(), other.getMath()) &&
+                Objects.equals(this.getScore(), other.getScore());
+        }
+    }
+}
diff --git a/style/copyright/Apache.xml b/style/copyright/Apache.xml
new file mode 100644
index 0000000..e3e3dec
--- /dev/null
+++ b/style/copyright/Apache.xml
@@ -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
+
+      http://www.apache.org/licenses/LICENSE-2.0
+
+  Unless required by applicable law or agreed to in writing, software
+  distributed under the License is distributed on an "AS IS" BASIS,
+  WITHOUT 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.a [...]
+    </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..747c7e2
--- /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..2872eb7
--- /dev/null
+++ b/style/rmq_checkstyle.xml
@@ -0,0 +1,134 @@
+<?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>
+
... 216 lines suppressed ...