You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@rocketmq.apache.org by hu...@apache.org on 2022/03/29 04:31:30 UTC

[rocketmq-mqtt] 02/43: init

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

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

commit 4dedf82c528bf0b441f02af6e1e1fe22806cc941
Author: pingww <pi...@gmail.com>
AuthorDate: Tue Mar 8 12:04:07 2022 +0800

    init
---
 .gitignore                                         |   3 +
 README.md                                          |  75 ++-
 assembly.xml                                       |  37 ++
 bin/mqtt.sh                                        |  93 ++++
 bin/runserver.sh                                   |  93 ++++
 conf/connect.conf                                  |  18 +
 conf/logback.xml                                   |  76 +++
 conf/service.conf                                  |  22 +
 conf/spring.xml                                    |  25 +
 mqtt-common/pom.xml                                |  50 ++
 .../rocketmq/mqtt/common/facade/AuthManager.java   |  33 ++
 .../mqtt/common/facade/LmqOffsetStore.java         |  45 ++
 .../rocketmq/mqtt/common/facade/LmqQueueStore.java |  92 ++++
 .../mqtt/common/facade/MetaPersistManager.java     |  44 ++
 .../mqtt/common/hook/AbstractUpstreamHook.java     |  70 +++
 .../org/apache/rocketmq/mqtt/common/hook/Hook.java |  36 ++
 .../rocketmq/mqtt/common/hook/HookResult.java      | 109 +++++
 .../rocketmq/mqtt/common/hook/UpstreamHook.java    |  37 ++
 .../mqtt/common/hook/UpstreamHookEnum.java         |  25 +
 .../mqtt/common/hook/UpstreamHookManager.java      |  45 ++
 .../rocketmq/mqtt/common/model/Constants.java      |  46 ++
 .../apache/rocketmq/mqtt/common/model/Message.java | 217 +++++++++
 .../rocketmq/mqtt/common/model/MessageEvent.java   |  79 +++
 .../mqtt/common/model/MqttMessageUpContext.java    |  59 +++
 .../rocketmq/mqtt/common/model/MqttTopic.java      |  47 ++
 .../rocketmq/mqtt/common/model/PullResult.java     |  65 +++
 .../apache/rocketmq/mqtt/common/model/Queue.java   | 116 +++++
 .../rocketmq/mqtt/common/model/QueueOffset.java    |  72 +++
 .../apache/rocketmq/mqtt/common/model/Remark.java  |  33 ++
 .../apache/rocketmq/mqtt/common/model/RpcCode.java |  28 ++
 .../rocketmq/mqtt/common/model/RpcHeader.java      |  25 +
 .../rocketmq/mqtt/common/model/StoreResult.java    |  41 ++
 .../rocketmq/mqtt/common/model/Subscription.java   | 114 +++++
 .../apache/rocketmq/mqtt/common/model/Trie.java    | 240 +++++++++
 .../rocketmq/mqtt/common/model/TrieException.java  |  41 ++
 .../rocketmq/mqtt/common/model/TrieMethod.java     |  33 ++
 .../rocketmq/mqtt/common/util/HmacSHA1Util.java    |  45 ++
 .../apache/rocketmq/mqtt/common/util/HostInfo.java |  56 +++
 .../rocketmq/mqtt/common/util/MessageUtil.java     | 119 +++++
 .../rocketmq/mqtt/common/util/NamespaceUtil.java   |  70 +++
 .../apache/rocketmq/mqtt/common/util/StatUtil.java | 472 ++++++++++++++++++
 .../rocketmq/mqtt/common/util/TopicUtils.java      | 195 ++++++++
 .../apache/rocketmq/mqtt/common/test/TestTrie.java |  37 ++
 mqtt-cs/pom.xml                                    |  71 +++
 .../rocketmq/mqtt/cs/channel/ChannelCloseFrom.java |  33 ++
 .../rocketmq/mqtt/cs/channel/ChannelException.java |  41 ++
 .../rocketmq/mqtt/cs/channel/ChannelInfo.java      | 254 ++++++++++
 .../rocketmq/mqtt/cs/channel/ChannelManager.java   |  64 +++
 .../rocketmq/mqtt/cs/channel/ConnectHandler.java   |  65 +++
 .../mqtt/cs/channel/DefaultChannelManager.java     | 151 ++++++
 .../rocketmq/mqtt/cs/config/ConnectConf.java       | 184 +++++++
 .../mqtt/cs/config/ConnectConfListener.java        |  73 +++
 .../mqtt/cs/hook/UpstreamHookManagerImpl.java      |  74 +++
 .../cs/protocol/mqtt/MqttPacketDispatcher.java     | 179 +++++++
 .../mqtt/cs/protocol/mqtt/MqttPacketHandler.java   |  37 ++
 .../protocol/mqtt/handler/MqttConnectHandler.java  | 111 +++++
 .../mqtt/handler/MqttDisconnectHandler.java        |  45 ++
 .../cs/protocol/mqtt/handler/MqttPingHandler.java  |  57 +++
 .../protocol/mqtt/handler/MqttPubAckHandler.java   |  58 +++
 .../protocol/mqtt/handler/MqttPubCompHandler.java  |  67 +++
 .../protocol/mqtt/handler/MqttPubRecHandler.java   |  60 +++
 .../protocol/mqtt/handler/MqttPubRelHandler.java   |  51 ++
 .../protocol/mqtt/handler/MqttPublishHandler.java  | 114 +++++
 .../mqtt/handler/MqttSubscribeHandler.java         | 130 +++++
 .../mqtt/handler/MqttUnSubscribeHandler.java       |  91 ++++
 .../mqtt/cs/protocol/rpc/RpcPacketDispatcher.java  |  86 ++++
 .../cs/protocol/ws/WebSocketServerHandler.java     | 109 +++++
 .../mqtt/cs/protocol/ws/WebsocketEncoder.java      |  38 ++
 .../rocketmq/mqtt/cs/session/QueueFresh.java       |  65 +++
 .../apache/rocketmq/mqtt/cs/session/Session.java   | 469 ++++++++++++++++++
 .../rocketmq/mqtt/cs/session/infly/InFlyCache.java | 191 ++++++++
 .../rocketmq/mqtt/cs/session/infly/MqttMsgId.java  |  93 ++++
 .../rocketmq/mqtt/cs/session/infly/PushAction.java | 187 +++++++
 .../mqtt/cs/session/infly/RetryDriver.java         | 327 +++++++++++++
 .../mqtt/cs/session/loop/PullResultStatus.java     |  26 +
 .../rocketmq/mqtt/cs/session/loop/QueueCache.java  | 318 ++++++++++++
 .../rocketmq/mqtt/cs/session/loop/SessionLoop.java | 100 ++++
 .../mqtt/cs/session/loop/SessionLoopImpl.java      | 536 +++++++++++++++++++++
 .../mqtt/cs/session/match/MatchAction.java         | 161 +++++++
 .../cs/session/notify/MessageNotifyAction.java     |  88 ++++
 .../rocketmq/mqtt/cs/starter/MqttServer.java       | 132 +++++
 .../apache/rocketmq/mqtt/cs/starter/RpcServer.java |  65 +++
 .../apache/rocketmq/mqtt/cs/starter/Startup.java   |  36 ++
 .../mqtt/cs/test/TestDefaultChannelManager.java    |  60 +++
 .../rocketmq/mqtt/cs/test/TestInFlyCache.java      |  49 ++
 .../rocketmq/mqtt/cs/test/TestMatchAction.java     |  69 +++
 .../mqtt/cs/test/TestMessageNotifyAction.java      |  87 ++++
 .../rocketmq/mqtt/cs/test/TestMqttMsgId.java       |  38 ++
 .../rocketmq/mqtt/cs/test/TestPushAction.java      |  96 ++++
 .../rocketmq/mqtt/cs/test/TestQueueCache.java      | 112 +++++
 .../rocketmq/mqtt/cs/test/TestRetryDriver.java     |  91 ++++
 .../apache/rocketmq/mqtt/cs/test/TestSession.java  |  69 +++
 .../rocketmq/mqtt/cs/test/TestSessionLoopImpl.java | 157 ++++++
 mqtt-ds/pom.xml                                    |  63 +++
 .../rocketmq/mqtt/ds/auth/AuthManagerSample.java   |  96 ++++
 .../rocketmq/mqtt/ds/config/ServiceConf.java       | 139 ++++++
 .../mqtt/ds/config/ServiceConfListener.java        |  73 +++
 .../rocketmq/mqtt/ds/meta/FirstTopicManager.java   | 162 +++++++
 .../mqtt/ds/meta/MetaPersistManagerSample.java     | 128 +++++
 .../mqtt/ds/meta/TopicNotExistException.java       |  41 ++
 .../rocketmq/mqtt/ds/meta/WildcardManager.java     | 129 +++++
 .../org/apache/rocketmq/mqtt/ds/mq/MqAdmin.java    |  53 ++
 .../org/apache/rocketmq/mqtt/ds/mq/MqConsumer.java |  84 ++++
 .../org/apache/rocketmq/mqtt/ds/mq/MqFactory.java  | 114 +++++
 .../org/apache/rocketmq/mqtt/ds/mq/MqProducer.java |  62 +++
 .../apache/rocketmq/mqtt/ds/mq/MqPullConsumer.java |  66 +++
 .../rocketmq/mqtt/ds/notify/NotifyManager.java     | 286 +++++++++++
 .../mqtt/ds/notify/NotifyRetryManager.java         |  97 ++++
 .../mqtt/ds/store/LmqOffsetStoreManager.java       | 155 ++++++
 .../mqtt/ds/store/LmqQueueStoreManager.java        | 427 ++++++++++++++++
 .../mqtt/ds/upstream/UpstreamProcessor.java        |  30 ++
 .../mqtt/ds/upstream/UpstreamProcessorManager.java |  79 +++
 .../mqtt/ds/upstream/processor/BaseProcessor.java  |  37 ++
 .../ds/upstream/processor/ConnectProcessor.java    |  46 ++
 .../ds/upstream/processor/DisconnectProcessor.java |  37 ++
 .../ds/upstream/processor/PublishProcessor.java    |  77 +++
 .../ds/upstream/processor/SubscribeProcessor.java  |  56 +++
 .../upstream/processor/UnSubscribeProcessor.java   |  53 ++
 .../mqtt/ds/test/TestFirstTopicManager.java        |  96 ++++
 .../mqtt/ds/test/TestLmqQueueStoreManager.java     | 115 +++++
 .../rocketmq/mqtt/ds/test/TestNotifyManager.java   |  90 ++++
 .../rocketmq/mqtt/ds/test/TestWildcardManager.java |  58 +++
 mqtt-example/pom.xml                               |  30 ++
 .../apache/rocketmq/mqtt/example/MqttConsumer.java |  98 ++++
 .../apache/rocketmq/mqtt/example/MqttProducer.java | 109 +++++
 .../rocketmq/mqtt/example/RocketMQConsumer.java    |  68 +++
 .../rocketmq/mqtt/example/RocketMQProducer.java    | 110 +++++
 pom.xml                                            | 189 ++++++++
 128 files changed, 12695 insertions(+), 1 deletion(-)

diff --git a/.gitignore b/.gitignore
new file mode 100644
index 0000000..7466b12
--- /dev/null
+++ b/.gitignore
@@ -0,0 +1,3 @@
+.idea
+*.iml
+target/
diff --git a/README.md b/README.md
index f2609de..c2676d6 100644
--- a/README.md
+++ b/README.md
@@ -1 +1,74 @@
-# rocketmq-mqtt
+## Apache RocketMQ MQTT
+A new MQTT protocol architecture model, based on which RocketMQ can better support messages from terminals such as IoT devices and Mobile APP. Based on the RocketMQ message unified storage engine, it supports both MQTT terminal and server message sending and receiving.
+
+## Architecture
+The relevant architecture design is introduced in [RIP-33](https://docs.google.com/document/d/1AD1GkV9mqE_YFA97uVem4SmB8ZJSXiJZvzt7-K6Jons/edit#)
+
+
+## Get Started
+
+### Prerequisites
+The queue model of MQTT needs to be based on the lightweight queue feature ([RIP-28](https://github.com/apache/rocketmq/pull/3694)) of RocketMQ. RocketMQ has only supported this feature since version 4.9.3. Please ensure that the installed version of RocketMQ already supports this feature.
+
+1. Clone
+```shell
+git clone https://github.com/apache/rocketmq-mqtt
+```
+2. Build the package
+```shell
+cd rocketmq-mqtt
+mvn clean package -DskipTests=true assembly:assembly
+```
+3. Config
+```shell
+cp -r  target/rocketmq-mqtt ~
+cd ~/rocketmq-mqtt
+cd conf
+```
+Some important configuration items in the **service.conf** configuration file
+
+**Config Key** | **Instruction**
+----- | ----  
+username   |  used for auth
+secretKey   | used for auth
+NAMESRV_ADDR   | specify namesrv address
+eventNotifyRetryTopic   | notify event retry topic
+clientRetryTopic   | client retry topic
+
+4. CreateTopic
+
+   create all first-level topics, including **eventNotifyRetryTopic** and **clientRetryTopic** in the configuration file above.
+```shell
+sh mqadmin updatetopic -c {cluster} -t {topic} -n {namesrv}
+```
+5. Initialize Meta
+- Configure Gateway Node List
+```shell
+sh mqadmin updateKvConfig -s LMQ -k LMQ_CONNECT_NODES -v {ip1,ip2} -n {namesrv}
+```
+- Configure the first-level topic list
+```shell
+sh mqadmin updateKvConfig -s LMQ -k ALL_FIRST_TOPICS -v {topic1,topic2} -n {namesrv}
+```
+- Configure a list of wildcard characters under each first-level topic
+```shell
+sh mqadmin updateKvConfig  -s LMQ -k {topic} -v {topic/+}  -n {namesrv}
+```
+6. Start Process
+```shell
+cd bin
+sh mqtt.sh start
+```
+### Example
+The mqtt-example module has written basic usage example code, which can be used for reference
+
+## Protocol Version
+The currently supported protocol version is [MQTT 3.1.1](http://docs.oasis-open.org/mqtt/mqtt/v3.1.1/os/mqtt-v3.1.1-os.pdf), but the will and retain features are not supported yet
+
+## Authentication
+At present, an implementation based on the HmacSHA1 signature algorithm is provided by default, Refer to **AuthManagerSample**. Users can customize other implementations to meet the needs of businesses to flexibly verify resources and identities.
+## Meta Persistence
+At present, meta data storage and management is simply implemented through the kvconfig mechanism of namesrv by default, Refer to **MetaPersistManagerSample**. Users can customize other implementations.
+
+## License
+[Apache License, Version 2.0](http://www.apache.org/licenses/LICENSE-2.0.html) Copyright (C) Apache Software Foundation.
diff --git a/assembly.xml b/assembly.xml
new file mode 100644
index 0000000..19ffe3d
--- /dev/null
+++ b/assembly.xml
@@ -0,0 +1,37 @@
+<?xml version="1.0" encoding="UTF-8"?>
+<assembly>
+    <formats>
+        <format>dir</format>
+        <format>tar.gz</format>
+    </formats>
+    <fileSets>
+        <fileSet>
+            <includes>
+                <include>bin/*</include>
+                <include>conf/**</include>
+            </includes>
+            <excludes>
+                <exclude>**/src/**</exclude>
+                <exclude>**/target/**</exclude>
+                <exclude>**/.*/**</exclude>
+            </excludes>
+        </fileSet>
+    </fileSets>
+    <moduleSets>
+        <moduleSet>
+            <includes>
+                <include>org.apache.rocketmq:mqtt-cs</include>
+                <include>org.apache.rocketmq:mqtt-ds</include>
+            </includes>
+            <binaries>
+                <outputDirectory>lib</outputDirectory>
+                <unpack>false</unpack>
+                <dependencySets>
+                    <dependencySet>
+                        <outputDirectory>lib</outputDirectory>
+                    </dependencySet>
+                </dependencySets>
+            </binaries>
+        </moduleSet>
+    </moduleSets>
+</assembly>
diff --git a/bin/mqtt.sh b/bin/mqtt.sh
new file mode 100644
index 0000000..d0bbbb4
--- /dev/null
+++ b/bin/mqtt.sh
@@ -0,0 +1,93 @@
+#!/bin/sh
+
+#
+# /*
+#  * Licensed to the Apache Software Foundation (ASF) under one or more
+#  * contributor license agreements.  See the NOTICE file distributed with
+#  * this work for additional information regarding copyright ownership.
+#  * The ASF licenses this file to You under the Apache License, Version 2.0
+#  * (the "License"); you may not use this file except in compliance with
+#  * the License.  You may obtain a copy of the License at
+#  *
+#  *     http://www.apache.org/licenses/LICENSE-2.0
+#  *
+#  * Unless required by applicable law or agreed to in writing, software
+#  * distributed under the License is distributed on an "AS IS" BASIS,
+#  * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+#  * See the License for the specific language governing permissions and
+#  * limitations under the License.
+#  */
+#
+
+if [ -z "$ROCKETMQ_MQTT_HOME" ]; then
+  ## resolve links - $0 may be a link to maven's home
+  PRG="$0"
+
+  # need this for relative symlinks
+  while [ -h "$PRG" ]; do
+    ls=$(ls -ld "$PRG")
+    link=$(expr "$ls" : '.*-> \(.*\)$')
+    if expr "$link" : '/.*' >/dev/null; then
+      PRG="$link"
+    else
+      PRG="$(dirname "$PRG")/$link"
+    fi
+  done
+
+  saveddir=$(pwd)
+
+  ROCKETMQ_MQTT_HOME=$(dirname "$PRG")/..
+
+  # make it fully qualified
+  ROCKETMQ_MQTT_HOME=$(cd "$ROCKETMQ_MQTT_HOME" && pwd)
+
+  cd "$saveddir"
+fi
+
+export ROCKETMQ_MQTT_HOME
+
+BASEDIR=$HOME
+mkdir -p $BASEDIR/logs
+
+mainClass="org.apache.rocketmq.mqtt.cs.starter.Startup"
+
+function startup() {
+  pid=`ps aux|grep $mainClass|grep -v grep |awk '{print $2}'`
+  if [ ! -z "$pid" ]; then
+    echo "java is runing..."
+    exit 1
+  fi
+  nohup sh ${ROCKETMQ_MQTT_HOME}/bin/runserver.sh $mainClass $@ >$BASEDIR/logs/start_out.log 2>&1 &
+}
+
+function stop() {
+  pid=`ps aux|grep $mainClass|grep -v grep |awk '{print $2}'`
+  if [ -z "$pid" ]; then
+    echo "no java to kill"
+  fi
+  printf 'stop...'
+  kill $pid
+  sleep 3
+  pid=`ps aux|grep $mainClass|grep -v grep |awk '{print $2}'`
+
+  if [ ! -z $pid ]; then
+    kill -9 $pid
+  fi
+}
+
+case "$1" in
+start)
+  startup $@
+  ;;
+stop)
+  stop
+  ;;
+restart)
+  stop
+  startup
+  ;;
+*)
+  printf "Usage: sh  $0 %s {start|stop|restart}\n"
+  exit 1
+  ;;
+esac
diff --git a/bin/runserver.sh b/bin/runserver.sh
new file mode 100644
index 0000000..de9541c
--- /dev/null
+++ b/bin/runserver.sh
@@ -0,0 +1,93 @@
+#!/bin/sh
+
+# Licensed to the Apache Software Foundation (ASF) under one or more
+# contributor license agreements.  See the NOTICE file distributed with
+# this work for additional information regarding copyright ownership.
+# The ASF licenses this file to You under the Apache License, Version 2.0
+# (the "License"); you may not use this file except in compliance with
+# the License.  You may obtain a copy of the License at
+#
+#     http://www.apache.org/licenses/LICENSE-2.0
+#
+# Unless required by applicable law or agreed to in writing, software
+# distributed under the License is distributed on an "AS IS" BASIS,
+# WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+# See the License for the specific language governing permissions and
+# limitations under the License.
+
+#===========================================================================================
+# Java Environment Setting
+#===========================================================================================
+error_exit ()
+{
+    echo "ERROR: $1 !!"
+    exit 1
+}
+
+[ ! -e "$JAVA_HOME/bin/java" ] && JAVA_HOME=$HOME/jdk/java
+[ ! -e "$JAVA_HOME/bin/java" ] && JAVA_HOME=/usr/java
+[ ! -e "$JAVA_HOME/bin/java" ] && error_exit "Please set the JAVA_HOME variable in your environment, We need java(x64)!"
+
+export JAVA_HOME
+export JAVA="$JAVA_HOME/bin/java"
+export BASE_DIR=$(dirname $0)/..
+export CLASSPATH=.:${BASE_DIR}/conf:${CLASSPATH}
+
+#===========================================================================================
+# JVM Configuration
+#===========================================================================================
+# The RAMDisk initializing size in MB on Darwin OS for gc-log
+DIR_SIZE_IN_MB=600
+
+choose_gc_log_directory()
+{
+    case "`uname`" in
+        Darwin)
+            if [ ! -d "/Volumes/RAMDisk" ]; then
+                # create ram disk on Darwin systems as gc-log directory
+                DEV=`hdiutil attach -nomount ram://$((2 * 1024 * DIR_SIZE_IN_MB))` > /dev/null
+                diskutil eraseVolume HFS+ RAMDisk ${DEV} > /dev/null
+                echo "Create RAMDisk /Volumes/RAMDisk for gc logging on Darwin OS."
+            fi
+            GC_LOG_DIR="/Volumes/RAMDisk"
+        ;;
+        *)
+            # check if /dev/shm exists on other systems
+            if [ -d "/dev/shm" ]; then
+                GC_LOG_DIR="/dev/shm"
+            else
+                GC_LOG_DIR=${BASE_DIR}
+            fi
+        ;;
+    esac
+}
+
+choose_gc_options()
+{
+    # Example of JAVA_MAJOR_VERSION value : '1', '9', '10', '11', ...
+    # '1' means releases befor Java 9
+    JAVA_MAJOR_VERSION=$("$JAVA" -version 2>&1 | sed -r -n 's/.* version "([0-9]*).*$/\1/p')
+    if [ -z "$JAVA_MAJOR_VERSION" ] || [ "$JAVA_MAJOR_VERSION" -lt "9" ] ; then
+      JAVA_OPT="${JAVA_OPT} -server -Xms4g -Xmx4g -Xmn2g -XX:MetaspaceSize=128m -XX:MaxMetaspaceSize=320m"
+      JAVA_OPT="${JAVA_OPT} -XX:+UseConcMarkSweepGC -XX:+UseCMSCompactAtFullCollection -XX:CMSInitiatingOccupancyFraction=70 -XX:+CMSParallelRemarkEnabled -XX:SoftRefLRUPolicyMSPerMB=0 -XX:+CMSClassUnloadingEnabled -XX:SurvivorRatio=8 -XX:-UseParNewGC"
+      JAVA_OPT="${JAVA_OPT} -verbose:gc -Xloggc:${GC_LOG_DIR}/rmq_srv_gc_%p_%t.log -XX:+PrintGCDetails -XX:+PrintGCDateStamps"
+      JAVA_OPT="${JAVA_OPT} -XX:+UseGCLogFileRotation -XX:NumberOfGCLogFiles=5 -XX:GCLogFileSize=30m"
+    else
+      JAVA_OPT="${JAVA_OPT} -server -Xms4g -Xmx4g -XX:MetaspaceSize=128m -XX:MaxMetaspaceSize=320m"
+      JAVA_OPT="${JAVA_OPT} -XX:+UseG1GC -XX:G1HeapRegionSize=16m -XX:G1ReservePercent=25 -XX:InitiatingHeapOccupancyPercent=30 -XX:SoftRefLRUPolicyMSPerMB=0"
+      JAVA_OPT="${JAVA_OPT} -Xlog:gc*:file=${GC_LOG_DIR}/rmq_srv_gc_%p_%t.log:time,tags:filecount=5,filesize=30M"
+    fi
+}
+
+choose_gc_log_directory
+choose_gc_options
+JAVA_OPT="${JAVA_OPT} -XX:-OmitStackTraceInFastThrow"
+JAVA_OPT="${JAVA_OPT} -XX:-UseLargePages"
+JAVA_OPT="${JAVA_OPT} -Djava.ext.dirs=${JAVA_HOME}/jre/lib/ext:${BASE_DIR}/lib:${JAVA_HOME}/lib/ext"
+#JAVA_OPT="${JAVA_OPT} -Xdebug -Xrunjdwp:transport=dt_socket,address=9555,server=y,suspend=n"
+JAVA_OPT="${JAVA_OPT} ${JAVA_OPT_EXT}"
+JAVA_OPT="${JAVA_OPT} -cp ${CLASSPATH}"
+
+JAVA_OPT="${JAVA_OPT} -Dlogback.configurationFile=${BASE_DIR}/conf/logback.xml"
+
+$JAVA ${JAVA_OPT} $@
\ No newline at end of file
diff --git a/conf/connect.conf b/conf/connect.conf
new file mode 100644
index 0000000..35f34b7
--- /dev/null
+++ b/conf/connect.conf
@@ -0,0 +1,18 @@
+# Licensed to the Apache Software Foundation (ASF) under one or more
+# contributor license agreements.  See the NOTICE file distributed with
+# this work for additional information regarding copyright ownership.
+# The ASF licenses this file to You under the Apache License, Version 2.0
+# (the "License"); you may not use this file except in compliance with
+# the License.  You may obtain a copy of the License at
+#
+#     http://www.apache.org/licenses/LICENSE-2.0
+#
+#  Unless required by applicable law or agreed to in writing, software
+#  distributed under the License is distributed on an "AS IS" BASIS,
+#  WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+#  See the License for the specific language governing permissions and
+#  limitations under the License.
+
+
+mqttPort=1883
+
diff --git a/conf/logback.xml b/conf/logback.xml
new file mode 100644
index 0000000..b43d147
--- /dev/null
+++ b/conf/logback.xml
@@ -0,0 +1,76 @@
+<?xml version="1.0" encoding="UTF-8"?>
+<configuration>
+    <appender name="MainAppender"
+              class="ch.qos.logback.core.rolling.RollingFileAppender">
+        <file>${user.home}/logs/mqtt.log</file>
+        <append>true</append>
+        <rollingPolicy class="ch.qos.logback.core.rolling.TimeBasedRollingPolicy">
+            <fileNamePattern>${user.home}/logs/mqtt.%d{yyyy-MM-dd}.log</fileNamePattern>
+            <maxHistory>10</maxHistory>
+        </rollingPolicy>
+        <encoder>
+            <pattern>%d{yyy-MM-dd HH:mm:ss:SSS,GMT+8} %p [%logger{0}] %m%n</pattern>
+            <charset class="java.nio.charset.Charset">UTF-8</charset>
+        </encoder>
+    </appender>
+    <appender name="AsyncMainAppender" class="ch.qos.logback.classic.AsyncAppender">
+        <appender-ref ref="MainAppender"/>
+    </appender>
+
+    <appender name="RmqAppender"
+              class="ch.qos.logback.core.rolling.RollingFileAppender">
+        <file>${user.home}/logs/rmq.log</file>
+        <append>true</append>
+        <rollingPolicy class="ch.qos.logback.core.rolling.TimeBasedRollingPolicy">
+            <fileNamePattern>${user.home}/logs/rmq.%d{yyyy-MM-dd}.log</fileNamePattern>
+            <maxHistory>10</maxHistory>
+        </rollingPolicy>
+        <encoder>
+            <pattern>%d{yyy-MM-dd HH:mm:ss,GMT+8} %p [%logger{0}] %m%n</pattern>
+            <charset class="java.nio.charset.Charset">UTF-8</charset>
+        </encoder>
+    </appender>
+    <appender name="AsyncRmqAppender" class="ch.qos.logback.classic.AsyncAppender">
+        <appender-ref ref="RmqAppender"/>
+    </appender>
+
+
+    <appender name="StatAppender"
+              class="ch.qos.logback.core.rolling.RollingFileAppender">
+        <file>${user.home}/logs/stat.log</file>
+        <append>true</append>
+        <rollingPolicy class="ch.qos.logback.core.rolling.TimeBasedRollingPolicy">
+            <fileNamePattern>${user.home}/logs/stat.%d{yyyy-MM-dd}.log</fileNamePattern>
+            <maxHistory>10</maxHistory>
+            <totalSizeCap>1GB</totalSizeCap>
+        </rollingPolicy>
+        <encoder>
+            <pattern>%d{yyy-MM-dd HH:mm:ss,GMT+8} %m%n</pattern>
+            <charset class="java.nio.charset.Charset">UTF-8</charset>
+        </encoder>
+    </appender>
+    <appender name="AsyncStatAppender" class="ch.qos.logback.classic.AsyncAppender">
+        <appender-ref ref="StatAppender"/>
+    </appender>
+
+    <root>
+        <level value="info"/>
+        <appender-ref ref="AsyncMainAppender"/>
+    </root>
+
+    <logger name="StatLogger" additivity="false">
+        <level value="info"/>
+        <appender-ref ref="AsyncStatAppender"/>
+    </logger>
+
+    <logger name="RocketmqClient" additivity="false">
+        <level value="INFO"/>
+        <appender-ref ref="AsyncRmqAppender"/>
+    </logger>
+
+    <logger name="RocketmqRemoting" additivity="false">
+        <level value="INFO"/>
+        <appender-ref ref="AsyncRmqAppender"/>
+    </logger>
+
+</configuration>
diff --git a/conf/service.conf b/conf/service.conf
new file mode 100644
index 0000000..2be8cdc
--- /dev/null
+++ b/conf/service.conf
@@ -0,0 +1,22 @@
+# Licensed to the Apache Software Foundation (ASF) under one or more
+# contributor license agreements.  See the NOTICE file distributed with
+# this work for additional information regarding copyright ownership.
+# The ASF licenses this file to You under the Apache License, Version 2.0
+# (the "License"); you may not use this file except in compliance with
+# the License.  You may obtain a copy of the License at
+#
+#     http://www.apache.org/licenses/LICENSE-2.0
+#
+#  Unless required by applicable law or agreed to in writing, software
+#  distributed under the License is distributed on an "AS IS" BASIS,
+#  WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+#  See the License for the specific language governing permissions and
+#  limitations under the License.
+
+
+username=
+secretKey=
+
+NAMESRV_ADDR=
+eventNotifyRetryTopic=
+clientRetryTopic=
diff --git a/conf/spring.xml b/conf/spring.xml
new file mode 100644
index 0000000..2f41526
--- /dev/null
+++ b/conf/spring.xml
@@ -0,0 +1,25 @@
+<?xml version="1.0" encoding="UTF-8"?>
+<beans xmlns="http://www.springframework.org/schema/beans"
+       xmlns:xsi="http://www.w3.org/2001/XMLSchema-instance"
+       xmlns:context="http://www.springframework.org/schema/context"
+       xmlns:task="http://www.springframework.org/schema/task"
+       xsi:schemaLocation="http://www.springframework.org/schema/beans
+        classpath:/org/springframework/beans/factory/xml/spring-beans-4.1.xsd
+        http://www.springframework.org/schema/context
+        classpath:/org/springframework/context/config/spring-context-4.1.xsd
+        http://www.springframework.org/schema/tool
+        classpath:/org/springframework/beans/factory/xml/spring-tool-4.1.xsd
+        http://www.springframework.org/schema/task
+        classpath:/org/springframework/scheduling/config/spring-task-4.1.xsd"
+       default-autowire="byName">
+
+    <context:component-scan base-package="org.apache.rocketmq.mqtt.ds"/>
+    <context:component-scan base-package="org.apache.rocketmq.mqtt.cs"/>
+
+
+    <bean id="authManager" class="org.apache.rocketmq.mqtt.ds.auth.AuthManagerSample" init-method="init"/>
+
+    <bean id="metaPersistManager" class="org.apache.rocketmq.mqtt.ds.meta.MetaPersistManagerSample" init-method="init"/>
+
+
+</beans>
\ No newline at end of file
diff --git a/mqtt-common/pom.xml b/mqtt-common/pom.xml
new file mode 100644
index 0000000..f5c75b4
--- /dev/null
+++ b/mqtt-common/pom.xml
@@ -0,0 +1,50 @@
+<?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">
+    <parent>
+        <artifactId>rocketmq-mqtt</artifactId>
+        <groupId>org.apache.rocketmq</groupId>
+        <version>1.0.0-SNAPSHOT</version>
+    </parent>
+    <modelVersion>4.0.0</modelVersion>
+
+    <artifactId>mqtt-common</artifactId>
+
+    <dependencies>
+        <dependency>
+            <groupId>junit</groupId>
+            <artifactId>junit</artifactId>
+            <scope>test</scope>
+        </dependency>
+        <dependency>
+            <groupId>org.mockito</groupId>
+            <artifactId>mockito-core</artifactId>
+            <scope>test</scope>
+        </dependency>
+        <dependency>
+            <groupId>org.apache.commons</groupId>
+            <artifactId>commons-lang3</artifactId>
+        </dependency>
+        <dependency>
+            <groupId>io.netty</groupId>
+            <artifactId>netty-all</artifactId>
+        </dependency>
+        <dependency>
+            <groupId>org.apache.rocketmq</groupId>
+            <artifactId>rocketmq-client</artifactId>
+        </dependency>
+        <dependency>
+            <groupId>com.alibaba</groupId>
+            <artifactId>fastjson</artifactId>
+        </dependency>
+        <dependency>
+            <groupId>org.slf4j</groupId>
+            <artifactId>slf4j-api</artifactId>
+        </dependency>
+        <dependency>
+            <groupId>commons-codec</groupId>
+            <artifactId>commons-codec</artifactId>
+        </dependency>
+    </dependencies>
+</project>
\ No newline at end of file
diff --git a/mqtt-common/src/main/java/org/apache/rocketmq/mqtt/common/facade/AuthManager.java b/mqtt-common/src/main/java/org/apache/rocketmq/mqtt/common/facade/AuthManager.java
new file mode 100644
index 0000000..16f620b
--- /dev/null
+++ b/mqtt-common/src/main/java/org/apache/rocketmq/mqtt/common/facade/AuthManager.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.mqtt.common.facade;
+
+import io.netty.handler.codec.mqtt.MqttMessage;
+import org.apache.rocketmq.mqtt.common.hook.HookResult;
+
+public interface AuthManager {
+    /**
+     * MQTT packet authentication
+     *
+     * @param message
+     * @return
+     */
+    HookResult doAuth(MqttMessage message);
+}
diff --git a/mqtt-common/src/main/java/org/apache/rocketmq/mqtt/common/facade/LmqOffsetStore.java b/mqtt-common/src/main/java/org/apache/rocketmq/mqtt/common/facade/LmqOffsetStore.java
new file mode 100644
index 0000000..b3182de
--- /dev/null
+++ b/mqtt-common/src/main/java/org/apache/rocketmq/mqtt/common/facade/LmqOffsetStore.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.mqtt.common.facade;
+
+import org.apache.rocketmq.mqtt.common.model.Queue;
+import org.apache.rocketmq.mqtt.common.model.QueueOffset;
+import org.apache.rocketmq.mqtt.common.model.Subscription;
+
+import java.util.Map;
+import java.util.Set;
+import java.util.concurrent.CompletableFuture;
+
+public interface LmqOffsetStore {
+    /**
+     *  save lmq offset
+     * @param clientId
+     * @param offsetMap
+     */
+    void save(String clientId, Map<Subscription, Map<Queue, QueueOffset>> offsetMap);
+
+    /**
+     *  get lmq offset
+     * @param clientId
+     * @param subscription
+     * @return
+     */
+    CompletableFuture<Map<Queue, QueueOffset>> getOffset(String clientId, Subscription subscription);
+}
diff --git a/mqtt-common/src/main/java/org/apache/rocketmq/mqtt/common/facade/LmqQueueStore.java b/mqtt-common/src/main/java/org/apache/rocketmq/mqtt/common/facade/LmqQueueStore.java
new file mode 100644
index 0000000..3e71ac7
--- /dev/null
+++ b/mqtt-common/src/main/java/org/apache/rocketmq/mqtt/common/facade/LmqQueueStore.java
@@ -0,0 +1,92 @@
+/*
+ *
+ *  * Licensed to the Apache Software Foundation (ASF) under one or more
+ *  * contributor license agreements.  See the NOTICE file distributed with
+ *  * this work for additional information regarding copyright ownership.
+ *  * The ASF licenses this file to You under the Apache License, Version 2.0
+ *  * (the "License"); you may not use this file except in compliance with
+ *  * the License.  You may obtain a copy of the License at
+ *  *
+ *  *     http://www.apache.org/licenses/LICENSE-2.0
+ *  *
+ *  * Unless required by applicable law or agreed to in writing, software
+ *  * distributed under the License is distributed on an "AS IS" BASIS,
+ *  * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ *  * See the License for the specific language governing permissions and
+ *  * limitations under the License.
+ *
+ */
+
+package org.apache.rocketmq.mqtt.common.facade;
+
+import org.apache.rocketmq.mqtt.common.model.*;
+
+import java.util.Set;
+import java.util.concurrent.CompletableFuture;
+
+public interface LmqQueueStore {
+    String LMQ_PREFIX = "%LMQ%";
+    String PROPERTY_INNER_MULTI_DISPATCH = "INNER_MULTI_DISPATCH";
+    String PROPERTY_INNER_MULTI_QUEUE_OFFSET = "INNER_MULTI_QUEUE_OFFSET";
+    String MULTI_DISPATCH_QUEUE_SPLITTER = ",";
+
+    /**
+     * put message and atomic dispatch to multiple queues
+     *
+     * @param queues
+     * @param message
+     * @return
+     */
+    CompletableFuture<StoreResult> putMessage(Set<String> queues, Message message);
+
+    /**
+     * pull messages
+     *
+     * @param firstTopic
+     * @param queue
+     * @param queueOffset
+     * @param count
+     * @return
+     */
+    CompletableFuture<PullResult> pullMessage(String firstTopic, Queue queue, QueueOffset queueOffset, long count);
+
+    /**
+     * pull last messages
+     *
+     * @param firstTopic
+     * @param queue
+     * @param count
+     * @return
+     */
+    CompletableFuture<PullResult> pullLastMessages(String firstTopic, Queue queue, long count);
+
+    /**
+     * query maxId of Queue
+     *
+     * @param queue
+     * @return
+     */
+    CompletableFuture<Long> queryQueueMaxOffset(Queue queue);
+
+    /**
+     * get readable brokers of the topic
+     *
+     * @param firstTopic
+     * @return
+     */
+    Set<String> getReadableBrokers(String firstTopic);
+
+    /**
+     * retry topic of one mqtt client
+     *
+     * @return
+     */
+    String getClientRetryTopic();
+
+    /**
+     * p2p topic of one mqtt client
+     *
+     * @return
+     */
+    String getClientP2pTopic();
+}
diff --git a/mqtt-common/src/main/java/org/apache/rocketmq/mqtt/common/facade/MetaPersistManager.java b/mqtt-common/src/main/java/org/apache/rocketmq/mqtt/common/facade/MetaPersistManager.java
new file mode 100644
index 0000000..805a3c8
--- /dev/null
+++ b/mqtt-common/src/main/java/org/apache/rocketmq/mqtt/common/facade/MetaPersistManager.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.mqtt.common.facade;
+
+import java.util.Set;
+
+public interface MetaPersistManager {
+
+    /**
+     * get wildcards of the first topic
+     * @param firstTopic
+     * @return
+     */
+    Set<String> getWildcards(String firstTopic);
+
+    /**
+     *  get all first topics
+     * @return
+     */
+    Set<String> getAllFirstTopics();
+
+    /**
+     * get all connect nodes
+     * @return
+     */
+    Set<String> getConnectNodeSet();
+}
diff --git a/mqtt-common/src/main/java/org/apache/rocketmq/mqtt/common/hook/AbstractUpstreamHook.java b/mqtt-common/src/main/java/org/apache/rocketmq/mqtt/common/hook/AbstractUpstreamHook.java
new file mode 100644
index 0000000..2ea972a
--- /dev/null
+++ b/mqtt-common/src/main/java/org/apache/rocketmq/mqtt/common/hook/AbstractUpstreamHook.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.mqtt.common.hook;
+
+import io.netty.handler.codec.mqtt.MqttMessage;
+import org.apache.rocketmq.mqtt.common.model.MqttMessageUpContext;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import java.util.concurrent.CompletableFuture;
+
+public abstract class AbstractUpstreamHook implements UpstreamHook {
+    public static Logger logger = LoggerFactory.getLogger(AbstractUpstreamHook.class);
+    public UpstreamHook nextUpstreamHook;
+
+    @Override
+    public void setNextHook(Hook hook) {
+        this.nextUpstreamHook = (UpstreamHook) hook;
+    }
+
+    @Override
+    public Hook getNextHook() {
+        return this.nextUpstreamHook;
+    }
+
+    @Override
+    public CompletableFuture<HookResult> doHook(MqttMessageUpContext context, MqttMessage msg) {
+        try {
+            CompletableFuture<HookResult> result = processMqttMessage(context,msg);
+            if (nextUpstreamHook == null) {
+                return result;
+            }
+            return result.thenCompose(hookResult -> {
+                if (!hookResult.isSuccess()) {
+                    CompletableFuture<HookResult> nextHookResult = new CompletableFuture<>();
+                    nextHookResult.complete(hookResult);
+                    return nextHookResult;
+                }
+                return nextUpstreamHook.doHook(context, msg);
+            });
+        } catch (Throwable t) {
+            logger.error("",t);
+            CompletableFuture<HookResult> result = new CompletableFuture<>();
+            result.completeExceptionally(t);
+            return result;
+        }
+    }
+
+    public abstract void register();
+
+    public abstract CompletableFuture<HookResult> processMqttMessage(MqttMessageUpContext context, MqttMessage message);
+
+}
diff --git a/mqtt-common/src/main/java/org/apache/rocketmq/mqtt/common/hook/Hook.java b/mqtt-common/src/main/java/org/apache/rocketmq/mqtt/common/hook/Hook.java
new file mode 100644
index 0000000..2d6824c
--- /dev/null
+++ b/mqtt-common/src/main/java/org/apache/rocketmq/mqtt/common/hook/Hook.java
@@ -0,0 +1,36 @@
+/*
+ *
+ *  * Licensed to the Apache Software Foundation (ASF) under one or more
+ *  * contributor license agreements.  See the NOTICE file distributed with
+ *  * this work for additional information regarding copyright ownership.
+ *  * The ASF licenses this file to You under the Apache License, Version 2.0
+ *  * (the "License"); you may not use this file except in compliance with
+ *  * the License.  You may obtain a copy of the License at
+ *  *
+ *  *     http://www.apache.org/licenses/LICENSE-2.0
+ *  *
+ *  * Unless required by applicable law or agreed to in writing, software
+ *  * distributed under the License is distributed on an "AS IS" BASIS,
+ *  * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ *  * See the License for the specific language governing permissions and
+ *  * limitations under the License.
+ *
+ */
+
+package org.apache.rocketmq.mqtt.common.hook;
+
+public interface Hook {
+    /**
+     * set next hook
+     *
+     * @param Hook
+     */
+    void setNextHook(Hook Hook);
+
+    /**
+     * get next hook
+     *
+     * @return
+     */
+    Hook getNextHook();
+}
diff --git a/mqtt-common/src/main/java/org/apache/rocketmq/mqtt/common/hook/HookResult.java b/mqtt-common/src/main/java/org/apache/rocketmq/mqtt/common/hook/HookResult.java
new file mode 100644
index 0000000..3502cdb
--- /dev/null
+++ b/mqtt-common/src/main/java/org/apache/rocketmq/mqtt/common/hook/HookResult.java
@@ -0,0 +1,109 @@
+/*
+ *
+ *  * Licensed to the Apache Software Foundation (ASF) under one or more
+ *  * contributor license agreements.  See the NOTICE file distributed with
+ *  * this work for additional information regarding copyright ownership.
+ *  * The ASF licenses this file to You under the Apache License, Version 2.0
+ *  * (the "License"); you may not use this file except in compliance with
+ *  * the License.  You may obtain a copy of the License at
+ *  *
+ *  *     http://www.apache.org/licenses/LICENSE-2.0
+ *  *
+ *  * Unless required by applicable law or agreed to in writing, software
+ *  * distributed under the License is distributed on an "AS IS" BASIS,
+ *  * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ *  * See the License for the specific language governing permissions and
+ *  * limitations under the License.
+ *
+ */
+
+package org.apache.rocketmq.mqtt.common.hook;
+
+import java.util.Arrays;
+import java.util.Objects;
+import java.util.concurrent.CompletableFuture;
+
+public class HookResult {
+    public static final int SUCCESS = 200;
+    public static final int FAIL = -200;
+    private int code;
+    private int subCode;
+    private String remark;
+    private byte[] data;
+
+    public HookResult(int code, int subCode, String remark, byte[] data) {
+        this.code = code;
+        this.subCode = subCode;
+        this.remark = remark;
+        this.data = data;
+    }
+
+    public HookResult(int code, String remark, byte[] data) {
+        this.code = code;
+        this.remark = remark;
+        this.data = data;
+    }
+
+    public static CompletableFuture<HookResult> newHookResult(int code, String remark, byte[] data) {
+        CompletableFuture<HookResult> result = new CompletableFuture<>();
+        result.complete(new HookResult(code, remark, data));
+        return result;
+    }
+
+    public static CompletableFuture<HookResult> newHookResult(int code, int subCode, String remark, byte[] data) {
+        CompletableFuture<HookResult> result = new CompletableFuture<>();
+        result.complete(new HookResult(code, subCode, remark, data));
+        return result;
+    }
+
+    public boolean isSuccess() {
+        return SUCCESS == code;
+    }
+
+    public int getCode() {
+        return code;
+    }
+
+    public void setCode(int code) {
+        this.code = code;
+    }
+
+    public int getSubCode() {
+        return subCode;
+    }
+
+    public void setSubCode(int subCode) {
+        this.subCode = subCode;
+    }
+
+    public String getRemark() {
+        return remark;
+    }
+
+    public void setRemark(String remark) {
+        this.remark = remark;
+    }
+
+    public byte[] getData() {
+        return data;
+    }
+
+    public void setData(byte[] data) {
+        this.data = data;
+    }
+
+    @Override
+    public boolean equals(Object o) {
+        if (this == o) return true;
+        if (o == null || getClass() != o.getClass()) return false;
+        HookResult that = (HookResult) o;
+        return code == that.code && subCode == that.subCode && Objects.equals(remark, that.remark) && Arrays.equals(data, that.data);
+    }
+
+    @Override
+    public int hashCode() {
+        int result = Objects.hash(code, subCode, remark);
+        result = 31 * result + Arrays.hashCode(data);
+        return result;
+    }
+}
diff --git a/mqtt-common/src/main/java/org/apache/rocketmq/mqtt/common/hook/UpstreamHook.java b/mqtt-common/src/main/java/org/apache/rocketmq/mqtt/common/hook/UpstreamHook.java
new file mode 100644
index 0000000..34b57b0
--- /dev/null
+++ b/mqtt-common/src/main/java/org/apache/rocketmq/mqtt/common/hook/UpstreamHook.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.mqtt.common.hook;
+
+import io.netty.handler.codec.mqtt.MqttMessage;
+import org.apache.rocketmq.mqtt.common.model.MqttMessageUpContext;
+
+import java.util.concurrent.CompletableFuture;
+
+public interface UpstreamHook extends Hook{
+
+    /**
+     *  do hook in upstream
+     * @param context
+     * @param msg
+     * @return
+     */
+    CompletableFuture<HookResult> doHook(MqttMessageUpContext context, MqttMessage msg);
+
+}
diff --git a/mqtt-common/src/main/java/org/apache/rocketmq/mqtt/common/hook/UpstreamHookEnum.java b/mqtt-common/src/main/java/org/apache/rocketmq/mqtt/common/hook/UpstreamHookEnum.java
new file mode 100644
index 0000000..0d49d1a
--- /dev/null
+++ b/mqtt-common/src/main/java/org/apache/rocketmq/mqtt/common/hook/UpstreamHookEnum.java
@@ -0,0 +1,25 @@
+/*
+ *
+ *  * Licensed to the Apache Software Foundation (ASF) under one or more
+ *  * contributor license agreements.  See the NOTICE file distributed with
+ *  * this work for additional information regarding copyright ownership.
+ *  * The ASF licenses this file to You under the Apache License, Version 2.0
+ *  * (the "License"); you may not use this file except in compliance with
+ *  * the License.  You may obtain a copy of the License at
+ *  *
+ *  *     http://www.apache.org/licenses/LICENSE-2.0
+ *  *
+ *  * Unless required by applicable law or agreed to in writing, software
+ *  * distributed under the License is distributed on an "AS IS" BASIS,
+ *  * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ *  * See the License for the specific language governing permissions and
+ *  * limitations under the License.
+ *
+ */
+
+package org.apache.rocketmq.mqtt.common.hook;
+
+public enum UpstreamHookEnum {
+    AUTH,
+    UPSTREAM_PROCESS
+}
diff --git a/mqtt-common/src/main/java/org/apache/rocketmq/mqtt/common/hook/UpstreamHookManager.java b/mqtt-common/src/main/java/org/apache/rocketmq/mqtt/common/hook/UpstreamHookManager.java
new file mode 100644
index 0000000..d38fcc7
--- /dev/null
+++ b/mqtt-common/src/main/java/org/apache/rocketmq/mqtt/common/hook/UpstreamHookManager.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.mqtt.common.hook;
+
+import io.netty.handler.codec.mqtt.MqttMessage;
+import org.apache.rocketmq.mqtt.common.model.MqttMessageUpContext;
+
+import java.util.concurrent.CompletableFuture;
+
+public interface UpstreamHookManager {
+    /**
+     * add a hook
+     *
+     * @param index
+     * @param upstreamHook
+     */
+    void addHook(int index, UpstreamHook upstreamHook);
+
+    /**
+     * do hook in upstream
+     *
+     * @param context
+     * @param msg
+     * @return
+     */
+    CompletableFuture<HookResult> doUpstreamHook(MqttMessageUpContext context, MqttMessage msg);
+
+}
diff --git a/mqtt-common/src/main/java/org/apache/rocketmq/mqtt/common/model/Constants.java b/mqtt-common/src/main/java/org/apache/rocketmq/mqtt/common/model/Constants.java
new file mode 100644
index 0000000..b2503a3
--- /dev/null
+++ b/mqtt-common/src/main/java/org/apache/rocketmq/mqtt/common/model/Constants.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.mqtt.common.model;
+
+public class Constants {
+    public static final String NAMESPACE_SPLITER = "%";
+    public static final String MQTT_TOPIC_DELIMITER = "/";
+
+    public static final String ADDFLAG = "+";
+    public static final String JINFLAG = "#";
+
+    public static final String P2P = "/p2p/";
+    public static final String RETRY = "/retry/";
+
+    public static String PROPERTY_NAMESPACE = "namespace";
+    public static final String PROPERTY_ORIGIN_MQTT_TOPIC = "originMqttTopic";
+    public static final String PROPERTY_MQTT_QOS = "qoslevel";
+    public static final String PROPERTY_MQTT_CLEAN_SESSION = "cleansessionflag";
+    public static final String PROPERTY_MQTT_CLIENT = "clientId";
+    public static final String PROPERTY_MQTT_RETRY_TIMES = "retryTimes";
+    public static final String PROPERTY_MQTT_EXT_DATA = "extData";
+
+
+    public static final String PROPERTY_MQTT_MSG_EVENT_RETRY_NODE = "retryNode";
+    public static final String PROPERTY_MQTT_MSG_EVENT_RETRY_TIME = "retryTime";
+
+    public static final String MQTT_TAG = "MQTT_COMMON";
+
+}
diff --git a/mqtt-common/src/main/java/org/apache/rocketmq/mqtt/common/model/Message.java b/mqtt-common/src/main/java/org/apache/rocketmq/mqtt/common/model/Message.java
new file mode 100644
index 0000000..d15a7fc
--- /dev/null
+++ b/mqtt-common/src/main/java/org/apache/rocketmq/mqtt/common/model/Message.java
@@ -0,0 +1,217 @@
+/*
+ *
+ *  * Licensed to the Apache Software Foundation (ASF) under one or more
+ *  * contributor license agreements.  See the NOTICE file distributed with
+ *  * this work for additional information regarding copyright ownership.
+ *  * The ASF licenses this file to You under the Apache License, Version 2.0
+ *  * (the "License"); you may not use this file except in compliance with
+ *  * the License.  You may obtain a copy of the License at
+ *  *
+ *  *     http://www.apache.org/licenses/LICENSE-2.0
+ *  *
+ *  * Unless required by applicable law or agreed to in writing, software
+ *  * distributed under the License is distributed on an "AS IS" BASIS,
+ *  * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ *  * See the License for the specific language governing permissions and
+ *  * limitations under the License.
+ *
+ */
+
+package org.apache.rocketmq.mqtt.common.model;
+
+import org.apache.commons.lang3.StringUtils;
+import org.apache.rocketmq.mqtt.common.util.TopicUtils;
+
+import java.util.HashMap;
+import java.util.Map;
+import java.util.Objects;
+
+
+public class Message {
+    private String msgId;
+    private String firstTopic;
+    private String originTopic;
+    private long offset;
+    private long nextOffset;
+    private int retry;
+    private byte[] payload;
+    private long bornTimestamp;
+    private long storeTimestamp;
+    private int ack = -1;
+    private Map<String, String> userProperties = new HashMap<>();
+
+    public static String propertyFirstTopic = "firstTopic";
+    public static String propertyOriginTopic = "originTopic";
+    public static String propertyOffset = "offset";
+    public static String propertyNextOffset = "nextOffset";
+    public static String propertyMsgId = "msgId";
+    public static String propertyRetry = "retry";
+    public static String propertyBornTime = "bornTime";
+    public static String propertyStoreTime = "storeTime";
+    public static String propertyUserProperties = "extData";
+
+    public static String extPropertyMqttRealTopic = "mqttRealTopic";
+    public static String extPropertyQoS = "qoslevel";
+    public static String extPropertyCleanSessionFlag = "cleansessionflag";
+    public static String extPropertyNamespaceId = "namespace";
+    public static String extPropertyClientId = "clientId";
+
+
+    public Message copy() {
+        Message message = new Message();
+        message.setMsgId(msgId);
+        message.setFirstTopic(this.firstTopic);
+        message.setOriginTopic(this.getOriginTopic());
+        message.setOffset(this.getOffset());
+        message.setNextOffset(this.getNextOffset());
+        message.setRetry(this.getRetry());
+        message.setPayload(this.getPayload());
+        message.setBornTimestamp(this.bornTimestamp);
+        message.setStoreTimestamp(this.storeTimestamp);
+        message.getUserProperties().putAll(this.userProperties);
+        return message;
+    }
+
+    public Integer qos() {
+        if (getUserProperties() == null) {
+            return null;
+        }
+        if (!getUserProperties().containsKey(extPropertyQoS)) {
+            return null;
+        }
+        return Integer.parseInt(getUserProperties().get(extPropertyQoS));
+    }
+
+    public String getMsgId() {
+        return msgId;
+    }
+
+    public void setMsgId(String msgId) {
+        this.msgId = msgId;
+    }
+
+    public String getFirstTopic() {
+        return firstTopic;
+    }
+
+    public void setFirstTopic(String firstTopic) {
+        this.firstTopic = firstTopic;
+    }
+
+    public String getOriginTopic() {
+        return originTopic;
+    }
+
+    public void setOriginTopic(String originTopic) {
+        this.originTopic = originTopic;
+    }
+
+    public long getOffset() {
+        return offset;
+    }
+
+    public void setOffset(long offset) {
+        this.offset = offset;
+    }
+
+    public int getRetry() {
+        return retry;
+    }
+
+    public void setRetry(int retry) {
+        this.retry = retry;
+    }
+
+    public byte[] getPayload() {
+        return payload;
+    }
+
+    public void setPayload(byte[] payload) {
+        this.payload = payload;
+    }
+
+    public long getNextOffset() {
+        return nextOffset;
+    }
+
+    public void setNextOffset(long nextOffset) {
+        this.nextOffset = nextOffset;
+    }
+
+    public long getBornTimestamp() {
+        return bornTimestamp;
+    }
+
+    public void setBornTimestamp(long bornTimestamp) {
+        this.bornTimestamp = bornTimestamp;
+    }
+
+    public long getStoreTimestamp() {
+        return storeTimestamp;
+    }
+
+    public void setStoreTimestamp(long storeTimestamp) {
+        this.storeTimestamp = storeTimestamp;
+    }
+
+    public int getAck() {
+        return ack;
+    }
+
+    public void setAck(int ack) {
+        this.ack = ack;
+    }
+
+    public Map<String, String> getUserProperties() {
+        return userProperties;
+    }
+
+    public void putUserProperty(String key, String value) {
+        if (StringUtils.isBlank(key) || StringUtils.isBlank(value)) {
+            return;
+        }
+        userProperties.put(key, value);
+    }
+
+    public String getUserProperty(String key) {
+        if (StringUtils.isBlank(key)) {
+            return null;
+        }
+        return userProperties.get(key);
+    }
+
+    public void clearUserProperty(String key) {
+        if (StringUtils.isBlank(key)) {
+            return;
+        }
+        if (userProperties == null) {
+            return;
+        }
+        userProperties.remove(key);
+    }
+
+    public boolean isP2P() {
+        if (TopicUtils.isP2P(TopicUtils.decode(firstTopic).getSecondTopic())) {
+            return true;
+        }
+        return false;
+    }
+
+    @Override
+    public boolean equals(Object o) {
+        if (this == o) {
+            return true;
+        }
+        if (o == null || getClass() != o.getClass()) {
+            return false;
+        }
+        Message message = (Message)o;
+        return offset == message.offset;
+    }
+
+    @Override
+    public int hashCode() {
+        return Objects.hash(offset);
+    }
+
+}
diff --git a/mqtt-common/src/main/java/org/apache/rocketmq/mqtt/common/model/MessageEvent.java b/mqtt-common/src/main/java/org/apache/rocketmq/mqtt/common/model/MessageEvent.java
new file mode 100644
index 0000000..05c3f33
--- /dev/null
+++ b/mqtt-common/src/main/java/org/apache/rocketmq/mqtt/common/model/MessageEvent.java
@@ -0,0 +1,79 @@
+/*
+ *
+ *  * Licensed to the Apache Software Foundation (ASF) under one or more
+ *  * contributor license agreements.  See the NOTICE file distributed with
+ *  * this work for additional information regarding copyright ownership.
+ *  * The ASF licenses this file to You under the Apache License, Version 2.0
+ *  * (the "License"); you may not use this file except in compliance with
+ *  * the License.  You may obtain a copy of the License at
+ *  *
+ *  *     http://www.apache.org/licenses/LICENSE-2.0
+ *  *
+ *  * Unless required by applicable law or agreed to in writing, software
+ *  * distributed under the License is distributed on an "AS IS" BASIS,
+ *  * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ *  * See the License for the specific language governing permissions and
+ *  * limitations under the License.
+ *
+ */
+
+package org.apache.rocketmq.mqtt.common.model;
+
+import java.util.Objects;
+
+
+public class MessageEvent {
+    private String pubTopic;
+    private String namespace;
+    private long queueId;
+    private String brokerName;
+
+    @Override
+    public boolean equals(Object o) {
+        if (this == o) {
+            return true;
+        }
+        if (o == null || getClass() != o.getClass()) {
+            return false;
+        }
+        MessageEvent that = (MessageEvent) o;
+        return Objects.equals(pubTopic, that.pubTopic) && Objects.equals(namespace, that.namespace) && Objects.equals(queueId, that.queueId) && Objects.equals(brokerName, that.brokerName);
+    }
+
+    @Override
+    public int hashCode() {
+        return Objects.hash(pubTopic, namespace, queueId, brokerName);
+    }
+
+    public String getPubTopic() {
+        return pubTopic;
+    }
+
+    public void setPubTopic(String pubTopic) {
+        this.pubTopic = pubTopic;
+    }
+
+    public String getNamespace() {
+        return namespace;
+    }
+
+    public void setNamespace(String namespace) {
+        this.namespace = namespace;
+    }
+
+    public long getQueueId() {
+        return queueId;
+    }
+
+    public void setQueueId(long queueId) {
+        this.queueId = queueId;
+    }
+
+    public String getBrokerName() {
+        return brokerName;
+    }
+
+    public void setBrokerName(String brokerName) {
+        this.brokerName = brokerName;
+    }
+}
diff --git a/mqtt-common/src/main/java/org/apache/rocketmq/mqtt/common/model/MqttMessageUpContext.java b/mqtt-common/src/main/java/org/apache/rocketmq/mqtt/common/model/MqttMessageUpContext.java
new file mode 100644
index 0000000..234c755
--- /dev/null
+++ b/mqtt-common/src/main/java/org/apache/rocketmq/mqtt/common/model/MqttMessageUpContext.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.mqtt.common.model;
+
+public class MqttMessageUpContext {
+    private String namespace;
+    private String clientId;
+    private String channelId;
+    private String node;
+
+    public String getNamespace() {
+        return namespace;
+    }
+
+    public void setNamespace(String namespace) {
+        this.namespace = namespace;
+    }
+
+    public String getClientId() {
+        return clientId;
+    }
+
+    public void setClientId(String clientId) {
+        this.clientId = clientId;
+    }
+
+    public String getChannelId() {
+        return channelId;
+    }
+
+    public void setChannelId(String channelId) {
+        this.channelId = channelId;
+    }
+
+    public String getNode() {
+        return node;
+    }
+
+    public void setNode(String node) {
+        this.node = node;
+    }
+}
diff --git a/mqtt-common/src/main/java/org/apache/rocketmq/mqtt/common/model/MqttTopic.java b/mqtt-common/src/main/java/org/apache/rocketmq/mqtt/common/model/MqttTopic.java
new file mode 100644
index 0000000..894a03b
--- /dev/null
+++ b/mqtt-common/src/main/java/org/apache/rocketmq/mqtt/common/model/MqttTopic.java
@@ -0,0 +1,47 @@
+/*
+ *
+ *  * Licensed to the Apache Software Foundation (ASF) under one or more
+ *  * contributor license agreements.  See the NOTICE file distributed with
+ *  * this work for additional information regarding copyright ownership.
+ *  * The ASF licenses this file to You under the Apache License, Version 2.0
+ *  * (the "License"); you may not use this file except in compliance with
+ *  * the License.  You may obtain a copy of the License at
+ *  *
+ *  *     http://www.apache.org/licenses/LICENSE-2.0
+ *  *
+ *  * Unless required by applicable law or agreed to in writing, software
+ *  * distributed under the License is distributed on an "AS IS" BASIS,
+ *  * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ *  * See the License for the specific language governing permissions and
+ *  * limitations under the License.
+ *
+ */
+
+package org.apache.rocketmq.mqtt.common.model;
+
+
+public class MqttTopic {
+    private String firstTopic;
+    private String secondTopic;
+
+    public MqttTopic(String firstTopic, String secondTopic) {
+        this.firstTopic = firstTopic;
+        this.secondTopic = secondTopic;
+    }
+
+    public String getFirstTopic() {
+        return firstTopic;
+    }
+
+    public void setFirstTopic(String firstTopic) {
+        this.firstTopic = firstTopic;
+    }
+
+    public String getSecondTopic() {
+        return secondTopic;
+    }
+
+    public void setSecondTopic(String secondTopic) {
+        this.secondTopic = secondTopic;
+    }
+}
diff --git a/mqtt-common/src/main/java/org/apache/rocketmq/mqtt/common/model/PullResult.java b/mqtt-common/src/main/java/org/apache/rocketmq/mqtt/common/model/PullResult.java
new file mode 100644
index 0000000..d208c90
--- /dev/null
+++ b/mqtt-common/src/main/java/org/apache/rocketmq/mqtt/common/model/PullResult.java
@@ -0,0 +1,65 @@
+/*
+ *
+ *  * Licensed to the Apache Software Foundation (ASF) under one or more
+ *  * contributor license agreements.  See the NOTICE file distributed with
+ *  * this work for additional information regarding copyright ownership.
+ *  * The ASF licenses this file to You under the Apache License, Version 2.0
+ *  * (the "License"); you may not use this file except in compliance with
+ *  * the License.  You may obtain a copy of the License at
+ *  *
+ *  *     http://www.apache.org/licenses/LICENSE-2.0
+ *  *
+ *  * Unless required by applicable law or agreed to in writing, software
+ *  * distributed under the License is distributed on an "AS IS" BASIS,
+ *  * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ *  * See the License for the specific language governing permissions and
+ *  * limitations under the License.
+ *
+ */
+
+package org.apache.rocketmq.mqtt.common.model;
+
+
+import java.util.List;
+
+
+public class PullResult {
+    public static final int PULL_SUCCESS = 301;
+    public static final int PULL_OFFSET_MOVED = 302;
+    private int code = PULL_SUCCESS;
+    private String remark;
+    private List<Message> messageList;
+    private QueueOffset nextQueueOffset;
+
+    public int getCode() {
+        return code;
+    }
+
+    public void setCode(int code) {
+        this.code = code;
+    }
+
+    public String getRemark() {
+        return remark;
+    }
+
+    public void setRemark(String remark) {
+        this.remark = remark;
+    }
+
+    public List<Message> getMessageList() {
+        return messageList;
+    }
+
+    public void setMessageList(List<Message> messageList) {
+        this.messageList = messageList;
+    }
+
+    public QueueOffset getNextQueueOffset() {
+        return nextQueueOffset;
+    }
+
+    public void setNextQueueOffset(QueueOffset nextQueueOffset) {
+        this.nextQueueOffset = nextQueueOffset;
+    }
+}
diff --git a/mqtt-common/src/main/java/org/apache/rocketmq/mqtt/common/model/Queue.java b/mqtt-common/src/main/java/org/apache/rocketmq/mqtt/common/model/Queue.java
new file mode 100644
index 0000000..e286827
--- /dev/null
+++ b/mqtt-common/src/main/java/org/apache/rocketmq/mqtt/common/model/Queue.java
@@ -0,0 +1,116 @@
+/*
+ *
+ *  * Licensed to the Apache Software Foundation (ASF) under one or more
+ *  * contributor license agreements.  See the NOTICE file distributed with
+ *  * this work for additional information regarding copyright ownership.
+ *  * The ASF licenses this file to You under the Apache License, Version 2.0
+ *  * (the "License"); you may not use this file except in compliance with
+ *  * the License.  You may obtain a copy of the License at
+ *  *
+ *  *     http://www.apache.org/licenses/LICENSE-2.0
+ *  *
+ *  * Unless required by applicable law or agreed to in writing, software
+ *  * distributed under the License is distributed on an "AS IS" BASIS,
+ *  * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ *  * See the License for the specific language governing permissions and
+ *  * limitations under the License.
+ *
+ */
+
+package org.apache.rocketmq.mqtt.common.model;
+
+import org.apache.commons.lang3.StringUtils;
+import org.apache.rocketmq.mqtt.common.util.TopicUtils;
+
+
+public class Queue {
+    protected long queueId;
+    protected String queueName;
+    protected String brokerName;
+
+    public Queue() {
+    }
+
+    public Queue(long queueId, String queueName, String brokerName) {
+        this.queueId = queueId;
+        this.queueName = queueName;
+        this.brokerName = brokerName;
+    }
+
+    public boolean isLmq() {
+        return StringUtils.isNotBlank(brokerName);
+    }
+
+    public String toFirstTopic() {
+        return TopicUtils.decode(queueName).getFirstTopic();
+    }
+
+    public boolean isRetry() {
+        return TopicUtils.isRetryTopic(queueName);
+    }
+
+    public boolean isP2p() {
+        return TopicUtils.isP2pTopic(queueName);
+    }
+
+    public long getQueueId() {
+        return queueId;
+    }
+
+    public void setQueueId(long queueId) {
+        this.queueId = queueId;
+    }
+
+    public String getBrokerName() {
+        return brokerName;
+    }
+
+    public void setBrokerName(String brokerName) {
+        this.brokerName = brokerName;
+    }
+
+    public String getQueueName() {
+        return queueName;
+    }
+
+    public void setQueueName(String queueName) {
+        this.queueName = queueName;
+    }
+
+    @Override
+    public boolean equals(Object o) {
+        if (this == o) {
+            return true;
+        }
+        if (o == null || getClass() != o.getClass()) {
+            return false;
+        }
+
+        Queue queue = (Queue) o;
+
+        if (queueId != queue.queueId) {
+            return false;
+        }
+        if (queueName != null ? !queueName.equals(queue.queueName) : queue.queueName != null) {
+            return false;
+        }
+        return brokerName != null ? brokerName.equals(queue.brokerName) : queue.brokerName == null;
+    }
+
+    @Override
+    public int hashCode() {
+        int result = (int) (queueId ^ (queueId >>> 32));
+        result = 31 * result + (queueName != null ? queueName.hashCode() : 0);
+        result = 31 * result + (brokerName != null ? brokerName.hashCode() : 0);
+        return result;
+    }
+
+    @Override
+    public String toString() {
+        return "Queue{" +
+                "queueId=" + queueId +
+                ", queueName='" + queueName + '\'' +
+                ", brokerName='" + brokerName + '\'' +
+                '}';
+    }
+}
diff --git a/mqtt-common/src/main/java/org/apache/rocketmq/mqtt/common/model/QueueOffset.java b/mqtt-common/src/main/java/org/apache/rocketmq/mqtt/common/model/QueueOffset.java
new file mode 100644
index 0000000..710bccc
--- /dev/null
+++ b/mqtt-common/src/main/java/org/apache/rocketmq/mqtt/common/model/QueueOffset.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.mqtt.common.model;
+
+import java.util.Objects;
+
+
+public class QueueOffset {
+    private volatile long offset = Long.MAX_VALUE;
+    private volatile byte initializingStatus = -1;
+
+    public QueueOffset() {
+    }
+
+    public QueueOffset(long offset) {
+        this.offset = offset;
+    }
+
+    public boolean isInitialized() {
+        return offset != Long.MAX_VALUE || initializingStatus == 1;
+    }
+
+    public boolean isInitializing() {
+        return initializingStatus == 0;
+    }
+
+    public void setInitialized() {
+        initializingStatus = 1;
+    }
+
+    public void setInitializing() {
+        initializingStatus = 0;
+    }
+
+    public long getOffset() {
+        return offset;
+    }
+
+    public void setOffset(long offset) {
+        this.offset = offset;
+    }
+
+    @Override
+    public boolean equals(Object o) {
+        if (this == o) return true;
+        if (o == null || getClass() != o.getClass()) return false;
+        QueueOffset that = (QueueOffset) o;
+        return offset == that.offset && initializingStatus == that.initializingStatus;
+    }
+
+    @Override
+    public int hashCode() {
+        return Objects.hash(offset, initializingStatus);
+    }
+}
diff --git a/mqtt-common/src/main/java/org/apache/rocketmq/mqtt/common/model/Remark.java b/mqtt-common/src/main/java/org/apache/rocketmq/mqtt/common/model/Remark.java
new file mode 100644
index 0000000..8ae1331
--- /dev/null
+++ b/mqtt-common/src/main/java/org/apache/rocketmq/mqtt/common/model/Remark.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.mqtt.common.model;
+
+
+public class Remark {
+
+    public static final String SUCCESS = "success";
+    public static final String FAIL = "fail";
+    public static final String CLIENT_ID_CONFLICT = "clientIdConflict";
+    public static final String INVALID_PARAM = "Invalid Param";
+    public static final String AUTH_FAILED = "Auth Failed";
+    public static final String OVERFLOW = "overflow";
+    public static final String EXCEPTION = "exception";
+    public static final String EXPIRED = "expire";
+}
diff --git a/mqtt-common/src/main/java/org/apache/rocketmq/mqtt/common/model/RpcCode.java b/mqtt-common/src/main/java/org/apache/rocketmq/mqtt/common/model/RpcCode.java
new file mode 100644
index 0000000..0a43185
--- /dev/null
+++ b/mqtt-common/src/main/java/org/apache/rocketmq/mqtt/common/model/RpcCode.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.mqtt.common.model;
+
+public class RpcCode {
+    public static final int SUCCESS = 1;
+    public static final int FAIL = -1;
+
+    public static final int CMD_NOTIFY_MQTT_MESSAGE = 201;
+    public static final int CMD_CLOSE_CHANNEL = 203;
+}
diff --git a/mqtt-common/src/main/java/org/apache/rocketmq/mqtt/common/model/RpcHeader.java b/mqtt-common/src/main/java/org/apache/rocketmq/mqtt/common/model/RpcHeader.java
new file mode 100644
index 0000000..2b38b83
--- /dev/null
+++ b/mqtt-common/src/main/java/org/apache/rocketmq/mqtt/common/model/RpcHeader.java
@@ -0,0 +1,25 @@
+/*
+ *
+ *  * Licensed to the Apache Software Foundation (ASF) under one or more
+ *  * contributor license agreements.  See the NOTICE file distributed with
+ *  * this work for additional information regarding copyright ownership.
+ *  * The ASF licenses this file to You under the Apache License, Version 2.0
+ *  * (the "License"); you may not use this file except in compliance with
+ *  * the License.  You may obtain a copy of the License at
+ *  *
+ *  *     http://www.apache.org/licenses/LICENSE-2.0
+ *  *
+ *  * Unless required by applicable law or agreed to in writing, software
+ *  * distributed under the License is distributed on an "AS IS" BASIS,
+ *  * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ *  * See the License for the specific language governing permissions and
+ *  * limitations under the License.
+ *
+ */
+
+package org.apache.rocketmq.mqtt.common.model;
+
+public class RpcHeader {
+    public static final String MQTT_CHANNEL_ID = "channelId";
+
+}
diff --git a/mqtt-common/src/main/java/org/apache/rocketmq/mqtt/common/model/StoreResult.java b/mqtt-common/src/main/java/org/apache/rocketmq/mqtt/common/model/StoreResult.java
new file mode 100644
index 0000000..23a8d22
--- /dev/null
+++ b/mqtt-common/src/main/java/org/apache/rocketmq/mqtt/common/model/StoreResult.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.mqtt.common.model;
+
+public class StoreResult {
+    private Queue queue;
+    private String msgId;
+
+    public Queue getQueue() {
+        return queue;
+    }
+
+    public void setQueue(Queue queue) {
+        this.queue = queue;
+    }
+
+    public String getMsgId() {
+        return msgId;
+    }
+
+    public void setMsgId(String msgId) {
+        this.msgId = msgId;
+    }
+}
diff --git a/mqtt-common/src/main/java/org/apache/rocketmq/mqtt/common/model/Subscription.java b/mqtt-common/src/main/java/org/apache/rocketmq/mqtt/common/model/Subscription.java
new file mode 100644
index 0000000..52788af
--- /dev/null
+++ b/mqtt-common/src/main/java/org/apache/rocketmq/mqtt/common/model/Subscription.java
@@ -0,0 +1,114 @@
+/*
+ *
+ *  * Licensed to the Apache Software Foundation (ASF) under one or more
+ *  * contributor license agreements.  See the NOTICE file distributed with
+ *  * this work for additional information regarding copyright ownership.
+ *  * The ASF licenses this file to You under the Apache License, Version 2.0
+ *  * (the "License"); you may not use this file except in compliance with
+ *  * the License.  You may obtain a copy of the License at
+ *  *
+ *  *     http://www.apache.org/licenses/LICENSE-2.0
+ *  *
+ *  * Unless required by applicable law or agreed to in writing, software
+ *  * distributed under the License is distributed on an "AS IS" BASIS,
+ *  * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ *  * See the License for the specific language governing permissions and
+ *  * limitations under the License.
+ *
+ */
+
+package org.apache.rocketmq.mqtt.common.model;
+
+import org.apache.rocketmq.mqtt.common.util.TopicUtils;
+
+
+public class Subscription {
+    private String topicFilter;
+    private int qos;
+
+    public Subscription() {
+    }
+
+    public Subscription(String topicFilter) {
+        this.topicFilter = topicFilter;
+    }
+
+    public Subscription(String topicFilter, int qos) {
+        this.topicFilter = topicFilter;
+        this.qos = qos;
+    }
+
+    public boolean isWildCard() {
+        return topicFilter != null &&
+                (topicFilter.contains(Constants.JINFLAG) || topicFilter.contains(Constants.ADDFLAG));
+    }
+
+    public String toFirstTopic() {
+        return TopicUtils.decode(topicFilter).getFirstTopic();
+    }
+
+    public String toQueueName() {
+        return topicFilter;
+    }
+
+    public static Subscription newP2pSubscription(String clientId) {
+        Subscription p2pSubscription = new Subscription();
+        p2pSubscription.setTopicFilter(TopicUtils.getP2pTopic(clientId));
+        p2pSubscription.setQos(1);
+        return p2pSubscription;
+    }
+
+    public static Subscription newRetrySubscription(String clientId) {
+        Subscription retrySubscription = new Subscription();
+        retrySubscription.setTopicFilter(TopicUtils.getRetryTopic(clientId));
+        retrySubscription.setQos(1);
+        return retrySubscription;
+    }
+
+    public boolean isRetry() {
+        return TopicUtils.isRetryTopic(topicFilter);
+    }
+
+    public boolean isP2p() {
+        return TopicUtils.isP2pTopic(topicFilter);
+    }
+
+    @Override
+    public boolean equals(Object o) {
+        if (this == o) { return true; }
+        if (o == null || getClass() != o.getClass()) { return false; }
+
+        Subscription that = (Subscription)o;
+
+        return topicFilter != null ? topicFilter.equals(that.topicFilter) : that.topicFilter == null;
+    }
+
+    @Override
+    public int hashCode() {
+        return topicFilter != null ? topicFilter.hashCode() : 0;
+    }
+
+    public String getTopicFilter() {
+        return topicFilter;
+    }
+
+    public void setTopicFilter(String topicFilter) {
+        this.topicFilter = topicFilter;
+    }
+
+    public int getQos() {
+        return qos;
+    }
+
+    public void setQos(int qos) {
+        this.qos = qos;
+    }
+
+    @Override
+    public String toString() {
+        return "Subscription{" +
+            "topicFilter='" + topicFilter + '\'' +
+            ", qos=" + qos +
+            '}';
+    }
+}
diff --git a/mqtt-common/src/main/java/org/apache/rocketmq/mqtt/common/model/Trie.java b/mqtt-common/src/main/java/org/apache/rocketmq/mqtt/common/model/Trie.java
new file mode 100644
index 0000000..10ffde9
--- /dev/null
+++ b/mqtt-common/src/main/java/org/apache/rocketmq/mqtt/common/model/Trie.java
@@ -0,0 +1,240 @@
+/*
+ *
+ *  * Licensed to the Apache Software Foundation (ASF) under one or more
+ *  * contributor license agreements.  See the NOTICE file distributed with
+ *  * this work for additional information regarding copyright ownership.
+ *  * The ASF licenses this file to You under the Apache License, Version 2.0
+ *  * (the "License"); you may not use this file except in compliance with
+ *  * the License.  You may obtain a copy of the License at
+ *  *
+ *  *     http://www.apache.org/licenses/LICENSE-2.0
+ *  *
+ *  * Unless required by applicable law or agreed to in writing, software
+ *  * distributed under the License is distributed on an "AS IS" BASIS,
+ *  * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ *  * See the License for the specific language governing permissions and
+ *  * limitations under the License.
+ *
+ */
+
+package org.apache.rocketmq.mqtt.common.model;
+
+
+import org.apache.rocketmq.mqtt.common.util.TopicUtils;
+
+import java.util.*;
+import java.util.concurrent.ConcurrentHashMap;
+
+
+public class Trie<K, V> {
+
+    private TrieNode<K, V> rootNode = new TrieNode(null);
+
+    public synchronized V addNode(String key, V nodeValue, K nodeKey) {
+        try {
+            String[] keyArray = key.split(Constants.MQTT_TOPIC_DELIMITER);
+            TrieNode<K, V> currentNode = rootNode;
+            int level = 0;
+            while (level < keyArray.length) {
+                TrieNode<K, V> trieNode = currentNode.children.get(keyArray[level]);
+                if (trieNode == null) {
+                    trieNode = new TrieNode(currentNode);
+                    TrieNode oldNode = currentNode.children.putIfAbsent(keyArray[level], trieNode);
+                    if (oldNode != null) {
+                        trieNode = oldNode;
+                    }
+                }
+                level++;
+                currentNode = trieNode;
+            }
+            V old = currentNode.valueSet.put(nodeKey, nodeValue);
+            return old;
+        } catch (Throwable e) {
+            throw new TrieException(e);
+        }
+    }
+
+    /**
+     * @param key
+     * @param valueKey
+     * @return null if can not find the key and valueKey or return the value
+     */
+    public synchronized V deleteNode(String key, K valueKey) {
+        try {
+            String[] keyArray = key.split(Constants.MQTT_TOPIC_DELIMITER);
+            TrieNode<K, V> currentNode = rootNode;
+            int level = 0;
+            while (level < keyArray.length) {
+                TrieNode trieNode = currentNode.children.get(keyArray[level]);
+                if (trieNode == null) {
+                    break;
+                }
+                level++;
+                currentNode = trieNode;
+            }
+            V oldValue = currentNode.valueSet.remove(valueKey);
+            //clean the empty node
+            while (currentNode.children.isEmpty() && currentNode.valueSet.isEmpty()) {
+                if (currentNode.parentNode != null) {
+                    currentNode.parentNode.children.remove(keyArray[--level]);
+                    currentNode = currentNode.parentNode;
+                } else {
+                    break;
+                }
+            }
+            return oldValue;
+        } catch (Throwable e) {
+            throw new TrieException(e);
+        }
+    }
+
+    public long countSubRecords() {
+        return countLevelRecords(rootNode);
+    }
+
+    private long countLevelRecords(TrieNode<K, V> currentNode) {
+        if (currentNode == null) {
+            return 0;
+        }
+        if (currentNode.children.isEmpty()) {
+            return currentNode.valueSet.size();
+        }
+        long childrenCount = 0;
+        for (Map.Entry<String, TrieNode<K, V>> entry : currentNode.children.entrySet()) {
+            childrenCount += countLevelRecords(entry.getValue());
+        }
+        return childrenCount + currentNode.valueSet.size();
+    }
+
+    public Map<K, V> getNode(String key) {
+        try {
+            String[] keyArray = key.split(Constants.MQTT_TOPIC_DELIMITER);
+            Map<K, V> result = findValueSet(rootNode, keyArray, 0, keyArray.length, false);
+            return result;
+        } catch (Throwable e) {
+            throw new TrieException(e);
+        }
+    }
+
+    public void traverseAll(TrieMethod<K, V> method) {
+        StringBuilder builder = new StringBuilder(128);
+        traverse(rootNode, method, builder);
+    }
+
+    public Set<String> getNodePath(String key) {
+        try {
+            String[] keyArray = key.split(Constants.MQTT_TOPIC_DELIMITER);
+            StringBuilder builder = new StringBuilder(key.length());
+            Set<String> result = findValuePath(rootNode, keyArray, 0, keyArray.length, builder, false);
+            return result;
+        } catch (Throwable e) {
+            throw new TrieException(e);
+        }
+    }
+
+    private Set<String> findValuePath(TrieNode<K, V> currentNode, String[] topicArray, int level, int maxLevel,
+                                      StringBuilder builder, boolean isJinFlag) {
+        Set<String> result = new HashSet<>();
+        if (level < maxLevel && !currentNode.children.isEmpty()) {
+            //first match the precise
+            TrieNode trieNode = currentNode.children.get(topicArray[level]);
+            if (trieNode != null) {
+                int start = builder.length();
+                builder.append(topicArray[level]).append(Constants.MQTT_TOPIC_DELIMITER);
+                result.addAll(findValuePath(trieNode, topicArray, level + 1, maxLevel, builder, false));
+                builder.delete(start, builder.length());
+            }
+            //match the #
+            TrieNode jinMatch = currentNode.children.get(Constants.JINFLAG);
+            if (jinMatch != null) {
+                int start = builder.length();
+                builder.append(Constants.JINFLAG).append(Constants.MQTT_TOPIC_DELIMITER);
+                result.addAll(findValuePath(jinMatch, topicArray, level + 1, maxLevel, builder, true));
+                builder.delete(start, builder.length());
+            }
+            //match the +
+            TrieNode jiaMatch = currentNode.children.get(Constants.ADDFLAG);
+            if (jiaMatch != null) {
+                int start = builder.length();
+                builder.append(Constants.ADDFLAG).append(Constants.MQTT_TOPIC_DELIMITER);
+                result.addAll(findValuePath(jiaMatch, topicArray, level + 1, maxLevel, builder, false));
+                builder.delete(start, builder.length());
+            }
+        } else {
+            //match the #
+            TrieNode jinMatch = currentNode.children.get(Constants.JINFLAG);
+            if (jinMatch != null) {
+                int start = builder.length();
+                builder.append(Constants.JINFLAG).append(Constants.MQTT_TOPIC_DELIMITER);
+                result.addAll(findValuePath(jinMatch, topicArray, level + 1, maxLevel, builder, true));
+                builder.delete(start, builder.length());
+            }
+            boolean jin = (level == maxLevel || isJinFlag) && !currentNode.valueSet.isEmpty() && builder.length() > 0;
+            if (jin) {
+                result.add(TopicUtils.normalizeTopic(builder.toString().substring(0, builder.length() - 1)));
+            }
+        }
+        return result;
+    }
+
+    private void traverse(TrieNode<K, V> currentNode, TrieMethod<K, V> method, StringBuilder builder) {
+        for (Map.Entry<String, TrieNode<K, V>> entry : currentNode.children.entrySet()) {
+            int start = builder.length();
+            builder.append(entry.getKey()).append(Constants.MQTT_TOPIC_DELIMITER);
+            traverse(entry.getValue(), method, builder);
+            builder.delete(start, builder.length());
+        }
+        Iterator<Map.Entry<K, V>> iterator = currentNode.valueSet.entrySet().iterator();
+        while (iterator.hasNext()) {
+            Map.Entry<K, V> entry = iterator.next();
+            try {
+                method.doMethod(builder.toString(), entry.getKey());
+            } catch (Throwable e) {
+            }
+        }
+    }
+
+    private Map<K, V> findValueSet(TrieNode<K, V> currentNode, String[] topicArray, int level, int maxLevel,
+                                   boolean isJinFlag) {
+        Map<K, V> result = new HashMap<>(16);
+        if (level < maxLevel && !currentNode.children.isEmpty()) {
+            //first match the precise
+            TrieNode trieNode = currentNode.children.get(topicArray[level]);
+            if (trieNode != null) {
+                result.putAll(findValueSet(trieNode, topicArray, level + 1, maxLevel, false));
+            }
+            //match the #
+            TrieNode jinMatch = currentNode.children.get(Constants.JINFLAG);
+            if (jinMatch != null) {
+                result.putAll(findValueSet(jinMatch, topicArray, level + 1, maxLevel, true));
+            }
+            //match the +
+            TrieNode jiaMatch = currentNode.children.get(Constants.ADDFLAG);
+            if (jiaMatch != null) {
+                result.putAll(findValueSet(jiaMatch, topicArray, level + 1, maxLevel, false));
+            }
+            return result;
+        } else {
+            //match the #
+            TrieNode jinMatch = currentNode.children.get(Constants.JINFLAG);
+            if (jinMatch != null) {
+                result.putAll(findValueSet(jinMatch, topicArray, level + 1, maxLevel, true));
+            }
+            if (level == maxLevel || isJinFlag) {
+                result.putAll(currentNode.valueSet);
+            }
+            return result;
+        }
+    }
+
+    class TrieNode<K, V> {
+        public TrieNode<K, V> parentNode;
+        public Map<String, TrieNode<K, V>> children = new ConcurrentHashMap<>();
+        public Map<K, V> valueSet = new ConcurrentHashMap<>();
+
+        public TrieNode(TrieNode<K, V> parentNode) {
+            this.parentNode = parentNode;
+        }
+    }
+
+}
diff --git a/mqtt-common/src/main/java/org/apache/rocketmq/mqtt/common/model/TrieException.java b/mqtt-common/src/main/java/org/apache/rocketmq/mqtt/common/model/TrieException.java
new file mode 100644
index 0000000..d11b692
--- /dev/null
+++ b/mqtt-common/src/main/java/org/apache/rocketmq/mqtt/common/model/TrieException.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.mqtt.common.model;
+
+public class TrieException  extends RuntimeException{
+    public TrieException() {
+    }
+
+    public TrieException(String message) {
+        super(message);
+    }
+
+    public TrieException(String message, Throwable cause) {
+        super(message, cause);
+    }
+
+    public TrieException(Throwable cause) {
+        super(cause);
+    }
+
+    public TrieException(String message, Throwable cause, boolean enableSuppression, boolean writableStackTrace) {
+        super(message, cause, enableSuppression, writableStackTrace);
+    }
+}
diff --git a/mqtt-common/src/main/java/org/apache/rocketmq/mqtt/common/model/TrieMethod.java b/mqtt-common/src/main/java/org/apache/rocketmq/mqtt/common/model/TrieMethod.java
new file mode 100644
index 0000000..a8e74b2
--- /dev/null
+++ b/mqtt-common/src/main/java/org/apache/rocketmq/mqtt/common/model/TrieMethod.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.mqtt.common.model;
+
+
+public interface TrieMethod<K, V> {
+
+    /**
+     * doMethod
+     *
+     * @param path
+     * @param nodeKey
+     */
+    void doMethod(String path, K nodeKey);
+
+}
diff --git a/mqtt-common/src/main/java/org/apache/rocketmq/mqtt/common/util/HmacSHA1Util.java b/mqtt-common/src/main/java/org/apache/rocketmq/mqtt/common/util/HmacSHA1Util.java
new file mode 100644
index 0000000..6c94830
--- /dev/null
+++ b/mqtt-common/src/main/java/org/apache/rocketmq/mqtt/common/util/HmacSHA1Util.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.mqtt.common.util;
+
+import org.apache.commons.codec.binary.Base64;
+
+import javax.crypto.Mac;
+import javax.crypto.spec.SecretKeySpec;
+import java.nio.charset.Charset;
+import java.security.InvalidKeyException;
+import java.security.NoSuchAlgorithmException;
+
+public class HmacSHA1Util {
+    private static Charset charset = Charset.forName("UTF-8");
+    private static String algorithm = "HmacSHA1";
+
+    public static String macSignature(String text, String secretKey) throws InvalidKeyException, NoSuchAlgorithmException {
+        Mac mac = Mac.getInstance(algorithm);
+        mac.init(new SecretKeySpec(secretKey.getBytes(charset), algorithm));
+        byte[] bytes = mac.doFinal(text.getBytes(charset));
+        return new String(Base64.encodeBase64(bytes), charset);
+    }
+
+    public static boolean validateSign(String text, byte[] input, String secretKey) throws NoSuchAlgorithmException, InvalidKeyException {
+        String sign = macSignature(text, secretKey);
+        return sign.equals(new String(input, charset));
+    }
+}
diff --git a/mqtt-common/src/main/java/org/apache/rocketmq/mqtt/common/util/HostInfo.java b/mqtt-common/src/main/java/org/apache/rocketmq/mqtt/common/util/HostInfo.java
new file mode 100644
index 0000000..044ca89
--- /dev/null
+++ b/mqtt-common/src/main/java/org/apache/rocketmq/mqtt/common/util/HostInfo.java
@@ -0,0 +1,56 @@
+/*
+ *
+ *  * Licensed to the Apache Software Foundation (ASF) under one or more
+ *  * contributor license agreements.  See the NOTICE file distributed with
+ *  * this work for additional information regarding copyright ownership.
+ *  * The ASF licenses this file to You under the Apache License, Version 2.0
+ *  * (the "License"); you may not use this file except in compliance with
+ *  * the License.  You may obtain a copy of the License at
+ *  *
+ *  *     http://www.apache.org/licenses/LICENSE-2.0
+ *  *
+ *  * Unless required by applicable law or agreed to in writing, software
+ *  * distributed under the License is distributed on an "AS IS" BASIS,
+ *  * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ *  * See the License for the specific language governing permissions and
+ *  * limitations under the License.
+ *
+ */
+
+package org.apache.rocketmq.mqtt.common.util;
+
+import java.net.InetAddress;
+import java.net.UnknownHostException;
+
+public class HostInfo {
+    private final String HOST_NAME;
+    private final String HOST_ADDRESS;
+
+    private static final HostInfo INSTALL = new HostInfo();
+
+    public static HostInfo getInstall() {
+        return INSTALL;
+    }
+
+    private HostInfo() {
+        String hostName;
+        String hostAddress;
+        try {
+            InetAddress localhost = InetAddress.getLocalHost();
+            hostName = localhost.getHostName();
+            hostAddress = localhost.getHostAddress();
+        } catch (UnknownHostException e) {
+            throw new RuntimeException(e);
+        }
+        HOST_NAME = hostName;
+        HOST_ADDRESS = hostAddress;
+    }
+
+    public final String getName() {
+        return HOST_NAME;
+    }
+
+    public final String getAddress() {
+        return HOST_ADDRESS;
+    }
+}
diff --git a/mqtt-common/src/main/java/org/apache/rocketmq/mqtt/common/util/MessageUtil.java b/mqtt-common/src/main/java/org/apache/rocketmq/mqtt/common/util/MessageUtil.java
new file mode 100644
index 0000000..8f4755d
--- /dev/null
+++ b/mqtt-common/src/main/java/org/apache/rocketmq/mqtt/common/util/MessageUtil.java
@@ -0,0 +1,119 @@
+/*
+ *
+ *  * Licensed to the Apache Software Foundation (ASF) under one or more
+ *  * contributor license agreements.  See the NOTICE file distributed with
+ *  * this work for additional information regarding copyright ownership.
+ *  * The ASF licenses this file to You under the Apache License, Version 2.0
+ *  * (the "License"); you may not use this file except in compliance with
+ *  * the License.  You may obtain a copy of the License at
+ *  *
+ *  *     http://www.apache.org/licenses/LICENSE-2.0
+ *  *
+ *  * Unless required by applicable law or agreed to in writing, software
+ *  * distributed under the License is distributed on an "AS IS" BASIS,
+ *  * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ *  * See the License for the specific language governing permissions and
+ *  * limitations under the License.
+ *
+ */
+
+package org.apache.rocketmq.mqtt.common.util;
+
+import com.alibaba.fastjson.JSONObject;
+import com.alibaba.fastjson.TypeReference;
+import io.netty.buffer.ByteBuf;
+import io.netty.buffer.ByteBufAllocator;
+import io.netty.buffer.UnpooledByteBufAllocator;
+import io.netty.handler.codec.mqtt.*;
+import org.apache.rocketmq.common.message.MessageDecoder;
+import org.apache.rocketmq.mqtt.common.model.Message;
+
+import java.nio.ByteBuffer;
+import java.util.ArrayList;
+import java.util.List;
+import java.util.Map;
+
+
+public class MessageUtil {
+    public static final ByteBufAllocator ALLOCATOR = new UnpooledByteBufAllocator(false);
+
+    public static MqttPublishMessage toMqttMessage(String topicName, byte[] body, int qos, int mqttId) {
+        ByteBuf payload = ALLOCATOR.buffer();
+        payload.writeBytes(body);
+        MqttFixedHeader mqttFixedHeader = new MqttFixedHeader(MqttMessageType.PUBLISH, false,
+            MqttQoS.valueOf(qos),
+            false, 0);
+        MqttPublishVariableHeader mqttPublishVariableHeader = new MqttPublishVariableHeader(topicName, mqttId);
+        MqttPublishMessage mqttPublishMessage = new MqttPublishMessage(mqttFixedHeader, mqttPublishVariableHeader,
+            payload);
+        return mqttPublishMessage;
+    }
+
+    public static Message toMessage(MqttPublishMessage mqttMessage) {
+        Message message = new Message();
+        message.setFirstTopic(TopicUtils.decode(mqttMessage.variableHeader().topicName()).getFirstTopic());
+        message.setOriginTopic(mqttMessage.variableHeader().topicName());
+        message.putUserProperty(Message.extPropertyQoS, String.valueOf(mqttMessage.fixedHeader().qosLevel().value()));
+        int readableBytes = mqttMessage.payload().readableBytes();
+        byte[] body = new byte[readableBytes];
+        mqttMessage.payload().readBytes(body);
+        message.setPayload(body);
+        return message;
+    }
+
+
+    public static byte[] encode(List<Message> messageList) {
+        if (messageList == null || messageList.isEmpty()) {
+            return null;
+        }
+        List<org.apache.rocketmq.common.message.Message> mqMessages = new ArrayList<>();
+        for (Message message : messageList) {
+            org.apache.rocketmq.common.message.Message mqMessage = new org.apache.rocketmq.common.message.Message();
+            mqMessage.setBody(message.getPayload());
+            mqMessage.putUserProperty(Message.propertyFirstTopic, message.getFirstTopic());
+            if (message.getOriginTopic() != null) {
+                mqMessage.putUserProperty(Message.propertyOriginTopic, message.getOriginTopic());
+            }
+            if (message.getMsgId() != null) {
+                mqMessage.putUserProperty(Message.propertyMsgId, message.getMsgId());
+            }
+            mqMessage.putUserProperty(Message.propertyOffset, String.valueOf(message.getOffset()));
+            mqMessage.putUserProperty(Message.propertyNextOffset, String.valueOf(message.getNextOffset()));
+            mqMessage.putUserProperty(Message.propertyRetry, String.valueOf(message.getRetry()));
+            mqMessage.putUserProperty(Message.propertyBornTime, String.valueOf(message.getBornTimestamp()));
+            mqMessage.putUserProperty(Message.propertyStoreTime, String.valueOf(message.getStoreTimestamp()));
+            mqMessage.putUserProperty(Message.propertyUserProperties,
+                JSONObject.toJSONString(message.getUserProperties()));
+            mqMessages.add(mqMessage);
+        }
+        return MessageDecoder.encodeMessages(mqMessages);
+    }
+
+    public static List<Message> decode(ByteBuffer byteBuffer) throws Exception {
+        List<org.apache.rocketmq.common.message.Message> mqMessages = MessageDecoder.decodeMessages(byteBuffer);
+        if (mqMessages == null) {
+            return null;
+        }
+        List<Message> messageList = new ArrayList<>();
+        for (org.apache.rocketmq.common.message.Message mqMessage : mqMessages) {
+            Message message = new Message();
+            message.setFirstTopic(mqMessage.getUserProperty(Message.propertyFirstTopic));
+            message.setOriginTopic(mqMessage.getUserProperty(Message.propertyOriginTopic));
+            message.setPayload(mqMessage.getBody());
+            message.setMsgId(mqMessage.getUserProperty(Message.propertyMsgId));
+            message.setOffset(Long.parseLong(mqMessage.getUserProperty(Message.propertyOffset)));
+            message.setNextOffset(Long.parseLong(mqMessage.getUserProperty(Message.propertyNextOffset)));
+            message.setStoreTimestamp(Long.parseLong(mqMessage.getUserProperty(Message.propertyStoreTime)));
+            message.setBornTimestamp(Long.parseLong(mqMessage.getUserProperty(Message.propertyBornTime)));
+            message.setRetry(Integer.parseInt(mqMessage.getUserProperty(Message.propertyRetry)));
+            String ext = mqMessage.getUserProperty(Message.propertyUserProperties);
+            if (ext != null) {
+                message.getUserProperties().putAll(
+                    com.alibaba.fastjson.JSONObject.parseObject(ext, new TypeReference<Map<String, String>>() {}));
+            }
+            messageList.add(message);
+        }
+        return messageList;
+    }
+
+}
diff --git a/mqtt-common/src/main/java/org/apache/rocketmq/mqtt/common/util/NamespaceUtil.java b/mqtt-common/src/main/java/org/apache/rocketmq/mqtt/common/util/NamespaceUtil.java
new file mode 100644
index 0000000..093629f
--- /dev/null
+++ b/mqtt-common/src/main/java/org/apache/rocketmq/mqtt/common/util/NamespaceUtil.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.mqtt.common.util;
+
+import org.apache.commons.lang3.StringUtils;
+
+public class NamespaceUtil {
+    public static final String NAMESPACE_SPLITER = "%";
+    private static int RESOURCE_LENGTH = 2;
+    public static final String MQ_DEFAULT_NAMESPACE_NAME = "DEFAULT_INSTANCE";
+
+    public NamespaceUtil() {
+    }
+
+    public static String encodeToNamespaceResource(String namespace, String resource) {
+        return resource != null && namespace != null ? StringUtils.join(new String[]{namespace, "%", resource}) : resource;
+    }
+
+    public static String decodeOriginResource(String resource) {
+        if (resource != null && resource.contains("%")) {
+            int firstIndex = resource.indexOf("%");
+            return resource.substring(firstIndex + 1);
+        } else {
+            return resource;
+        }
+    }
+
+    public static String decodeMqttNamespaceIdFromKey(String key) {
+        return decodeMqttNamespaceIdFromClientId(key);
+    }
+
+    public static String decodeMqttNamespaceIdFromClientId(String clientId) {
+        if (clientId != null && clientId.contains("%")) {
+            String mqttNamespaceId = clientId.split("%")[0];
+            return mqttNamespaceId;
+        } else {
+            return null;
+        }
+    }
+
+    public static String decodeStoreNamespaceIdFromTopic(String topic) {
+        if (topic != null && topic.contains("%")) {
+            String storeNamespaceId = topic.split("%")[0];
+            return storeNamespaceId;
+        } else {
+            return null;
+        }
+    }
+
+    public static String decodeNamespaceId(String resource) {
+        return resource != null && resource.contains("%") ? resource.split("%")[0] : null;
+    }
+}
diff --git a/mqtt-common/src/main/java/org/apache/rocketmq/mqtt/common/util/StatUtil.java b/mqtt-common/src/main/java/org/apache/rocketmq/mqtt/common/util/StatUtil.java
new file mode 100644
index 0000000..cd6ecd8
--- /dev/null
+++ b/mqtt-common/src/main/java/org/apache/rocketmq/mqtt/common/util/StatUtil.java
@@ -0,0 +1,472 @@
+/*
+ *
+ *  * Licensed to the Apache Software Foundation (ASF) under one or more
+ *  * contributor license agreements.  See the NOTICE file distributed with
+ *  * this work for additional information regarding copyright ownership.
+ *  * The ASF licenses this file to You under the Apache License, Version 2.0
+ *  * (the "License"); you may not use this file except in compliance with
+ *  * the License.  You may obtain a copy of the License at
+ *  *
+ *  *     http://www.apache.org/licenses/LICENSE-2.0
+ *  *
+ *  * Unless required by applicable law or agreed to in writing, software
+ *  * distributed under the License is distributed on an "AS IS" BASIS,
+ *  * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ *  * See the License for the specific language governing permissions and
+ *  * limitations under the License.
+ *
+ */
+
+package org.apache.rocketmq.mqtt.common.util;
+
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import javax.annotation.Generated;
+import java.math.BigDecimal;
+import java.util.ArrayList;
+import java.util.Collections;
+import java.util.List;
+import java.util.Map;
+import java.util.concurrent.*;
+import java.util.concurrent.atomic.AtomicInteger;
+import java.util.concurrent.atomic.AtomicLong;
+
+import static java.math.BigDecimal.ROUND_HALF_UP;
+
+public class StatUtil {
+    private static Logger sysLogger = LoggerFactory.getLogger(StatUtil.class);
+    private static Logger logger = LoggerFactory.getLogger("StatLogger");
+    private static final int MAX_KEY_NUM = Integer.parseInt(System.getProperty("stat.util.key.max.num", "10000"));
+    private static volatile ConcurrentMap<String, Invoke> invokeCache = new ConcurrentHashMap<>(64);
+    private static volatile ConcurrentMap<String, Map<Long, SecondInvoke>> secondInvokeCache = new ConcurrentHashMap<>(
+        64);
+
+    private static int STAT_WINDOW_SECONDS = Integer.parseInt(System.getProperty("stat.win.seconds", "60"));
+    private static String SPLITTER = "|";
+    private static ScheduledExecutorService daemon = Executors.newSingleThreadScheduledExecutor();
+
+    static class Invoke {
+        AtomicLong totalPv = new AtomicLong();
+        AtomicLong failPv = new AtomicLong();
+        AtomicLong sumRt = new AtomicLong();
+        AtomicLong maxRt = new AtomicLong();
+        AtomicLong minRt = new AtomicLong();
+        AtomicInteger topSecondPv = new AtomicInteger();
+        AtomicInteger secondPv = new AtomicInteger();
+        AtomicLong second = new AtomicLong(System.currentTimeMillis() / 1000L);
+    }
+
+    static class SecondInvoke implements Comparable<SecondInvoke> {
+        AtomicLong total = new AtomicLong();
+        AtomicLong fail = new AtomicLong();
+        AtomicLong sumRt = new AtomicLong();
+        AtomicLong maxRt = new AtomicLong();
+        AtomicLong minRt = new AtomicLong();
+        Long second = nowSecond();
+
+        @Override
+        public int compareTo(SecondInvoke o) {
+            return o.second.compareTo(second);
+        }
+    }
+
+    static {
+        daemon.scheduleAtFixedRate(new Runnable() {
+            @Override
+            public void run() {
+                try {
+                    printInvokeStat();
+                    printSecondInvokeStat();
+                } catch (Exception e) {
+                    logger.error("", e);
+                }
+            }
+        }, STAT_WINDOW_SECONDS, STAT_WINDOW_SECONDS, TimeUnit.SECONDS);
+    }
+
+    private static void printInvokeStat() {
+        Map<String, Invoke> tmp = invokeCache;
+        invokeCache = new ConcurrentHashMap<>(64);
+        for (Map.Entry<String, Invoke> entry : tmp.entrySet()) {
+            String key = entry.getKey();
+            Invoke invoke = entry.getValue();
+            logger.warn("{}",
+                buildLog(key, invoke.topSecondPv.get(), invoke.totalPv.get(), invoke.failPv.get(), invoke.minRt.get(),
+                    invoke.maxRt.get(), invoke.sumRt.get()));
+        }
+    }
+
+    private static void printSecondInvokeStat() {
+        for (Map.Entry<String, Map<Long, SecondInvoke>> entry : secondInvokeCache.entrySet()) {
+            String key = entry.getKey();
+            Map<Long, SecondInvoke> secondInvokeMap = entry.getValue();
+            long totalPv = 0L;
+            long failPv = 0L;
+            long topSecondPv = 0L;
+            long sumRt = 0L;
+            long maxRt = 0L;
+            long minRt = 0L;
+
+            for (Map.Entry<Long, SecondInvoke> invokeEntry : secondInvokeMap.entrySet()) {
+                long second = invokeEntry.getKey();
+                SecondInvoke secondInvoke = invokeEntry.getValue();
+                if (nowSecond() - second >= STAT_WINDOW_SECONDS) {
+                    secondInvokeMap.remove(second);
+                    continue;
+                }
+                long secondPv = secondInvoke.total.get();
+                totalPv += secondPv;
+                failPv += secondInvoke.fail.get();
+                sumRt += secondInvoke.sumRt.get();
+                if (maxRt < secondInvoke.maxRt.get()) {
+                    maxRt = secondInvoke.maxRt.get();
+                }
+                if (minRt > secondInvoke.minRt.get()) {
+                    minRt = secondInvoke.minRt.get();
+                }
+                if (topSecondPv < secondPv) {
+                    topSecondPv = secondPv;
+                }
+            }
+            if (secondInvokeMap.isEmpty()) {
+                secondInvokeCache.remove(key);
+                continue;
+            }
+            logger.warn("{}", buildLog(key, topSecondPv, totalPv, failPv, minRt, maxRt, sumRt));
+        }
+    }
+
+    private static String buildLog(String key, long topSecondPv, long totalPv, long failPv, long minRt, long maxRt,
+        long sumRt) {
+        StringBuilder sb = new StringBuilder();
+        sb.append(SPLITTER);
+        sb.append(key);
+        sb.append(SPLITTER);
+        sb.append(topSecondPv);
+        sb.append(SPLITTER);
+        int tps = new BigDecimal(totalPv).divide(new BigDecimal(STAT_WINDOW_SECONDS),
+            ROUND_HALF_UP).intValue();
+        sb.append(tps);
+        sb.append(SPLITTER);
+        sb.append(totalPv);
+        sb.append(SPLITTER);
+        sb.append(failPv);
+        sb.append(SPLITTER);
+        sb.append(minRt);
+        sb.append(SPLITTER);
+        long avg = new BigDecimal(sumRt).divide(new BigDecimal(totalPv),
+            ROUND_HALF_UP).longValue();
+        sb.append(avg);
+        sb.append(SPLITTER);
+        sb.append(maxRt);
+        return sb.toString();
+    }
+
+    public static String buildKey(String... keys) {
+        if (keys == null || keys.length <= 0) {
+            return null;
+        }
+        StringBuilder sb = new StringBuilder();
+        for (String key : keys) {
+            sb.append(key);
+            sb.append(",");
+        }
+        sb.deleteCharAt(sb.length() - 1);
+        return sb.toString();
+    }
+
+    public static void addInvoke(String key, long rt) {
+        addInvoke(key, rt, true);
+    }
+
+    private static Invoke getAndSetInvoke(String key) {
+        Invoke invoke = invokeCache.get(key);
+        if (invoke == null) {
+            invokeCache.putIfAbsent(key, new Invoke());
+        }
+        return invokeCache.get(key);
+    }
+
+    public static void addInvoke(String key, int num, long rt, boolean success) {
+        if (invokeCache.size() > MAX_KEY_NUM || secondInvokeCache.size() > MAX_KEY_NUM) {
+            return;
+        }
+        Invoke invoke = getAndSetInvoke(key);
+        if (invoke == null) {
+            return;
+        }
+
+        invoke.totalPv.getAndAdd(num);
+        if (!success) {
+            invoke.failPv.getAndAdd(num);
+        }
+        long now = nowSecond();
+        AtomicLong oldSecond = invoke.second;
+        if (oldSecond.get() == now) {
+            invoke.secondPv.getAndAdd(num);
+        } else {
+            if (oldSecond.compareAndSet(oldSecond.get(), now)) {
+                if (invoke.secondPv.get() > invoke.topSecondPv.get()) {
+                    invoke.topSecondPv.set(invoke.secondPv.get());
+                }
+                invoke.secondPv.set(num);
+            } else {
+                invoke.secondPv.getAndAdd(num);
+            }
+        }
+
+        invoke.sumRt.addAndGet(rt);
+        if (invoke.maxRt.get() < rt) {
+            invoke.maxRt.set(rt);
+        }
+        if (invoke.minRt.get() > rt) {
+            invoke.minRt.set(rt);
+        }
+    }
+
+    public static void addInvoke(String key, long rt, boolean success) {
+        if (invokeCache.size() > MAX_KEY_NUM || secondInvokeCache.size() > MAX_KEY_NUM) {
+            return;
+        }
+        Invoke invoke = getAndSetInvoke(key);
+        if (invoke == null) {
+            return;
+        }
+
+        invoke.totalPv.getAndIncrement();
+        if (!success) {
+            invoke.failPv.getAndIncrement();
+        }
+        long now = nowSecond();
+        AtomicLong oldSecond = invoke.second;
+        if (oldSecond.get() == now) {
+            invoke.secondPv.getAndIncrement();
+        } else {
+            if (oldSecond.compareAndSet(oldSecond.get(), now)) {
+                if (invoke.secondPv.get() > invoke.topSecondPv.get()) {
+                    invoke.topSecondPv.set(invoke.secondPv.get());
+                }
+                invoke.secondPv.set(1);
+            } else {
+                invoke.secondPv.getAndIncrement();
+            }
+        }
+
+        invoke.sumRt.addAndGet(rt);
+        if (invoke.maxRt.get() < rt) {
+            invoke.maxRt.set(rt);
+        }
+        if (invoke.minRt.get() > rt) {
+            invoke.minRt.set(rt);
+        }
+    }
+
+    public static SecondInvoke getAndSetSecondInvoke(String key) {
+        if (!secondInvokeCache.containsKey(key)) {
+            secondInvokeCache.putIfAbsent(key, new ConcurrentHashMap<>(STAT_WINDOW_SECONDS));
+        }
+        Map<Long, SecondInvoke> secondInvokeMap = secondInvokeCache.get(key);
+        if (secondInvokeMap == null) {
+            return null;
+        }
+        long second = nowSecond();
+        if (!secondInvokeMap.containsKey(second)) {
+            secondInvokeMap.putIfAbsent(second, new SecondInvoke());
+        }
+        return secondInvokeMap.get(second);
+    }
+
+    public static void addSecondInvoke(String key, long rt) {
+        addSecondInvoke(key, rt, true);
+    }
+
+    public static void addSecondInvoke(String key, long rt, boolean success) {
+        if (invokeCache.size() > MAX_KEY_NUM || secondInvokeCache.size() > MAX_KEY_NUM) {
+            return;
+        }
+        SecondInvoke secondInvoke = getAndSetSecondInvoke(key);
+        if (secondInvoke == null) {
+            return;
+        }
+        secondInvoke.total.addAndGet(1);
+        if (!success) {
+            secondInvoke.fail.addAndGet(1);
+        }
+        secondInvoke.sumRt.addAndGet(rt);
+        if (secondInvoke.maxRt.get() < rt) {
+            secondInvoke.maxRt.set(rt);
+        }
+        if (secondInvoke.minRt.get() > rt) {
+            secondInvoke.minRt.set(rt);
+        }
+    }
+
+    public static void addPv(String key, long totalPv) {
+        addPv(key, totalPv, true);
+    }
+
+    public static void addPv(String key, long totalPv, boolean success) {
+        if (invokeCache.size() > MAX_KEY_NUM || secondInvokeCache.size() > MAX_KEY_NUM) {
+            return;
+        }
+        if (totalPv <= 0) {
+            return;
+        }
+        Invoke invoke = getAndSetInvoke(key);
+        if (invoke == null) {
+            return;
+        }
+        invoke.totalPv.addAndGet(totalPv);
+        if (!success) {
+            invoke.failPv.addAndGet(totalPv);
+        }
+        long now = nowSecond();
+        AtomicLong oldSecond = invoke.second;
+        if (oldSecond.get() == now) {
+            invoke.secondPv.addAndGet((int)totalPv);
+        } else {
+            if (oldSecond.compareAndSet(oldSecond.get(), now)) {
+                if (invoke.secondPv.get() > invoke.topSecondPv.get()) {
+                    invoke.topSecondPv.set(invoke.secondPv.get());
+                }
+                invoke.secondPv.set((int)totalPv);
+            } else {
+                invoke.secondPv.addAndGet((int)totalPv);
+            }
+        }
+    }
+
+    public static void addSecondPv(String key, long totalPv) {
+        addSecondPv(key, totalPv, true);
+    }
+
+    public static void addSecondPv(String key, long totalPv, boolean success) {
+        if (invokeCache.size() > MAX_KEY_NUM || secondInvokeCache.size() > MAX_KEY_NUM) {
+            return;
+        }
+        if (totalPv <= 0) {
+            return;
+        }
+        SecondInvoke secondInvoke = getAndSetSecondInvoke(key);
+        if (secondInvoke == null) {
+            return;
+        }
+        secondInvoke.total.addAndGet(totalPv);
+        if (!success) {
+            secondInvoke.fail.addAndGet(totalPv);
+        }
+    }
+
+    public static boolean isOverFlow(String key, int tps) {
+        return nowTps(key) >= tps;
+    }
+
+    public static int nowTps(String key) {
+        Map<Long, SecondInvoke> secondInvokeMap = secondInvokeCache.get(key);
+        if (secondInvokeMap != null) {
+            SecondInvoke secondInvoke = secondInvokeMap.get(nowSecond());
+            if (secondInvoke != null) {
+                return (int)secondInvoke.total.get();
+            }
+        }
+        Invoke invoke = invokeCache.get(key);
+        if (invoke == null) {
+            return 0;
+        }
+        AtomicLong oldSecond = invoke.second;
+        if (oldSecond.get() == nowSecond()) {
+            return invoke.secondPv.get();
+        }
+        return 0;
+    }
+
+    public static int totalPvInWindow(String key, int windowSeconds) {
+        List<SecondInvoke> list = secondInvokeList(key, windowSeconds);
+        long totalPv = 0;
+        for (int i = 0; i < windowSeconds && i < list.size(); i++) {
+            totalPv += list.get(i).total.get();
+        }
+        return (int)totalPv;
+    }
+
+    public static int failPvInWindow(String key, int windowSeconds) {
+        List<SecondInvoke> list = secondInvokeList(key, windowSeconds);
+        long failPv = 0;
+        for (int i = 0; i < windowSeconds && i < list.size(); i++) {
+            failPv += list.get(i).fail.get();
+        }
+        return (int)failPv;
+    }
+
+    public static int topTpsInWindow(String key, int windowSeconds) {
+        List<SecondInvoke> list = secondInvokeList(key, windowSeconds);
+        long topTps = 0;
+        for (int i = 0; i < windowSeconds && i < list.size(); i++) {
+            long secondPv = list.get(i).total.get();
+            if (topTps < secondPv) {
+                topTps = secondPv;
+            }
+        }
+        return (int)topTps;
+    }
+
+    public static int avgRtInWindow(String key, int windowSeconds) {
+        List<SecondInvoke> list = secondInvokeList(key, windowSeconds);
+        long sumRt = 0;
+        long totalPv = 0;
+        for (int i = 0; i < windowSeconds && i < list.size(); i++) {
+            sumRt += list.get(i).sumRt.get();
+            totalPv += list.get(i).total.get();
+        }
+        if (totalPv <= 0) {
+            return 0;
+        }
+        long avg = new BigDecimal(sumRt).divide(new BigDecimal(totalPv),
+            ROUND_HALF_UP).longValue();
+        return (int)avg;
+    }
+
+    public static int maxRtInWindow(String key, int windowSeconds) {
+        List<SecondInvoke> list = secondInvokeList(key, windowSeconds);
+        long maxRt = 0;
+        long totalPv = 0;
+        for (int i = 0; i < windowSeconds && i < list.size(); i++) {
+            if (maxRt < list.get(i).maxRt.get()) {
+                maxRt = list.get(i).maxRt.get();
+            }
+        }
+        return (int)maxRt;
+    }
+
+    public static int minRtInWindow(String key, int windowSeconds) {
+        List<SecondInvoke> list = secondInvokeList(key, windowSeconds);
+        long minRt = 0;
+        long totalPv = 0;
+        for (int i = 0; i < windowSeconds && i < list.size(); i++) {
+            if (minRt < list.get(i).minRt.get()) {
+                minRt = list.get(i).minRt.get();
+            }
+        }
+        return (int)minRt;
+    }
+
+    private static List<SecondInvoke> secondInvokeList(String key, int windowSeconds) {
+        if (windowSeconds > STAT_WINDOW_SECONDS || windowSeconds <= 0) {
+            throw new IllegalArgumentException("windowSeconds Must Not be great than " + STAT_WINDOW_SECONDS);
+        }
+        Map<Long, SecondInvoke> secondInvokeMap = secondInvokeCache.get(key);
+        if (secondInvokeMap == null || secondInvokeMap.isEmpty()) {
+            return new ArrayList<>();
+        }
+        List<SecondInvoke> list = new ArrayList<>();
+        list.addAll(secondInvokeMap.values());
+        Collections.sort(list);
+        return list;
+    }
+
+    private static long nowSecond() {
+        return System.currentTimeMillis() / 1000L;
+    }
+
+}
diff --git a/mqtt-common/src/main/java/org/apache/rocketmq/mqtt/common/util/TopicUtils.java b/mqtt-common/src/main/java/org/apache/rocketmq/mqtt/common/util/TopicUtils.java
new file mode 100644
index 0000000..3622ee5
--- /dev/null
+++ b/mqtt-common/src/main/java/org/apache/rocketmq/mqtt/common/util/TopicUtils.java
@@ -0,0 +1,195 @@
+/*
+ *
+ *  * Licensed to the Apache Software Foundation (ASF) under one or more
+ *  * contributor license agreements.  See the NOTICE file distributed with
+ *  * this work for additional information regarding copyright ownership.
+ *  * The ASF licenses this file to You under the Apache License, Version 2.0
+ *  * (the "License"); you may not use this file except in compliance with
+ *  * the License.  You may obtain a copy of the License at
+ *  *
+ *  *     http://www.apache.org/licenses/LICENSE-2.0
+ *  *
+ *  * Unless required by applicable law or agreed to in writing, software
+ *  * distributed under the License is distributed on an "AS IS" BASIS,
+ *  * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ *  * See the License for the specific language governing permissions and
+ *  * limitations under the License.
+ *
+ */
+
+package org.apache.rocketmq.mqtt.common.util;
+
+import org.apache.commons.lang3.StringUtils;
+import org.apache.rocketmq.mqtt.common.model.Constants;
+import org.apache.rocketmq.mqtt.common.model.MqttTopic;
+
+
+public class TopicUtils {
+
+    /**
+     * t1/t2/t3/
+     *
+     * @param topic
+     * @return
+     */
+    public static String normalizeTopic(String topic) {
+        if (topic == null) {
+            return null;
+        }
+        if (!topic.contains(Constants.MQTT_TOPIC_DELIMITER)) {
+            return topic;
+        }
+        if (!topic.endsWith(Constants.MQTT_TOPIC_DELIMITER)) {
+            return topic + Constants.MQTT_TOPIC_DELIMITER;
+        }
+        return topic;
+    }
+
+    /**
+     * /t2/t3/t4/
+     *
+     * @param secondtopic
+     * @return
+     */
+    public static String normalizeSecondTopic(String secondtopic) {
+        if (secondtopic == null || secondtopic.isEmpty()) {
+            return null;
+        }
+        if (!secondtopic.startsWith(Constants.MQTT_TOPIC_DELIMITER)) {
+            secondtopic = Constants.MQTT_TOPIC_DELIMITER + secondtopic;
+        }
+        if (!secondtopic.endsWith(Constants.MQTT_TOPIC_DELIMITER)) {
+            return secondtopic + Constants.MQTT_TOPIC_DELIMITER;
+        }
+        return secondtopic;
+    }
+
+    public static boolean isP2P(String secondTopic) {
+        return secondTopic != null && secondTopic.startsWith(Constants.P2P);
+    }
+
+    public static String getClientIdFromP2pTopic(String p2pTopic) {
+        String tmp = p2pTopic.substring(Constants.P2P.length());
+        return tmp.substring(0, tmp.length() - 1);
+    }
+
+    public static String getClientIdFromRetryTopic(String retryTopic) {
+        String tmp = retryTopic.substring(Constants.RETRY.length());
+        return tmp.substring(0, tmp.length() - 1);
+    }
+
+    public static String getP2pTopic(String clientId) {
+        return normalizeTopic(Constants.P2P + clientId + Constants.MQTT_TOPIC_DELIMITER);
+    }
+
+    public static String getRetryTopic(String clientId) {
+        return normalizeTopic(Constants.RETRY + clientId + Constants.MQTT_TOPIC_DELIMITER);
+    }
+
+    public static boolean isRetryTopic(String topic) {
+        return topic != null && topic.startsWith(Constants.RETRY);
+    }
+
+    public static boolean isP2pTopic(String topic) {
+        return topic != null && topic.startsWith(Constants.P2P);
+    }
+
+    public static String getP2Peer(MqttTopic mqttTopic, String namespace) {
+        if (!isP2P(mqttTopic.getSecondTopic())) {
+            return null;
+        }
+        if (mqttTopic.getSecondTopic() == null || mqttTopic.getFirstTopic() == null) {
+            return null;
+        }
+        if (mqttTopic.getFirstTopic().contains(Constants.NAMESPACE_SPLITER) && StringUtils.isNotBlank(namespace)) {
+            return StringUtils.join(namespace, Constants.NAMESPACE_SPLITER, mqttTopic.getSecondTopic().split(Constants.MQTT_TOPIC_DELIMITER)[2]);
+        }
+        return mqttTopic.getSecondTopic().split(Constants.MQTT_TOPIC_DELIMITER)[2];
+    }
+
+    public static String encode(String topic, String secondTopic) {
+        if (secondTopic != null && secondTopic.length() > 1) {
+            return topic + secondTopic;
+        }
+        return topic;
+    }
+
+    public static MqttTopic decode(String topics) {
+        if (topics.startsWith(Constants.MQTT_TOPIC_DELIMITER)) {
+            topics = topics.substring(1);
+        }
+        String topic;
+        String secondTopic = null;
+        int index = topics.indexOf(Constants.MQTT_TOPIC_DELIMITER, 1);
+        if (index > 0) {
+            topic = topics.substring(0, index);
+            secondTopic = topics.substring(index);
+        } else {
+            topic = topics;
+        }
+        return new MqttTopic(topic, secondTopic);
+    }
+
+    public static boolean isWildCard(String topicFilter) {
+        return topicFilter != null &&
+                (topicFilter.contains(Constants.JINFLAG) || topicFilter.contains(Constants.ADDFLAG));
+    }
+
+    public static boolean isMatch(String topic, String topicFilter) {
+        if (topic.equals(topicFilter)) {
+            return true;
+        }
+        if (!isWildCard(topicFilter)) {
+            return false;
+        }
+
+        String[] subscribeTopics = topicFilter.split(Constants.MQTT_TOPIC_DELIMITER);
+        String[] messageTopics = topic.split(Constants.MQTT_TOPIC_DELIMITER);
+        int targetTopicLength = messageTopics.length;
+        int sourceTopicLength = subscribeTopics.length;
+        int minTopicLength = Math.min(targetTopicLength, sourceTopicLength);
+
+        for (int i = 0; i < minTopicLength; i++) {
+            String sourceTopic = subscribeTopics[i];
+
+            if (!Constants.JINFLAG.equals(sourceTopic) &&
+                    !Constants.ADDFLAG.equals(sourceTopic)) {
+                if (!sourceTopic.equals(messageTopics[i])) {
+                    return false;
+                }
+            }
+            //多级
+            if (Constants.JINFLAG.equals(sourceTopic)) {
+                return true;
+            }
+            boolean last = i == minTopicLength - 1 &&
+                    (sourceTopicLength == targetTopicLength ||
+                            (sourceTopicLength == targetTopicLength + 1 &&
+                                    Constants.JINFLAG.equals(subscribeTopics[sourceTopicLength - 1])
+                            )
+                    );
+            if (last) {
+                return true;
+            }
+        }
+
+        return false;
+    }
+
+    public static String wrapLmq(String firstTopic, String secondTopic) {
+        if (StringUtils.isBlank(secondTopic)) {
+            return firstTopic;
+        }
+        return firstTopic + normalizeSecondTopic(secondTopic);
+    }
+
+    public static String wrapP2pLmq(String clientId) {
+        return normalizeTopic(Constants.P2P + clientId);
+    }
+
+    public static void main(String[] args) {
+        String topic = "/t/t1/t2";
+        String topicFilter = "/t/t1/t2";
+        System.out.println(TopicUtils.isMatch(topic, topicFilter));
+    }
+}
diff --git a/mqtt-common/src/test/java/org/apache/rocketmq/mqtt/common/test/TestTrie.java b/mqtt-common/src/test/java/org/apache/rocketmq/mqtt/common/test/TestTrie.java
new file mode 100644
index 0000000..3b12714
--- /dev/null
+++ b/mqtt-common/src/test/java/org/apache/rocketmq/mqtt/common/test/TestTrie.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.mqtt.common.test;
+
+import org.apache.commons.collections.CollectionUtils;
+import org.apache.rocketmq.mqtt.common.model.Trie;
+import org.junit.Assert;
+import org.junit.Test;
+
+public class TestTrie {
+
+    @Test
+    public void test() {
+        Trie<String, String> trie = new Trie<>();
+        trie.addNode("test", "test", "test");
+        Assert.assertTrue(trie.getNodePath("test").contains("test"));
+        trie.deleteNode("test", "test");
+        Assert.assertTrue(CollectionUtils.isEmpty(trie.getNodePath("test")));
+    }
+}
diff --git a/mqtt-cs/pom.xml b/mqtt-cs/pom.xml
new file mode 100644
index 0000000..582771e
--- /dev/null
+++ b/mqtt-cs/pom.xml
@@ -0,0 +1,71 @@
+<?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">
+    <parent>
+        <artifactId>rocketmq-mqtt</artifactId>
+        <groupId>org.apache.rocketmq</groupId>
+        <version>1.0.0-SNAPSHOT</version>
+    </parent>
+    <modelVersion>4.0.0</modelVersion>
+
+    <artifactId>mqtt-cs</artifactId>
+
+    <dependencies>
+        <dependency>
+            <groupId>org.apache.rocketmq</groupId>
+            <artifactId>mqtt-common</artifactId>
+        </dependency>
+        <dependency>
+            <groupId>org.apache.rocketmq</groupId>
+            <artifactId>rocketmq-client</artifactId>
+        </dependency>
+        <dependency>
+            <groupId>io.netty</groupId>
+            <artifactId>netty-all</artifactId>
+        </dependency>
+        <dependency>
+            <groupId>org.slf4j</groupId>
+            <artifactId>slf4j-api</artifactId>
+        </dependency>
+        <dependency>
+            <groupId>org.springframework</groupId>
+            <artifactId>spring-core</artifactId>
+        </dependency>
+        <dependency>
+            <groupId>org.springframework</groupId>
+            <artifactId>spring-context</artifactId>
+        </dependency>
+        <dependency>
+            <groupId>org.springframework</groupId>
+            <artifactId>spring-beans</artifactId>
+        </dependency>
+        <dependency>
+            <groupId>org.apache.commons</groupId>
+            <artifactId>commons-lang3</artifactId>
+        </dependency>
+        <dependency>
+            <groupId>com.alibaba</groupId>
+            <artifactId>fastjson</artifactId>
+        </dependency>
+        <dependency>
+            <groupId>com.github.ben-manes.caffeine</groupId>
+            <artifactId>caffeine</artifactId>
+        </dependency>
+        <dependency>
+            <groupId>com.google.code.findbugs</groupId>
+            <artifactId>jsr305</artifactId>
+        </dependency>
+        <dependency>
+            <groupId>junit</groupId>
+            <artifactId>junit</artifactId>
+            <scope>test</scope>
+        </dependency>
+        <dependency>
+            <groupId>org.mockito</groupId>
+            <artifactId>mockito-core</artifactId>
+            <scope>test</scope>
+        </dependency>
+    </dependencies>
+
+</project>
\ No newline at end of file
diff --git a/mqtt-cs/src/main/java/org/apache/rocketmq/mqtt/cs/channel/ChannelCloseFrom.java b/mqtt-cs/src/main/java/org/apache/rocketmq/mqtt/cs/channel/ChannelCloseFrom.java
new file mode 100644
index 0000000..700658f
--- /dev/null
+++ b/mqtt-cs/src/main/java/org/apache/rocketmq/mqtt/cs/channel/ChannelCloseFrom.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.mqtt.cs.channel;
+
+
+public enum ChannelCloseFrom {
+    /**
+     *
+     */
+    CLIENT,
+
+    /**
+     *
+     */
+    SERVER
+}
diff --git a/mqtt-cs/src/main/java/org/apache/rocketmq/mqtt/cs/channel/ChannelException.java b/mqtt-cs/src/main/java/org/apache/rocketmq/mqtt/cs/channel/ChannelException.java
new file mode 100644
index 0000000..dbdbbfd
--- /dev/null
+++ b/mqtt-cs/src/main/java/org/apache/rocketmq/mqtt/cs/channel/ChannelException.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.mqtt.cs.channel;
+
+public class ChannelException extends RuntimeException{
+    public ChannelException() {
+    }
+
+    public ChannelException(String message) {
+        super(message);
+    }
+
+    public ChannelException(String message, Throwable cause) {
+        super(message, cause);
+    }
+
+    public ChannelException(Throwable cause) {
+        super(cause);
+    }
+
+    public ChannelException(String message, Throwable cause, boolean enableSuppression, boolean writableStackTrace) {
+        super(message, cause, enableSuppression, writableStackTrace);
+    }
+}
diff --git a/mqtt-cs/src/main/java/org/apache/rocketmq/mqtt/cs/channel/ChannelInfo.java b/mqtt-cs/src/main/java/org/apache/rocketmq/mqtt/cs/channel/ChannelInfo.java
new file mode 100644
index 0000000..695d6c6
--- /dev/null
+++ b/mqtt-cs/src/main/java/org/apache/rocketmq/mqtt/cs/channel/ChannelInfo.java
@@ -0,0 +1,254 @@
+/*
+ *
+ *  * Licensed to the Apache Software Foundation (ASF) under one or more
+ *  * contributor license agreements.  See the NOTICE file distributed with
+ *  * this work for additional information regarding copyright ownership.
+ *  * The ASF licenses this file to You under the Apache License, Version 2.0
+ *  * (the "License"); you may not use this file except in compliance with
+ *  * the License.  You may obtain a copy of the License at
+ *  *
+ *  *     http://www.apache.org/licenses/LICENSE-2.0
+ *  *
+ *  * Unless required by applicable law or agreed to in writing, software
+ *  * distributed under the License is distributed on an "AS IS" BASIS,
+ *  * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ *  * See the License for the specific language governing permissions and
+ *  * limitations under the License.
+ *
+ */
+
+package org.apache.rocketmq.mqtt.cs.channel;
+
+import com.alibaba.fastjson.JSON;
+import io.netty.channel.Channel;
+import io.netty.util.Attribute;
+import io.netty.util.AttributeKey;
+import org.apache.commons.lang3.StringUtils;
+
+import java.util.*;
+import java.util.concurrent.CompletableFuture;
+import java.util.concurrent.ConcurrentHashMap;
+import java.util.concurrent.ConcurrentMap;
+import java.util.concurrent.atomic.AtomicInteger;
+
+public class ChannelInfo {
+    private static final String CHANNEL_ID_KEY = "0";
+    private static final String CHANNEL_CLIENT_ID_KEY = "1";
+    private static final String CHANNEL_TIMESTAMP_KEY = "2";
+    private static final String CHANNEL_KEEPLIVE_KEY = "3";
+    private static final String CHANNEL_OWNER_KEY = "4";
+    private static final String CHANNEL_NAMESPACE_KEY = "5";
+    private static final String CHANNEL_EXT_CHANGE_KEY = "6";
+    private static final String CHANNEL_STORE_NAMESPACE_KEY = "7";
+    private static final String CHANNEL_CLEAN_SESSION_KEY = "8";
+    private static final String CHANNEL_SUB_NUM_KEY = "9";
+    private static final String CHANNEL_FUTRUE_KEY = "10";
+    private static final String CHANNEL_LIFE_CYCLE = "11";
+    private static final String CHANNEL_LAST_ACTIVE_TIMESTAMP_KEY = "12";
+    private static final String CHANNEL_IS_FLUSHING = "13";
+    private static final String CHANNEL_REMOTE_IP = "14";
+    private static final String CHANNEL_TUNNEL_ID = "15";
+
+    public static final String FUTURE_CONNECT = "connect";
+    public static final String FUTURE_SUBSCRIBE = "subscribe";
+
+    public static final AttributeKey<Map<String, Object>> CHANNEL_INFO_ATTRIBUTE_KEY = AttributeKey.valueOf("I");
+
+    public static final AttributeKey<ConcurrentMap<String, String>> CHANNEL_EXTDATA_ATTRIBUTE_KEY = AttributeKey
+        .valueOf("E");
+
+    public static final AttributeKey<String> CHANNEL_GA_ATTRIBUTE_KEY = AttributeKey.valueOf("GA");
+
+
+    public static Map<String, String> getExtData(Channel channel) {
+        Attribute<ConcurrentMap<String, String>> extAttribute = channel.attr(CHANNEL_EXTDATA_ATTRIBUTE_KEY);
+        if (extAttribute.get() == null) {
+            extAttribute.setIfAbsent(new ConcurrentHashMap<>());
+        }
+        return extAttribute.get();
+    }
+
+    public static String encodeExtData(Channel channel) {
+        Map<String, String> extData = getExtData(channel);
+        return JSON.toJSONString(extData);
+    }
+
+    public static boolean updateExtData(Channel channel, String extDataStr) {
+        if (StringUtils.isBlank(extDataStr)) {
+            return false;
+        }
+        updateExtData(channel, JSON.parseObject(extDataStr, Map.class));
+        return true;
+    }
+
+    private static void updateExtData(Channel channel, Map<String, String> extData) {
+        Map<String, String> currentExt = getExtData(channel);
+        currentExt.putAll(extData);
+        setExtDataChange(channel, true);
+    }
+
+    public static void setExtDataChange(Channel channel, boolean flag) {
+        getInfo(channel).put(CHANNEL_EXT_CHANGE_KEY, flag);
+    }
+
+    public static boolean checkExtDataChange(Channel channel) {
+        if (!getInfo(channel).containsKey(CHANNEL_EXT_CHANGE_KEY)) {
+            getInfo(channel).put(CHANNEL_EXT_CHANGE_KEY, false);
+        }
+        Object obj = getInfo(channel).get(CHANNEL_EXT_CHANGE_KEY);
+        if (obj == null) {
+            return false;
+        }
+        return (boolean)obj;
+    }
+
+    public static String getId(Channel channel) {
+        Map<String, Object> info = getInfo(channel);
+        if (info.containsKey(CHANNEL_ID_KEY)) {
+            return (String)info.get(CHANNEL_ID_KEY);
+        }
+        String channelIdStr = UUID.randomUUID().toString().replaceAll("-", "").toLowerCase();
+        info.put(CHANNEL_ID_KEY, channelIdStr);
+        return channelIdStr;
+    }
+
+    public static Boolean getCleanSessionFlag(Channel channel) {
+        if (!getInfo(channel).containsKey(CHANNEL_CLEAN_SESSION_KEY)) {
+            getInfo(channel).put(CHANNEL_CLEAN_SESSION_KEY, true);
+        }
+        Object obj = getInfo(channel).get(CHANNEL_CLEAN_SESSION_KEY);
+        if (obj == null) {
+            return true;
+        }
+        return (Boolean)obj;
+    }
+
+    public static void setCleanSessionFlag(Channel channel, Boolean cleanSessionFalg) {
+        getInfo(channel).put(CHANNEL_CLEAN_SESSION_KEY, cleanSessionFalg);
+    }
+
+    public static String getClientId(Channel channel) {
+        return (String)getInfo(channel).get(CHANNEL_CLIENT_ID_KEY);
+    }
+
+    public static long getChannelLifeCycle(Channel channel) {
+        Long expireTime = (Long) getInfo(channel).get(CHANNEL_LIFE_CYCLE);
+        if (expireTime == null) {
+            return Long.MAX_VALUE;
+        }
+        return expireTime;
+    }
+
+    public static void setChannelLifeCycle(Channel channel, Long expireTime) {
+        getInfo(channel).put(CHANNEL_LIFE_CYCLE, expireTime == null ? Long.MAX_VALUE : expireTime);
+    }
+
+    public static void setFuture(Channel channel, String futureKey, CompletableFuture<Void> future) {
+        getInfo(channel).put(CHANNEL_FUTRUE_KEY + futureKey, future);
+    }
+
+    public static CompletableFuture<Void> getFuture(Channel channel, String futureKey) {
+        Object future = getInfo(channel).get(CHANNEL_FUTRUE_KEY + futureKey);
+        if (future != null) {
+            return (CompletableFuture<Void>)future;
+        }
+        return null;
+    }
+
+    public static void removeFuture(Channel channel, String futureKey) {
+        getInfo(channel).remove(CHANNEL_FUTRUE_KEY + futureKey);
+    }
+
+    public static void setClientId(Channel channel, String clientId) {
+        getInfo(channel).put(CHANNEL_CLIENT_ID_KEY, clientId);
+    }
+
+    public static void touch(Channel channel) {
+        getInfo(channel).put(CHANNEL_TIMESTAMP_KEY, System.currentTimeMillis());
+    }
+
+    public static long getLastTouch(Channel channel) {
+        Object t = getInfo(channel).get(CHANNEL_TIMESTAMP_KEY);
+        return t != null ? (long)t : 0;
+    }
+
+    public static void lastActive(Channel channel, long timeStamp) {
+        getInfo(channel).put(CHANNEL_LAST_ACTIVE_TIMESTAMP_KEY, timeStamp);
+    }
+
+    public static long getLastActive(Channel channel) {
+        Object t = getInfo(channel).get(CHANNEL_LAST_ACTIVE_TIMESTAMP_KEY);
+        return t != null ? (long)t : 0;
+    }
+
+    public static void setRemoteIP(Channel channel, String ip) {
+        getInfo(channel).put(CHANNEL_REMOTE_IP, ip);
+    }
+
+    public static String getRemoteIP(Channel channel) {
+        Object t = getInfo(channel).get(CHANNEL_REMOTE_IP);
+        return t == null ? "" : (String) t;
+    }
+
+    public static void setKeepLive(Channel channel, int seconds) {
+        getInfo(channel).put(CHANNEL_KEEPLIVE_KEY, seconds);
+    }
+
+    public static Integer getKeepLive(Channel channel) {
+        return (Integer)getInfo(channel).get(CHANNEL_KEEPLIVE_KEY);
+    }
+
+    public static boolean isExpired(Channel channel) {
+        Long timestamp = (Long)getInfo(channel).get(CHANNEL_TIMESTAMP_KEY);
+        if (timestamp == null) {
+            return true;
+        }
+        Integer keepLiveT = getKeepLive(channel);
+        if (keepLiveT == null) {
+            return true;
+        }
+        return System.currentTimeMillis() - timestamp > keepLiveT * 1000L * 1.5;
+    }
+
+    public static void setOwner(Channel channel, String owner) {
+        getInfo(channel).put(CHANNEL_OWNER_KEY, owner);
+    }
+
+    public static String getOwner(Channel channel) {
+        return (String)getInfo(channel).get(CHANNEL_OWNER_KEY);
+    }
+
+    public static void setNamespace(Channel channel, String namespace) {
+        getInfo(channel).put(CHANNEL_NAMESPACE_KEY, namespace);
+    }
+
+    public static String getNamespace(Channel channel) {
+        return (String)getInfo(channel).get(CHANNEL_NAMESPACE_KEY);
+    }
+
+    /**
+     * clear channelInfo except the channelId、namespace
+     *
+     * @param channel
+     */
+    public static void clear(Channel channel) {
+        String channelId = getId(channel);
+        String namespace = getNamespace(channel);
+        Map<String, Object> newInfoAttribute = new ConcurrentHashMap<>(8);
+        newInfoAttribute.put(CHANNEL_ID_KEY, channelId);
+        if (namespace != null) {
+            newInfoAttribute.put(CHANNEL_NAMESPACE_KEY, namespace);
+        }
+        channel.attr(CHANNEL_INFO_ATTRIBUTE_KEY).set(newInfoAttribute);
+        channel.attr(CHANNEL_EXTDATA_ATTRIBUTE_KEY).set(null);
+    }
+
+    public static Map<String, Object> getInfo(Channel channel) {
+        Attribute<Map<String, Object>> infoAttribute = channel.attr(CHANNEL_INFO_ATTRIBUTE_KEY);
+        if (infoAttribute.get() == null) {
+            infoAttribute.setIfAbsent(new ConcurrentHashMap<>(8));
+        }
+        return infoAttribute.get();
+    }
+
+}
diff --git a/mqtt-cs/src/main/java/org/apache/rocketmq/mqtt/cs/channel/ChannelManager.java b/mqtt-cs/src/main/java/org/apache/rocketmq/mqtt/cs/channel/ChannelManager.java
new file mode 100644
index 0000000..fb1e209
--- /dev/null
+++ b/mqtt-cs/src/main/java/org/apache/rocketmq/mqtt/cs/channel/ChannelManager.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.mqtt.cs.channel;
+
+import io.netty.channel.Channel;
+
+
+public interface ChannelManager {
+
+    /**
+     * addChannel
+     *
+     * @param channel
+     */
+    void addChannel(Channel channel);
+
+    /**
+     * closeConnect
+     *
+     * @param channel
+     * @param from
+     * @param reason
+     */
+    void closeConnect(Channel channel, ChannelCloseFrom from, String reason);
+
+    /**
+     *  closeConnect
+     * @param channelId
+     * @param reason
+     */
+    void closeConnect(String channelId, String reason);
+
+    /**
+     *  get channel by Id
+     * @param channelId
+     * @return
+     */
+    Channel getChannelById(String channelId);
+
+    /**
+     * totalConn
+     *
+     * @return
+     */
+    int totalConn();
+
+}
diff --git a/mqtt-cs/src/main/java/org/apache/rocketmq/mqtt/cs/channel/ConnectHandler.java b/mqtt-cs/src/main/java/org/apache/rocketmq/mqtt/cs/channel/ConnectHandler.java
new file mode 100644
index 0000000..9ee8071
--- /dev/null
+++ b/mqtt-cs/src/main/java/org/apache/rocketmq/mqtt/cs/channel/ConnectHandler.java
@@ -0,0 +1,65 @@
+/*
+ *
+ *  * Licensed to the Apache Software Foundation (ASF) under one or more
+ *  * contributor license agreements.  See the NOTICE file distributed with
+ *  * this work for additional information regarding copyright ownership.
+ *  * The ASF licenses this file to You under the Apache License, Version 2.0
+ *  * (the "License"); you may not use this file except in compliance with
+ *  * the License.  You may obtain a copy of the License at
+ *  *
+ *  *     http://www.apache.org/licenses/LICENSE-2.0
+ *  *
+ *  * Unless required by applicable law or agreed to in writing, software
+ *  * distributed under the License is distributed on an "AS IS" BASIS,
+ *  * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ *  * See the License for the specific language governing permissions and
+ *  * limitations under the License.
+ *
+ */
+
+package org.apache.rocketmq.mqtt.cs.channel;
+
+import io.netty.channel.ChannelHandler;
+import io.netty.channel.ChannelHandlerContext;
+import io.netty.channel.ChannelInboundHandlerAdapter;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+import org.springframework.stereotype.Component;
+
+import javax.annotation.Resource;
+import java.util.Arrays;
+import java.util.List;
+
+
+@ChannelHandler.Sharable
+@Component
+public class ConnectHandler extends ChannelInboundHandlerAdapter {
+    private static Logger logger = LoggerFactory.getLogger(ConnectHandler.class);
+
+    @Resource
+    private ChannelManager channelManager;
+
+    @Override
+    public void channelActive(ChannelHandlerContext ctx) throws Exception {
+        ctx.fireChannelActive();
+        channelManager.addChannel(ctx.channel());
+    }
+
+    @Override
+    public void channelInactive(ChannelHandlerContext ctx) throws Exception {
+        super.channelInactive(ctx);
+        channelManager.closeConnect(ctx.channel(), ChannelCloseFrom.CLIENT, "be closed");
+    }
+
+    public final List<String> simpleExceptions = Arrays.asList("Connection reset by peer");
+
+    @Override
+    public void exceptionCaught(ChannelHandlerContext ctx, Throwable cause) throws Exception {
+        if (cause.getMessage() != null && simpleExceptions.contains(cause.getMessage())) {
+        } else {
+            logger.error("exceptionCaught {}", ctx.channel(), cause);
+        }
+        channelManager.closeConnect(ctx.channel(), ChannelCloseFrom.SERVER, cause.getMessage());
+    }
+
+}
diff --git a/mqtt-cs/src/main/java/org/apache/rocketmq/mqtt/cs/channel/DefaultChannelManager.java b/mqtt-cs/src/main/java/org/apache/rocketmq/mqtt/cs/channel/DefaultChannelManager.java
new file mode 100644
index 0000000..fcbd449
--- /dev/null
+++ b/mqtt-cs/src/main/java/org/apache/rocketmq/mqtt/cs/channel/DefaultChannelManager.java
@@ -0,0 +1,151 @@
+/*
+ *
+ *  * Licensed to the Apache Software Foundation (ASF) under one or more
+ *  * contributor license agreements.  See the NOTICE file distributed with
+ *  * this work for additional information regarding copyright ownership.
+ *  * The ASF licenses this file to You under the Apache License, Version 2.0
+ *  * (the "License"); you may not use this file except in compliance with
+ *  * the License.  You may obtain a copy of the License at
+ *  *
+ *  *     http://www.apache.org/licenses/LICENSE-2.0
+ *  *
+ *  * Unless required by applicable law or agreed to in writing, software
+ *  * distributed under the License is distributed on an "AS IS" BASIS,
+ *  * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ *  * See the License for the specific language governing permissions and
+ *  * limitations under the License.
+ *
+ */
+
+package org.apache.rocketmq.mqtt.cs.channel;
+
+import io.netty.channel.Channel;
+import io.netty.util.HashedWheelTimer;
+import io.netty.util.Timeout;
+import org.apache.commons.lang3.StringUtils;
+import org.apache.rocketmq.mqtt.cs.config.ConnectConf;
+import org.apache.rocketmq.mqtt.cs.session.Session;
+import org.apache.rocketmq.mqtt.cs.session.infly.RetryDriver;
+import org.apache.rocketmq.mqtt.cs.session.loop.SessionLoop;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+import org.springframework.stereotype.Component;
+
+import javax.annotation.PostConstruct;
+import javax.annotation.Resource;
+import java.util.Map;
+import java.util.concurrent.ConcurrentHashMap;
+import java.util.concurrent.ScheduledThreadPoolExecutor;
+import java.util.concurrent.TimeUnit;
+
+@Component
+public class DefaultChannelManager implements ChannelManager {
+    private static Logger logger = LoggerFactory.getLogger(DefaultChannelManager.class);
+    private Map<String, Channel> channelMap = new ConcurrentHashMap<>(1024);
+    private HashedWheelTimer hashedWheelTimer;
+    private static int MinBlankChannelSeconds = 10;
+    private ScheduledThreadPoolExecutor scheduler;
+
+    @Resource
+    private ConnectConf connectConf;
+
+    @Resource
+    private SessionLoop sessionLoop;
+
+    @Resource
+    private RetryDriver retryDriver;
+
+
+    @PostConstruct
+    public void init() {
+        sessionLoop.setChannelManager(this);
+        hashedWheelTimer = new HashedWheelTimer(1, TimeUnit.SECONDS);
+        hashedWheelTimer.start();
+        Runtime.getRuntime().addShutdownHook(new Thread(() -> {
+            for(Channel channel : channelMap.values()) {
+                closeConnect(channel, ChannelCloseFrom.SERVER, "ServerShutdown");
+            }
+        }));
+    }
+
+    @Override
+    public void addChannel(Channel channel) {
+        if (channelMap.size() > connectConf.getMaxConn()) {
+            closeConnect(channel, ChannelCloseFrom.SERVER, "overflow");
+            logger.error("channel is too many {}", channelMap.size());
+            return;
+        }
+        ChannelInfo.touch(channel);
+        channelMap.put(ChannelInfo.getId(channel), channel);
+        hashedWheelTimer.newTimeout(timeout -> doPing(timeout, channel), MinBlankChannelSeconds, TimeUnit.SECONDS);
+    }
+
+    private void doPing(Timeout timeout, Channel channel) {
+        try {
+            if (StringUtils.isBlank(ChannelInfo.getClientId(channel))) {
+                //close
+                closeConnect(channel, ChannelCloseFrom.SERVER, "No CONNECT");
+                return;
+            }
+            long channelLifeCycle = ChannelInfo.getChannelLifeCycle(channel);
+            if (System.currentTimeMillis() > channelLifeCycle) {
+                closeConnect(channel, ChannelCloseFrom.SERVER, "Channel Auth Expire");
+                return;
+            }
+            if (ChannelInfo.isExpired(channel)) {
+                closeConnect(channel, ChannelCloseFrom.SERVER, "No Heart");
+            } else {
+                int keepAliveTimeSeconds = ChannelInfo.getKeepLive(channel);
+                hashedWheelTimer.newTimeout(timeout.task(), (long)Math.ceil(keepAliveTimeSeconds * 1.5 + 1),
+                    TimeUnit.SECONDS);
+            }
+        } catch (Exception e) {
+            logger.error("", e);
+        }
+    }
+
+    @Override
+    public void closeConnect(Channel channel, ChannelCloseFrom from, String reason) {
+        String clientId = ChannelInfo.getClientId(channel);
+        String channelId = ChannelInfo.getId(channel);
+        if (clientId == null) {
+            channelMap.remove(channelId);
+            sessionLoop.unloadSession(clientId, channelId);
+            if (channel.isActive()) {
+                channel.close();
+            }
+            return;
+        }
+
+        //session maybe null
+        Session session = sessionLoop.unloadSession(clientId, channelId);
+        retryDriver.unloadSession(session);
+        channelMap.remove(channelId);
+
+        ChannelInfo.clear(channel);
+
+        if (channel.isActive()) {
+            channel.close();
+        }
+    }
+
+    @Override
+    public void closeConnect(String channelId, String reason) {
+        Channel channel = channelMap.get(channelId);
+        if (channel == null) {
+            return;
+        }
+        closeConnect(channel, ChannelCloseFrom.SERVER, reason);
+    }
+
+    @Override
+    public Channel getChannelById(String channelId) {
+        return channelMap.get(channelId);
+    }
+
+    @Override
+    public int totalConn() {
+        return channelMap.size();
+    }
+
+}
diff --git a/mqtt-cs/src/main/java/org/apache/rocketmq/mqtt/cs/config/ConnectConf.java b/mqtt-cs/src/main/java/org/apache/rocketmq/mqtt/cs/config/ConnectConf.java
new file mode 100644
index 0000000..153fbaa
--- /dev/null
+++ b/mqtt-cs/src/main/java/org/apache/rocketmq/mqtt/cs/config/ConnectConf.java
@@ -0,0 +1,184 @@
+/*
+ *
+ *  * Licensed to the Apache Software Foundation (ASF) under one or more
+ *  * contributor license agreements.  See the NOTICE file distributed with
+ *  * this work for additional information regarding copyright ownership.
+ *  * The ASF licenses this file to You under the Apache License, Version 2.0
+ *  * (the "License"); you may not use this file except in compliance with
+ *  * the License.  You may obtain a copy of the License at
+ *  *
+ *  *     http://www.apache.org/licenses/LICENSE-2.0
+ *  *
+ *  * Unless required by applicable law or agreed to in writing, software
+ *  * distributed under the License is distributed on an "AS IS" BASIS,
+ *  * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ *  * See the License for the specific language governing permissions and
+ *  * limitations under the License.
+ *
+ */
+
+package org.apache.rocketmq.mqtt.cs.config;
+
+import org.apache.rocketmq.common.MixAll;
+import org.springframework.core.io.ClassPathResource;
+import org.springframework.stereotype.Component;
+
+import java.io.File;
+import java.io.IOException;
+import java.io.InputStream;
+import java.util.Properties;
+
+@Component
+public class ConnectConf {
+    private static final String CONF_FILE_NAME = "connect.conf";
+    private File confFile;
+    private int nettySelectThreadNum = 1;
+    private int nettyWorkerThreadNum = Runtime.getRuntime().availableProcessors() * 2;
+    private int mqttPort = 1883;
+    private int mqttWsPort = 8888;
+    private int maxPacketSizeInByte = 64 * 1024;
+    private int highWater = 256 * 1024;
+    private int lowWater = 16 * 1024;
+    private int maxConn = 10 * 10000;
+    private boolean order;
+    private int maxRetryTime = 15;
+    private int sizeOfNotRollWhenAckSlow = 32;
+    private int queueCacheSize = 128;
+    private int pullBatchSize = 32;
+    private int rpcListenPort = 7001;
+    private int retryIntervalSeconds = 3;
+
+    public ConnectConf() throws IOException {
+        ClassPathResource classPathResource = new ClassPathResource(CONF_FILE_NAME);
+        InputStream in = classPathResource.getInputStream();
+        Properties properties = new Properties();
+        properties.load(in);
+        in.close();
+        MixAll.properties2Object(properties, this);
+        this.confFile = new File(classPathResource.getURL().getFile());
+    }
+
+    public File getConfFile() {
+        return confFile;
+    }
+
+    public int getNettySelectThreadNum() {
+        return nettySelectThreadNum;
+    }
+
+    public void setNettySelectThreadNum(int nettySelectThreadNum) {
+        this.nettySelectThreadNum = nettySelectThreadNum;
+    }
+
+    public int getNettyWorkerThreadNum() {
+        return nettyWorkerThreadNum;
+    }
+
+    public void setNettyWorkerThreadNum(int nettyWorkerThreadNum) {
+        this.nettyWorkerThreadNum = nettyWorkerThreadNum;
+    }
+
+    public int getMqttPort() {
+        return mqttPort;
+    }
+
+    public void setMqttPort(int mqttPort) {
+        this.mqttPort = mqttPort;
+    }
+
+    public int getMqttWsPort() {
+        return mqttWsPort;
+    }
+
+    public void setMqttWsPort(int mqttWsPort) {
+        this.mqttWsPort = mqttWsPort;
+    }
+
+    public int getMaxPacketSizeInByte() {
+        return maxPacketSizeInByte;
+    }
+
+    public void setMaxPacketSizeInByte(int maxPacketSizeInByte) {
+        this.maxPacketSizeInByte = maxPacketSizeInByte;
+    }
+
+    public int getHighWater() {
+        return highWater;
+    }
+
+    public void setHighWater(int highWater) {
+        this.highWater = highWater;
+    }
+
+    public int getLowWater() {
+        return lowWater;
+    }
+
+    public void setLowWater(int lowWater) {
+        this.lowWater = lowWater;
+    }
+
+    public int getMaxConn() {
+        return maxConn;
+    }
+
+    public void setMaxConn(int maxConn) {
+        this.maxConn = maxConn;
+    }
+
+    public boolean isOrder() {
+        return order;
+    }
+
+    public void setOrder(boolean order) {
+        this.order = order;
+    }
+
+    public int getMaxRetryTime() {
+        return maxRetryTime;
+    }
+
+    public void setMaxRetryTime(int maxRetryTime) {
+        this.maxRetryTime = maxRetryTime;
+    }
+
+    public int getSizeOfNotRollWhenAckSlow() {
+        return sizeOfNotRollWhenAckSlow;
+    }
+
+    public void setSizeOfNotRollWhenAckSlow(int sizeOfNotRollWhenAckSlow) {
+        this.sizeOfNotRollWhenAckSlow = sizeOfNotRollWhenAckSlow;
+    }
+
+    public int getPullBatchSize() {
+        return pullBatchSize;
+    }
+
+    public void setPullBatchSize(int pullBatchSize) {
+        this.pullBatchSize = pullBatchSize;
+    }
+
+    public int getQueueCacheSize() {
+        return queueCacheSize;
+    }
+
+    public void setQueueCacheSize(int queueCacheSize) {
+        this.queueCacheSize = queueCacheSize;
+    }
+
+    public int getRpcListenPort() {
+        return rpcListenPort;
+    }
+
+    public void setRpcListenPort(int rpcListenPort) {
+        this.rpcListenPort = rpcListenPort;
+    }
+
+    public int getRetryIntervalSeconds() {
+        return retryIntervalSeconds;
+    }
+
+    public void setRetryIntervalSeconds(int retryIntervalSeconds) {
+        this.retryIntervalSeconds = retryIntervalSeconds;
+    }
+}
diff --git a/mqtt-cs/src/main/java/org/apache/rocketmq/mqtt/cs/config/ConnectConfListener.java b/mqtt-cs/src/main/java/org/apache/rocketmq/mqtt/cs/config/ConnectConfListener.java
new file mode 100644
index 0000000..6a8694b
--- /dev/null
+++ b/mqtt-cs/src/main/java/org/apache/rocketmq/mqtt/cs/config/ConnectConfListener.java
@@ -0,0 +1,73 @@
+/*
+ *
+ *  * Licensed to the Apache Software Foundation (ASF) under one or more
+ *  * contributor license agreements.  See the NOTICE file distributed with
+ *  * this work for additional information regarding copyright ownership.
+ *  * The ASF licenses this file to You under the Apache License, Version 2.0
+ *  * (the "License"); you may not use this file except in compliance with
+ *  * the License.  You may obtain a copy of the License at
+ *  *
+ *  *     http://www.apache.org/licenses/LICENSE-2.0
+ *  *
+ *  * Unless required by applicable law or agreed to in writing, software
+ *  * distributed under the License is distributed on an "AS IS" BASIS,
+ *  * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ *  * See the License for the specific language governing permissions and
+ *  * limitations under the License.
+ *
+ */
+
+package org.apache.rocketmq.mqtt.cs.config;
+
+import org.apache.rocketmq.common.MixAll;
+import org.apache.rocketmq.common.ThreadFactoryImpl;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+import org.springframework.stereotype.Component;
+
+import javax.annotation.PostConstruct;
+import javax.annotation.Resource;
+import java.io.File;
+import java.io.FileInputStream;
+import java.io.InputStream;
+import java.util.Properties;
+import java.util.concurrent.ScheduledThreadPoolExecutor;
+import java.util.concurrent.TimeUnit;
+import java.util.concurrent.atomic.AtomicLong;
+
+
+@Component
+public class ConnectConfListener {
+    private static Logger logger = LoggerFactory.getLogger(ConnectConfListener.class);
+
+    @Resource
+    private ConnectConf connectConf;
+
+    private File confFile;
+    private ScheduledThreadPoolExecutor scheduler;
+    private AtomicLong gmt = new AtomicLong();
+
+    @PostConstruct
+    public void start() {
+        confFile = connectConf.getConfFile();
+        gmt.set(confFile.lastModified());
+        scheduler = new ScheduledThreadPoolExecutor(1, new ThreadFactoryImpl("ConnectConfListener"));
+        scheduler.scheduleWithFixedDelay(() -> {
+            try {
+                if (gmt.get() == confFile.lastModified()) {
+                    return;
+                }
+                gmt.set(confFile.lastModified());
+                InputStream in = new FileInputStream(confFile.getAbsoluteFile());
+                Properties properties = new Properties();
+                properties.load(in);
+                in.close();
+                MixAll.properties2Object(properties, connectConf);
+                logger.warn("UpdateConf:{}", confFile.getAbsolutePath());
+            } catch (Exception e) {
+                logger.error("", e);
+            }
+        }, 3, 3, TimeUnit.SECONDS);
+    }
+
+}
diff --git a/mqtt-cs/src/main/java/org/apache/rocketmq/mqtt/cs/hook/UpstreamHookManagerImpl.java b/mqtt-cs/src/main/java/org/apache/rocketmq/mqtt/cs/hook/UpstreamHookManagerImpl.java
new file mode 100644
index 0000000..763b572
--- /dev/null
+++ b/mqtt-cs/src/main/java/org/apache/rocketmq/mqtt/cs/hook/UpstreamHookManagerImpl.java
@@ -0,0 +1,74 @@
+/*
+ *
+ *  * Licensed to the Apache Software Foundation (ASF) under one or more
+ *  * contributor license agreements.  See the NOTICE file distributed with
+ *  * this work for additional information regarding copyright ownership.
+ *  * The ASF licenses this file to You under the Apache License, Version 2.0
+ *  * (the "License"); you may not use this file except in compliance with
+ *  * the License.  You may obtain a copy of the License at
+ *  *
+ *  *     http://www.apache.org/licenses/LICENSE-2.0
+ *  *
+ *  * Unless required by applicable law or agreed to in writing, software
+ *  * distributed under the License is distributed on an "AS IS" BASIS,
+ *  * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ *  * See the License for the specific language governing permissions and
+ *  * limitations under the License.
+ *
+ */
+
+package org.apache.rocketmq.mqtt.cs.hook;
+
+import io.netty.handler.codec.mqtt.MqttMessage;
+import org.apache.rocketmq.mqtt.common.hook.HookResult;
+import org.apache.rocketmq.mqtt.common.hook.UpstreamHook;
+import org.apache.rocketmq.mqtt.common.hook.UpstreamHookEnum;
+import org.apache.rocketmq.mqtt.common.hook.UpstreamHookManager;
+import org.apache.rocketmq.mqtt.common.model.MqttMessageUpContext;
+import org.springframework.stereotype.Component;
+
+import java.util.concurrent.CompletableFuture;
+import java.util.concurrent.atomic.AtomicBoolean;
+
+@Component
+public class UpstreamHookManagerImpl implements UpstreamHookManager {
+
+    private UpstreamHook[] upstreamHookList = new UpstreamHook[UpstreamHookEnum.values().length];
+    private AtomicBoolean isAssembled = new AtomicBoolean(false);
+
+    @Override
+    public void addHook(int index, UpstreamHook upstreamHook) {
+        if (isAssembled.get()) {
+            throw new IllegalArgumentException("Hook Was Assembled");
+        }
+        synchronized (upstreamHookList) {
+            upstreamHookList[index] = upstreamHook;
+        }
+    }
+
+    @Override
+    public CompletableFuture<HookResult> doUpstreamHook(MqttMessageUpContext context, MqttMessage msg) {
+        assembleNextHook();
+        CompletableFuture<HookResult> hookResult = new CompletableFuture<>();
+        if (upstreamHookList.length <= 0) {
+            hookResult.complete(new HookResult(HookResult.SUCCESS, -1, null, null));
+            return hookResult;
+        }
+        return upstreamHookList[0].doHook(context, msg);
+    }
+
+    private void assembleNextHook() {
+        if (isAssembled.compareAndSet(false, true)) {
+            synchronized (upstreamHookList) {
+                for (int i = 0; i < upstreamHookList.length - 1; i++) {
+                    UpstreamHook upstreamHook = upstreamHookList[i];
+                    if (upstreamHook.getNextHook() != null) {
+                        continue;
+                    }
+                    upstreamHook.setNextHook(upstreamHookList[i + 1]);
+                }
+            }
+        }
+    }
+
+}
diff --git a/mqtt-cs/src/main/java/org/apache/rocketmq/mqtt/cs/protocol/mqtt/MqttPacketDispatcher.java b/mqtt-cs/src/main/java/org/apache/rocketmq/mqtt/cs/protocol/mqtt/MqttPacketDispatcher.java
new file mode 100644
index 0000000..0cb302c
--- /dev/null
+++ b/mqtt-cs/src/main/java/org/apache/rocketmq/mqtt/cs/protocol/mqtt/MqttPacketDispatcher.java
@@ -0,0 +1,179 @@
+/*
+ *
+ *  * Licensed to the Apache Software Foundation (ASF) under one or more
+ *  * contributor license agreements.  See the NOTICE file distributed with
+ *  * this work for additional information regarding copyright ownership.
+ *  * The ASF licenses this file to You under the Apache License, Version 2.0
+ *  * (the "License"); you may not use this file except in compliance with
+ *  * the License.  You may obtain a copy of the License at
+ *  *
+ *  *     http://www.apache.org/licenses/LICENSE-2.0
+ *  *
+ *  * Unless required by applicable law or agreed to in writing, software
+ *  * distributed under the License is distributed on an "AS IS" BASIS,
+ *  * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ *  * See the License for the specific language governing permissions and
+ *  * limitations under the License.
+ *
+ */
+
+package org.apache.rocketmq.mqtt.cs.protocol.mqtt;
+
+import io.netty.channel.Channel;
+import io.netty.channel.ChannelHandler;
+import io.netty.channel.ChannelHandlerContext;
+import io.netty.channel.SimpleChannelInboundHandler;
+import io.netty.handler.codec.mqtt.*;
+import io.netty.util.ReferenceCountUtil;
+import org.apache.rocketmq.mqtt.common.hook.HookResult;
+import org.apache.rocketmq.mqtt.common.hook.UpstreamHookManager;
+import org.apache.rocketmq.mqtt.common.model.MqttMessageUpContext;
+import org.apache.rocketmq.mqtt.common.util.HostInfo;
+import org.apache.rocketmq.mqtt.cs.channel.ChannelException;
+import org.apache.rocketmq.mqtt.cs.channel.ChannelInfo;
+import org.apache.rocketmq.mqtt.cs.channel.ChannelManager;
+import org.apache.rocketmq.mqtt.cs.protocol.mqtt.handler.*;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+import org.springframework.stereotype.Component;
+
+import javax.annotation.Resource;
+import java.util.concurrent.CompletableFuture;
+
+
+@ChannelHandler.Sharable
+@Component
+public class MqttPacketDispatcher extends SimpleChannelInboundHandler<MqttMessage> {
+    private static Logger logger = LoggerFactory.getLogger(MqttPacketDispatcher.class);
+
+    @Resource
+    private MqttConnectHandler mqttConnectHandler;
+
+    @Resource
+    private MqttDisconnectHandler mqttDisconnectHandler;
+
+    @Resource
+    private MqttPublishHandler mqttPublishHandler;
+
+    @Resource
+    private MqttSubscribeHandler mqttSubscribeHandler;
+
+    @Resource
+    private MqttPubAckHandler mqttPubAckHandler;
+
+    @Resource
+    private MqttPingHandler mqttPingHandler;
+
+    @Resource
+    private MqttUnSubscribeHandler mqttUnSubscribeHandler;
+
+    @Resource
+    private MqttPubRelHandler mqttPubRelHandler;
+
+    @Resource
+    private MqttPubRecHandler mqttPubRecHandler;
+
+    @Resource
+    private MqttPubCompHandler mqttPubCompHandler;
+
+    @Resource
+    private UpstreamHookManager upstreamHookManager;
+
+    @Resource
+    private ChannelManager channelManager;
+
+    @Override
+    protected void channelRead0(ChannelHandlerContext ctx, MqttMessage msg) throws Exception {
+        if (!ctx.channel().isActive()) {
+            return;
+        }
+        if (!msg.decoderResult().isSuccess()) {
+            throw new RuntimeException(ChannelInfo.getClientId(ctx.channel()) + "," + msg.decoderResult());
+        }
+        ChannelInfo.touch(ctx.channel());
+        CompletableFuture<HookResult> upstreamHookResult;
+        try {
+            if (msg instanceof MqttPublishMessage) {
+                ((MqttPublishMessage) msg).retain();
+            }
+            upstreamHookResult = upstreamHookManager.doUpstreamHook(buildMqttMessageUpContext(ctx), msg);
+            if (upstreamHookResult == null) {
+                _channelRead0(ctx, msg, null);
+                return;
+            }
+        } catch (Throwable t) {
+            logger.error("", t);
+            if (msg instanceof MqttPublishMessage) {
+                ReferenceCountUtil.release(msg);
+            }
+            throw new ChannelException(t.getMessage());
+        }
+        upstreamHookResult.whenComplete((hookResult, throwable) -> {
+            if (msg instanceof MqttPublishMessage) {
+                ReferenceCountUtil.release(msg);
+            }
+            if (throwable != null) {
+                logger.error("", throwable);
+                ctx.fireExceptionCaught(new ChannelException(throwable.getMessage()));
+                return;
+            }
+            if (hookResult == null) {
+                ctx.fireExceptionCaught(new ChannelException("UpstreamHook Result Unknown"));
+                return;
+            }
+            try {
+                _channelRead0(ctx, msg, hookResult);
+            } catch (Throwable t) {
+                logger.error("", t);
+                ctx.fireExceptionCaught(new ChannelException(t.getMessage()));
+            }
+        });
+    }
+
+    private void _channelRead0(ChannelHandlerContext ctx, MqttMessage msg, HookResult upstreamHookResult) {
+        switch (msg.fixedHeader().messageType()) {
+            case CONNECT:
+                mqttConnectHandler.doHandler(ctx, (MqttConnectMessage) msg, upstreamHookResult);
+                break;
+            case PUBLISH:
+                mqttPublishHandler.doHandler(ctx, (MqttPublishMessage) msg, upstreamHookResult);
+                break;
+            case SUBSCRIBE:
+                mqttSubscribeHandler.doHandler(ctx, (MqttSubscribeMessage) msg, upstreamHookResult);
+                break;
+            case PUBACK:
+                mqttPubAckHandler.doHandler(ctx, (MqttPubAckMessage) msg, upstreamHookResult);
+                break;
+            case PINGREQ:
+                mqttPingHandler.doHandler(ctx, msg, upstreamHookResult);
+                break;
+            case UNSUBSCRIBE:
+                mqttUnSubscribeHandler.doHandler(ctx, (MqttUnsubscribeMessage) msg, upstreamHookResult);
+                break;
+            case PUBREL:
+                mqttPubRelHandler.doHandler(ctx, msg, upstreamHookResult);
+                break;
+            case PUBREC:
+                mqttPubRecHandler.doHandler(ctx, msg, upstreamHookResult);
+                break;
+            case PUBCOMP:
+                mqttPubCompHandler.doHandler(ctx, msg, upstreamHookResult);
+                break;
+            case DISCONNECT:
+                mqttDisconnectHandler.doHandler(ctx, msg, upstreamHookResult);
+                break;
+            default:
+        }
+    }
+
+    public MqttMessageUpContext buildMqttMessageUpContext(ChannelHandlerContext ctx) {
+        MqttMessageUpContext context = new MqttMessageUpContext();
+        Channel channel = ctx.channel();
+        context.setClientId(ChannelInfo.getClientId(channel));
+        context.setChannelId(ChannelInfo.getId(channel));
+        context.setNode(HostInfo.getInstall().getAddress());
+        context.setNamespace(ChannelInfo.getNamespace(channel));
+        return context;
+    }
+
+}
diff --git a/mqtt-cs/src/main/java/org/apache/rocketmq/mqtt/cs/protocol/mqtt/MqttPacketHandler.java b/mqtt-cs/src/main/java/org/apache/rocketmq/mqtt/cs/protocol/mqtt/MqttPacketHandler.java
new file mode 100644
index 0000000..73094f3
--- /dev/null
+++ b/mqtt-cs/src/main/java/org/apache/rocketmq/mqtt/cs/protocol/mqtt/MqttPacketHandler.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.mqtt.cs.protocol.mqtt;
+
+import io.netty.channel.ChannelHandlerContext;
+import io.netty.handler.codec.mqtt.MqttMessage;
+import org.apache.rocketmq.mqtt.common.hook.HookResult;
+
+
+public interface MqttPacketHandler<T extends MqttMessage> {
+
+    /**
+     * doHandler
+     *
+     * @param ctx
+     * @param mqttMessage
+     */
+    void doHandler(ChannelHandlerContext ctx, T mqttMessage, HookResult upstreamHookResult);
+
+}
diff --git a/mqtt-cs/src/main/java/org/apache/rocketmq/mqtt/cs/protocol/mqtt/handler/MqttConnectHandler.java b/mqtt-cs/src/main/java/org/apache/rocketmq/mqtt/cs/protocol/mqtt/handler/MqttConnectHandler.java
new file mode 100644
index 0000000..9fbcbac
--- /dev/null
+++ b/mqtt-cs/src/main/java/org/apache/rocketmq/mqtt/cs/protocol/mqtt/handler/MqttConnectHandler.java
@@ -0,0 +1,111 @@
+/*
+ *
+ *  * Licensed to the Apache Software Foundation (ASF) under one or more
+ *  * contributor license agreements.  See the NOTICE file distributed with
+ *  * this work for additional information regarding copyright ownership.
+ *  * The ASF licenses this file to You under the Apache License, Version 2.0
+ *  * (the "License"); you may not use this file except in compliance with
+ *  * the License.  You may obtain a copy of the License at
+ *  *
+ *  *     http://www.apache.org/licenses/LICENSE-2.0
+ *  *
+ *  * Unless required by applicable law or agreed to in writing, software
+ *  * distributed under the License is distributed on an "AS IS" BASIS,
+ *  * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ *  * See the License for the specific language governing permissions and
+ *  * limitations under the License.
+ *
+ */
+
+package org.apache.rocketmq.mqtt.cs.protocol.mqtt.handler;
+
+
+import io.netty.channel.Channel;
+import io.netty.channel.ChannelHandlerContext;
+import io.netty.handler.codec.mqtt.*;
+import org.apache.rocketmq.common.ThreadFactoryImpl;
+import org.apache.rocketmq.mqtt.common.hook.HookResult;
+import org.apache.rocketmq.mqtt.cs.channel.ChannelCloseFrom;
+import org.apache.rocketmq.mqtt.cs.channel.ChannelInfo;
+import org.apache.rocketmq.mqtt.cs.channel.ChannelManager;
+import org.apache.rocketmq.mqtt.cs.config.ConnectConf;
+import org.apache.rocketmq.mqtt.cs.protocol.mqtt.MqttPacketHandler;
+import org.apache.rocketmq.mqtt.cs.session.loop.SessionLoop;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+import org.springframework.stereotype.Component;
+
+import javax.annotation.Resource;
+import java.util.concurrent.CompletableFuture;
+import java.util.concurrent.ScheduledThreadPoolExecutor;
+import java.util.concurrent.TimeUnit;
+
+
+@Component
+public class MqttConnectHandler implements MqttPacketHandler<MqttConnectMessage> {
+    private static Logger logger = LoggerFactory.getLogger(MqttConnectHandler.class);
+
+    @Resource
+    private ChannelManager channelManager;
+
+    @Resource
+    private SessionLoop sessionLoop;
+
+    @Resource
+    private ConnectConf connectConf;
+
+    private ScheduledThreadPoolExecutor scheduler = new ScheduledThreadPoolExecutor(1, new ThreadFactoryImpl("check_connect_future"));
+
+    @Override
+    public void doHandler(ChannelHandlerContext ctx, MqttConnectMessage connectMessage, HookResult upstreamHookResult) {
+        MqttConnectVariableHeader variableHeader = connectMessage.variableHeader();
+        Channel channel = ctx.channel();
+        ChannelInfo.setKeepLive(channel, variableHeader.keepAliveTimeSeconds());
+        ChannelInfo.setClientId(channel, connectMessage.payload().clientIdentifier());
+        ChannelInfo.setCleanSessionFlag(channel, variableHeader.isCleanSession());
+        CompletableFuture<Void> future = new CompletableFuture<>();
+        ChannelInfo.setFuture(channel, ChannelInfo.FUTURE_CONNECT, future);
+        scheduler.schedule(() -> {
+            if (!future.isDone()) {
+                future.complete(null);
+            }
+        }, 1, TimeUnit.SECONDS);
+        String remark = upstreamHookResult.getRemark();
+        if (!upstreamHookResult.isSuccess()) {
+            byte connAckCode = (byte) upstreamHookResult.getSubCode();
+            MqttConnectReturnCode mqttConnectReturnCode = MqttConnectReturnCode.valueOf(connAckCode);
+            if (mqttConnectReturnCode == null) {
+                channelManager.closeConnect(channel, ChannelCloseFrom.SERVER, remark);
+                return;
+            }
+            channel.writeAndFlush(getMqttConnAckMessage(mqttConnectReturnCode));
+            channelManager.closeConnect(channel, ChannelCloseFrom.SERVER, remark);
+            return;
+        }
+        try {
+            MqttConnAckMessage mqttConnAckMessage = getMqttConnAckMessage(MqttConnectReturnCode.CONNECTION_ACCEPTED);
+            future.thenAccept(aVoid -> {
+                if (!channel.isActive()) {
+                    return;
+                }
+                ChannelInfo.removeFuture(channel, ChannelInfo.FUTURE_CONNECT);
+                channel.writeAndFlush(mqttConnAckMessage);
+            });
+            sessionLoop.loadSession(ChannelInfo.getClientId(channel), channel);
+        } catch (Exception e) {
+            logger.error("Connect:{}", connectMessage.payload().clientIdentifier(), e);
+            channelManager.closeConnect(channel, ChannelCloseFrom.SERVER, "ConnectException");
+        }
+    }
+
+    private MqttConnAckMessage getMqttConnAckMessage(MqttConnectReturnCode returnCode) {
+        MqttConnAckVariableHeader mqttConnAckVariableHeader =
+                new MqttConnAckVariableHeader(returnCode, false);
+        MqttFixedHeader mqttFixedHeader =
+                new MqttFixedHeader(MqttMessageType.CONNACK, false, MqttQoS.AT_MOST_ONCE, false, 0);
+        MqttConnAckMessage mqttConnAckMessage =
+                new MqttConnAckMessage(mqttFixedHeader, mqttConnAckVariableHeader);
+        return mqttConnAckMessage;
+    }
+
+}
diff --git a/mqtt-cs/src/main/java/org/apache/rocketmq/mqtt/cs/protocol/mqtt/handler/MqttDisconnectHandler.java b/mqtt-cs/src/main/java/org/apache/rocketmq/mqtt/cs/protocol/mqtt/handler/MqttDisconnectHandler.java
new file mode 100644
index 0000000..1c563c0
--- /dev/null
+++ b/mqtt-cs/src/main/java/org/apache/rocketmq/mqtt/cs/protocol/mqtt/handler/MqttDisconnectHandler.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.mqtt.cs.protocol.mqtt.handler;
+
+import io.netty.channel.ChannelHandlerContext;
+import io.netty.handler.codec.mqtt.MqttMessage;
+import org.apache.rocketmq.mqtt.common.hook.HookResult;
+import org.apache.rocketmq.mqtt.cs.channel.ChannelCloseFrom;
+import org.apache.rocketmq.mqtt.cs.channel.ChannelManager;
+import org.apache.rocketmq.mqtt.cs.protocol.mqtt.MqttPacketHandler;
+import org.springframework.stereotype.Component;
+
+import javax.annotation.Resource;
+
+
+
+@Component
+public class MqttDisconnectHandler implements MqttPacketHandler<MqttMessage> {
+
+    @Resource
+    private ChannelManager channelManager;
+
+    @Override
+    public void doHandler(ChannelHandlerContext ctx, MqttMessage mqttMessage, HookResult upstreamHookResult) {
+        channelManager.closeConnect(ctx.channel(), ChannelCloseFrom.CLIENT, "disconnect");
+    }
+
+}
diff --git a/mqtt-cs/src/main/java/org/apache/rocketmq/mqtt/cs/protocol/mqtt/handler/MqttPingHandler.java b/mqtt-cs/src/main/java/org/apache/rocketmq/mqtt/cs/protocol/mqtt/handler/MqttPingHandler.java
new file mode 100644
index 0000000..d68ff79
--- /dev/null
+++ b/mqtt-cs/src/main/java/org/apache/rocketmq/mqtt/cs/protocol/mqtt/handler/MqttPingHandler.java
@@ -0,0 +1,57 @@
+/*
+ *
+ *  * Licensed to the Apache Software Foundation (ASF) under one or more
+ *  * contributor license agreements.  See the NOTICE file distributed with
+ *  * this work for additional information regarding copyright ownership.
+ *  * The ASF licenses this file to You under the Apache License, Version 2.0
+ *  * (the "License"); you may not use this file except in compliance with
+ *  * the License.  You may obtain a copy of the License at
+ *  *
+ *  *     http://www.apache.org/licenses/LICENSE-2.0
+ *  *
+ *  * Unless required by applicable law or agreed to in writing, software
+ *  * distributed under the License is distributed on an "AS IS" BASIS,
+ *  * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ *  * See the License for the specific language governing permissions and
+ *  * limitations under the License.
+ *
+ */
+
+package org.apache.rocketmq.mqtt.cs.protocol.mqtt.handler;
+
+
+import io.netty.channel.Channel;
+import io.netty.channel.ChannelHandlerContext;
+import io.netty.handler.codec.mqtt.MqttFixedHeader;
+import io.netty.handler.codec.mqtt.MqttMessage;
+import io.netty.handler.codec.mqtt.MqttMessageType;
+import io.netty.handler.codec.mqtt.MqttQoS;
+import org.apache.rocketmq.mqtt.common.hook.HookResult;
+import org.apache.rocketmq.mqtt.cs.channel.ChannelInfo;
+import org.apache.rocketmq.mqtt.cs.channel.ChannelManager;
+import org.apache.rocketmq.mqtt.cs.protocol.mqtt.MqttPacketHandler;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+import org.springframework.stereotype.Component;
+
+import javax.annotation.Resource;
+
+
+
+@Component
+public class MqttPingHandler implements MqttPacketHandler<MqttMessage> {
+    private static Logger logger = LoggerFactory.getLogger(MqttPingHandler.class);
+
+    @Resource
+    private ChannelManager channelManager;
+
+    @Override
+    public void doHandler(ChannelHandlerContext ctx, MqttMessage mqttMessage, HookResult upstreamHookResult) {
+        MqttFixedHeader mqttFixedHeader =
+                new MqttFixedHeader(MqttMessageType.PINGRESP, false, MqttQoS.AT_MOST_ONCE, false, 0);
+        Channel channel = ctx.channel();
+        ChannelInfo.touch(channel);
+        MqttMessage pingMessage = new MqttMessage(mqttFixedHeader);
+        channel.writeAndFlush(pingMessage);
+    }
+}
diff --git a/mqtt-cs/src/main/java/org/apache/rocketmq/mqtt/cs/protocol/mqtt/handler/MqttPubAckHandler.java b/mqtt-cs/src/main/java/org/apache/rocketmq/mqtt/cs/protocol/mqtt/handler/MqttPubAckHandler.java
new file mode 100644
index 0000000..ba2cbd4
--- /dev/null
+++ b/mqtt-cs/src/main/java/org/apache/rocketmq/mqtt/cs/protocol/mqtt/handler/MqttPubAckHandler.java
@@ -0,0 +1,58 @@
+/*
+ *
+ *  * Licensed to the Apache Software Foundation (ASF) under one or more
+ *  * contributor license agreements.  See the NOTICE file distributed with
+ *  * this work for additional information regarding copyright ownership.
+ *  * The ASF licenses this file to You under the Apache License, Version 2.0
+ *  * (the "License"); you may not use this file except in compliance with
+ *  * the License.  You may obtain a copy of the License at
+ *  *
+ *  *     http://www.apache.org/licenses/LICENSE-2.0
+ *  *
+ *  * Unless required by applicable law or agreed to in writing, software
+ *  * distributed under the License is distributed on an "AS IS" BASIS,
+ *  * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ *  * See the License for the specific language governing permissions and
+ *  * limitations under the License.
+ *
+ */
+
+package org.apache.rocketmq.mqtt.cs.protocol.mqtt.handler;
+
+
+import io.netty.channel.ChannelHandlerContext;
+import io.netty.handler.codec.mqtt.MqttPubAckMessage;
+import org.apache.rocketmq.mqtt.common.hook.HookResult;
+import org.apache.rocketmq.mqtt.cs.channel.ChannelInfo;
+import org.apache.rocketmq.mqtt.cs.protocol.mqtt.MqttPacketHandler;
+import org.apache.rocketmq.mqtt.cs.session.infly.PushAction;
+import org.apache.rocketmq.mqtt.cs.session.infly.RetryDriver;
+import org.apache.rocketmq.mqtt.cs.session.loop.SessionLoop;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+import org.springframework.stereotype.Component;
+
+import javax.annotation.Resource;
+
+
+
+@Component
+public class MqttPubAckHandler implements MqttPacketHandler<MqttPubAckMessage> {
+    private static Logger logger = LoggerFactory.getLogger(MqttPubAckHandler.class);
+
+    @Resource
+    private PushAction pushAction;
+
+    @Resource
+    private RetryDriver retryDriver;
+
+    @Resource
+    private SessionLoop sessionLoop;
+
+    @Override
+    public void doHandler(ChannelHandlerContext ctx, MqttPubAckMessage mqttMessage, HookResult upstreamHookResult) {
+        int messageId = mqttMessage.variableHeader().messageId();
+        retryDriver.unMountPublish(messageId, ChannelInfo.getId(ctx.channel()));
+        pushAction.rollNextByAck(sessionLoop.getSession(ChannelInfo.getId(ctx.channel())), messageId);
+    }
+}
diff --git a/mqtt-cs/src/main/java/org/apache/rocketmq/mqtt/cs/protocol/mqtt/handler/MqttPubCompHandler.java b/mqtt-cs/src/main/java/org/apache/rocketmq/mqtt/cs/protocol/mqtt/handler/MqttPubCompHandler.java
new file mode 100644
index 0000000..17913a6
--- /dev/null
+++ b/mqtt-cs/src/main/java/org/apache/rocketmq/mqtt/cs/protocol/mqtt/handler/MqttPubCompHandler.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.mqtt.cs.protocol.mqtt.handler;
+
+import io.netty.channel.ChannelHandlerContext;
+import io.netty.handler.codec.mqtt.MqttMessage;
+import io.netty.handler.codec.mqtt.MqttMessageIdVariableHeader;
+import org.apache.rocketmq.mqtt.common.hook.HookResult;
+import org.apache.rocketmq.mqtt.cs.channel.ChannelInfo;
+import org.apache.rocketmq.mqtt.cs.protocol.mqtt.MqttPacketHandler;
+import org.apache.rocketmq.mqtt.cs.session.infly.InFlyCache;
+import org.apache.rocketmq.mqtt.cs.session.infly.MqttMsgId;
+import org.apache.rocketmq.mqtt.cs.session.infly.PushAction;
+import org.apache.rocketmq.mqtt.cs.session.infly.RetryDriver;
+import org.apache.rocketmq.mqtt.cs.session.loop.SessionLoop;
+import org.springframework.stereotype.Component;
+
+import javax.annotation.Resource;
+
+
+@Component
+public class MqttPubCompHandler implements MqttPacketHandler<MqttMessage> {
+
+    @Resource
+    private RetryDriver retryDriver;
+
+    @Resource
+    private InFlyCache inFlyCache;
+
+    @Resource
+    private MqttMsgId mqttMsgId;
+
+    @Resource
+    private PushAction pushAction;
+
+    @Resource
+    private SessionLoop sessionLoop;
+
+    @Override
+    public void doHandler(ChannelHandlerContext ctx, MqttMessage mqttMessage, HookResult upstreamHookResult) {
+        MqttMessageIdVariableHeader variableHeader = (MqttMessageIdVariableHeader) mqttMessage.variableHeader();
+        String channelId = ChannelInfo.getId(ctx.channel());
+
+        retryDriver.unMountPubRel(variableHeader.messageId(), ChannelInfo.getId(ctx.channel()));
+
+        //The Packet Identifier becomes available for reuse once the Sender has received the PUBCOMP Packet.
+        pushAction.rollNextByAck(sessionLoop.getSession(channelId), variableHeader.messageId());
+    }
+
+}
diff --git a/mqtt-cs/src/main/java/org/apache/rocketmq/mqtt/cs/protocol/mqtt/handler/MqttPubRecHandler.java b/mqtt-cs/src/main/java/org/apache/rocketmq/mqtt/cs/protocol/mqtt/handler/MqttPubRecHandler.java
new file mode 100644
index 0000000..cfe0c0d
--- /dev/null
+++ b/mqtt-cs/src/main/java/org/apache/rocketmq/mqtt/cs/protocol/mqtt/handler/MqttPubRecHandler.java
@@ -0,0 +1,60 @@
+/*
+ *
+ *  * Licensed to the Apache Software Foundation (ASF) under one or more
+ *  * contributor license agreements.  See the NOTICE file distributed with
+ *  * this work for additional information regarding copyright ownership.
+ *  * The ASF licenses this file to You under the Apache License, Version 2.0
+ *  * (the "License"); you may not use this file except in compliance with
+ *  * the License.  You may obtain a copy of the License at
+ *  *
+ *  *     http://www.apache.org/licenses/LICENSE-2.0
+ *  *
+ *  * Unless required by applicable law or agreed to in writing, software
+ *  * distributed under the License is distributed on an "AS IS" BASIS,
+ *  * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ *  * See the License for the specific language governing permissions and
+ *  * limitations under the License.
+ *
+ */
+
+package org.apache.rocketmq.mqtt.cs.protocol.mqtt.handler;
+
+
+import io.netty.channel.ChannelHandlerContext;
+import io.netty.handler.codec.mqtt.*;
+import org.apache.rocketmq.mqtt.common.hook.HookResult;
+import org.apache.rocketmq.mqtt.cs.channel.ChannelInfo;
+import org.apache.rocketmq.mqtt.cs.protocol.mqtt.MqttPacketHandler;
+import org.apache.rocketmq.mqtt.cs.session.infly.InFlyCache;
+import org.apache.rocketmq.mqtt.cs.session.infly.RetryDriver;
+import org.apache.rocketmq.mqtt.cs.session.loop.SessionLoop;
+import org.springframework.stereotype.Component;
+
+import javax.annotation.Resource;
+
+
+@Component
+public class MqttPubRecHandler implements MqttPacketHandler<MqttMessage> {
+
+    @Resource
+    private RetryDriver retryDriver;
+
+    @Resource
+    private InFlyCache inFlyCache;
+
+    @Resource
+    private SessionLoop sessionLoop;
+
+    @Override
+    public void doHandler(ChannelHandlerContext ctx, MqttMessage mqttMessage, HookResult upstreamHookResult) {
+        MqttMessageIdVariableHeader variableHeader = (MqttMessageIdVariableHeader) mqttMessage.variableHeader();
+        String channelId = ChannelInfo.getId(ctx.channel());
+        retryDriver.unMountPublish(variableHeader.messageId(), channelId);
+        retryDriver.mountPubRel(variableHeader.messageId(), channelId);
+
+        MqttFixedHeader pubRelMqttFixedHeader = new MqttFixedHeader(MqttMessageType.PUBREL, false,
+                MqttQoS.AT_LEAST_ONCE, false, 0);
+        MqttMessage pubRelMqttMessage = new MqttMessage(pubRelMqttFixedHeader, variableHeader);
+        ctx.channel().writeAndFlush(pubRelMqttMessage);
+    }
+}
diff --git a/mqtt-cs/src/main/java/org/apache/rocketmq/mqtt/cs/protocol/mqtt/handler/MqttPubRelHandler.java b/mqtt-cs/src/main/java/org/apache/rocketmq/mqtt/cs/protocol/mqtt/handler/MqttPubRelHandler.java
new file mode 100644
index 0000000..11db612
--- /dev/null
+++ b/mqtt-cs/src/main/java/org/apache/rocketmq/mqtt/cs/protocol/mqtt/handler/MqttPubRelHandler.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.mqtt.cs.protocol.mqtt.handler;
+
+
+import io.netty.channel.ChannelHandlerContext;
+import io.netty.handler.codec.mqtt.*;
+import org.apache.rocketmq.mqtt.common.hook.HookResult;
+import org.apache.rocketmq.mqtt.cs.channel.ChannelInfo;
+import org.apache.rocketmq.mqtt.cs.protocol.mqtt.MqttPacketHandler;
+import org.apache.rocketmq.mqtt.cs.session.infly.InFlyCache;
+import org.springframework.stereotype.Component;
+
+import javax.annotation.Resource;
+
+
+@Component
+public class MqttPubRelHandler implements MqttPacketHandler<MqttMessage> {
+
+    @Resource
+    private InFlyCache inFlyCache;
+
+    @Override
+    public void doHandler(ChannelHandlerContext ctx, MqttMessage mqttMessage, HookResult upstreamHookResult) {
+        final MqttMessageIdVariableHeader variableHeader = (MqttMessageIdVariableHeader) mqttMessage.variableHeader();
+        String channelId = ChannelInfo.getId(ctx.channel());
+        inFlyCache.remove(InFlyCache.CacheType.PUB, channelId, variableHeader.messageId());
+
+        MqttFixedHeader pubcompFixedHeader = new MqttFixedHeader(MqttMessageType.PUBCOMP, false, MqttQoS.AT_MOST_ONCE,
+                false, 0);
+        MqttMessage pubcomMqttMessage = new MqttMessage(pubcompFixedHeader, variableHeader);
+        ctx.channel().writeAndFlush(pubcomMqttMessage);
+    }
+}
diff --git a/mqtt-cs/src/main/java/org/apache/rocketmq/mqtt/cs/protocol/mqtt/handler/MqttPublishHandler.java b/mqtt-cs/src/main/java/org/apache/rocketmq/mqtt/cs/protocol/mqtt/handler/MqttPublishHandler.java
new file mode 100644
index 0000000..8825637
--- /dev/null
+++ b/mqtt-cs/src/main/java/org/apache/rocketmq/mqtt/cs/protocol/mqtt/handler/MqttPublishHandler.java
@@ -0,0 +1,114 @@
+/*
+ *
+ *  * Licensed to the Apache Software Foundation (ASF) under one or more
+ *  * contributor license agreements.  See the NOTICE file distributed with
+ *  * this work for additional information regarding copyright ownership.
+ *  * The ASF licenses this file to You under the Apache License, Version 2.0
+ *  * (the "License"); you may not use this file except in compliance with
+ *  * the License.  You may obtain a copy of the License at
+ *  *
+ *  *     http://www.apache.org/licenses/LICENSE-2.0
+ *  *
+ *  * Unless required by applicable law or agreed to in writing, software
+ *  * distributed under the License is distributed on an "AS IS" BASIS,
+ *  * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ *  * See the License for the specific language governing permissions and
+ *  * limitations under the License.
+ *
+ */
+
+package org.apache.rocketmq.mqtt.cs.protocol.mqtt.handler;
+
+import io.netty.channel.Channel;
+import io.netty.channel.ChannelHandlerContext;
+import io.netty.handler.codec.mqtt.*;
+import org.apache.rocketmq.mqtt.common.hook.HookResult;
+import org.apache.rocketmq.mqtt.cs.channel.ChannelCloseFrom;
+import org.apache.rocketmq.mqtt.cs.channel.ChannelInfo;
+import org.apache.rocketmq.mqtt.cs.channel.ChannelManager;
+import org.apache.rocketmq.mqtt.cs.config.ConnectConf;
+import org.apache.rocketmq.mqtt.cs.protocol.mqtt.MqttPacketHandler;
+import org.apache.rocketmq.mqtt.cs.session.infly.InFlyCache;
+import org.apache.rocketmq.mqtt.cs.session.loop.SessionLoop;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+import org.springframework.stereotype.Component;
+
+import javax.annotation.Resource;
+
+@Component
+public class MqttPublishHandler implements MqttPacketHandler<MqttPublishMessage> {
+    private static Logger logger = LoggerFactory.getLogger(MqttPublishHandler.class);
+
+    @Resource
+    private InFlyCache inFlyCache;
+
+    @Resource
+    private ChannelManager channelManager;
+
+    @Resource
+    private SessionLoop sessionLoop;
+
+    @Resource
+    private ConnectConf connectConf;
+
+
+    @Override
+    public void doHandler(ChannelHandlerContext ctx,
+                          MqttPublishMessage mqttMessage,
+                          HookResult upstreamHookResult) {
+        final MqttPublishVariableHeader variableHeader = mqttMessage.variableHeader();
+        Channel channel = ctx.channel();
+        String channelId = ChannelInfo.getId(channel);
+        final boolean isQos2Message = isQos2Message(mqttMessage);
+        if (isQos2Message) {
+            if (inFlyCache.contains(InFlyCache.CacheType.PUB, channelId, variableHeader.messageId())) {
+                doResponse(ctx, mqttMessage);
+                return;
+            }
+        }
+        String remark = upstreamHookResult.getRemark();
+        if (!upstreamHookResult.isSuccess()) {
+            channelManager.closeConnect(channel, ChannelCloseFrom.SERVER, remark);
+            return;
+        }
+        doResponse(ctx, mqttMessage);
+        if (isQos2Message) {
+            inFlyCache.put(InFlyCache.CacheType.PUB, channelId, variableHeader.messageId());
+        }
+    }
+
+    private boolean isQos2Message(MqttPublishMessage mqttPublishMessage) {
+        return MqttQoS.EXACTLY_ONCE.equals(mqttPublishMessage.fixedHeader().qosLevel());
+    }
+
+    private void doResponse(ChannelHandlerContext ctx, MqttPublishMessage mqttMessage) {
+        MqttFixedHeader fixedHeader = mqttMessage.fixedHeader();
+        MqttPublishVariableHeader variableHeader = mqttMessage.variableHeader();
+        switch (fixedHeader.qosLevel()) {
+            case AT_MOST_ONCE:
+                break;
+            case AT_LEAST_ONCE:
+                MqttFixedHeader mqttFixedHeader = new MqttFixedHeader(MqttMessageType.PUBACK, false,
+                    MqttQoS.AT_MOST_ONCE,
+                    false, 0);
+                MqttMessageIdVariableHeader mqttMessageIdVariableHeader = MqttMessageIdVariableHeader
+                    .from(variableHeader.messageId());
+                MqttPubAckMessage pubackMessage = new MqttPubAckMessage(mqttFixedHeader, mqttMessageIdVariableHeader);
+                ctx.channel().writeAndFlush(pubackMessage);
+                break;
+            case EXACTLY_ONCE:
+                MqttFixedHeader pubrecMqttHeader = new MqttFixedHeader(MqttMessageType.PUBREC, false,
+                    MqttQoS.AT_MOST_ONCE,
+                    false, 0);
+                MqttMessageIdVariableHeader pubrecMessageIdVariableHeader = MqttMessageIdVariableHeader
+                    .from(variableHeader.messageId());
+                MqttMessage pubrecMqttMessage = new MqttMessage(pubrecMqttHeader, pubrecMessageIdVariableHeader);
+                ctx.channel().writeAndFlush(pubrecMqttMessage);
+                break;
+            default:
+                throw new IllegalArgumentException("unknown qos:" + fixedHeader.qosLevel());
+        }
+    }
+
+}
diff --git a/mqtt-cs/src/main/java/org/apache/rocketmq/mqtt/cs/protocol/mqtt/handler/MqttSubscribeHandler.java b/mqtt-cs/src/main/java/org/apache/rocketmq/mqtt/cs/protocol/mqtt/handler/MqttSubscribeHandler.java
new file mode 100644
index 0000000..3435ff3
--- /dev/null
+++ b/mqtt-cs/src/main/java/org/apache/rocketmq/mqtt/cs/protocol/mqtt/handler/MqttSubscribeHandler.java
@@ -0,0 +1,130 @@
+/*
+ *
+ *  * Licensed to the Apache Software Foundation (ASF) under one or more
+ *  * contributor license agreements.  See the NOTICE file distributed with
+ *  * this work for additional information regarding copyright ownership.
+ *  * The ASF licenses this file to You under the Apache License, Version 2.0
+ *  * (the "License"); you may not use this file except in compliance with
+ *  * the License.  You may obtain a copy of the License at
+ *  *
+ *  *     http://www.apache.org/licenses/LICENSE-2.0
+ *  *
+ *  * Unless required by applicable law or agreed to in writing, software
+ *  * distributed under the License is distributed on an "AS IS" BASIS,
+ *  * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ *  * See the License for the specific language governing permissions and
+ *  * limitations under the License.
+ *
+ */
+
+package org.apache.rocketmq.mqtt.cs.protocol.mqtt.handler;
+
+
+import io.netty.channel.Channel;
+import io.netty.channel.ChannelHandlerContext;
+import io.netty.handler.codec.mqtt.*;
+import org.apache.rocketmq.common.ThreadFactoryImpl;
+import org.apache.rocketmq.mqtt.common.hook.HookResult;
+import org.apache.rocketmq.mqtt.common.model.Subscription;
+import org.apache.rocketmq.mqtt.common.util.TopicUtils;
+import org.apache.rocketmq.mqtt.cs.channel.ChannelCloseFrom;
+import org.apache.rocketmq.mqtt.cs.channel.ChannelInfo;
+import org.apache.rocketmq.mqtt.cs.channel.ChannelManager;
+import org.apache.rocketmq.mqtt.cs.config.ConnectConf;
+import org.apache.rocketmq.mqtt.cs.protocol.mqtt.MqttPacketHandler;
+import org.apache.rocketmq.mqtt.cs.session.loop.SessionLoop;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+import org.springframework.stereotype.Component;
+
+import javax.annotation.Resource;
+import java.util.HashSet;
+import java.util.List;
+import java.util.Set;
+import java.util.concurrent.CompletableFuture;
+import java.util.concurrent.ScheduledThreadPoolExecutor;
+import java.util.concurrent.TimeUnit;
+
+import static io.netty.handler.codec.mqtt.MqttMessageIdVariableHeader.from;
+import static io.netty.handler.codec.mqtt.MqttMessageType.SUBACK;
+import static io.netty.handler.codec.mqtt.MqttQoS.AT_MOST_ONCE;
+
+
+
+@Component
+public class MqttSubscribeHandler implements MqttPacketHandler<MqttSubscribeMessage> {
+    private static Logger logger = LoggerFactory.getLogger(MqttSubscribeHandler.class);
+
+
+    @Resource
+    private SessionLoop sessionLoop;
+
+    @Resource
+    private ChannelManager channelManager;
+
+    @Resource
+    private ConnectConf connectConf;
+
+    private ScheduledThreadPoolExecutor scheduler = new ScheduledThreadPoolExecutor(1, new ThreadFactoryImpl("check_connect_future"));
+
+
+    @Override
+    public void doHandler(ChannelHandlerContext ctx, MqttSubscribeMessage mqttMessage, HookResult upstreamHookResult) {
+        String clientId = ChannelInfo.getClientId(ctx.channel());
+        Channel channel = ctx.channel();
+        CompletableFuture<Void> future = new CompletableFuture<>();
+        ChannelInfo.setFuture(channel, ChannelInfo.FUTURE_SUBSCRIBE, future);
+        scheduler.schedule(() -> {
+            if(!future.isDone()){
+                future.complete(null);
+            }
+        },1,TimeUnit.SECONDS);
+        String remark = upstreamHookResult.getRemark();
+        if(!upstreamHookResult.isSuccess()){
+            channelManager.closeConnect(channel, ChannelCloseFrom.SERVER, remark);
+            return;
+        }
+        try {
+            MqttSubscribePayload payload = mqttMessage.payload();
+            List<MqttTopicSubscription> mqttTopicSubscriptions = payload.topicSubscriptions();
+            if (mqttTopicSubscriptions != null && !mqttTopicSubscriptions.isEmpty()) {
+                Set<Subscription> subscriptions = new HashSet<>(mqttTopicSubscriptions.size());
+                for (MqttTopicSubscription mqttTopicSubscription : mqttTopicSubscriptions) {
+                    Subscription subscription = new Subscription();
+                    subscription.setQos(mqttTopicSubscription.qualityOfService().value());
+                    subscription.setTopicFilter(TopicUtils.normalizeTopic(mqttTopicSubscription.topicName()));
+                    subscriptions.add(subscription);
+                }
+                sessionLoop.addSubscription(ChannelInfo.getId(ctx.channel()), subscriptions);
+            }
+            future.thenAccept(aVoid -> {
+                if (!channel.isActive()) {
+                    return;
+                }
+                ChannelInfo.removeFuture(channel, ChannelInfo.FUTURE_SUBSCRIBE);
+                channel.writeAndFlush(getResponse(mqttMessage));
+            });
+        } catch (Exception e) {
+            logger.error("Subscribe:{}", clientId, e);
+            channelManager.closeConnect(channel, ChannelCloseFrom.SERVER, "SubscribeException");
+        }
+    }
+
+
+    private MqttSubAckMessage getResponse(MqttSubscribeMessage mqttSubscribeMessage) {
+        MqttSubscribePayload payload = mqttSubscribeMessage.payload();
+        List<MqttTopicSubscription> mqttTopicSubscriptions = payload.topicSubscriptions();
+        // AT_MOST_ONCE
+        int[] qoss = new int[mqttTopicSubscriptions.size()];
+        int i = 0;
+        for (MqttTopicSubscription sub : mqttTopicSubscriptions) {
+            qoss[i++] = sub.qualityOfService().value();
+        }
+        MqttFixedHeader fixedHeader = new MqttFixedHeader(SUBACK, false, AT_MOST_ONCE, false, 0);
+        MqttMessageIdVariableHeader variableHeader = from(mqttSubscribeMessage.variableHeader().messageId());
+        MqttSubAckMessage mqttSubAckMessage = new MqttSubAckMessage(fixedHeader, variableHeader,
+            new MqttSubAckPayload(qoss));
+        return mqttSubAckMessage;
+    }
+
+}
diff --git a/mqtt-cs/src/main/java/org/apache/rocketmq/mqtt/cs/protocol/mqtt/handler/MqttUnSubscribeHandler.java b/mqtt-cs/src/main/java/org/apache/rocketmq/mqtt/cs/protocol/mqtt/handler/MqttUnSubscribeHandler.java
new file mode 100644
index 0000000..4cc32b1
--- /dev/null
+++ b/mqtt-cs/src/main/java/org/apache/rocketmq/mqtt/cs/protocol/mqtt/handler/MqttUnSubscribeHandler.java
@@ -0,0 +1,91 @@
+/*
+ *
+ *  * Licensed to the Apache Software Foundation (ASF) under one or more
+ *  * contributor license agreements.  See the NOTICE file distributed with
+ *  * this work for additional information regarding copyright ownership.
+ *  * The ASF licenses this file to You under the Apache License, Version 2.0
+ *  * (the "License"); you may not use this file except in compliance with
+ *  * the License.  You may obtain a copy of the License at
+ *  *
+ *  *     http://www.apache.org/licenses/LICENSE-2.0
+ *  *
+ *  * Unless required by applicable law or agreed to in writing, software
+ *  * distributed under the License is distributed on an "AS IS" BASIS,
+ *  * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ *  * See the License for the specific language governing permissions and
+ *  * limitations under the License.
+ *
+ */
+
+package org.apache.rocketmq.mqtt.cs.protocol.mqtt.handler;
+
+
+import io.netty.channel.Channel;
+import io.netty.channel.ChannelHandlerContext;
+import io.netty.handler.codec.mqtt.*;
+import org.apache.rocketmq.mqtt.common.hook.HookResult;
+import org.apache.rocketmq.mqtt.common.model.Subscription;
+import org.apache.rocketmq.mqtt.common.util.TopicUtils;
+import org.apache.rocketmq.mqtt.cs.channel.ChannelCloseFrom;
+import org.apache.rocketmq.mqtt.cs.channel.ChannelInfo;
+import org.apache.rocketmq.mqtt.cs.channel.ChannelManager;
+import org.apache.rocketmq.mqtt.cs.protocol.mqtt.MqttPacketHandler;
+import org.apache.rocketmq.mqtt.cs.session.loop.SessionLoop;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+import org.springframework.stereotype.Component;
+
+import javax.annotation.Resource;
+import java.util.HashSet;
+import java.util.Set;
+
+import static io.netty.handler.codec.mqtt.MqttMessageType.UNSUBACK;
+import static io.netty.handler.codec.mqtt.MqttQoS.AT_MOST_ONCE;
+
+
+@Component
+public class MqttUnSubscribeHandler implements MqttPacketHandler<MqttUnsubscribeMessage> {
+    private static Logger logger = LoggerFactory.getLogger(MqttUnSubscribeHandler.class);
+
+
+    @Resource
+    private SessionLoop sessionLoop;
+
+    @Resource
+    private ChannelManager channelManager;
+
+    @Override
+    public void doHandler(ChannelHandlerContext ctx, MqttUnsubscribeMessage mqttMessage, HookResult upstreamHookResult) {
+        long start = System.currentTimeMillis();
+        String clientId = ChannelInfo.getClientId(ctx.channel());
+        Channel channel = ctx.channel();
+        String remark = upstreamHookResult.getRemark();
+        if (!upstreamHookResult.isSuccess()) {
+            channelManager.closeConnect(channel, ChannelCloseFrom.SERVER, remark);
+            return;
+        }
+        try {
+            MqttUnsubscribePayload payload = mqttMessage.payload();
+            if (payload.topics() != null && !payload.topics().isEmpty()) {
+                Set<Subscription> subscriptions = new HashSet<>();
+                for (String topic : payload.topics()) {
+                    subscriptions.add(new Subscription(TopicUtils.normalizeTopic(topic)));
+                }
+                sessionLoop.removeSubscription(ChannelInfo.getId(ctx.channel()), subscriptions);
+            }
+            channel.writeAndFlush(getResponse(mqttMessage));
+        } catch (Exception e) {
+            logger.error("UnSubscribe:{}", clientId, e);
+            channelManager.closeConnect(channel, ChannelCloseFrom.SERVER, "UnSubscribeException");
+        }
+    }
+
+    private MqttUnsubAckMessage getResponse(MqttUnsubscribeMessage mqttUnsubscribeMessage) {
+        MqttFixedHeader fixedHeader = new MqttFixedHeader(UNSUBACK, false, AT_MOST_ONCE, false, 0);
+        MqttMessageIdVariableHeader variableHeader = MqttMessageIdVariableHeader
+                .from(mqttUnsubscribeMessage.variableHeader().messageId());
+        MqttUnsubAckMessage mqttUnsubAckMessage = new MqttUnsubAckMessage(fixedHeader, variableHeader);
+        return mqttUnsubAckMessage;
+    }
+
+}
diff --git a/mqtt-cs/src/main/java/org/apache/rocketmq/mqtt/cs/protocol/rpc/RpcPacketDispatcher.java b/mqtt-cs/src/main/java/org/apache/rocketmq/mqtt/cs/protocol/rpc/RpcPacketDispatcher.java
new file mode 100644
index 0000000..bc0c316
--- /dev/null
+++ b/mqtt-cs/src/main/java/org/apache/rocketmq/mqtt/cs/protocol/rpc/RpcPacketDispatcher.java
@@ -0,0 +1,86 @@
+/*
+ *
+ *  * Licensed to the Apache Software Foundation (ASF) under one or more
+ *  * contributor license agreements.  See the NOTICE file distributed with
+ *  * this work for additional information regarding copyright ownership.
+ *  * The ASF licenses this file to You under the Apache License, Version 2.0
+ *  * (the "License"); you may not use this file except in compliance with
+ *  * the License.  You may obtain a copy of the License at
+ *  *
+ *  *     http://www.apache.org/licenses/LICENSE-2.0
+ *  *
+ *  * Unless required by applicable law or agreed to in writing, software
+ *  * distributed under the License is distributed on an "AS IS" BASIS,
+ *  * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ *  * See the License for the specific language governing permissions and
+ *  * limitations under the License.
+ *
+ */
+
+package org.apache.rocketmq.mqtt.cs.protocol.rpc;
+
+
+import com.alibaba.fastjson.JSONObject;
+import io.netty.channel.ChannelHandlerContext;
+import org.apache.rocketmq.mqtt.common.model.MessageEvent;
+import org.apache.rocketmq.mqtt.common.model.RpcCode;
+import org.apache.rocketmq.mqtt.common.model.RpcHeader;
+import org.apache.rocketmq.mqtt.cs.channel.ChannelManager;
+import org.apache.rocketmq.mqtt.cs.session.notify.MessageNotifyAction;
+import org.apache.rocketmq.remoting.netty.NettyRequestProcessor;
+import org.apache.rocketmq.remoting.protocol.RemotingCommand;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+import org.springframework.stereotype.Component;
+
+import javax.annotation.Resource;
+import java.nio.charset.StandardCharsets;
+import java.util.List;
+
+
+@Component
+public class RpcPacketDispatcher implements NettyRequestProcessor {
+    private static Logger logger = LoggerFactory.getLogger(RpcPacketDispatcher.class);
+
+    @Resource
+    private MessageNotifyAction messageNotifyAction;
+
+    @Resource
+    private ChannelManager channelManager;
+
+    @Override
+    public RemotingCommand processRequest(ChannelHandlerContext ctx, RemotingCommand request) throws Exception {
+        RemotingCommand response = RemotingCommand.createResponseCommand(RpcCode.SUCCESS, null);
+        response.setOpaque(request.getOpaque());
+        int code = request.getCode();
+        try {
+            if (RpcCode.CMD_NOTIFY_MQTT_MESSAGE == code) {
+                doNotify(request);
+            } else if (RpcCode.CMD_CLOSE_CHANNEL == code) {
+                closeChannel(request);
+            }
+        } catch (Throwable t) {
+            logger.error("", t);
+            response.setCode(RpcCode.FAIL);
+        }
+        return response;
+    }
+
+    @Override
+    public boolean rejectRequest() {
+        return false;
+    }
+
+    private void doNotify(RemotingCommand request) {
+        String payload = new String(request.getBody(), StandardCharsets.UTF_8);
+        List<MessageEvent> events = JSONObject.parseArray(payload, MessageEvent.class);
+        messageNotifyAction.notify(events);
+    }
+
+    private void closeChannel(RemotingCommand request) {
+        String channelId = request.getExtFields() != null ?
+                request.getExtFields().get(RpcHeader.MQTT_CHANNEL_ID) : null;
+        channelManager.closeConnect(channelId, request.getRemark());
+    }
+
+}
diff --git a/mqtt-cs/src/main/java/org/apache/rocketmq/mqtt/cs/protocol/ws/WebSocketServerHandler.java b/mqtt-cs/src/main/java/org/apache/rocketmq/mqtt/cs/protocol/ws/WebSocketServerHandler.java
new file mode 100644
index 0000000..12e8a1a
--- /dev/null
+++ b/mqtt-cs/src/main/java/org/apache/rocketmq/mqtt/cs/protocol/ws/WebSocketServerHandler.java
@@ -0,0 +1,109 @@
+/*
+ *
+ *  * Licensed to the Apache Software Foundation (ASF) under one or more
+ *  * contributor license agreements.  See the NOTICE file distributed with
+ *  * this work for additional information regarding copyright ownership.
+ *  * The ASF licenses this file to You under the Apache License, Version 2.0
+ *  * (the "License"); you may not use this file except in compliance with
+ *  * the License.  You may obtain a copy of the License at
+ *  *
+ *  *     http://www.apache.org/licenses/LICENSE-2.0
+ *  *
+ *  * Unless required by applicable law or agreed to in writing, software
+ *  * distributed under the License is distributed on an "AS IS" BASIS,
+ *  * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ *  * See the License for the specific language governing permissions and
+ *  * limitations under the License.
+ *
+ */
+package org.apache.rocketmq.mqtt.cs.protocol.ws;
+
+import io.netty.channel.*;
+import io.netty.handler.codec.http.DefaultFullHttpResponse;
+import io.netty.handler.codec.http.FullHttpRequest;
+import io.netty.handler.codec.http.FullHttpResponse;
+import io.netty.handler.codec.http.websocketx.*;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+import org.springframework.stereotype.Component;
+
+import static io.netty.handler.codec.http.HttpResponseStatus.BAD_REQUEST;
+import static io.netty.handler.codec.http.HttpUtil.isKeepAlive;
+import static io.netty.handler.codec.http.HttpVersion.HTTP_1_1;
+
+
+@ChannelHandler.Sharable
+@Component
+public class WebSocketServerHandler extends SimpleChannelInboundHandler<Object> {
+    private static Logger sysLogger = LoggerFactory.getLogger(WebSocketServerHandler.class);
+
+    private WebSocketServerHandshaker handshaker;
+
+
+    @Override
+    public void channelReadComplete(ChannelHandlerContext ctx) throws Exception {
+        ctx.flush();
+    }
+
+    private void handleHttpRequest(ChannelHandlerContext ctx, FullHttpRequest req) throws Exception {
+        if (!req.decoderResult().isSuccess()) {
+            sendHttpResponse(ctx, req, new DefaultFullHttpResponse(HTTP_1_1, BAD_REQUEST));
+            return;
+        }
+        String upgrade = req.headers().get("Upgrade");
+        if (upgrade == null || (!"websocket".equals(upgrade.toLowerCase()))) {
+            return;
+        }
+        WebSocketServerHandshakerFactory wsFactory = new WebSocketServerHandshakerFactory("ws://localhost:8888/mqtt",
+                "*",
+                false);
+        handshaker = wsFactory.newHandshaker(req);
+        if (handshaker == null) {
+            WebSocketServerHandshakerFactory.sendUnsupportedVersionResponse(ctx.channel());
+        } else {
+            handshaker.handshake(ctx.channel(), req);
+        }
+    }
+
+    private void handleWebSocketFrame(ChannelHandlerContext ctx, WebSocketFrame frame) {
+        if (frame instanceof CloseWebSocketFrame) {
+            handshaker.close(ctx.channel(), (CloseWebSocketFrame) frame.retain());
+            return;
+        }
+        if (frame instanceof PingWebSocketFrame) {
+            ctx.channel().write(new PongWebSocketFrame(frame.content().retain()));
+            return;
+        }
+        if (frame instanceof PongWebSocketFrame) {
+            return;
+        }
+        if (frame instanceof BinaryWebSocketFrame) {
+            throw new UnsupportedOperationException(
+                    String.format("%s frame types not supported", frame.getClass().getName()));
+        }
+        String request = ((TextWebSocketFrame) frame).text();
+        ctx.channel().write(
+                new TextWebSocketFrame(request + " , welcome netty websocket:" + new java.util.Date().toString()));
+    }
+
+    public static void sendHttpResponse(ChannelHandlerContext ctx, FullHttpRequest req, FullHttpResponse res) {
+        ChannelFuture f = ctx.channel().writeAndFlush(res);
+        if (!isKeepAlive(req) || res.status().code() != 200) {
+            f.addListener(ChannelFutureListener.CLOSE);
+        }
+    }
+
+    @Override
+    protected void channelRead0(ChannelHandlerContext ctx, Object msg) throws Exception {
+        if (msg instanceof FullHttpRequest) {
+            handleHttpRequest(ctx, (FullHttpRequest) msg);
+        } else if (msg instanceof WebSocketFrame) {
+            if (msg instanceof BinaryWebSocketFrame) {
+                ((WebSocketFrame) msg).retain();
+                ctx.fireChannelRead(((WebSocketFrame) msg).content());
+            } else {
+                handleWebSocketFrame(ctx, (WebSocketFrame) msg);
+            }
+        }
+    }
+}
diff --git a/mqtt-cs/src/main/java/org/apache/rocketmq/mqtt/cs/protocol/ws/WebsocketEncoder.java b/mqtt-cs/src/main/java/org/apache/rocketmq/mqtt/cs/protocol/ws/WebsocketEncoder.java
new file mode 100644
index 0000000..dab6306
--- /dev/null
+++ b/mqtt-cs/src/main/java/org/apache/rocketmq/mqtt/cs/protocol/ws/WebsocketEncoder.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.mqtt.cs.protocol.ws;
+
+import io.netty.buffer.ByteBuf;
+import io.netty.channel.ChannelHandlerContext;
+import io.netty.handler.codec.MessageToMessageEncoder;
+import io.netty.handler.codec.http.websocketx.BinaryWebSocketFrame;
+
+import java.util.List;
+
+public class WebsocketEncoder extends MessageToMessageEncoder<ByteBuf>{
+
+	@Override
+	protected void encode(ChannelHandlerContext ctx, ByteBuf msg, List<Object> out) throws Exception {
+		msg.retain();
+		BinaryWebSocketFrame binaryWebSocketFrame=new BinaryWebSocketFrame(msg);
+		out.add(binaryWebSocketFrame);
+	}
+
+}
diff --git a/mqtt-cs/src/main/java/org/apache/rocketmq/mqtt/cs/session/QueueFresh.java b/mqtt-cs/src/main/java/org/apache/rocketmq/mqtt/cs/session/QueueFresh.java
new file mode 100644
index 0000000..194dd4e
--- /dev/null
+++ b/mqtt-cs/src/main/java/org/apache/rocketmq/mqtt/cs/session/QueueFresh.java
@@ -0,0 +1,65 @@
+/*
+ *
+ *  * Licensed to the Apache Software Foundation (ASF) under one or more
+ *  * contributor license agreements.  See the NOTICE file distributed with
+ *  * this work for additional information regarding copyright ownership.
+ *  * The ASF licenses this file to You under the Apache License, Version 2.0
+ *  * (the "License"); you may not use this file except in compliance with
+ *  * the License.  You may obtain a copy of the License at
+ *  *
+ *  *     http://www.apache.org/licenses/LICENSE-2.0
+ *  *
+ *  * Unless required by applicable law or agreed to in writing, software
+ *  * distributed under the License is distributed on an "AS IS" BASIS,
+ *  * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ *  * See the License for the specific language governing permissions and
+ *  * limitations under the License.
+ *
+ */
+
+package org.apache.rocketmq.mqtt.cs.session;
+
+import org.apache.commons.lang3.StringUtils;
+import org.apache.rocketmq.mqtt.common.facade.LmqQueueStore;
+import org.apache.rocketmq.mqtt.common.model.Queue;
+import org.apache.rocketmq.mqtt.common.model.Subscription;
+import org.springframework.stereotype.Component;
+
+import javax.annotation.Resource;
+import java.util.HashSet;
+import java.util.Set;
+
+@Component
+public class QueueFresh {
+
+    @Resource
+    private LmqQueueStore lmqQueueStore;
+
+    public Set<Queue> freshQueue(Session session, Subscription subscription) {
+        Set<Queue> queues = new HashSet<>();
+        Set<String> brokers;
+        if (subscription.isP2p()) {
+            String findTopic = lmqQueueStore.getClientP2pTopic();
+            if(StringUtils.isBlank(findTopic)){
+                findTopic = lmqQueueStore.getClientRetryTopic();
+            }
+            brokers = lmqQueueStore.getReadableBrokers(findTopic);
+        } else if (subscription.isRetry()) {
+            brokers = lmqQueueStore.getReadableBrokers(lmqQueueStore.getClientRetryTopic());
+        } else {
+            brokers = lmqQueueStore.getReadableBrokers(subscription.toFirstTopic());
+        }
+        if (brokers == null || brokers.isEmpty()) {
+            return queues;
+        }
+        for (String broker : brokers) {
+            Queue moreQueue = new Queue();
+            moreQueue.setQueueName(subscription.toQueueName());
+            moreQueue.setBrokerName(broker);
+            queues.add(moreQueue);
+        }
+        session.freshQueue(subscription, queues);
+        return queues;
+    }
+
+}
diff --git a/mqtt-cs/src/main/java/org/apache/rocketmq/mqtt/cs/session/Session.java b/mqtt-cs/src/main/java/org/apache/rocketmq/mqtt/cs/session/Session.java
new file mode 100644
index 0000000..f3538b6
--- /dev/null
+++ b/mqtt-cs/src/main/java/org/apache/rocketmq/mqtt/cs/session/Session.java
@@ -0,0 +1,469 @@
+/*
+ *
+ *  * Licensed to the Apache Software Foundation (ASF) under one or more
+ *  * contributor license agreements.  See the NOTICE file distributed with
+ *  * this work for additional information regarding copyright ownership.
+ *  * The ASF licenses this file to You under the Apache License, Version 2.0
+ *  * (the "License"); you may not use this file except in compliance with
+ *  * the License.  You may obtain a copy of the License at
+ *  *
+ *  *     http://www.apache.org/licenses/LICENSE-2.0
+ *  *
+ *  * Unless required by applicable law or agreed to in writing, software
+ *  * distributed under the License is distributed on an "AS IS" BASIS,
+ *  * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ *  * See the License for the specific language governing permissions and
+ *  * limitations under the License.
+ *
+ */
+
+package org.apache.rocketmq.mqtt.cs.session;
+
+
+import io.netty.channel.Channel;
+import org.apache.rocketmq.mqtt.common.model.Message;
+import org.apache.rocketmq.mqtt.common.model.Queue;
+import org.apache.rocketmq.mqtt.common.model.QueueOffset;
+import org.apache.rocketmq.mqtt.common.model.Subscription;
+import org.apache.rocketmq.mqtt.cs.channel.ChannelInfo;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+import org.springframework.util.CollectionUtils;
+
+import java.util.*;
+import java.util.concurrent.ConcurrentHashMap;
+import java.util.concurrent.ConcurrentMap;
+import java.util.concurrent.atomic.AtomicBoolean;
+
+public class Session {
+    private static Logger logger = LoggerFactory.getLogger(Session.class);
+    private final long startTime = System.currentTimeMillis();
+    private Channel channel;
+    private volatile boolean destroyed = false;
+    private volatile int loadStatus = -1;
+    private volatile int pullSize;
+    private String clientId;
+    private String channelId;
+    private AtomicBoolean needPersistOffset = new AtomicBoolean(false);
+    private ConcurrentMap<String, Map<Queue, QueueOffset>> offsetMap = new ConcurrentHashMap<>(16);
+    private Map<String, Subscription> subscriptions = new ConcurrentHashMap<>();
+    private ConcurrentMap<Subscription, Map<Queue, LinkedHashSet<Message>>> sendingMessages = new ConcurrentHashMap<>(16);
+    private ConcurrentMap<Subscription, Integer> loadStatusMap = new ConcurrentHashMap<>();
+
+    public Session() {
+    }
+
+    public ConcurrentMap<Subscription, Integer> getLoadStatusMap() {
+        return loadStatusMap;
+    }
+
+    public long getStartTime() {
+        return startTime;
+    }
+
+    public Channel getChannel() {
+        return channel;
+    }
+
+    public void setChannel(Channel channel) {
+        this.channel = channel;
+    }
+
+    public boolean isClean() {
+        return Boolean.TRUE.equals(ChannelInfo.getCleanSessionFlag(channel));
+    }
+
+    public boolean isLoaded() {
+        return this.loadStatus == 1;
+    }
+
+    public void setLoaded() {
+        this.loadStatus = 1;
+    }
+
+    public void setLoading() {
+        this.loadStatus = 0;
+    }
+
+    public boolean isLoading() {
+        return loadStatus == 0;
+    }
+
+    public void resetLoad() {
+        this.loadStatus = -1;
+    }
+
+    public String getClientId() {
+        return clientId;
+    }
+
+    public void setClientId(String clientId) {
+        this.clientId = clientId;
+    }
+
+    public String getChannelId() {
+        return channelId;
+    }
+
+    public void setChannelId(String channelId) {
+        this.channelId = channelId;
+    }
+
+    public boolean isDestroyed() {
+        return destroyed;
+    }
+
+    public int getPullSize() {
+        return pullSize;
+    }
+
+    public void setPullSize(int pullSize) {
+        this.pullSize = pullSize;
+    }
+
+    public void destroy() {
+        this.destroyed = true;
+        this.offsetMap.clear();
+        this.sendingMessages.clear();
+        this.subscriptions.clear();
+    }
+
+    public Map<Subscription, Map<Queue, QueueOffset>> offsetMapSnapshot() {
+        Map<Subscription, Map<Queue, QueueOffset>> tmp = new HashMap<>(8);
+        for (String queueName : offsetMap.keySet()) {
+            Subscription subscription = subscriptions.get(queueName);
+            if (subscription == null) {
+                continue;
+            }
+            Map<Queue, QueueOffset> queueMap = new HashMap<>(8);
+            tmp.put(subscription, queueMap);
+            for (Map.Entry<Queue, QueueOffset> entry : offsetMap.get(queueName).entrySet()) {
+                queueMap.put(entry.getKey(), entry.getValue());
+            }
+        }
+        return tmp;
+    }
+
+    public Set<Subscription> subscriptionSnapshot() {
+        Set<Subscription> tmp = new HashSet<>();
+        tmp.addAll(subscriptions.values());
+        return tmp;
+    }
+
+    public void removeSubscription(Subscription subscription) {
+        if (subscription == null) {
+            throw new RuntimeException("subscription is null");
+        }
+        offsetMap.remove(subscription.toQueueName());
+        sendingMessages.remove(subscription);
+        subscriptions.remove(subscription.getTopicFilter());
+    }
+
+    public void freshQueue(Subscription subscription, Set<Queue> queues) {
+        if (subscription == null) {
+            throw new RuntimeException("subscription is null");
+        }
+        if (queues == null) {
+            logger.warn("queues is null when freshQueue,{},{}", getClientId(), subscription);
+            return;
+        }
+        if (!subscriptions.containsKey(subscription.getTopicFilter())) {
+            return;
+        }
+
+        String queueName = subscription.toQueueName();
+        if (!offsetMap.containsKey(queueName)) {
+            offsetMap.putIfAbsent(queueName, new ConcurrentHashMap<>(16));
+        }
+        for (Queue memQueue : offsetMap.get(queueName).keySet()) {
+            if (!queues.contains(memQueue)) {
+                offsetMap.get(queueName).remove(memQueue);
+            }
+        }
+        // init queueOffset
+        for (Queue nowQueue : queues) {
+            if (!offsetMap.get(queueName).containsKey(nowQueue)) {
+                QueueOffset queueOffset = new QueueOffset();
+                //if no offset  use init offset
+                offsetMap.get(queueName).put(nowQueue, queueOffset);
+                this.markPersistOffsetFlag(true);
+            }
+        }
+
+        if (!sendingMessages.containsKey(subscription)) {
+            sendingMessages.putIfAbsent(subscription, new ConcurrentHashMap<>(16));
+        }
+        for (Queue memQueue : sendingMessages.get(subscription).keySet()) {
+            if (!queues.contains(memQueue)) {
+                sendingMessages.get(subscription).remove(memQueue);
+            }
+        }
+        if (queues.isEmpty()) {
+            logger.warn("queues is empty when freshQueue,{},{}", getClientId(), subscription);
+        }
+    }
+
+    public void addOffset(String queueName, Map<Queue, QueueOffset> map) {
+        if (queueName == null) {
+            throw new RuntimeException("queueName is null");
+        }
+
+        if (!offsetMap.containsKey(queueName)) {
+            offsetMap.putIfAbsent(queueName, new ConcurrentHashMap<>(16));
+        }
+        offsetMap.get(queueName).putAll(map);
+    }
+
+    public void addOffset(Map<String, Map<Queue, QueueOffset>> offsetMapParam) {
+        if (offsetMapParam != null && !offsetMapParam.isEmpty()) {
+            for (String queueName : offsetMapParam.keySet()) {
+                if (!subscriptions.containsKey(queueName)) {
+                    continue;
+                }
+                addOffset(queueName, offsetMapParam.get(queueName));
+            }
+        }
+    }
+
+    public void addSubscription(Set<Subscription> subscriptionsParam) {
+        if (CollectionUtils.isEmpty(subscriptionsParam)) {
+            return;
+        }
+        for (Subscription subscription : subscriptionsParam) {
+            addSubscription(subscription);
+        }
+    }
+
+    public void addSubscription(Subscription subscriptionParam) {
+        if (subscriptionParam != null) {
+            subscriptions.put(subscriptionParam.getTopicFilter(), subscriptionParam);
+        }
+    }
+
+    public QueueOffset getQueueOffset(Subscription subscription, Queue queue) {
+        if (subscription == null) {
+            throw new RuntimeException("subscription is null");
+        }
+        if (queue == null) {
+            throw new RuntimeException("queue is null");
+        }
+        String queueName = subscription.toQueueName();
+        Map<Queue, QueueOffset> queueQueueOffsetMap = offsetMap.get(queueName);
+        if (queueQueueOffsetMap != null) {
+            return queueQueueOffsetMap.get(queue);
+        }
+        return null;
+    }
+
+    public Map<Queue, QueueOffset> getQueueOffset(Subscription subscription) {
+        if (subscription == null) {
+            throw new RuntimeException("subscription is null");
+        }
+        String queueName = subscription.toQueueName();
+        return offsetMap.get(queueName);
+    }
+
+    public boolean addSendingMessages(Subscription subscription, Queue queue, List<Message> messages) {
+        if (subscription == null) {
+            throw new RuntimeException("subscription is null");
+        }
+        if (queue == null) {
+            throw new RuntimeException("queue is null");
+        }
+        if (messages == null || messages.isEmpty()) {
+            return false;
+        }
+        if (!subscriptions.containsKey(subscription.getTopicFilter())) {
+            return false;
+        }
+        if (!sendingMessages.containsKey(subscription)) {
+            sendingMessages.putIfAbsent(subscription, new ConcurrentHashMap<>(16));
+        }
+        if (!sendingMessages.get(subscription).containsKey(queue)) {
+            sendingMessages.get(subscription).putIfAbsent(queue, new LinkedHashSet<>(8));
+        }
+        String queueName = subscription.toQueueName();
+        Map<Queue, QueueOffset> queueOffsetMap = offsetMap.get(queueName);
+        if (queueOffsetMap == null || !queueOffsetMap.containsKey(queue)) {
+            logger.warn("not found queueOffset,{},{},{}", getClientId(), subscription, queue);
+            return false;
+        }
+        boolean add = false;
+        QueueOffset queueOffset;
+        queueOffset = queueOffsetMap.get(queue);
+        Iterator<Message> iterator = messages.iterator();
+        while (iterator.hasNext()) {
+            Message message = iterator.next();
+            if (message.getOffset() < queueOffset.getOffset() && queueOffset.getOffset() != Long.MAX_VALUE) {
+                continue;
+            }
+            synchronized (this) {
+                if (sendingMessages.get(subscription).get(queue).add(message.copy())) {
+                    add = true;
+                }
+            }
+        }
+        return add;
+    }
+
+    public Message rollNext(Subscription subscription, Queue pendingQueue, long pendingDownSeqId) {
+        if (subscription == null) {
+            throw new RuntimeException("subscription is null");
+        }
+        if (pendingQueue == null) {
+            throw new RuntimeException("queue is null");
+        }
+        Map<Queue, LinkedHashSet<Message>> queueSendingMsgs = sendingMessages.get(subscription);
+        if (queueSendingMsgs == null || queueSendingMsgs.isEmpty()) {
+            return null;
+        }
+        LinkedHashSet<Message> messages = queueSendingMsgs.get(pendingQueue);
+        if (messages == null) {
+            return null;
+        }
+        Message message;
+        Message nextMessage = null;
+        synchronized (this) {
+            if (messages.isEmpty()) {
+                return null;
+            }
+            message = messages.iterator().next();
+            if (message.getOffset() != pendingDownSeqId) {
+                return null;
+            }
+            messages.remove(message);
+            updateQueueOffset(subscription, pendingQueue, message);
+            this.markPersistOffsetFlag(true);
+            if (!messages.isEmpty()) {
+                nextMessage = messages.iterator().next();
+            }
+        }
+        return nextMessage;
+    }
+
+    public boolean sendingMessageIsEmpty(Subscription subscription, Queue queue) {
+        if (subscription == null) {
+            throw new RuntimeException("subscription is null");
+        }
+        if (queue == null) {
+            throw new RuntimeException("queue is null");
+        }
+        Map<Queue, LinkedHashSet<Message>> queueSendingMsgs = sendingMessages.get(subscription);
+        if (queueSendingMsgs == null || queueSendingMsgs.isEmpty()) {
+            return true;
+        }
+        LinkedHashSet<Message> messages = queueSendingMsgs.get(queue);
+        if (messages == null) {
+            return true;
+        }
+        synchronized (this) {
+            return messages.size() <= 0;
+        }
+    }
+
+    public Message nextSendMessageByOrder(Subscription subscription, Queue queue) {
+        if (subscription == null) {
+            throw new RuntimeException("subscription is null");
+        }
+        if (queue == null) {
+            throw new RuntimeException("queue is null");
+        }
+        Map<Queue, LinkedHashSet<Message>> tmp = sendingMessages.get(subscription);
+        if (tmp != null && !tmp.isEmpty()) {
+            LinkedHashSet<Message> messages = tmp.get(queue);
+            if (messages == null) {
+                return null;
+            }
+            synchronized (this) {
+                return messages.isEmpty() ? null : messages.iterator().next();
+            }
+        }
+        return null;
+    }
+
+    public List<Message> pendMessageList(Subscription subscription, Queue queue) {
+        if (subscription == null) {
+            throw new RuntimeException("subscription is null");
+        }
+        if (queue == null) {
+            throw new RuntimeException("queue is null");
+        }
+        List<Message> list = new ArrayList<>();
+        Map<Queue, LinkedHashSet<Message>> tmp = sendingMessages.get(subscription);
+        if (tmp != null && !tmp.isEmpty()) {
+            LinkedHashSet<Message> messages = tmp.get(queue);
+            if (messages == null) {
+                return null;
+            }
+            synchronized (this) {
+                if (!messages.isEmpty()) {
+                    for (Message message : messages) {
+                        if (message.getAck() == -1) {
+                            list.add(message);
+                        }
+                    }
+                }
+            }
+        }
+        return list;
+    }
+
+    public void ack(Subscription subscription, Queue pendingQueue, long pendingDownSeqId) {
+        if (subscription == null) {
+            throw new RuntimeException("subscription is null");
+        }
+        if (pendingQueue == null) {
+            throw new RuntimeException("queue is null");
+        }
+        Map<Queue, LinkedHashSet<Message>> queueSendingMsgs = sendingMessages.get(subscription);
+        if (queueSendingMsgs == null || queueSendingMsgs.isEmpty()) {
+            return;
+        }
+        LinkedHashSet<Message> messages = queueSendingMsgs.get(pendingQueue);
+        if (messages == null) {
+            return;
+        }
+        synchronized (this) {
+            if (messages.isEmpty()) {
+                return;
+            }
+            boolean flag = true;
+            Iterator<Message> iterator = messages.iterator();
+            while (iterator.hasNext()) {
+                Message message = iterator.next();
+                if (message.getOffset() == pendingDownSeqId) {
+                    message.setAck(1);
+                }
+                if (message.getAck() != 1) {
+                    flag = false;
+                }
+                if (flag) {
+                    updateQueueOffset(subscription, pendingQueue, message);
+                    this.markPersistOffsetFlag(true);
+                    iterator.remove();
+                }
+            }
+        }
+    }
+
+    private void updateQueueOffset(Subscription subscription, Queue queue, Message message) {
+        String queueName = subscription.toQueueName();
+        Map<Queue, QueueOffset> queueOffsetMap = offsetMap.get(queueName);
+        if (queueOffsetMap == null || !queueOffsetMap.containsKey(queue)) {
+            logger.warn("failed update queue offset,not found queueOffset,{},{},{}", getClientId(), subscription,
+                    queue);
+            return;
+        }
+        QueueOffset queueOffset = queueOffsetMap.get(queue);
+        queueOffset.setOffset(message.getOffset() + 1);
+    }
+
+    public boolean markPersistOffsetFlag(boolean flag) {
+        return this.needPersistOffset.compareAndSet(!flag, flag);
+    }
+
+    public boolean getPersistOffsetFlag() {
+        return needPersistOffset.get();
+    }
+}
+
diff --git a/mqtt-cs/src/main/java/org/apache/rocketmq/mqtt/cs/session/infly/InFlyCache.java b/mqtt-cs/src/main/java/org/apache/rocketmq/mqtt/cs/session/infly/InFlyCache.java
new file mode 100644
index 0000000..93e21fb
--- /dev/null
+++ b/mqtt-cs/src/main/java/org/apache/rocketmq/mqtt/cs/session/infly/InFlyCache.java
@@ -0,0 +1,191 @@
+/*
+ *
+ *  * Licensed to the Apache Software Foundation (ASF) under one or more
+ *  * contributor license agreements.  See the NOTICE file distributed with
+ *  * this work for additional information regarding copyright ownership.
+ *  * The ASF licenses this file to You under the Apache License, Version 2.0
+ *  * (the "License"); you may not use this file except in compliance with
+ *  * the License.  You may obtain a copy of the License at
+ *  *
+ *  *     http://www.apache.org/licenses/LICENSE-2.0
+ *  *
+ *  * Unless required by applicable law or agreed to in writing, software
+ *  * distributed under the License is distributed on an "AS IS" BASIS,
+ *  * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ *  * See the License for the specific language governing permissions and
+ *  * limitations under the License.
+ *
+ */
+
+package org.apache.rocketmq.mqtt.cs.session.infly;
+
+
+import org.apache.commons.lang3.StringUtils;
+import org.apache.rocketmq.mqtt.common.model.Message;
+import org.apache.rocketmq.mqtt.common.model.Queue;
+import org.apache.rocketmq.mqtt.common.model.Subscription;
+import org.springframework.stereotype.Component;
+
+import javax.annotation.Resource;
+import java.util.HashSet;
+import java.util.Map;
+import java.util.Set;
+import java.util.concurrent.ConcurrentHashMap;
+import java.util.concurrent.ConcurrentMap;
+
+
+@Component
+public class InFlyCache {
+
+    @Resource
+    private MqttMsgId mqttMsgId;
+
+    private ConcurrentMap<String, Set<Integer>> pubCache = new ConcurrentHashMap<>(128);
+    private PendingDownCache pendingDownCache = new PendingDownCache();
+
+    public void cleanResource(String clientId,String channelId) {
+        pubCache.remove(channelId);
+        pendingDownCache.clear(clientId, channelId);
+    }
+
+    public enum CacheType {
+        PUB,
+    }
+
+    private ConcurrentMap<String, Set<Integer>> whichCache(CacheType cacheType) {
+        switch (cacheType) {
+            case PUB:
+                return pubCache;
+            default:
+                throw new RuntimeException("invalid cache type");
+        }
+    }
+
+    public void put(CacheType cacheType, String channelId, int mqttMsgId) {
+        ConcurrentMap<String, Set<Integer>> cache = whichCache(cacheType);
+        if (!cache.containsKey(channelId)) {
+            cache.putIfAbsent(channelId, new HashSet<>());
+        }
+        Set<Integer> idCache = cache.get(channelId);
+        if (idCache == null) {
+            return;
+        }
+        synchronized (idCache) {
+            cache.get(channelId).add(mqttMsgId);
+        }
+    }
+
+    public boolean contains(CacheType cacheType, String channelId, int mqttMsgId) {
+        ConcurrentMap<String, Set<Integer>> cache = whichCache(cacheType);
+        Set<Integer> idCache = cache.get(channelId);
+        if (idCache == null) {
+            return false;
+        }
+        synchronized (idCache) {
+            return idCache.contains(mqttMsgId);
+        }
+    }
+
+    public void remove(CacheType cacheType, String channelId, int mqttMsgId) {
+        ConcurrentMap<String, Set<Integer>> cache = whichCache(cacheType);
+        Set<Integer> idCache = cache.get(channelId);
+        if (idCache == null) {
+            return;
+        }
+        synchronized (idCache) {
+            idCache.remove(mqttMsgId);
+            if (idCache.isEmpty()) {
+                cache.remove(channelId);
+            }
+        }
+    }
+
+    public PendingDownCache getPendingDownCache() {
+        return pendingDownCache;
+    }
+
+    public class PendingDownCache {
+        private ConcurrentMap<String, Map<Integer, PendingDown>> cache = new ConcurrentHashMap<>(128);
+
+        public PendingDown put(String channelId, int mqttMsgId, Subscription subscription, Queue queue,
+                               Message message) {
+            PendingDown pendingDown = new PendingDown();
+            pendingDown.setSubscription(subscription);
+            pendingDown.setQueue(queue);
+            pendingDown.setSeqId(message.getOffset());
+            if (!cache.containsKey(channelId)) {
+                cache.putIfAbsent(channelId, new ConcurrentHashMap<>(16));
+            }
+            cache.get(channelId).put(mqttMsgId, pendingDown);
+            return pendingDown;
+        }
+
+        public Map<Integer, PendingDown> all(String channelId) {
+            if (StringUtils.isBlank(channelId)) {
+                return null;
+            }
+            return cache.get(channelId);
+        }
+
+        public PendingDown remove(String channelId, int mqttMsgId) {
+            Map<Integer, PendingDown> map = cache.get(channelId);
+            if (map != null) {
+                return map.remove(mqttMsgId);
+
+            }
+            return null;
+        }
+
+        public PendingDown get(String channelId, int mqttMsgId) {
+            Map<Integer, PendingDown> map = cache.get(channelId);
+            if (map != null) {
+                return map.get(mqttMsgId);
+
+            }
+            return null;
+        }
+
+        public void clear(String clientId, String channelId) {
+            Map<Integer, PendingDown> pendingDownMap = cache.remove(channelId);
+            if (clientId != null && pendingDownMap != null) {
+                pendingDownMap.forEach((mqttId, pendingDown) -> mqttMsgId.releaseId(mqttId, clientId));
+            }
+        }
+    }
+
+    public class PendingDown {
+        private Subscription subscription;
+        private Queue queue;
+        private long seqId;
+        private long t = System.currentTimeMillis();
+
+        public Subscription getSubscription() {
+            return subscription;
+        }
+
+        public void setSubscription(Subscription subscription) {
+            this.subscription = subscription;
+        }
+
+        public Queue getQueue() {
+            return queue;
+        }
+
+        public void setQueue(Queue queue) {
+            this.queue = queue;
+        }
+
+        public long getSeqId() {
+            return seqId;
+        }
+
+        public void setSeqId(long seqId) {
+            this.seqId = seqId;
+        }
+
+        public long getT() {
+            return t;
+        }
+    }
+
+}
diff --git a/mqtt-cs/src/main/java/org/apache/rocketmq/mqtt/cs/session/infly/MqttMsgId.java b/mqtt-cs/src/main/java/org/apache/rocketmq/mqtt/cs/session/infly/MqttMsgId.java
new file mode 100644
index 0000000..e4838b1
--- /dev/null
+++ b/mqtt-cs/src/main/java/org/apache/rocketmq/mqtt/cs/session/infly/MqttMsgId.java
@@ -0,0 +1,93 @@
+/*
+ *
+ *  * Licensed to the Apache Software Foundation (ASF) under one or more
+ *  * contributor license agreements.  See the NOTICE file distributed with
+ *  * this work for additional information regarding copyright ownership.
+ *  * The ASF licenses this file to You under the Apache License, Version 2.0
+ *  * (the "License"); you may not use this file except in compliance with
+ *  * the License.  You may obtain a copy of the License at
+ *  *
+ *  *     http://www.apache.org/licenses/LICENSE-2.0
+ *  *
+ *  * Unless required by applicable law or agreed to in writing, software
+ *  * distributed under the License is distributed on an "AS IS" BASIS,
+ *  * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ *  * See the License for the specific language governing permissions and
+ *  * limitations under the License.
+ *
+ */
+
+package org.apache.rocketmq.mqtt.cs.session.infly;
+
+import org.apache.commons.lang3.StringUtils;
+import org.springframework.stereotype.Component;
+
+import javax.annotation.PostConstruct;
+import java.util.ArrayList;
+import java.util.List;
+import java.util.Map;
+import java.util.concurrent.ConcurrentHashMap;
+
+@Component
+public class MqttMsgId {
+
+    private static final int MAX_MSG_ID = 65535;
+    private static final int MIN_MSG_ID = 1;
+
+    private static final int ID_POOL_SIZE = 8192;
+    private static final List<MsgIdEntry> ID_POOL = new ArrayList<>(ID_POOL_SIZE);
+
+    @PostConstruct
+    public void init() {
+        for (int i = 0; i < ID_POOL_SIZE; i++) {
+            ID_POOL.add(new MsgIdEntry());
+        }
+    }
+
+    class MsgIdEntry {
+        private int nextMsgId = MIN_MSG_ID - 1;
+        private Map<Integer, Integer> inUseMsgIds = new ConcurrentHashMap<>();
+    }
+
+    private MsgIdEntry hashMsgID(String clientId) {
+        int hashCode = clientId.hashCode();
+        if (hashCode < 0) {
+            hashCode *= -1;
+        }
+        return ID_POOL.get(hashCode % ID_POOL_SIZE);
+    }
+ 
+    public int nextId(String clientId) {
+        MsgIdEntry msgIdEntry = hashMsgID(clientId);
+        synchronized (msgIdEntry) {
+            int startingMessageId = msgIdEntry.nextMsgId;
+            int loopCount = 0;
+            int maxLoopCount = 2;
+            do {
+                msgIdEntry.nextMsgId++;
+                if (msgIdEntry.nextMsgId > MAX_MSG_ID) {
+                    msgIdEntry.nextMsgId = MIN_MSG_ID;
+                }
+                if (msgIdEntry.nextMsgId == startingMessageId) {
+                    loopCount++;
+                    if (loopCount >= maxLoopCount) {
+                        msgIdEntry.nextMsgId++;
+                        break;
+                    }
+                }
+            } while (msgIdEntry.inUseMsgIds.containsKey(new Integer(msgIdEntry.nextMsgId)));
+            Integer id = new Integer(msgIdEntry.nextMsgId);
+            msgIdEntry.inUseMsgIds.put(id, id);
+            return msgIdEntry.nextMsgId;
+        }
+    }
+
+    public void releaseId(int msgId, String clientId) {
+        if (StringUtils.isBlank(clientId)) {
+            return;
+        }
+        MsgIdEntry msgIdEntry = hashMsgID(clientId);
+        msgIdEntry.inUseMsgIds.remove(msgId);
+    }
+
+}
diff --git a/mqtt-cs/src/main/java/org/apache/rocketmq/mqtt/cs/session/infly/PushAction.java b/mqtt-cs/src/main/java/org/apache/rocketmq/mqtt/cs/session/infly/PushAction.java
new file mode 100644
index 0000000..fb3776a
--- /dev/null
+++ b/mqtt-cs/src/main/java/org/apache/rocketmq/mqtt/cs/session/infly/PushAction.java
@@ -0,0 +1,187 @@
+/*
+ *
+ *  * Licensed to the Apache Software Foundation (ASF) under one or more
+ *  * contributor license agreements.  See the NOTICE file distributed with
+ *  * this work for additional information regarding copyright ownership.
+ *  * The ASF licenses this file to You under the Apache License, Version 2.0
+ *  * (the "License"); you may not use this file except in compliance with
+ *  * the License.  You may obtain a copy of the License at
+ *  *
+ *  *     http://www.apache.org/licenses/LICENSE-2.0
+ *  *
+ *  * Unless required by applicable law or agreed to in writing, software
+ *  * distributed under the License is distributed on an "AS IS" BASIS,
+ *  * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ *  * See the License for the specific language governing permissions and
+ *  * limitations under the License.
+ *
+ */
+
+package org.apache.rocketmq.mqtt.cs.session.infly;
+
+import io.netty.channel.Channel;
+import io.netty.channel.ChannelFuture;
+import io.netty.channel.ChannelFutureListener;
+import org.apache.commons.lang3.StringUtils;
+import org.apache.rocketmq.mqtt.common.model.Message;
+import org.apache.rocketmq.mqtt.common.model.Queue;
+import org.apache.rocketmq.mqtt.common.model.Subscription;
+import org.apache.rocketmq.mqtt.common.util.MessageUtil;
+import org.apache.rocketmq.mqtt.common.util.TopicUtils;
+import org.apache.rocketmq.mqtt.cs.channel.ChannelInfo;
+import org.apache.rocketmq.mqtt.cs.config.ConnectConf;
+import org.apache.rocketmq.mqtt.cs.session.Session;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+import org.springframework.stereotype.Component;
+
+import javax.annotation.Resource;
+import java.util.List;
+
+
+@Component
+public class PushAction {
+    private static Logger logger = LoggerFactory.getLogger(PushAction.class);
+
+    @Resource
+    private MqttMsgId mqttMsgId;
+
+    @Resource
+    private RetryDriver retryDriver;
+
+    @Resource
+    private InFlyCache inFlyCache;
+
+    @Resource
+    private ConnectConf connectConf;
+
+
+    public void messageArrive(Session session, Subscription subscription, Queue queue) {
+        if (session == null) {
+            return;
+        }
+        if (!connectConf.isOrder()) {
+            List<Message> list = session.pendMessageList(subscription, queue);
+            if (list != null && !list.isEmpty()) {
+                for (Message message : list) {
+                    message.setAck(0);
+                    push(message, subscription, session, queue);
+                }
+            }
+            return;
+        }
+        if (retryDriver.needRetryBefore(subscription, queue, session)) {
+            return;
+        }
+        Message message = session.nextSendMessageByOrder(subscription, queue);
+        if (message != null) {
+            push(message, subscription, session, queue);
+        }
+    }
+
+    public void push(Message message, Subscription subscription, Session session, Queue queue) {
+        String clientId = session.getClientId();
+        int mqttId = mqttMsgId.nextId(clientId);
+        inFlyCache.getPendingDownCache().put(session.getChannelId(), mqttId, subscription, queue, message);
+        try {
+            if (session.isClean()) {
+                if (message.getStoreTimestamp() > 0 &&
+                        message.getStoreTimestamp() < session.getStartTime()) {
+                    logger.warn("old msg:{},{},{},{}", session.getClientId(), message.getMsgId(),
+                            message.getStoreTimestamp(), session.getStartTime());
+                    rollNext(session, mqttId);
+                    return;
+                }
+            }
+        } catch (Exception e) {
+            logger.error("", e);
+        }
+        int qos = subscription.getQos();
+        if (subscription.isP2p() && message.qos() != null) {
+            qos = message.qos();
+        }
+        if (qos == 0) {
+            write(session, message, mqttId, 0, subscription);
+            rollNextByAck(session, mqttId);
+        } else {
+            retryDriver.mountPublish(mqttId, message, subscription.getQos(), ChannelInfo.getId(session.getChannel()), subscription);
+            write(session, message, mqttId, qos, subscription);
+        }
+    }
+
+    public void write(Session session, Message message, int mqttId, int qos, Subscription subscription) {
+        Channel channel = session.getChannel();
+        String owner = ChannelInfo.getOwner(channel);
+        String clientId = session.getClientId();
+        String topicName = message.getOriginTopic();
+        String mqttRealTopic = message.getUserProperty(Message.extPropertyMqttRealTopic);
+        if (StringUtils.isNotBlank(mqttRealTopic)) {
+            topicName = mqttRealTopic;
+        }
+        if (StringUtils.isBlank(topicName)) {
+            topicName = message.getFirstTopic();
+        }
+        boolean isP2P = TopicUtils.isP2P(TopicUtils.decode(topicName).getSecondTopic());
+        if (!channel.isWritable()) {
+            logger.error("UnWritable:{}", clientId);
+            return;
+        }
+        Object data = MessageUtil.toMqttMessage(topicName, message.getPayload(), qos, mqttId);
+        ChannelFuture writeFuture = session.getChannel().writeAndFlush(data);
+        int bodySize = message.getPayload() != null ? message.getPayload().length : 0;
+        writeFuture.addListener((ChannelFutureListener) future -> {
+            if (subscription.isRetry()) {
+                message.setRetry(message.getRetry() + 1);
+                logger.warn("retryPush:{},{},{}", session.getClientId(), message.getMsgId(), message.getRetry());
+            }
+        });
+    }
+
+    public void rollNextByAck(Session session, int mqttId) {
+        InFlyCache.PendingDown pendingDown = inFlyCache.getPendingDownCache().get(session.getChannelId(), mqttId);
+        if (pendingDown == null) {
+            return;
+        }
+        rollNext(session, mqttId);
+    }
+
+    public void rollNext(Session session, int mqttId) {
+        if (session == null || session.isDestroyed()) {
+            return;
+        }
+        mqttMsgId.releaseId(mqttId, session.getClientId());
+        InFlyCache.PendingDown pendingDown = inFlyCache.getPendingDownCache().remove(session.getChannelId(), mqttId);
+        if (pendingDown == null) {
+            return;
+        }
+        _rollNext(session, pendingDown);
+    }
+
+    public void rollNextNoWaitRetry(Session session, int mqttId) {
+        if (session == null || session.isDestroyed()) {
+            return;
+        }
+        InFlyCache.PendingDown pendingDown = inFlyCache.getPendingDownCache().get(session.getChannelId(), mqttId);
+        if (pendingDown == null) {
+            return;
+        }
+        _rollNext(session, pendingDown);
+    }
+
+    public void _rollNext(Session session, InFlyCache.PendingDown pendingDown) {
+        Subscription subscription = pendingDown.getSubscription();
+        Queue pendingQueue = pendingDown.getQueue();
+        long pendingDownSeqId = pendingDown.getSeqId();
+
+        if (!connectConf.isOrder()) {
+            session.ack(subscription, pendingQueue, pendingDownSeqId);
+            return;
+        }
+
+        Message nextSendOne = session.rollNext(subscription, pendingQueue, pendingDownSeqId);
+        if (nextSendOne != null) {
+            push(nextSendOne, subscription, session, pendingQueue);
+        }
+    }
+
+}
diff --git a/mqtt-cs/src/main/java/org/apache/rocketmq/mqtt/cs/session/infly/RetryDriver.java b/mqtt-cs/src/main/java/org/apache/rocketmq/mqtt/cs/session/infly/RetryDriver.java
new file mode 100644
index 0000000..d4c3ae6
--- /dev/null
+++ b/mqtt-cs/src/main/java/org/apache/rocketmq/mqtt/cs/session/infly/RetryDriver.java
@@ -0,0 +1,327 @@
+/*
+ *
+ *  * Licensed to the Apache Software Foundation (ASF) under one or more
+ *  * contributor license agreements.  See the NOTICE file distributed with
+ *  * this work for additional information regarding copyright ownership.
+ *  * The ASF licenses this file to You under the Apache License, Version 2.0
+ *  * (the "License"); you may not use this file except in compliance with
+ *  * the License.  You may obtain a copy of the License at
+ *  *
+ *  *     http://www.apache.org/licenses/LICENSE-2.0
+ *  *
+ *  * Unless required by applicable law or agreed to in writing, software
+ *  * distributed under the License is distributed on an "AS IS" BASIS,
+ *  * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ *  * See the License for the specific language governing permissions and
+ *  * limitations under the License.
+ *
+ */
+
+package org.apache.rocketmq.mqtt.cs.session.infly;
+
+import com.github.benmanes.caffeine.cache.Cache;
+import com.github.benmanes.caffeine.cache.Caffeine;
+import com.github.benmanes.caffeine.cache.RemovalListener;
+import io.netty.handler.codec.mqtt.*;
+import org.apache.rocketmq.common.ThreadFactoryImpl;
+import org.apache.rocketmq.mqtt.common.facade.LmqQueueStore;
+import org.apache.rocketmq.mqtt.common.model.Message;
+import org.apache.rocketmq.mqtt.common.model.Queue;
+import org.apache.rocketmq.mqtt.common.model.StoreResult;
+import org.apache.rocketmq.mqtt.common.model.Subscription;
+import org.apache.rocketmq.mqtt.cs.channel.ChannelManager;
+import org.apache.rocketmq.mqtt.cs.config.ConnectConf;
+import org.apache.rocketmq.mqtt.cs.session.QueueFresh;
+import org.apache.rocketmq.mqtt.cs.session.Session;
+import org.apache.rocketmq.mqtt.cs.session.loop.SessionLoop;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+import org.springframework.stereotype.Component;
+
+import javax.annotation.PostConstruct;
+import javax.annotation.Resource;
+import java.util.*;
+import java.util.concurrent.CompletableFuture;
+import java.util.concurrent.ConcurrentHashMap;
+import java.util.concurrent.ScheduledThreadPoolExecutor;
+import java.util.concurrent.TimeUnit;
+
+
+@Component
+public class RetryDriver {
+    private static Logger logger = LoggerFactory.getLogger(RetryDriver.class);
+
+    @Resource
+    private InFlyCache inFlyCache;
+
+    @Resource
+    private MqttMsgId mqttMsgId;
+
+    @Resource
+    private SessionLoop sessionLoop;
+
+    @Resource
+    private PushAction pushAction;
+
+    @Resource
+    private ChannelManager channelManager;
+
+    @Resource
+    private ConnectConf connectConf;
+
+    @Resource
+    private LmqQueueStore lmqQueueStore;
+
+    @Resource
+    private QueueFresh queueFresh;
+
+    private Cache<String, RetryMessage> retryCache;
+    private final int MAX_CACHE = 50000;
+    private Map<String, Map<Integer, RetryMessage>> sessionNoWaitRetryMsgMap = new ConcurrentHashMap<>(16);
+    private ScheduledThreadPoolExecutor scheduler = new ScheduledThreadPoolExecutor(2,
+            new ThreadFactoryImpl("retry_msg_thread_"));
+
+    @PostConstruct
+    public void init() {
+        retryCache = Caffeine.newBuilder().maximumSize(MAX_CACHE).removalListener((RemovalListener<String, RetryMessage>) (key, value, cause) -> {
+            if (value == null || key == null) {
+                return;
+            }
+            if (cause.wasEvicted()) {
+                saveRetryQueue(key, value);
+            }
+        }).build();
+
+        scheduler.scheduleWithFixedDelay(() -> doRetryCache(), 3, connectConf.getRetryIntervalSeconds(), TimeUnit.SECONDS);
+
+        Runtime.getRuntime().addShutdownHook(new Thread(() -> {
+            Map<String, RetryMessage> map = retryCache.asMap();
+            if (map == null) {
+                return;
+            }
+            for (Map.Entry<String, RetryMessage> entry : map.entrySet()) {
+                saveRetryQueue(entry.getKey(), entry.getValue());
+            }
+        }));
+    }
+
+    public void unloadSession(Session session) {
+        if (session == null) {
+            return;
+        }
+        Map<Integer, RetryMessage> map = sessionNoWaitRetryMsgMap.remove(session.getChannelId());
+        if (map == null || map.isEmpty()) {
+            return;
+        }
+        for (Map.Entry<Integer, RetryMessage> entry : map.entrySet()) {
+            String cacheKey = buildKey(entry.getKey(), session.getChannelId());
+            retryCache.invalidate(cacheKey);
+            RetryMessage retryMessage = entry.getValue();
+            saveRetryQueue(cacheKey, retryMessage);
+        }
+    }
+
+    private void saveRetryQueue(String key, RetryMessage retryMessage) {
+        Message message = retryMessage.message.copy();
+        message.setFirstTopic(lmqQueueStore.getClientRetryTopic());
+        Session session = retryMessage.session;
+        int mqttMsgId = retryMessage.mqttMsgId;
+        String clientId = session.getClientId();
+        if (message.getRetry() >= connectConf.getMaxRetryTime()) {
+            pushAction.rollNext(session, retryMessage.mqttMsgId);
+            return;
+        }
+        String retryQueue = Subscription.newRetrySubscription(clientId).toQueueName();
+        CompletableFuture<StoreResult> result = lmqQueueStore.putMessage(new HashSet<>(Arrays.asList(retryQueue)), message);
+        result.whenComplete((storeResult, throwable) -> {
+            if (throwable != null) {
+                retryCache.put(key, retryMessage);
+                return;
+            }
+            long queueId = storeResult.getQueue().getQueueId();
+            String brokerName = storeResult.getQueue().getBrokerName();
+            pushAction.rollNext(session, mqttMsgId);
+            scheduler.schedule(() -> {
+                Subscription subscription = Subscription.newRetrySubscription(clientId);
+                List<Session> sessionList = sessionLoop.getSessionList(clientId);
+                if (sessionList != null) {
+                    for (Session eachSession : sessionList) {
+                        Set<Queue> set = queueFresh.freshQueue(eachSession, subscription);
+                        if (set == null || set.isEmpty()) {
+                            continue;
+                        }
+                        for (Queue queue : set) {
+                            if (Objects.equals(queue.getBrokerName(), brokerName)) {
+                                sessionLoop.notifyPullMessage(eachSession, subscription, queue);
+                            }
+                        }
+                    }
+                }
+            }, 3, TimeUnit.SECONDS);
+        });
+    }
+
+    private void doRetryCache() {
+        try {
+            for (Map.Entry<String, RetryMessage> entry : retryCache.asMap().entrySet()) {
+                try {
+                    RetryMessage retryMessage = entry.getValue();
+                    Message message = retryMessage.message;
+                    Session session = retryMessage.session;
+                    int mqttMsgId = retryMessage.mqttMsgId;
+                    if (System.currentTimeMillis() - retryMessage.timestamp < 3000) {
+                        continue;
+                    }
+                    if (MqttMessageType.PUBLISH.equals(retryMessage.mqttMessageType)) {
+                        if (session == null || session.isDestroyed()) {
+                            cleanRetryMessage(mqttMsgId, session.getChannelId());
+                            continue;
+                        }
+                        if (retryMessage.mountTimeout()) {
+                            saveRetryQueue(entry.getKey(), retryMessage);
+                            cleanRetryMessage(mqttMsgId, session.getChannelId());
+                            continue;
+                        }
+                        pushAction.write(session, message, mqttMsgId, retryMessage.qos, retryMessage.subscription);
+                        retryMessage.timestamp = System.currentTimeMillis();
+                        retryMessage.localRetryTime++;
+                    } else if (MqttMessageType.PUBREL.equals(retryMessage.mqttMessageType)) {
+                        if (session == null || session.isDestroyed() || retryMessage.mountRelTimeout()) {
+                            retryCache.invalidate(entry.getKey());
+                            logger.error("failed to retry pub rel more times,{},{}", session.getClientId(), mqttMsgId);
+                            pushAction.rollNextByAck(session, mqttMsgId);
+                            continue;
+                        }
+                        MqttFixedHeader pubRelMqttFixedHeader = new MqttFixedHeader(MqttMessageType.PUBREL, false,
+                                MqttQoS.valueOf(retryMessage.qos), false, 0);
+                        MqttMessage pubRelMqttMessage = new MqttMessage(pubRelMqttFixedHeader,
+                                MqttMessageIdVariableHeader.from(mqttMsgId));
+                        session.getChannel().writeAndFlush(pubRelMqttMessage);
+                        retryMessage.localRetryTime++;
+                        retryMessage.timestamp = System.currentTimeMillis();
+                        logger.warn("retryPubRel:{},{}", session.getClientId(), mqttMsgId);
+                    } else {
+                        logger.error("error retry message type:{}", retryMessage.mqttMessageType);
+                    }
+                } catch (Exception e) {
+                    logger.error("", e);
+                }
+            }
+        } catch (Exception e) {
+            logger.error("", e);
+        }
+    }
+
+    public void mountPublish(int mqttMsgId, Message message, int qos, String channelId, Subscription subscription) {
+        Session session = sessionLoop.getSession(channelId);
+        if (session == null) {
+            return;
+        }
+
+        RetryMessage retryMessage = new RetryMessage(session, message, qos, mqttMsgId, MqttMessageType.PUBLISH, subscription);
+        retryCache.put(buildKey(mqttMsgId, channelId), retryMessage);
+        Map<Integer, RetryMessage> noWaitRetryMsgMap = sessionNoWaitRetryMsgMap.get(channelId);
+        if (noWaitRetryMsgMap == null) {
+            noWaitRetryMsgMap = new ConcurrentHashMap<>(2);
+            sessionNoWaitRetryMsgMap.putIfAbsent(channelId, noWaitRetryMsgMap);
+        }
+
+        if (!subscription.isRetry() &&
+                noWaitRetryMsgMap.size() < connectConf.getSizeOfNotRollWhenAckSlow()) {
+            noWaitRetryMsgMap.put(mqttMsgId, retryMessage);
+            pushAction.rollNextNoWaitRetry(session, mqttMsgId);
+        }
+    }
+
+    private RetryMessage cleanRetryMessage(int mqttMsgId, String channelId) {
+        Map<Integer, RetryMessage> retryMsgMap = sessionNoWaitRetryMsgMap.get(channelId);
+        if (retryMsgMap != null) {
+            retryMsgMap.remove(mqttMsgId);
+        }
+        String key = buildKey(mqttMsgId, channelId);
+        return unMount(key);
+    }
+
+    public void mountPubRel(int mqttMsgId, String channelId) {
+        Session session = sessionLoop.getSession(channelId);
+        if (session == null) {
+            return;
+        }
+        RetryMessage retryMessage = new RetryMessage(session, null, MqttQoS.AT_LEAST_ONCE.value(), mqttMsgId,
+                MqttMessageType.PUBREL, null);
+        retryCache.put(buildKey(mqttMsgId, channelId), retryMessage);
+    }
+
+    public RetryMessage unMountPublish(int mqttMsgId, String channelId) {
+        RetryMessage retryMessage = cleanRetryMessage(mqttMsgId, channelId);
+        return retryMessage;
+    }
+
+    public RetryMessage unMountPubRel(int mqttMsgId, String channelId) {
+        String key = buildKey(mqttMsgId, channelId);
+        return unMount(key);
+    }
+
+    private RetryMessage unMount(String key) {
+        RetryMessage message = retryCache.getIfPresent(key);
+        if (message != null) {
+            retryCache.invalidate(key);
+        }
+        return message;
+    }
+
+    public boolean needRetryBefore(Subscription subscription, Queue queue, Session session) {
+        Map<Integer, InFlyCache.PendingDown> pendingDowns = inFlyCache.getPendingDownCache().all(
+                session.getChannelId());
+        if (pendingDowns == null || pendingDowns.isEmpty()) {
+            return false;
+        }
+        InFlyCache.PendingDown pendingDown = null;
+        for (Map.Entry<Integer, InFlyCache.PendingDown> entry : pendingDowns.entrySet()) {
+            InFlyCache.PendingDown each = entry.getValue();
+            if (each.getSubscription().equals(subscription) && each.getQueue().equals(queue)) {
+                pendingDown = each;
+                break;
+            }
+        }
+        return pendingDown != null;
+    }
+
+    private String buildKey(int mqttMsgId, String channelId) {
+        StringBuilder sb = new StringBuilder();
+        sb.append(String.valueOf(mqttMsgId));
+        sb.append("_");
+        sb.append(channelId);
+        return sb.toString();
+    }
+
+    public class RetryMessage {
+        private Session session;
+        private Message message;
+        private Subscription subscription;
+        private int qos;
+        private int mqttMsgId;
+        private MqttMessageType mqttMessageType;
+        private int localRetryTime = 0;
+        private static final int MAX_LOCAL_RETRY = 1;
+        private long timestamp = System.currentTimeMillis();
+
+        public RetryMessage(Session session, Message message, int qos, int mqttMsgId, MqttMessageType mqttMessageType, Subscription subscription) {
+            this.session = session;
+            this.message = message;
+            this.qos = qos;
+            this.mqttMsgId = mqttMsgId;
+            this.mqttMessageType = mqttMessageType;
+            this.subscription = subscription;
+        }
+
+        private boolean mountTimeout() {
+            return localRetryTime >= MAX_LOCAL_RETRY;
+        }
+
+        private boolean mountRelTimeout() {
+            return localRetryTime >= 3;
+        }
+    }
+
+}
diff --git a/mqtt-cs/src/main/java/org/apache/rocketmq/mqtt/cs/session/loop/PullResultStatus.java b/mqtt-cs/src/main/java/org/apache/rocketmq/mqtt/cs/session/loop/PullResultStatus.java
new file mode 100644
index 0000000..b97f5cb
--- /dev/null
+++ b/mqtt-cs/src/main/java/org/apache/rocketmq/mqtt/cs/session/loop/PullResultStatus.java
@@ -0,0 +1,26 @@
+/*
+ *
+ *  * Licensed to the Apache Software Foundation (ASF) under one or more
+ *  * contributor license agreements.  See the NOTICE file distributed with
+ *  * this work for additional information regarding copyright ownership.
+ *  * The ASF licenses this file to You under the Apache License, Version 2.0
+ *  * (the "License"); you may not use this file except in compliance with
+ *  * the License.  You may obtain a copy of the License at
+ *  *
+ *  *     http://www.apache.org/licenses/LICENSE-2.0
+ *  *
+ *  * Unless required by applicable law or agreed to in writing, software
+ *  * distributed under the License is distributed on an "AS IS" BASIS,
+ *  * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ *  * See the License for the specific language governing permissions and
+ *  * limitations under the License.
+ *
+ */
+
+package org.apache.rocketmq.mqtt.cs.session.loop;
+
+
+public enum PullResultStatus {
+    DONE,
+    LATER
+}
diff --git a/mqtt-cs/src/main/java/org/apache/rocketmq/mqtt/cs/session/loop/QueueCache.java b/mqtt-cs/src/main/java/org/apache/rocketmq/mqtt/cs/session/loop/QueueCache.java
new file mode 100644
index 0000000..ebb3385
--- /dev/null
+++ b/mqtt-cs/src/main/java/org/apache/rocketmq/mqtt/cs/session/loop/QueueCache.java
@@ -0,0 +1,318 @@
+/*
+ *
+ *  * Licensed to the Apache Software Foundation (ASF) under one or more
+ *  * contributor license agreements.  See the NOTICE file distributed with
+ *  * this work for additional information regarding copyright ownership.
+ *  * The ASF licenses this file to You under the Apache License, Version 2.0
+ *  * (the "License"); you may not use this file except in compliance with
+ *  * the License.  You may obtain a copy of the License at
+ *  *
+ *  *     http://www.apache.org/licenses/LICENSE-2.0
+ *  *
+ *  * Unless required by applicable law or agreed to in writing, software
+ *  * distributed under the License is distributed on an "AS IS" BASIS,
+ *  * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ *  * See the License for the specific language governing permissions and
+ *  * limitations under the License.
+ *
+ */
+
+package org.apache.rocketmq.mqtt.cs.session.loop;
+
+
+import com.github.benmanes.caffeine.cache.Cache;
+import com.github.benmanes.caffeine.cache.Caffeine;
+import org.apache.commons.lang3.StringUtils;
+import org.apache.commons.lang3.tuple.Pair;
+import org.apache.rocketmq.common.ThreadFactoryImpl;
+import org.apache.rocketmq.mqtt.common.facade.LmqQueueStore;
+import org.apache.rocketmq.mqtt.common.model.Queue;
+import org.apache.rocketmq.mqtt.common.model.*;
+import org.apache.rocketmq.mqtt.common.util.StatUtil;
+import org.apache.rocketmq.mqtt.cs.config.ConnectConf;
+import org.apache.rocketmq.mqtt.cs.session.Session;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+import org.springframework.stereotype.Component;
+import org.springframework.util.CollectionUtils;
+
+import javax.annotation.PostConstruct;
+import javax.annotation.Resource;
+import java.util.*;
+import java.util.concurrent.CompletableFuture;
+import java.util.concurrent.ConcurrentHashMap;
+import java.util.concurrent.ScheduledThreadPoolExecutor;
+import java.util.concurrent.TimeUnit;
+import java.util.concurrent.atomic.AtomicBoolean;
+import java.util.concurrent.atomic.AtomicLong;
+
+import static org.apache.rocketmq.mqtt.cs.session.loop.PullResultStatus.DONE;
+import static org.apache.rocketmq.mqtt.cs.session.loop.PullResultStatus.LATER;
+
+
+@Component
+public class QueueCache {
+    private static Logger logger = LoggerFactory.getLogger(QueueCache.class);
+
+    @Resource
+    private ConnectConf connectConf;
+
+    @Resource
+    private LmqQueueStore lmqQueueStore;
+
+    private ScheduledThreadPoolExecutor loadCacheService = new ScheduledThreadPoolExecutor(1,
+            new ThreadFactoryImpl("loadCache_"));
+
+    private AtomicLong rid = new AtomicLong();
+    private Map<Queue, QueueEvent> loadEvent = new ConcurrentHashMap<>();
+    private Map<Queue, Boolean> loadStatus = new ConcurrentHashMap<>();
+
+    private Cache<Queue, CacheEntry> cache = Caffeine.newBuilder()
+            .expireAfterAccess(10, TimeUnit.MINUTES)
+            .maximumSize(1_000)
+            .build();
+
+
+    @PostConstruct
+    public void init() {
+        loadCacheService.scheduleWithFixedDelay(() -> {
+            for (Map.Entry<Queue, QueueEvent> entry : loadEvent.entrySet()) {
+                Queue queue = entry.getKey();
+                QueueEvent event = entry.getValue();
+                if (Boolean.TRUE.equals(loadStatus.get(queue))) {
+                    continue;
+                }
+                CacheEntry cacheEntry = cache.getIfPresent(queue);
+                if (cacheEntry == null) {
+                    cacheEntry = new CacheEntry();
+                    cache.put(queue, cacheEntry);
+                }
+                if (CollectionUtils.isEmpty(cacheEntry.messageList)) {
+                    loadCache(true, queue.toFirstTopic(), queue, null, 1, event);
+                    continue;
+                }
+                QueueOffset queueOffset = new QueueOffset();
+                Message lastMessage = cacheEntry.messageList.get(cacheEntry.messageList.size() - 1);
+                queueOffset.setOffset(lastMessage.getOffset() + 1);
+                loadCache(false, queue.toFirstTopic(), queue, queueOffset, connectConf.getPullBatchSize(), event);
+            }
+        }, 10, 10, TimeUnit.MILLISECONDS);
+    }
+
+    public void refreshCache(Pair<Queue, Session> pair) {
+        Queue queue = pair.getLeft();
+        if (queue == null) {
+            return;
+        }
+        if (queue.isP2p() || queue.isRetry()) {
+            return;
+        }
+        addLoadEvent(queue, pair.getRight());
+    }
+
+    class QueueEvent {
+        long id;
+        Session session;
+
+        public QueueEvent(long id, Session session) {
+            this.id = id;
+            this.session = session;
+        }
+
+        @Override
+        public boolean equals(Object o) {
+            if (this == o) {
+                return true;
+            }
+            if (o == null || getClass() != o.getClass()) {
+                return false;
+            }
+            QueueEvent that = (QueueEvent) o;
+            return id == that.id;
+        }
+
+        @Override
+        public int hashCode() {
+            return Objects.hash(id);
+        }
+    }
+
+    private void addLoadEvent(Queue queue, Session session) {
+        loadEvent.put(queue, new QueueEvent(rid.incrementAndGet(), session));
+        CacheEntry cacheEntry = cache.getIfPresent(queue);
+        if (cacheEntry == null) {
+            cacheEntry = new CacheEntry();
+            cache.put(queue, cacheEntry);
+        }
+    }
+
+    private void callbackResult(CompletableFuture<PullResult> pullResult, CompletableFuture<PullResult> callBackResult) {
+        pullResult.whenComplete((pullResult1, throwable) -> {
+            if (throwable != null) {
+                callBackResult.completeExceptionally(throwable);
+            } else {
+                callBackResult.complete(pullResult1);
+            }
+        });
+    }
+
+    private String toFirstTopic(Subscription subscription) {
+        String firstTopic = subscription.toFirstTopic();
+        if (subscription.isRetry()) {
+            firstTopic = lmqQueueStore.getClientRetryTopic();
+        }
+        if (subscription.isP2p()) {
+            if (StringUtils.isNotBlank(lmqQueueStore.getClientP2pTopic())) {
+                firstTopic = lmqQueueStore.getClientP2pTopic();
+            } else {
+                firstTopic = lmqQueueStore.getClientRetryTopic();
+            }
+        }
+        return firstTopic;
+    }
+
+    public PullResultStatus pullMessage(Session session, Subscription subscription, Queue queue,
+                                        QueueOffset queueOffset, int count,
+                                        CompletableFuture<PullResult> callBackResult) {
+        if (subscription.isP2p() || subscription.isRetry()) {
+            StatUtil.addPv("NotPullCache", 1);
+            CompletableFuture<PullResult> pullResult = lmqQueueStore.pullMessage(toFirstTopic(subscription), queue, queueOffset, count);
+            callbackResult(pullResult, callBackResult);
+            return DONE;
+        }
+        CacheEntry cacheEntry = cache.getIfPresent(queue);
+        if (cacheEntry == null) {
+            StatUtil.addPv("NoPullCache", 1);
+            CompletableFuture<PullResult> pullResult = lmqQueueStore.pullMessage(toFirstTopic(subscription), queue, queueOffset, count);
+            callbackResult(pullResult, callBackResult);
+            return DONE;
+        }
+        if (cacheEntry.loading.get()) {
+            if (System.currentTimeMillis() - cacheEntry.startLoadingT > 1000) {
+                StatUtil.addPv("LoadPullCacheTimeout", 1);
+                CompletableFuture<PullResult> pullResult = lmqQueueStore.pullMessage(toFirstTopic(subscription), queue, queueOffset, count);
+                callbackResult(pullResult, callBackResult);
+                return DONE;
+            }
+            return LATER;
+        }
+
+        List<Message> cacheMsgList = cacheEntry.messageList;
+        if (cacheMsgList.isEmpty()) {
+            if (loadEvent.get(queue) != null) {
+                StatUtil.addPv("EmptyPullCacheLATER", 1);
+                return LATER;
+            }
+            StatUtil.addPv("EmptyPullCache", 1);
+            CompletableFuture<PullResult> pullResult = lmqQueueStore.pullMessage(toFirstTopic(subscription), queue, queueOffset, count);
+            callbackResult(pullResult, callBackResult);
+            return DONE;
+        }
+
+        if (queueOffset.getOffset() < cacheMsgList.get(0).getOffset()) {
+            StatUtil.addPv("OutPullCache", 1);
+            CompletableFuture<PullResult> pullResult = lmqQueueStore.pullMessage(toFirstTopic(subscription), queue, queueOffset, count);
+            callbackResult(pullResult, callBackResult);
+            return DONE;
+        }
+        List<Message> resultMsgs = new ArrayList<>();
+        synchronized (cacheEntry) {
+            for (Message message : cacheMsgList) {
+                if (message.getOffset() >= queueOffset.getOffset()) {
+                    resultMsgs.add(message);
+                }
+                if (resultMsgs.size() >= count) {
+                    break;
+                }
+            }
+        }
+        if (resultMsgs.isEmpty()) {
+            if (loadEvent.get(queue) != null) {
+                StatUtil.addPv("PullCacheLATER", 1);
+                return LATER;
+            }
+            StatUtil.addPv("OutPullCache2", 1);
+            CompletableFuture<PullResult> pullResult = lmqQueueStore.pullMessage(toFirstTopic(subscription), queue, queueOffset, count);
+            callbackResult(pullResult, callBackResult);
+            return DONE;
+        }
+        PullResult pullResult = new PullResult();
+        pullResult.setMessageList(resultMsgs);
+        callBackResult.complete(pullResult);
+        StatUtil.addPv("PullFromCache", 1);
+        if (loadEvent.get(queue) != null) {
+            return LATER;
+        }
+        return DONE;
+    }
+
+    private void loadCache(boolean isFirst, String firstTopic, Queue queue, QueueOffset queueOffset, int count,
+                           QueueEvent event) {
+        loadStatus.put(queue, true);
+        CacheEntry cacheEntry = cache.getIfPresent(queue);
+        if (cacheEntry == null) {
+            cacheEntry = new CacheEntry();
+            cache.put(queue, cacheEntry);
+        }
+        cacheEntry.startLoad();
+        CacheEntry finalCacheEntry = cacheEntry;
+        CompletableFuture<PullResult> result;
+        if (isFirst) {
+            result = lmqQueueStore.pullLastMessages(firstTopic, queue, count);
+        } else {
+            result = lmqQueueStore.pullMessage(firstTopic, queue, queueOffset, count);
+        }
+        result.whenComplete((pullResult, throwable) -> {
+            if (throwable != null) {
+                logger.error("", throwable);
+                loadEvent.remove(queue, event);
+                loadStatus.remove(queue);
+                finalCacheEntry.endLoad();
+                addLoadEvent(queue, event.session);
+                return;
+            }
+            try {
+                if (pullResult != null && !CollectionUtils.isEmpty(pullResult.getMessageList())) {
+                    synchronized (finalCacheEntry) {
+                        finalCacheEntry.messageList.addAll(pullResult.getMessageList());
+                        if (isFirst) {
+                            Collections.sort(finalCacheEntry.messageList, Comparator.comparingLong(Message::getOffset));
+                        }
+                        int overNum = finalCacheEntry.messageList.size() - connectConf.getQueueCacheSize();
+                        for (int i = 0; i < overNum; i++) {
+                            finalCacheEntry.messageList.remove(0);
+                        }
+                    }
+                    if (pullResult.getMessageList().size() >= count && !isFirst) {
+                        addLoadEvent(queue, event.session);
+                        return;
+                    }
+                }
+            } catch (Exception e) {
+                logger.error("loadCache failed {}", queue.getQueueName(), e);
+                addLoadEvent(queue, event.session);
+            } finally {
+                loadEvent.remove(queue, event);
+                loadStatus.remove(queue);
+                finalCacheEntry.endLoad();
+            }
+        });
+    }
+
+    class CacheEntry {
+        private AtomicBoolean loading = new AtomicBoolean(false);
+        private List<Message> messageList = new ArrayList<>();
+        private volatile long startLoadingT = System.currentTimeMillis();
+
+        private void startLoad() {
+            if (loading.compareAndSet(false, true)) {
+                startLoadingT = System.currentTimeMillis();
+            }
+        }
+
+        private void endLoad() {
+            loading.compareAndSet(true, false);
+        }
+    }
+
+}
diff --git a/mqtt-cs/src/main/java/org/apache/rocketmq/mqtt/cs/session/loop/SessionLoop.java b/mqtt-cs/src/main/java/org/apache/rocketmq/mqtt/cs/session/loop/SessionLoop.java
new file mode 100644
index 0000000..ddcab25
--- /dev/null
+++ b/mqtt-cs/src/main/java/org/apache/rocketmq/mqtt/cs/session/loop/SessionLoop.java
@@ -0,0 +1,100 @@
+/*
+ *
+ *  * Licensed to the Apache Software Foundation (ASF) under one or more
+ *  * contributor license agreements.  See the NOTICE file distributed with
+ *  * this work for additional information regarding copyright ownership.
+ *  * The ASF licenses this file to You under the Apache License, Version 2.0
+ *  * (the "License"); you may not use this file except in compliance with
+ *  * the License.  You may obtain a copy of the License at
+ *  *
+ *  *     http://www.apache.org/licenses/LICENSE-2.0
+ *  *
+ *  * Unless required by applicable law or agreed to in writing, software
+ *  * distributed under the License is distributed on an "AS IS" BASIS,
+ *  * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ *  * See the License for the specific language governing permissions and
+ *  * limitations under the License.
+ *
+ */
+
+package org.apache.rocketmq.mqtt.cs.session.loop;
+
+
+import io.netty.channel.Channel;
+import org.apache.rocketmq.mqtt.common.model.Queue;
+import org.apache.rocketmq.mqtt.common.model.Subscription;
+import org.apache.rocketmq.mqtt.cs.channel.ChannelManager;
+import org.apache.rocketmq.mqtt.cs.session.Session;
+
+import java.util.List;
+import java.util.Set;
+
+
+public interface SessionLoop {
+
+    /**
+     * set ChannelManager
+     *
+     * @param channelManager
+     */
+    void setChannelManager(ChannelManager channelManager);
+
+    /**
+     * load one mqtt session
+     *
+     * @param clientId
+     * @param channel
+     */
+    void loadSession(String clientId, Channel channel);
+
+    /**
+     * unload one mqtt session
+     *
+     * @param clientId
+     * @param channelId
+     * @return
+     */
+    Session unloadSession(String clientId, String channelId);
+
+    /**
+     * get the session by channelId
+     *
+     * @param channelId
+     * @return
+     */
+    Session getSession(String channelId);
+
+    /**
+     * get session list by clientId
+     *
+     * @param clientId
+     * @return
+     */
+    List<Session> getSessionList(String clientId);
+
+    /**
+     * add subscription
+     *
+     * @param channelId
+     * @param subscriptions
+     */
+    void addSubscription(String channelId, Set<Subscription> subscriptions);
+
+    /**
+     * remove subscription
+     *
+     * @param channelId
+     * @param subscriptions
+     */
+    void removeSubscription(String channelId, Set<Subscription> subscriptions);
+
+    /**
+     * notify to pull message from queue
+     *
+     * @param session
+     * @param subscription
+     * @param queue
+     */
+    void notifyPullMessage(Session session, Subscription subscription, Queue queue);
+
+}
diff --git a/mqtt-cs/src/main/java/org/apache/rocketmq/mqtt/cs/session/loop/SessionLoopImpl.java b/mqtt-cs/src/main/java/org/apache/rocketmq/mqtt/cs/session/loop/SessionLoopImpl.java
new file mode 100644
index 0000000..00a938a
--- /dev/null
+++ b/mqtt-cs/src/main/java/org/apache/rocketmq/mqtt/cs/session/loop/SessionLoopImpl.java
@@ -0,0 +1,536 @@
+/*
+ *
+ *  * Licensed to the Apache Software Foundation (ASF) under one or more
+ *  * contributor license agreements.  See the NOTICE file distributed with
+ *  * this work for additional information regarding copyright ownership.
+ *  * The ASF licenses this file to You under the Apache License, Version 2.0
+ *  * (the "License"); you may not use this file except in compliance with
+ *  * the License.  You may obtain a copy of the License at
+ *  *
+ *  *     http://www.apache.org/licenses/LICENSE-2.0
+ *  *
+ *  * Unless required by applicable law or agreed to in writing, software
+ *  * distributed under the License is distributed on an "AS IS" BASIS,
+ *  * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ *  * See the License for the specific language governing permissions and
+ *  * limitations under the License.
+ *
+ */
+
+package org.apache.rocketmq.mqtt.cs.session.loop;
+
+import com.alibaba.fastjson.JSONObject;
+import io.netty.channel.Channel;
+import org.apache.commons.lang3.StringUtils;
+import org.apache.rocketmq.common.ThreadFactoryImpl;
+import org.apache.rocketmq.mqtt.common.facade.LmqOffsetStore;
+import org.apache.rocketmq.mqtt.common.facade.LmqQueueStore;
+import org.apache.rocketmq.mqtt.common.model.PullResult;
+import org.apache.rocketmq.mqtt.common.model.Queue;
+import org.apache.rocketmq.mqtt.common.model.QueueOffset;
+import org.apache.rocketmq.mqtt.common.model.Subscription;
+import org.apache.rocketmq.mqtt.cs.channel.ChannelInfo;
+import org.apache.rocketmq.mqtt.cs.channel.ChannelManager;
+import org.apache.rocketmq.mqtt.cs.config.ConnectConf;
+import org.apache.rocketmq.mqtt.cs.session.QueueFresh;
+import org.apache.rocketmq.mqtt.cs.session.Session;
+import org.apache.rocketmq.mqtt.cs.session.infly.InFlyCache;
+import org.apache.rocketmq.mqtt.cs.session.infly.PushAction;
+import org.apache.rocketmq.mqtt.cs.session.match.MatchAction;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+import org.springframework.stereotype.Component;
+
+import javax.annotation.PostConstruct;
+import javax.annotation.Resource;
+import java.util.*;
+import java.util.concurrent.CompletableFuture;
+import java.util.concurrent.ConcurrentHashMap;
+import java.util.concurrent.ScheduledThreadPoolExecutor;
+import java.util.concurrent.TimeUnit;
+import java.util.concurrent.atomic.AtomicInteger;
+import java.util.concurrent.atomic.AtomicLong;
+
+
+@Component
+public class SessionLoopImpl implements SessionLoop {
+    private static Logger logger = LoggerFactory.getLogger(SessionLoopImpl.class);
+
+    @Resource
+    private PushAction pushAction;
+
+    @Resource
+    private MatchAction matchAction;
+
+    @Resource
+    private ConnectConf connectConf;
+
+    @Resource
+    private InFlyCache inFlyCache;
+
+    @Resource
+    private QueueCache queueCache;
+
+    @Resource
+    private LmqQueueStore lmqQueueStore;
+
+    @Resource
+    private LmqOffsetStore lmqOffsetStore;
+
+    @Resource
+    private QueueFresh queueFresh;
+
+    private ChannelManager channelManager;
+    private ScheduledThreadPoolExecutor pullService;
+    private ScheduledThreadPoolExecutor scheduler;
+    private ScheduledThreadPoolExecutor persistOffsetScheduler;
+
+    /**
+     * channelId->session
+     */
+    private Map<String, Session> sessionMap = new ConcurrentHashMap<>(1024);
+    private Map<String, Map<String, Session>> clientMap = new ConcurrentHashMap<>(1024);
+    private Map<String, PullEvent> pullEventMap = new ConcurrentHashMap<>(1024);
+    private Map<String, Boolean> pullStatus = new ConcurrentHashMap<>(1024);
+
+    private AtomicLong rid = new AtomicLong();
+    private long pullIntervalMillis = 10;
+
+    @PostConstruct
+    public void init() {
+        pullService = new ScheduledThreadPoolExecutor(1, new ThreadFactoryImpl("pull_message_thread_"));
+        scheduler = new ScheduledThreadPoolExecutor(2, new ThreadFactoryImpl("loop_scheduler_"));
+        persistOffsetScheduler = new ScheduledThreadPoolExecutor(1, new ThreadFactoryImpl("persistOffset_scheduler_"));
+        persistOffsetScheduler.scheduleWithFixedDelay(() -> persistAllOffset(true), 5000, 5000, TimeUnit.MILLISECONDS);
+        pullService.scheduleWithFixedDelay(() -> pullLoop(), pullIntervalMillis, pullIntervalMillis, TimeUnit.MILLISECONDS);
+    }
+
+    private void pullLoop() {
+        try {
+            for (Map.Entry<String, PullEvent> entry : pullEventMap.entrySet()) {
+                PullEvent pullEvent = entry.getValue();
+                Session session = pullEvent.session;
+                if (!session.getChannel().isActive()) {
+                    pullStatus.remove(eventQueueKey(session, pullEvent.queue));
+                    pullEventMap.remove(entry.getKey());
+                    continue;
+                }
+                if (Boolean.TRUE.equals(pullStatus.get(eventQueueKey(session, pullEvent.queue)))) {
+                    continue;
+                }
+                if (!session.getChannel().isWritable()) {
+                    continue;
+                }
+                doPull(pullEvent);
+            }
+        } catch (Exception e) {
+            logger.error("", e);
+        }
+    }
+
+    @Override
+    public void setChannelManager(ChannelManager channelManager) {
+        this.channelManager = channelManager;
+    }
+
+    @Override
+    public void loadSession(String clientId, Channel channel) {
+        if (StringUtils.isBlank(clientId)) {
+            return;
+        }
+        if (!channel.isActive()) {
+            return;
+        }
+        String channelId = ChannelInfo.getId(channel);
+        if (sessionMap.containsKey(channelId)) {
+            return;
+        }
+        Session session = new Session();
+        session.setClientId(clientId);
+        session.setChannelId(channelId);
+        session.setChannel(channel);
+        addSubscriptionAndInit(session,
+                new HashSet<>(
+                        Arrays.asList(Subscription.newP2pSubscription(clientId), Subscription.newRetrySubscription(clientId))),
+                ChannelInfo.getFuture(channel, ChannelInfo.FUTURE_CONNECT));
+        synchronized (this) {
+            sessionMap.put(channelId, session);
+            if (!clientMap.containsKey(clientId)) {
+                clientMap.putIfAbsent(clientId, new ConcurrentHashMap<>(2));
+            }
+            clientMap.get(clientId).put(channelId, session);
+        }
+        if (!channel.isActive()) {
+            unloadSession(clientId, channelId);
+            return;
+        }
+        if (!session.isClean()) {
+            notifyPullMessage(session, null, null);
+        }
+    }
+
+    @Override
+    public Session unloadSession(String clientId, String channelId) {
+        Session session = null;
+        try {
+            synchronized (this) {
+                session = sessionMap.remove(channelId);
+                if (clientId == null) {
+                    if (session != null) {
+                        clientId = session.getClientId();
+                    }
+                }
+                if (clientId != null && clientMap.containsKey(clientId)) {
+                    clientMap.get(clientId).remove(channelId);
+                    if (clientMap.get(clientId).isEmpty()) {
+                        clientMap.remove(clientId);
+                    }
+                }
+            }
+            inFlyCache.cleanResource(clientId, channelId);
+            if (session != null) {
+                matchAction.removeSubscription(session, session.subscriptionSnapshot());
+                persistOffset(session);
+            }
+        } catch (Exception e) {
+            logger.error("unloadSession fail:{},{}", clientId, channelId, e);
+        } finally {
+            if (session != null) {
+                session.destroy();
+            }
+        }
+        return session;
+    }
+
+    @Override
+    public Session getSession(String channelId) {
+        return sessionMap.get(channelId);
+    }
+
+    @Override
+    public List<Session> getSessionList(String clientId) {
+        List<Session> list = new ArrayList<>();
+        Map<String, Session> sessions = clientMap.get(clientId);
+        if (sessions != null && !sessions.isEmpty()) {
+            for (Session session : sessions.values()) {
+                if (!session.isDestroyed()) {
+                    list.add(session);
+                } else {
+                    logger.error("the session was destroyed,{}", clientId);
+                    sessions.remove(session.getChannelId());
+                }
+            }
+        }
+        return list;
+    }
+
+    @Override
+    public void addSubscription(String channelId, Set<Subscription> subscriptions) {
+        if (subscriptions == null || subscriptions.isEmpty()) {
+            return;
+        }
+        Session session = getSession(channelId);
+        if (session == null) {
+            return;
+        }
+        addSubscriptionAndInit(session, subscriptions,
+                ChannelInfo.getFuture(session.getChannel(), ChannelInfo.FUTURE_SUBSCRIBE));
+        matchAction.addSubscription(session, subscriptions);
+
+        if (!session.isClean()) {
+            for (Subscription subscription : subscriptions) {
+                notifyPullMessage(session, subscription, null);
+            }
+        }
+    }
+
+    @Override
+    public void removeSubscription(String channelId, Set<Subscription> subscriptions) {
+        if (subscriptions == null || subscriptions.isEmpty()) {
+            return;
+        }
+        Session session = getSession(channelId);
+        if (session == null) {
+            return;
+        }
+        for (Subscription subscription : subscriptions) {
+            session.removeSubscription(subscription);
+        }
+        matchAction.removeSubscription(session, subscriptions);
+    }
+
+    private void addSubscriptionAndInit(Session session, Set<Subscription> subscriptions,
+                                        CompletableFuture<Void> future) {
+        if (session == null) {
+            return;
+        }
+        if (subscriptions == null) {
+            return;
+        }
+        session.addSubscription(subscriptions);
+        AtomicInteger result = new AtomicInteger(subscriptions.size());
+        for (Subscription subscription : subscriptions) {
+            queueFresh.freshQueue(session, subscription);
+            Map<Queue, QueueOffset> queueOffsets = session.getQueueOffset(subscription);
+            if (queueOffsets != null) {
+                for (Map.Entry<Queue, QueueOffset> entry : queueOffsets.entrySet()) {
+                    initOffset(session, subscription, entry.getKey(), entry.getValue(), future, result);
+                }
+            }
+        }
+    }
+
+    private void futureDone(CompletableFuture<Void> future, AtomicInteger result) {
+        if (future == null) {
+            return;
+        }
+        if (result == null) {
+            return;
+        }
+        if (result.decrementAndGet() <= 0) {
+            future.complete(null);
+        }
+    }
+
+    private void initOffset(Session session, Subscription subscription, Queue queue, QueueOffset queueOffset,
+                            CompletableFuture<Void> future, AtomicInteger result) {
+        if (queueOffset.isInitialized()) {
+            futureDone(future, result);
+            return;
+        }
+        if (queueOffset.isInitializing()) {
+            return;
+        }
+        queueOffset.setInitializing();
+        CompletableFuture<Long> queryResult = lmqQueueStore.queryQueueMaxOffset(queue);
+        queryResult.whenComplete((maxOffset, throwable) -> {
+            if (throwable != null) {
+                logger.error("queryQueueMaxId onException {}", queue.getQueueName(), throwable);
+                QueueOffset _queueOffset = session.getQueueOffset(subscription, queue);
+                if (_queueOffset != null) {
+                    if (!_queueOffset.isInitialized()) {
+                        _queueOffset.setOffset(Long.MAX_VALUE);
+                    }
+                    _queueOffset.setInitialized();
+                }
+                futureDone(future, result);
+                return;
+            }
+            QueueOffset _queueOffset = session.getQueueOffset(subscription, queue);
+            if (_queueOffset != null) {
+                if (!_queueOffset.isInitialized()) {
+                    _queueOffset.setOffset(maxOffset);
+                }
+                _queueOffset.setInitialized();
+            }
+            futureDone(future, result);
+        });
+    }
+
+    @Override
+    public void notifyPullMessage(Session session, Subscription subscription, Queue queue) {
+        if (session == null || session.isDestroyed()) {
+            return;
+        }
+        if (queue != null) {
+            if (subscription == null) {
+                throw new RuntimeException(
+                        "invalid notifyPullMessage, subscription is null, but queue is not null," + session.getClientId());
+            }
+            queueFresh.freshQueue(session, subscription);
+            pullMessage(session, subscription, queue);
+            return;
+        }
+        for (Subscription each : session.subscriptionSnapshot()) {
+            if (subscription != null && !each.equals(subscription)) {
+                continue;
+            }
+            queueFresh.freshQueue(session, each);
+            Map<Queue, QueueOffset> queueOffsets = session.getQueueOffset(each);
+            if (queueOffsets != null) {
+                for (Map.Entry<Queue, QueueOffset> entry : queueOffsets.entrySet()) {
+                    pullMessage(session, each, entry.getKey());
+                }
+            }
+        }
+    }
+
+    private String eventQueueKey(Session session, Queue queue) {
+        StringBuilder sb = new StringBuilder();
+        sb.append(ChannelInfo.getId(session.getChannel()));
+        sb.append("-");
+        sb.append(queue.getQueueId());
+        sb.append("-");
+        sb.append(queue.getQueueName());
+        sb.append("-");
+        sb.append(queue.getBrokerName());
+        return sb.toString();
+    }
+
+    private boolean needLoadPersistedOffset(Session session, Subscription subscription, Queue queue) {
+        if (session.isClean()) {
+            return false;
+        }
+        Integer status = session.getLoadStatusMap().get(subscription);
+        if (status != null && status == 1) {
+            return false;
+        }
+        if (status != null && status == 0) {
+            return true;
+        }
+        session.getLoadStatusMap().put(subscription, 0);
+        CompletableFuture<Map<Queue, QueueOffset>> result = lmqOffsetStore.getOffset(session.getClientId(), subscription);
+        result.whenComplete((offsetMap, throwable) -> {
+            if (throwable != null) {
+                // retry
+                scheduler.schedule(() -> {
+                    session.getLoadStatusMap().put(subscription, -1);
+                    pullMessage(session, subscription, queue);
+                }, 3, TimeUnit.SECONDS);
+                return;
+            }
+            session.addOffset(subscription.toQueueName(), offsetMap);
+            session.getLoadStatusMap().put(subscription, 1);
+            pullMessage(session, subscription, queue);
+        });
+        return true;
+    }
+
+    private void pullMessage(Session session, Subscription subscription, Queue queue) {
+        if (queue == null) {
+            return;
+        }
+        if (session == null || session.isDestroyed()) {
+            return;
+        }
+        if (needLoadPersistedOffset(session, subscription, queue)) {
+            return;
+        }
+        if (!session.sendingMessageIsEmpty(subscription, queue)) {
+            scheduler.schedule(() -> pullMessage(session, subscription, queue), pullIntervalMillis, TimeUnit.MILLISECONDS);
+        } else {
+            PullEvent pullEvent = new PullEvent(session, subscription, queue);
+            pullEventMap.put(eventQueueKey(session, queue), pullEvent);
+        }
+    }
+
+    private void doPull(PullEvent pullEvent) {
+        Session session = pullEvent.session;
+        Subscription subscription = pullEvent.subscription;
+        Queue queue = pullEvent.queue;
+        QueueOffset queueOffset = session.getQueueOffset(subscription, queue);
+        if (session.isDestroyed() || queueOffset == null) {
+            clearPullStatus(session, queue, pullEvent);
+            return;
+        }
+
+        if (!queueOffset.isInitialized()) {
+            initOffset(session, subscription, queue, queueOffset, null, null);
+            scheduler.schedule(() -> pullMessage(session, subscription, queue), pullIntervalMillis, TimeUnit.MILLISECONDS);
+            return;
+        }
+
+        pullStatus.put(eventQueueKey(session, queue), true);
+        int count = session.getPullSize() > 0 ? session.getPullSize() : connectConf.getPullBatchSize();
+        CompletableFuture<PullResult> result = new CompletableFuture<>();
+        queueCache.pullMessage(session, subscription, queue, queueOffset, count, result);
+        result.whenComplete((pullResult, throwable) -> {
+            if (throwable != null) {
+                clearPullStatus(session, queue, pullEvent);
+                logger.error("{}", session.getClientId(), throwable);
+                if (session.isDestroyed()) {
+                    return;
+                }
+                scheduler.schedule(() -> pullMessage(session, subscription, queue), 1, TimeUnit.SECONDS);
+                return;
+            }
+            try {
+                if (session.isDestroyed()) {
+                    return;
+                }
+                if (PullResult.PULL_SUCCESS == pullResult.getCode()) {
+                    if (pullResult.getMessageList() != null &&
+                            pullResult.getMessageList().size() >= count) {
+                        scheduler.schedule(() -> pullMessage(session, subscription, queue), pullIntervalMillis, TimeUnit.MILLISECONDS);
+                    }
+                    boolean add = session.addSendingMessages(subscription, queue, pullResult.getMessageList());
+                    if (add) {
+                        pushAction.messageArrive(session, subscription, queue);
+                    }
+                } else if (PullResult.PULL_OFFSET_MOVED == pullResult.getCode()) {
+                    queueOffset.setOffset(pullResult.getNextQueueOffset().getOffset());
+                    queueOffset.setOffset(pullResult.getNextQueueOffset().getOffset());
+                    session.markPersistOffsetFlag(true);
+                    pullMessage(session, subscription, queue);
+                } else {
+                    logger.error("response:{},{}", session.getClientId(), JSONObject.toJSONString(pullResult));
+                }
+            } finally {
+                clearPullStatus(session, queue, pullEvent);
+            }
+        });
+
+        PullResultStatus pullResultStatus = queueCache.pullMessage(session, subscription, queue, queueOffset, count, result);
+        if (PullResultStatus.LATER.equals(pullResultStatus)) {
+            clearPullStatus(session, queue, pullEvent);
+            scheduler.schedule(() -> pullMessage(session, subscription, queue), pullIntervalMillis, TimeUnit.MILLISECONDS);
+        }
+    }
+
+    private void clearPullStatus(Session session, Queue queue, PullEvent pullEvent) {
+        pullEventMap.remove(eventQueueKey(session, queue), pullEvent);
+        pullStatus.remove(eventQueueKey(session, queue));
+    }
+
+    private void persistAllOffset(boolean needSleep) {
+        try {
+            for (Session session : sessionMap.values()) {
+                if (persistOffset(session) && needSleep) {
+                    Thread.sleep(5L);
+                }
+            }
+        } catch (Exception e) {
+            logger.error("", e);
+        }
+    }
+
+    private boolean persistOffset(Session session) {
+        try {
+            if (!session.getPersistOffsetFlag()) {
+                return false;
+            }
+            lmqOffsetStore.save(session.getClientId(), session.offsetMapSnapshot());
+        } catch (Exception e) {
+            logger.error("{}", session.getClientId(), e);
+        }
+        return true;
+    }
+
+    class PullEvent {
+        private Session session;
+        private Subscription subscription;
+        private Queue queue;
+        private long id = rid.getAndIncrement();
+
+        public PullEvent(Session session, Subscription subscription, Queue queue) {
+            this.session = session;
+            this.subscription = subscription;
+            this.queue = queue;
+        }
+
+        @Override
+        public boolean equals(Object o) {
+            if (this == o) {
+                return true;
+            }
+            if (o == null || getClass() != o.getClass()) {
+                return false;
+            }
+
+            PullEvent pullEvent = (PullEvent) o;
+
+            return id == pullEvent.id;
+        }
+    }
+
+}
diff --git a/mqtt-cs/src/main/java/org/apache/rocketmq/mqtt/cs/session/match/MatchAction.java b/mqtt-cs/src/main/java/org/apache/rocketmq/mqtt/cs/session/match/MatchAction.java
new file mode 100644
index 0000000..471185d
--- /dev/null
+++ b/mqtt-cs/src/main/java/org/apache/rocketmq/mqtt/cs/session/match/MatchAction.java
@@ -0,0 +1,161 @@
+/*
+ *
+ *  * Licensed to the Apache Software Foundation (ASF) under one or more
+ *  * contributor license agreements.  See the NOTICE file distributed with
+ *  * this work for additional information regarding copyright ownership.
+ *  * The ASF licenses this file to You under the Apache License, Version 2.0
+ *  * (the "License"); you may not use this file except in compliance with
+ *  * the License.  You may obtain a copy of the License at
+ *  *
+ *  *     http://www.apache.org/licenses/LICENSE-2.0
+ *  *
+ *  * Unless required by applicable law or agreed to in writing, software
+ *  * distributed under the License is distributed on an "AS IS" BASIS,
+ *  * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ *  * See the License for the specific language governing permissions and
+ *  * limitations under the License.
+ *
+ */
+
+package org.apache.rocketmq.mqtt.cs.session.match;
+
+
+import org.apache.commons.lang3.tuple.Pair;
+import org.apache.rocketmq.mqtt.common.model.MqttTopic;
+import org.apache.rocketmq.mqtt.common.model.Subscription;
+import org.apache.rocketmq.mqtt.common.model.Trie;
+import org.apache.rocketmq.mqtt.common.util.TopicUtils;
+import org.apache.rocketmq.mqtt.cs.session.Session;
+import org.apache.rocketmq.mqtt.cs.session.loop.SessionLoop;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+import org.springframework.stereotype.Component;
+
+import javax.annotation.Resource;
+import java.util.HashSet;
+import java.util.List;
+import java.util.Map;
+import java.util.Set;
+import java.util.concurrent.ConcurrentHashMap;
+import java.util.concurrent.ConcurrentMap;
+
+
+@Component
+public class MatchAction {
+    private static Logger logger = LoggerFactory.getLogger(MatchAction.class);
+
+    @Resource
+    private SessionLoop sessionLoop;
+
+    private Trie<String, Integer> trie = new Trie<>();
+    private ConcurrentMap<String, Set<String>> topicCache = new ConcurrentHashMap<>(16);
+
+
+    public Set<Pair<Session, Subscription>> matchClients(String topic, String namespace) {
+        Set<Pair<Session, Subscription>> result = new HashSet<>();
+        MqttTopic mqttTopic = TopicUtils.decode(topic);
+        String secondTopic = TopicUtils.normalizeSecondTopic(mqttTopic.getSecondTopic());
+        if (TopicUtils.isRetryTopic(topic)) {
+            String clientId = TopicUtils.getClientIdFromRetryTopic(topic);
+            List<Session> sessions = sessionLoop.getSessionList(clientId);
+            for (Session session : sessions) {
+                result.add(Pair.of(session, Subscription.newRetrySubscription(clientId)));
+            }
+        } else if (TopicUtils.isP2P(secondTopic)) {
+            String clientId = TopicUtils.getP2Peer(mqttTopic, namespace);
+            List<Session> sessions = sessionLoop.getSessionList(clientId);
+            for (Session session : sessions) {
+                result.add(Pair.of(session, Subscription.newP2pSubscription(clientId)));
+            }
+        } else if(TopicUtils.isP2pTopic(topic)){
+            // may be produced by rmq
+            String clientId = TopicUtils.getClientIdFromP2pTopic(topic);
+            List<Session> sessions = sessionLoop.getSessionList(clientId);
+            for (Session session : sessions) {
+                result.add(Pair.of(session, Subscription.newP2pSubscription(clientId)));
+            }
+        } else {
+            Set<String> channelIdSet = new HashSet<>();
+            synchronized (topicCache) {
+                Set<String> precises = topicCache.get(topic);
+                if (precises != null && !precises.isEmpty()) {
+                    channelIdSet.addAll(precises);
+                }
+            }
+            Map<String, Integer> map = trie.getNode(topic);
+            if (map != null && !map.isEmpty()) {
+                channelIdSet.addAll(map.keySet());
+            }
+
+            for (String channelId : channelIdSet) {
+                Session session = sessionLoop.getSession(channelId);
+                if (session == null) {
+                    continue;
+                }
+                Set<Subscription> tmp = session.subscriptionSnapshot();
+                if (tmp != null && !tmp.isEmpty()) {
+                    for (Subscription subscription : tmp) {
+                        if (TopicUtils.isMatch(topic, subscription.getTopicFilter())) {
+                            result.add(Pair.of(session, subscription));
+                        }
+                    }
+                }
+            }
+        }
+        return result;
+    }
+
+    public void addSubscription(Session session, Set<Subscription> subscriptions) {
+        String channelId = session.getChannelId();
+        if (channelId == null || subscriptions == null || subscriptions.isEmpty()) {
+            return;
+        }
+        for (Subscription subscription : subscriptions) {
+            if (subscription.isRetry() || subscription.isP2p()) {
+                continue;
+            }
+            String topicFilter = subscription.getTopicFilter();
+            boolean isWildCard = TopicUtils.isWildCard(topicFilter);
+            if (isWildCard) {
+                trie.addNode(topicFilter, subscription.getQos(), channelId);
+                continue;
+            }
+
+            synchronized (topicCache) {
+                if (!topicCache.containsKey(topicFilter)) {
+                    topicCache.putIfAbsent(topicFilter, new HashSet<>());
+                }
+                topicCache.get(topicFilter).add(channelId);
+            }
+        }
+    }
+
+    public void removeSubscription(Session session, Set<Subscription> subscriptions) {
+        String channelId = session.getChannelId();
+        if (channelId == null || subscriptions == null || subscriptions.isEmpty()) {
+            return;
+        }
+        for (Subscription subscription : subscriptions) {
+            if (subscription.isRetry() || subscription.isP2p()) {
+                continue;
+            }
+            String topicFilter = subscription.getTopicFilter();
+            boolean isWildCard = TopicUtils.isWildCard(topicFilter);
+            if (isWildCard) {
+                trie.deleteNode(topicFilter, channelId);
+                continue;
+            }
+
+            synchronized (topicCache) {
+                Set<String> channelIdSet = topicCache.get(topicFilter);
+                if (channelIdSet != null) {
+                    channelIdSet.remove(channelId);
+                    if (channelIdSet.isEmpty()) {
+                        topicCache.remove(topicFilter);
+                    }
+                }
+            }
+        }
+    }
+
+}
diff --git a/mqtt-cs/src/main/java/org/apache/rocketmq/mqtt/cs/session/notify/MessageNotifyAction.java b/mqtt-cs/src/main/java/org/apache/rocketmq/mqtt/cs/session/notify/MessageNotifyAction.java
new file mode 100644
index 0000000..299abb7
--- /dev/null
+++ b/mqtt-cs/src/main/java/org/apache/rocketmq/mqtt/cs/session/notify/MessageNotifyAction.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.mqtt.cs.session.notify;
+
+import org.apache.commons.lang3.tuple.Pair;
+import org.apache.rocketmq.mqtt.common.model.MessageEvent;
+import org.apache.rocketmq.mqtt.common.model.Queue;
+import org.apache.rocketmq.mqtt.common.model.Subscription;
+import org.apache.rocketmq.mqtt.common.util.TopicUtils;
+import org.apache.rocketmq.mqtt.cs.session.QueueFresh;
+import org.apache.rocketmq.mqtt.cs.session.Session;
+import org.apache.rocketmq.mqtt.cs.session.loop.QueueCache;
+import org.apache.rocketmq.mqtt.cs.session.loop.SessionLoop;
+import org.apache.rocketmq.mqtt.cs.session.match.MatchAction;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+import org.springframework.stereotype.Component;
+
+import javax.annotation.Resource;
+import java.util.List;
+import java.util.Objects;
+import java.util.Set;
+
+@Component
+public class MessageNotifyAction {
+    private static Logger logger = LoggerFactory.getLogger(MessageNotifyAction.class);
+
+    @Resource
+    private MatchAction matchAction;
+
+    @Resource
+    private SessionLoop sessionLoop;
+
+    @Resource
+    private QueueCache queueCache;
+
+    @Resource
+    private QueueFresh queueFresh;
+
+    public void notify(List<MessageEvent> events) {
+        if (events == null || events.isEmpty()) {
+            return;
+        }
+        for (MessageEvent event : events) {
+            Set<Pair<Session, Subscription>> result = matchAction.matchClients(
+                    TopicUtils.normalizeTopic(event.getPubTopic()), event.getNamespace());
+            if (result == null || result.isEmpty()) {
+                continue;
+            }
+            for (Pair<Session, Subscription> pair : result) {
+                Session session = pair.getLeft();
+                Subscription subscription = pair.getRight();
+                Set<Queue> set = queueFresh.freshQueue(session, subscription);
+                if (set == null || set.isEmpty()) {
+                    continue;
+                }
+                for (Queue queue : set) {
+                    if (isTargetQueue(queue, event)) {
+                        queueCache.refreshCache(Pair.of(queue, session));
+                        sessionLoop.notifyPullMessage(session, subscription, queue);
+                    }
+                }
+            }
+        }
+    }
+
+    private boolean isTargetQueue(Queue queue, MessageEvent event) {
+        return Objects.equals(queue.getBrokerName(), event.getBrokerName());
+    }
+
+}
diff --git a/mqtt-cs/src/main/java/org/apache/rocketmq/mqtt/cs/starter/MqttServer.java b/mqtt-cs/src/main/java/org/apache/rocketmq/mqtt/cs/starter/MqttServer.java
new file mode 100644
index 0000000..1273989
--- /dev/null
+++ b/mqtt-cs/src/main/java/org/apache/rocketmq/mqtt/cs/starter/MqttServer.java
@@ -0,0 +1,132 @@
+/*
+ *
+ *  * Licensed to the Apache Software Foundation (ASF) under one or more
+ *  * contributor license agreements.  See the NOTICE file distributed with
+ *  * this work for additional information regarding copyright ownership.
+ *  * The ASF licenses this file to You under the Apache License, Version 2.0
+ *  * (the "License"); you may not use this file except in compliance with
+ *  * the License.  You may obtain a copy of the License at
+ *  *
+ *  *     http://www.apache.org/licenses/LICENSE-2.0
+ *  *
+ *  * Unless required by applicable law or agreed to in writing, software
+ *  * distributed under the License is distributed on an "AS IS" BASIS,
+ *  * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ *  * See the License for the specific language governing permissions and
+ *  * limitations under the License.
+ *
+ */
+
+package org.apache.rocketmq.mqtt.cs.starter;
+
+import io.netty.bootstrap.ServerBootstrap;
+import io.netty.buffer.PooledByteBufAllocator;
+import io.netty.channel.ChannelInitializer;
+import io.netty.channel.ChannelOption;
+import io.netty.channel.ChannelPipeline;
+import io.netty.channel.WriteBufferWaterMark;
+import io.netty.channel.nio.NioEventLoopGroup;
+import io.netty.channel.socket.SocketChannel;
+import io.netty.channel.socket.nio.NioServerSocketChannel;
+import io.netty.handler.codec.http.HttpObjectAggregator;
+import io.netty.handler.codec.http.HttpServerCodec;
+import io.netty.handler.codec.mqtt.MqttDecoder;
+import io.netty.handler.codec.mqtt.MqttEncoder;
+import io.netty.handler.stream.ChunkedWriteHandler;
+import org.apache.rocketmq.mqtt.cs.channel.ChannelManager;
+import org.apache.rocketmq.mqtt.cs.channel.ConnectHandler;
+import org.apache.rocketmq.mqtt.cs.config.ConnectConf;
+import org.apache.rocketmq.mqtt.cs.protocol.mqtt.MqttPacketDispatcher;
+import org.apache.rocketmq.mqtt.cs.protocol.ws.WebSocketServerHandler;
+import org.apache.rocketmq.mqtt.cs.protocol.ws.WebsocketEncoder;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+import org.springframework.stereotype.Service;
+
+import javax.annotation.PostConstruct;
+import javax.annotation.Resource;
+import java.net.InetSocketAddress;
+
+@Service
+public class MqttServer {
+    private static Logger logger = LoggerFactory.getLogger(MqttServer.class);
+
+    private ServerBootstrap serverBootstrap = new ServerBootstrap();
+    private ServerBootstrap wsServerBootstrap = new ServerBootstrap();
+
+    @Resource
+    private ConnectHandler connectHandler;
+
+    @Resource
+    private ConnectConf connectConf;
+
+    @Resource
+    private MqttPacketDispatcher mqttPacketDispatcher;
+
+    @Resource
+    private WebSocketServerHandler webSocketServerHandler;
+
+    @Resource
+    private ChannelManager channelManager;
+
+    @PostConstruct
+    public void init() throws Exception {
+        start();
+        startWs();
+    }
+
+    private void start() {
+        int port = connectConf.getMqttPort();
+        serverBootstrap
+            .group(new NioEventLoopGroup(connectConf.getNettySelectThreadNum()), new NioEventLoopGroup(connectConf.getNettyWorkerThreadNum()))
+            .channel(NioServerSocketChannel.class)
+            .option(ChannelOption.SO_BACKLOG, 8 * 1024)
+            .childOption(ChannelOption.ALLOCATOR, PooledByteBufAllocator.DEFAULT)
+            .childOption(ChannelOption.WRITE_BUFFER_WATER_MARK,new WriteBufferWaterMark(connectConf.getLowWater(), connectConf.getHighWater()))
+            .childOption(ChannelOption.TCP_NODELAY, true)
+            .localAddress(new InetSocketAddress(port))
+            .childHandler(new ChannelInitializer<SocketChannel>() {
+                @Override
+                public void initChannel(SocketChannel ch) throws Exception {
+                    ChannelPipeline pipeline = ch.pipeline();
+                    pipeline.addLast("connectHandler", connectHandler);
+                    pipeline.addLast("decoder", new MqttDecoder(connectConf.getMaxPacketSizeInByte()));
+                    pipeline.addLast("encoder", MqttEncoder.INSTANCE);
+                    pipeline.addLast("dispatcher", mqttPacketDispatcher);
+                }
+            });
+        serverBootstrap.bind();
+        logger.warn("start mqtt server , port:{}", port);
+    }
+
+    private void startWs() {
+        int port = connectConf.getMqttWsPort();
+        wsServerBootstrap
+            .group(new NioEventLoopGroup(connectConf.getNettySelectThreadNum()), new NioEventLoopGroup(connectConf.getNettyWorkerThreadNum()))
+            .channel(NioServerSocketChannel.class)
+            .option(ChannelOption.SO_BACKLOG, 8 * 1024)
+            .option(ChannelOption.SO_KEEPALIVE, true)
+            .childOption(ChannelOption.ALLOCATOR, PooledByteBufAllocator.DEFAULT)
+            .childOption(ChannelOption.WRITE_BUFFER_WATER_MARK,new WriteBufferWaterMark(connectConf.getLowWater(), connectConf.getHighWater()))
+            .childOption(ChannelOption.TCP_NODELAY, true)
+            .localAddress(new InetSocketAddress(port))
+            .childHandler(new ChannelInitializer<SocketChannel>() {
+                @Override
+                public void initChannel(SocketChannel ch) throws Exception {
+                    ChannelPipeline pipeline = ch.pipeline();
+                    pipeline.addLast("connectHandler", connectHandler);
+                    pipeline.addLast("http-codec", new HttpServerCodec(1024, 32 * 1024, connectConf.getMaxPacketSizeInByte() * 2, true));
+                    pipeline.addLast("aggregator", new HttpObjectAggregator(connectConf.getMaxPacketSizeInByte() * 2));
+                    pipeline.addLast("http-chunked", new ChunkedWriteHandler());
+                    pipeline.addLast("websocket-handler", webSocketServerHandler);
+                    pipeline.addLast("websocket-encoder", new WebsocketEncoder());
+                    pipeline.addLast("decoder", new MqttDecoder(connectConf.getMaxPacketSizeInByte()));
+                    pipeline.addLast("encoder", MqttEncoder.INSTANCE);
+                    pipeline.addLast("dispatcher", mqttPacketDispatcher);
+                }
+            });
+        wsServerBootstrap.bind();
+        logger.warn("start mqtt ws server , port:{}", port);
+    }
+
+}
diff --git a/mqtt-cs/src/main/java/org/apache/rocketmq/mqtt/cs/starter/RpcServer.java b/mqtt-cs/src/main/java/org/apache/rocketmq/mqtt/cs/starter/RpcServer.java
new file mode 100644
index 0000000..bd4ba1b
--- /dev/null
+++ b/mqtt-cs/src/main/java/org/apache/rocketmq/mqtt/cs/starter/RpcServer.java
@@ -0,0 +1,65 @@
+/*
+ *
+ *  * Licensed to the Apache Software Foundation (ASF) under one or more
+ *  * contributor license agreements.  See the NOTICE file distributed with
+ *  * this work for additional information regarding copyright ownership.
+ *  * The ASF licenses this file to You under the Apache License, Version 2.0
+ *  * (the "License"); you may not use this file except in compliance with
+ *  * the License.  You may obtain a copy of the License at
+ *  *
+ *  *     http://www.apache.org/licenses/LICENSE-2.0
+ *  *
+ *  * Unless required by applicable law or agreed to in writing, software
+ *  * distributed under the License is distributed on an "AS IS" BASIS,
+ *  * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ *  * See the License for the specific language governing permissions and
+ *  * limitations under the License.
+ *
+ */
+
+package org.apache.rocketmq.mqtt.cs.starter;
+
+
+import org.apache.rocketmq.common.ThreadFactoryImpl;
+import org.apache.rocketmq.mqtt.cs.config.ConnectConf;
+import org.apache.rocketmq.mqtt.cs.protocol.rpc.RpcPacketDispatcher;
+import org.apache.rocketmq.remoting.netty.NettyRemotingServer;
+import org.apache.rocketmq.remoting.netty.NettyServerConfig;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+import org.springframework.stereotype.Service;
+
+import javax.annotation.PostConstruct;
+import javax.annotation.Resource;
+import java.util.concurrent.BlockingQueue;
+import java.util.concurrent.LinkedBlockingQueue;
+import java.util.concurrent.ThreadPoolExecutor;
+import java.util.concurrent.TimeUnit;
+
+@Service
+public class RpcServer {
+    private static Logger logger = LoggerFactory.getLogger(RpcServer.class);
+
+    @Resource
+    private ConnectConf connectConf;
+
+    @Resource
+    private RpcPacketDispatcher rpcPacketDispatcher;
+
+    private NettyRemotingServer remotingServer;
+    private BlockingQueue<Runnable> csBridgeRpcQueue;
+
+    @PostConstruct
+    public void start() {
+        NettyServerConfig nettyServerConfig = new NettyServerConfig();
+        nettyServerConfig.setListenPort(connectConf.getRpcListenPort());
+        remotingServer = new NettyRemotingServer(nettyServerConfig);
+        csBridgeRpcQueue = new LinkedBlockingQueue<>(10000);
+        ThreadPoolExecutor executor = new ThreadPoolExecutor(8, 16, 1, TimeUnit.MINUTES,
+                csBridgeRpcQueue, new ThreadFactoryImpl("Rpc_Server_Thread_"));
+        remotingServer.registerDefaultProcessor(rpcPacketDispatcher, executor);
+        remotingServer.start();
+        logger.warn("start  rpc server , port:{}", connectConf.getRpcListenPort());
+    }
+
+}
diff --git a/mqtt-cs/src/main/java/org/apache/rocketmq/mqtt/cs/starter/Startup.java b/mqtt-cs/src/main/java/org/apache/rocketmq/mqtt/cs/starter/Startup.java
new file mode 100644
index 0000000..2c74958
--- /dev/null
+++ b/mqtt-cs/src/main/java/org/apache/rocketmq/mqtt/cs/starter/Startup.java
@@ -0,0 +1,36 @@
+/*
+ *
+ *  * Licensed to the Apache Software Foundation (ASF) under one or more
+ *  * contributor license agreements.  See the NOTICE file distributed with
+ *  * this work for additional information regarding copyright ownership.
+ *  * The ASF licenses this file to You under the Apache License, Version 2.0
+ *  * (the "License"); you may not use this file except in compliance with
+ *  * the License.  You may obtain a copy of the License at
+ *  *
+ *  *     http://www.apache.org/licenses/LICENSE-2.0
+ *  *
+ *  * Unless required by applicable law or agreed to in writing, software
+ *  * distributed under the License is distributed on an "AS IS" BASIS,
+ *  * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ *  * See the License for the specific language governing permissions and
+ *  * limitations under the License.
+ *
+ */
+
+package org.apache.rocketmq.mqtt.cs.starter;
+
+import org.apache.rocketmq.client.log.ClientLogger;
+import org.springframework.context.support.ClassPathXmlApplicationContext;
+
+
+public class Startup {
+
+    public static void main(String[] args) {
+        System.setProperty(ClientLogger.CLIENT_LOG_USESLF4J, "true");
+
+        ClassPathXmlApplicationContext applicationContext = new ClassPathXmlApplicationContext("classpath:spring.xml");
+
+        System.out.println("start rocketmq mqtt  ...");
+    }
+
+}
diff --git a/mqtt-cs/src/test/java/org/apache/rocketmq/mqtt/cs/test/TestDefaultChannelManager.java b/mqtt-cs/src/test/java/org/apache/rocketmq/mqtt/cs/test/TestDefaultChannelManager.java
new file mode 100644
index 0000000..bb3acfb
--- /dev/null
+++ b/mqtt-cs/src/test/java/org/apache/rocketmq/mqtt/cs/test/TestDefaultChannelManager.java
@@ -0,0 +1,60 @@
+/*
+ *
+ *  * Licensed to the Apache Software Foundation (ASF) under one or more
+ *  * contributor license agreements.  See the NOTICE file distributed with
+ *  * this work for additional information regarding copyright ownership.
+ *  * The ASF licenses this file to You under the Apache License, Version 2.0
+ *  * (the "License"); you may not use this file except in compliance with
+ *  * the License.  You may obtain a copy of the License at
+ *  *
+ *  *     http://www.apache.org/licenses/LICENSE-2.0
+ *  *
+ *  * Unless required by applicable law or agreed to in writing, software
+ *  * distributed under the License is distributed on an "AS IS" BASIS,
+ *  * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ *  * See the License for the specific language governing permissions and
+ *  * limitations under the License.
+ *
+ */
+
+package org.apache.rocketmq.mqtt.cs.test;
+
+import io.netty.channel.socket.nio.NioSocketChannel;
+import io.netty.util.Timeout;
+import org.apache.commons.lang3.reflect.FieldUtils;
+import org.apache.commons.lang3.reflect.MethodUtils;
+import org.apache.rocketmq.mqtt.cs.channel.ChannelInfo;
+import org.apache.rocketmq.mqtt.cs.channel.DefaultChannelManager;
+import org.apache.rocketmq.mqtt.cs.config.ConnectConf;
+import org.apache.rocketmq.mqtt.cs.session.infly.RetryDriver;
+import org.apache.rocketmq.mqtt.cs.session.loop.SessionLoop;
+import org.junit.Assert;
+import org.junit.Test;
+import org.junit.runner.RunWith;
+import org.mockito.junit.MockitoJUnitRunner;
+
+import java.lang.reflect.InvocationTargetException;
+
+import static org.mockito.Mockito.*;
+
+public class TestDefaultChannelManager {
+
+    @Test
+    public void test() throws IllegalAccessException, InterruptedException, InvocationTargetException, NoSuchMethodException {
+        DefaultChannelManager defaultChannelManager = new DefaultChannelManager();
+        SessionLoop sessionLoop = mock(SessionLoop.class);
+        FieldUtils.writeDeclaredField(defaultChannelManager, "sessionLoop", sessionLoop, true);
+        FieldUtils.writeDeclaredField(defaultChannelManager, "connectConf", mock(ConnectConf.class), true);
+        FieldUtils.writeDeclaredField(defaultChannelManager, "retryDriver", mock(RetryDriver.class), true);
+        FieldUtils.writeStaticField(DefaultChannelManager.class, "MinBlankChannelSeconds", 1, true);
+        defaultChannelManager.init();
+        NioSocketChannel channel = spy(new NioSocketChannel());
+        when(channel.isActive()).thenReturn(false);
+        ChannelInfo.setClientId(channel, "test");
+        ChannelInfo.setKeepLive(channel, 0);
+        defaultChannelManager.addChannel(channel);
+        MethodUtils.invokeMethod(defaultChannelManager, true, "doPing", mock(Timeout.class), channel);
+        verify(sessionLoop).unloadSession(anyString(), anyString());
+    }
+
+}
diff --git a/mqtt-cs/src/test/java/org/apache/rocketmq/mqtt/cs/test/TestInFlyCache.java b/mqtt-cs/src/test/java/org/apache/rocketmq/mqtt/cs/test/TestInFlyCache.java
new file mode 100644
index 0000000..a3fb6e8
--- /dev/null
+++ b/mqtt-cs/src/test/java/org/apache/rocketmq/mqtt/cs/test/TestInFlyCache.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.mqtt.cs.test;
+
+import org.apache.rocketmq.mqtt.common.model.Message;
+import org.apache.rocketmq.mqtt.common.model.Queue;
+import org.apache.rocketmq.mqtt.common.model.Subscription;
+import org.apache.rocketmq.mqtt.cs.session.infly.InFlyCache;
+import org.junit.Assert;
+import org.junit.Test;
+import org.junit.runner.RunWith;
+import org.mockito.junit.MockitoJUnitRunner;
+
+import static org.mockito.Mockito.mock;
+
+@RunWith(MockitoJUnitRunner.class)
+public class TestInFlyCache {
+
+    @Test
+    public void test() {
+        InFlyCache inFlyCache = new InFlyCache();
+        inFlyCache.put(InFlyCache.CacheType.PUB, "test", 1);
+        Assert.assertTrue(inFlyCache.contains(InFlyCache.CacheType.PUB, "test", 1));
+
+        inFlyCache.getPendingDownCache().put("test", 1, mock(Subscription.class), mock(Queue.class), mock(Message.class));
+        Assert.assertTrue(null != inFlyCache.getPendingDownCache().get("test", 1));
+
+        inFlyCache.getPendingDownCache().remove("test", 1);
+        Assert.assertTrue(null == inFlyCache.getPendingDownCache().get("test", 1));
+
+    }
+}
diff --git a/mqtt-cs/src/test/java/org/apache/rocketmq/mqtt/cs/test/TestMatchAction.java b/mqtt-cs/src/test/java/org/apache/rocketmq/mqtt/cs/test/TestMatchAction.java
new file mode 100644
index 0000000..0369b70
--- /dev/null
+++ b/mqtt-cs/src/test/java/org/apache/rocketmq/mqtt/cs/test/TestMatchAction.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.mqtt.cs.test;
+
+import org.apache.commons.lang3.reflect.FieldUtils;
+import org.apache.commons.lang3.tuple.Pair;
+import org.apache.rocketmq.mqtt.common.model.Subscription;
+import org.apache.rocketmq.mqtt.cs.session.Session;
+import org.apache.rocketmq.mqtt.cs.session.loop.SessionLoop;
+import org.apache.rocketmq.mqtt.cs.session.match.MatchAction;
+import org.junit.Assert;
+import org.junit.Test;
+import org.junit.runner.RunWith;
+import org.mockito.Mock;
+import org.mockito.junit.MockitoJUnitRunner;
+
+import java.util.Arrays;
+import java.util.HashSet;
+import java.util.Set;
+
+import static org.mockito.ArgumentMatchers.any;
+import static org.mockito.Mockito.mock;
+import static org.mockito.Mockito.when;
+
+@RunWith(MockitoJUnitRunner.class)
+public class TestMatchAction {
+
+    @Mock
+    private SessionLoop sessionLoop;
+
+    @Test
+    public void test() throws IllegalAccessException {
+        MatchAction matchAction = new MatchAction();
+        FieldUtils.writeDeclaredField(matchAction, "sessionLoop", sessionLoop, true);
+
+        Session session = mock(Session.class);
+        when(session.getChannelId()).thenReturn("test");
+        when(sessionLoop.getSession(any())).thenReturn(session);
+        Subscription subscription = new Subscription("test");
+        Set<Subscription> subscriptions = new HashSet<>(Arrays.asList(subscription));
+        when(session.subscriptionSnapshot()).thenReturn(subscriptions);
+
+        matchAction.addSubscription(session, subscriptions);
+        Set<Pair<Session, Subscription>> set =  matchAction.matchClients("test","");
+        Assert.assertFalse(set.isEmpty());
+
+        matchAction.removeSubscription(session,subscriptions);
+        set =  matchAction.matchClients("test","");
+        Assert.assertTrue(set.isEmpty());
+    }
+
+}
diff --git a/mqtt-cs/src/test/java/org/apache/rocketmq/mqtt/cs/test/TestMessageNotifyAction.java b/mqtt-cs/src/test/java/org/apache/rocketmq/mqtt/cs/test/TestMessageNotifyAction.java
new file mode 100644
index 0000000..c969483
--- /dev/null
+++ b/mqtt-cs/src/test/java/org/apache/rocketmq/mqtt/cs/test/TestMessageNotifyAction.java
@@ -0,0 +1,87 @@
+/*
+ *
+ *  * Licensed to the Apache Software Foundation (ASF) under one or more
+ *  * contributor license agreements.  See the NOTICE file distributed with
+ *  * this work for additional information regarding copyright ownership.
+ *  * The ASF licenses this file to You under the Apache License, Version 2.0
+ *  * (the "License"); you may not use this file except in compliance with
+ *  * the License.  You may obtain a copy of the License at
+ *  *
+ *  *     http://www.apache.org/licenses/LICENSE-2.0
+ *  *
+ *  * Unless required by applicable law or agreed to in writing, software
+ *  * distributed under the License is distributed on an "AS IS" BASIS,
+ *  * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ *  * See the License for the specific language governing permissions and
+ *  * limitations under the License.
+ *
+ */
+
+package org.apache.rocketmq.mqtt.cs.test;
+
+import org.apache.commons.lang3.reflect.FieldUtils;
+import org.apache.rocketmq.mqtt.common.model.MessageEvent;
+import org.apache.rocketmq.mqtt.common.model.Queue;
+import org.apache.rocketmq.mqtt.common.model.Subscription;
+import org.apache.rocketmq.mqtt.cs.session.QueueFresh;
+import org.apache.rocketmq.mqtt.cs.session.Session;
+import org.apache.rocketmq.mqtt.cs.session.loop.QueueCache;
+import org.apache.rocketmq.mqtt.cs.session.loop.SessionLoop;
+import org.apache.rocketmq.mqtt.cs.session.match.MatchAction;
+import org.apache.rocketmq.mqtt.cs.session.notify.MessageNotifyAction;
+import org.junit.Test;
+import org.junit.runner.RunWith;
+import org.mockito.Mock;
+import org.mockito.junit.MockitoJUnitRunner;
+
+import java.util.Arrays;
+import java.util.HashSet;
+import java.util.Set;
+
+import static org.mockito.ArgumentMatchers.any;
+import static org.mockito.ArgumentMatchers.eq;
+import static org.mockito.Mockito.*;
+
+@RunWith(MockitoJUnitRunner.class)
+public class TestMessageNotifyAction {
+
+    @Mock
+    private SessionLoop sessionLoop;
+
+    @Mock
+    private QueueFresh queueFresh;
+
+    @Mock
+    private QueueCache queueCache;
+
+    @Test
+    public void test() throws IllegalAccessException {
+        MatchAction matchAction = new MatchAction();
+        FieldUtils.writeDeclaredField(matchAction, "sessionLoop", sessionLoop, true);
+
+        MessageNotifyAction messageNotifyAction = new MessageNotifyAction();
+        FieldUtils.writeDeclaredField(messageNotifyAction, "sessionLoop", sessionLoop, true);
+        FieldUtils.writeDeclaredField(messageNotifyAction, "queueFresh", queueFresh, true);
+        FieldUtils.writeDeclaredField(messageNotifyAction, "queueCache", queueCache, true);
+        FieldUtils.writeDeclaredField(messageNotifyAction, "matchAction", matchAction, true);
+
+        Session session = mock(Session.class);
+        when(session.getChannelId()).thenReturn("test");
+        when(sessionLoop.getSession(any())).thenReturn(session);
+        Subscription subscription = new Subscription("test");
+        Set<Subscription> subscriptions = new HashSet<>(Arrays.asList(subscription));
+        when(session.subscriptionSnapshot()).thenReturn(subscriptions);
+        matchAction.addSubscription(session, subscriptions);
+
+        Queue queue = new Queue(0, "test", "test");
+        when(queueFresh.freshQueue(eq(session), eq(subscription))).thenReturn(new HashSet<>(Arrays.asList(queue)));
+
+        MessageEvent messageEvent = new MessageEvent();
+        messageEvent.setPubTopic("test");
+        messageEvent.setBrokerName("test");
+        messageEvent.setQueueId(0);
+        messageNotifyAction.notify(Arrays.asList(messageEvent));
+        verify(sessionLoop).notifyPullMessage(eq(session), eq(subscription), eq(queue));
+    }
+
+}
diff --git a/mqtt-cs/src/test/java/org/apache/rocketmq/mqtt/cs/test/TestMqttMsgId.java b/mqtt-cs/src/test/java/org/apache/rocketmq/mqtt/cs/test/TestMqttMsgId.java
new file mode 100644
index 0000000..b4efbc4
--- /dev/null
+++ b/mqtt-cs/src/test/java/org/apache/rocketmq/mqtt/cs/test/TestMqttMsgId.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.mqtt.cs.test;
+
+import org.apache.rocketmq.mqtt.cs.session.infly.MqttMsgId;
+import org.junit.Assert;
+import org.junit.Test;
+import org.junit.runner.RunWith;
+import org.mockito.junit.MockitoJUnitRunner;
+
+@RunWith(MockitoJUnitRunner.class)
+public class TestMqttMsgId {
+
+    @Test
+    public void test() {
+        MqttMsgId mqttMsgId = new MqttMsgId();
+        mqttMsgId.init();
+        int id = mqttMsgId.nextId("test");
+        Assert.assertFalse(mqttMsgId.nextId("test") == id);
+    }
+}
diff --git a/mqtt-cs/src/test/java/org/apache/rocketmq/mqtt/cs/test/TestPushAction.java b/mqtt-cs/src/test/java/org/apache/rocketmq/mqtt/cs/test/TestPushAction.java
new file mode 100644
index 0000000..de7f988
--- /dev/null
+++ b/mqtt-cs/src/test/java/org/apache/rocketmq/mqtt/cs/test/TestPushAction.java
@@ -0,0 +1,96 @@
+/*
+ *
+ *  * Licensed to the Apache Software Foundation (ASF) under one or more
+ *  * contributor license agreements.  See the NOTICE file distributed with
+ *  * this work for additional information regarding copyright ownership.
+ *  * The ASF licenses this file to You under the Apache License, Version 2.0
+ *  * (the "License"); you may not use this file except in compliance with
+ *  * the License.  You may obtain a copy of the License at
+ *  *
+ *  *     http://www.apache.org/licenses/LICENSE-2.0
+ *  *
+ *  * Unless required by applicable law or agreed to in writing, software
+ *  * distributed under the License is distributed on an "AS IS" BASIS,
+ *  * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ *  * See the License for the specific language governing permissions and
+ *  * limitations under the License.
+ *
+ */
+
+package org.apache.rocketmq.mqtt.cs.test;
+
+import org.apache.commons.lang3.reflect.FieldUtils;
+import org.apache.rocketmq.mqtt.common.model.Message;
+import org.apache.rocketmq.mqtt.common.model.Queue;
+import org.apache.rocketmq.mqtt.common.model.Subscription;
+import org.apache.rocketmq.mqtt.cs.config.ConnectConf;
+import org.apache.rocketmq.mqtt.cs.session.Session;
+import org.apache.rocketmq.mqtt.cs.session.infly.InFlyCache;
+import org.apache.rocketmq.mqtt.cs.session.infly.MqttMsgId;
+import org.apache.rocketmq.mqtt.cs.session.infly.PushAction;
+import org.apache.rocketmq.mqtt.cs.session.infly.RetryDriver;
+import org.junit.Before;
+import org.junit.Test;
+import org.junit.runner.RunWith;
+import org.mockito.Mock;
+import org.mockito.junit.MockitoJUnitRunner;
+
+import java.util.ArrayList;
+import java.util.List;
+
+import static org.mockito.ArgumentMatchers.any;
+import static org.mockito.Mockito.*;
+
+@RunWith(MockitoJUnitRunner.class)
+public class TestPushAction {
+
+    @Mock
+    private MqttMsgId mqttMsgId;
+
+    @Mock
+    private RetryDriver retryDriver;
+
+    @Mock
+    private InFlyCache inFlyCache;
+
+    @Mock
+    private ConnectConf connectConf;
+
+    private PushAction pushAction = new PushAction();
+
+    @Before
+    public void before() throws IllegalAccessException {
+        FieldUtils.writeDeclaredField(pushAction, "mqttMsgId", mqttMsgId, true);
+        FieldUtils.writeDeclaredField(pushAction, "retryDriver", retryDriver, true);
+        FieldUtils.writeDeclaredField(pushAction, "inFlyCache", inFlyCache, true);
+        FieldUtils.writeDeclaredField(pushAction, "connectConf", connectConf, true);
+    }
+
+    @Test
+    public void testMessageArrive() {
+        Session session = mock(Session.class);
+        Subscription subscription = mock(Subscription.class);
+        Queue queue = mock(Queue.class);
+        List<Message> messages = new ArrayList<>();
+        messages.add(mock(Message.class));
+        when(session.pendMessageList(any(), any())).thenReturn(messages);
+        when(connectConf.isOrder()).thenReturn(false);
+        PushAction spyPushAction = spy(pushAction);
+        doNothing().when(spyPushAction).push(any(), any(), any(), any());
+        spyPushAction.messageArrive(session, subscription, queue);
+        verify(spyPushAction, atLeastOnce()).push(any(), any(), any(), any());
+    }
+
+    @Test
+    public void testPush() {
+        Session session = mock(Session.class);
+        when(session.getChannelId()).thenReturn("test");
+        when(session.getClientId()).thenReturn("test");
+        PushAction spyPushAction = spy(pushAction);
+        doNothing().when(spyPushAction).write(any(), any(), anyInt(), anyInt(), any());
+        when(inFlyCache.getPendingDownCache()).thenReturn(new InFlyCache().getPendingDownCache());
+        spyPushAction.push(mock(Message.class), mock(Subscription.class), session, mock(Queue.class));
+        verify(spyPushAction, atLeastOnce()).write(any(), any(), anyInt(), anyInt(), any());
+    }
+
+}
diff --git a/mqtt-cs/src/test/java/org/apache/rocketmq/mqtt/cs/test/TestQueueCache.java b/mqtt-cs/src/test/java/org/apache/rocketmq/mqtt/cs/test/TestQueueCache.java
new file mode 100644
index 0000000..9f1a2c6
--- /dev/null
+++ b/mqtt-cs/src/test/java/org/apache/rocketmq/mqtt/cs/test/TestQueueCache.java
@@ -0,0 +1,112 @@
+/*
+ *
+ *  * Licensed to the Apache Software Foundation (ASF) under one or more
+ *  * contributor license agreements.  See the NOTICE file distributed with
+ *  * this work for additional information regarding copyright ownership.
+ *  * The ASF licenses this file to You under the Apache License, Version 2.0
+ *  * (the "License"); you may not use this file except in compliance with
+ *  * the License.  You may obtain a copy of the License at
+ *  *
+ *  *     http://www.apache.org/licenses/LICENSE-2.0
+ *  *
+ *  * Unless required by applicable law or agreed to in writing, software
+ *  * distributed under the License is distributed on an "AS IS" BASIS,
+ *  * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ *  * See the License for the specific language governing permissions and
+ *  * limitations under the License.
+ *
+ */
+
+package org.apache.rocketmq.mqtt.cs.test;
+
+import org.apache.commons.lang3.reflect.FieldUtils;
+import org.apache.commons.lang3.tuple.Pair;
+import org.apache.rocketmq.mqtt.common.facade.LmqQueueStore;
+import org.apache.rocketmq.mqtt.common.model.*;
+import org.apache.rocketmq.mqtt.cs.config.ConnectConf;
+import org.apache.rocketmq.mqtt.cs.session.Session;
+import org.apache.rocketmq.mqtt.cs.session.loop.QueueCache;
+import org.junit.Assert;
+import org.junit.Before;
+import org.junit.Test;
+import org.junit.runner.RunWith;
+import org.mockito.Mock;
+import org.mockito.junit.MockitoJUnitRunner;
+
+import java.util.ArrayList;
+import java.util.List;
+import java.util.concurrent.CompletableFuture;
+import java.util.concurrent.ExecutionException;
+import java.util.concurrent.TimeUnit;
+import java.util.concurrent.TimeoutException;
+
+import static org.mockito.ArgumentMatchers.*;
+import static org.mockito.Mockito.mock;
+import static org.mockito.Mockito.when;
+
+@RunWith(MockitoJUnitRunner.class)
+public class TestQueueCache {
+
+    @Mock
+    private ConnectConf connectConf;
+
+    @Mock
+    private LmqQueueStore lmqQueueStore;
+
+    private QueueCache queueCache = new QueueCache();
+
+    @Before
+    public void before() throws IllegalAccessException {
+        FieldUtils.writeDeclaredField(queueCache, "connectConf", connectConf, true);
+        FieldUtils.writeDeclaredField(queueCache, "lmqQueueStore", lmqQueueStore, true);
+        queueCache.init();
+    }
+
+    @Test
+    public void test() throws InterruptedException, ExecutionException, TimeoutException {
+        when(connectConf.getQueueCacheSize()).thenReturn(32);
+        when(connectConf.getPullBatchSize()).thenReturn(32);
+        QueueOffset queueOffset = new QueueOffset();
+        Queue queue = new Queue();
+        queue.setQueueName("test");
+        queue.setBrokerName("test");
+
+        List<Message> messageList = new ArrayList<>();
+        messageList.add(new Message());
+        messageList.add(new Message());
+        messageList.get(0).setOffset(1);
+        messageList.get(1).setOffset(2);
+
+        CompletableFuture<PullResult> resultPullLast = new CompletableFuture<>();
+        PullResult pullResult = new PullResult();
+        pullResult.setCode(PullResult.PULL_SUCCESS);
+        pullResult.setMessageList(messageList.subList(0, 1));
+        resultPullLast.complete(pullResult);
+        when(lmqQueueStore.pullLastMessages(any(), any(), anyLong())).thenReturn(resultPullLast);
+
+        CompletableFuture<PullResult> resultPull = new CompletableFuture<>();
+        pullResult = new PullResult();
+        pullResult.setCode(PullResult.PULL_SUCCESS);
+        pullResult.setMessageList(messageList.subList(1, messageList.size()));
+        resultPull.complete(pullResult);
+        when(lmqQueueStore.pullMessage(any(), any(), any(), anyLong())).thenReturn(resultPull);
+
+        Session session = mock(Session.class);
+        queueCache.refreshCache(Pair.of(queue, session));
+        Thread.sleep(1000);
+        CompletableFuture<PullResult> callBackResult = new CompletableFuture<>();
+        queueOffset.setOffset(1);
+        queueCache.pullMessage(session, new Subscription("test"), queue, queueOffset, 32, callBackResult);
+        pullResult = callBackResult.get(1, TimeUnit.SECONDS);
+        Assert.assertTrue(pullResult.getMessageList().get(0).getOffset() == 1);
+
+        queueCache.refreshCache(Pair.of(queue, session));
+        Thread.sleep(1000);
+        callBackResult = new CompletableFuture<>();
+        queueOffset.setOffset(2);
+        queueCache.pullMessage(session, new Subscription("test"), queue, queueOffset, 32, callBackResult);
+        pullResult = callBackResult.get(1, TimeUnit.SECONDS);
+        Assert.assertTrue(pullResult.getMessageList().get(0).getOffset() == 2);
+    }
+
+}
diff --git a/mqtt-cs/src/test/java/org/apache/rocketmq/mqtt/cs/test/TestRetryDriver.java b/mqtt-cs/src/test/java/org/apache/rocketmq/mqtt/cs/test/TestRetryDriver.java
new file mode 100644
index 0000000..cb7d37f
--- /dev/null
+++ b/mqtt-cs/src/test/java/org/apache/rocketmq/mqtt/cs/test/TestRetryDriver.java
@@ -0,0 +1,91 @@
+/*
+ *
+ *  * Licensed to the Apache Software Foundation (ASF) under one or more
+ *  * contributor license agreements.  See the NOTICE file distributed with
+ *  * this work for additional information regarding copyright ownership.
+ *  * The ASF licenses this file to You under the Apache License, Version 2.0
+ *  * (the "License"); you may not use this file except in compliance with
+ *  * the License.  You may obtain a copy of the License at
+ *  *
+ *  *     http://www.apache.org/licenses/LICENSE-2.0
+ *  *
+ *  * Unless required by applicable law or agreed to in writing, software
+ *  * distributed under the License is distributed on an "AS IS" BASIS,
+ *  * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ *  * See the License for the specific language governing permissions and
+ *  * limitations under the License.
+ *
+ */
+
+package org.apache.rocketmq.mqtt.cs.test;
+
+import org.apache.commons.lang3.reflect.FieldUtils;
+import org.apache.commons.lang3.reflect.MethodUtils;
+import org.apache.rocketmq.mqtt.common.facade.LmqQueueStore;
+import org.apache.rocketmq.mqtt.common.model.Message;
+import org.apache.rocketmq.mqtt.common.model.Subscription;
+import org.apache.rocketmq.mqtt.cs.config.ConnectConf;
+import org.apache.rocketmq.mqtt.cs.session.Session;
+import org.apache.rocketmq.mqtt.cs.session.infly.PushAction;
+import org.apache.rocketmq.mqtt.cs.session.infly.RetryDriver;
+import org.apache.rocketmq.mqtt.cs.session.loop.SessionLoop;
+import org.junit.Before;
+import org.junit.Test;
+import org.junit.runner.RunWith;
+import org.mockito.Mock;
+import org.mockito.junit.MockitoJUnitRunner;
+
+import java.lang.reflect.InvocationTargetException;
+import java.util.concurrent.CompletableFuture;
+
+import static org.mockito.ArgumentMatchers.any;
+import static org.mockito.Mockito.*;
+
+@RunWith(MockitoJUnitRunner.class)
+public class TestRetryDriver {
+
+    @Mock
+    private SessionLoop sessionLoop;
+
+    @Mock
+    private PushAction pushAction;
+
+    @Mock
+    private ConnectConf connectConf;
+
+    @Mock
+    private LmqQueueStore lmqQueueStore;
+
+    private RetryDriver retryDriver = new RetryDriver();
+
+    @Before
+    public void before() throws IllegalAccessException {
+        FieldUtils.writeDeclaredField(retryDriver, "sessionLoop", sessionLoop, true);
+        FieldUtils.writeDeclaredField(retryDriver, "pushAction", pushAction, true);
+        FieldUtils.writeDeclaredField(retryDriver, "connectConf", connectConf, true);
+        FieldUtils.writeDeclaredField(retryDriver, "lmqQueueStore", lmqQueueStore, true);
+
+        when(connectConf.getRetryIntervalSeconds()).thenReturn(1);
+        when(connectConf.getMaxRetryTime()).thenReturn(1);
+        retryDriver.init();
+    }
+
+    @Test
+    public void test() throws InvocationTargetException, NoSuchMethodException, IllegalAccessException, InterruptedException {
+        Session session = mock(Session.class);
+        when(session.getChannelId()).thenReturn("test");
+        when(session.isDestroyed()).thenReturn(false);
+        when(sessionLoop.getSession(any())).thenReturn(session);
+        when(lmqQueueStore.putMessage(any(), any())).thenReturn(mock(CompletableFuture.class));
+        Message message = mock(Message.class);
+        when(message.copy()).thenReturn(mock(Message.class));
+        retryDriver.mountPublish(1,message , 1, "test", mock(Subscription.class));
+        Thread.sleep(3000);
+        MethodUtils.invokeMethod(retryDriver, true, "doRetryCache");
+        verify(pushAction, atLeastOnce()).write(any(), any(), eq(1), eq(1), any());
+        Thread.sleep(3000);
+        MethodUtils.invokeMethod(retryDriver, true, "doRetryCache");
+        verify(lmqQueueStore, atLeastOnce()).putMessage(any(), any());
+    }
+
+}
diff --git a/mqtt-cs/src/test/java/org/apache/rocketmq/mqtt/cs/test/TestSession.java b/mqtt-cs/src/test/java/org/apache/rocketmq/mqtt/cs/test/TestSession.java
new file mode 100644
index 0000000..cfcfa6e
--- /dev/null
+++ b/mqtt-cs/src/test/java/org/apache/rocketmq/mqtt/cs/test/TestSession.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.mqtt.cs.test;
+
+import org.apache.rocketmq.mqtt.common.model.Message;
+import org.apache.rocketmq.mqtt.common.model.Queue;
+import org.apache.rocketmq.mqtt.common.model.QueueOffset;
+import org.apache.rocketmq.mqtt.common.model.Subscription;
+import org.apache.rocketmq.mqtt.cs.session.Session;
+import org.junit.Assert;
+import org.junit.Test;
+import org.junit.runner.RunWith;
+import org.mockito.junit.MockitoJUnitRunner;
+
+import java.util.*;
+
+@RunWith(MockitoJUnitRunner.class)
+public class TestSession {
+
+    @Test
+    public void test() {
+        Session session = new Session();
+
+        Set<Subscription> subscriptions = new HashSet<>();
+        Subscription subscription = new Subscription("test");
+        subscriptions.add(subscription);
+        session.addSubscription(subscriptions);
+        Set<Subscription> subscriptionSnapshot = session.subscriptionSnapshot();
+        Assert.assertTrue(subscriptionSnapshot.iterator().next().equals(subscription));
+
+        Queue queue = new Queue(0, "test", "test");
+        QueueOffset queueOffset = new QueueOffset();
+        Map<Queue, QueueOffset> offsetMap = new HashMap<>();
+        offsetMap.put(queue, queueOffset);
+        session.addOffset(subscription.toQueueName(), offsetMap);
+        Assert.assertTrue(queueOffset.equals(session.getQueueOffset(subscription, queue)));
+
+        session.freshQueue(subscription, new HashSet<>(Arrays.asList(queue)));
+        List<Message> messages = new ArrayList<>();
+        Message message = new Message();
+        message.setOffset(1);
+        messages.add(message);
+        session.addSendingMessages(subscription, queue, messages);
+        Assert.assertFalse(session.sendingMessageIsEmpty(subscription, queue));
+        Assert.assertTrue(message.equals(session.nextSendMessageByOrder(subscription,queue)));
+        Assert.assertTrue(message.equals(session.pendMessageList(subscription,queue).iterator().next()));
+
+        session.ack(subscription,queue,1);
+        Assert.assertTrue(session.sendingMessageIsEmpty(subscription, queue));
+    }
+
+}
diff --git a/mqtt-cs/src/test/java/org/apache/rocketmq/mqtt/cs/test/TestSessionLoopImpl.java b/mqtt-cs/src/test/java/org/apache/rocketmq/mqtt/cs/test/TestSessionLoopImpl.java
new file mode 100644
index 0000000..b3eb307
--- /dev/null
+++ b/mqtt-cs/src/test/java/org/apache/rocketmq/mqtt/cs/test/TestSessionLoopImpl.java
@@ -0,0 +1,157 @@
+/*
+ *
+ *  * Licensed to the Apache Software Foundation (ASF) under one or more
+ *  * contributor license agreements.  See the NOTICE file distributed with
+ *  * this work for additional information regarding copyright ownership.
+ *  * The ASF licenses this file to You under the Apache License, Version 2.0
+ *  * (the "License"); you may not use this file except in compliance with
+ *  * the License.  You may obtain a copy of the License at
+ *  *
+ *  *     http://www.apache.org/licenses/LICENSE-2.0
+ *  *
+ *  * Unless required by applicable law or agreed to in writing, software
+ *  * distributed under the License is distributed on an "AS IS" BASIS,
+ *  * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ *  * See the License for the specific language governing permissions and
+ *  * limitations under the License.
+ *
+ */
+
+package org.apache.rocketmq.mqtt.cs.test;
+
+import io.netty.channel.socket.nio.NioSocketChannel;
+import org.apache.commons.lang3.reflect.FieldUtils;
+import org.apache.rocketmq.mqtt.common.facade.LmqOffsetStore;
+import org.apache.rocketmq.mqtt.common.facade.LmqQueueStore;
+import org.apache.rocketmq.mqtt.common.model.Queue;
+import org.apache.rocketmq.mqtt.common.model.QueueOffset;
+import org.apache.rocketmq.mqtt.common.model.Subscription;
+import org.apache.rocketmq.mqtt.cs.config.ConnectConf;
+import org.apache.rocketmq.mqtt.cs.session.QueueFresh;
+import org.apache.rocketmq.mqtt.cs.session.Session;
+import org.apache.rocketmq.mqtt.cs.session.infly.InFlyCache;
+import org.apache.rocketmq.mqtt.cs.session.loop.QueueCache;
+import org.apache.rocketmq.mqtt.cs.session.loop.SessionLoopImpl;
+import org.apache.rocketmq.mqtt.cs.session.match.MatchAction;
+import org.junit.Assert;
+import org.junit.Before;
+import org.junit.Test;
+import org.junit.runner.RunWith;
+import org.mockito.Mock;
+import org.mockito.junit.MockitoJUnitRunner;
+
+import java.lang.reflect.Field;
+import java.util.*;
+import java.util.concurrent.CompletableFuture;
+import java.util.concurrent.ConcurrentHashMap;
+
+import static org.mockito.Mockito.*;
+
+@RunWith(MockitoJUnitRunner.class)
+public class TestSessionLoopImpl {
+
+    @Mock
+    private MatchAction matchAction;
+
+    @Mock
+    private QueueFresh queueFresh;
+
+    @Mock
+    private InFlyCache inFlyCache;
+
+    @Mock
+    private LmqOffsetStore lmqOffsetStore;
+
+    @Mock
+    private LmqQueueStore lmqQueueStore;
+
+    @Mock
+    private QueueCache queueCache;
+
+    @Mock
+    private ConnectConf connectConf;
+
+    private SessionLoopImpl sessionLoop = new SessionLoopImpl();
+
+    @Before
+    public void before() throws IllegalAccessException {
+        FieldUtils.writeDeclaredField(sessionLoop, "matchAction", matchAction, true);
+        FieldUtils.writeDeclaredField(sessionLoop, "queueFresh", queueFresh, true);
+        FieldUtils.writeDeclaredField(sessionLoop, "inFlyCache", inFlyCache, true);
+        FieldUtils.writeDeclaredField(sessionLoop, "lmqOffsetStore", lmqOffsetStore, true);
+        FieldUtils.writeDeclaredField(sessionLoop, "lmqQueueStore", lmqQueueStore, true);
+        FieldUtils.writeDeclaredField(sessionLoop, "queueCache", queueCache, true);
+        FieldUtils.writeDeclaredField(sessionLoop, "connectConf", connectConf, true);
+
+    }
+
+    @Test
+    public void testSessionLoad() throws IllegalAccessException {
+        SessionLoopImpl spySessionLoop = spy(sessionLoop);
+
+        NioSocketChannel channel = spy(new NioSocketChannel());
+        when(channel.isActive()).thenReturn(true);
+        spySessionLoop.loadSession("test", channel);
+
+        Field field = FieldUtils.getField(SessionLoopImpl.class, "sessionMap", true);
+        Map<String, Session> sessionMap = (Map<String, Session>) field.get(spySessionLoop);
+        Assert.assertFalse(sessionMap.isEmpty());
+
+        List<Session> sessionList = spySessionLoop.getSessionList("test");
+        Assert.assertFalse(sessionList.isEmpty());
+
+        spySessionLoop.unloadSession("test", sessionMap.keySet().iterator().next());
+        Assert.assertTrue(sessionMap.isEmpty());
+    }
+
+    @Test
+    public void testAddSubscription() {
+        SessionLoopImpl spySessionLoop = spy(sessionLoop);
+        Session session = mock(Session.class);
+        NioSocketChannel channel = spy(new NioSocketChannel());
+        when(session.getChannel()).thenReturn(channel);
+        when(spySessionLoop.getSession(anyString())).thenReturn(session);
+        QueueOffset queueOffset = new QueueOffset();
+        when(session.getQueueOffset(any(), any())).thenReturn(queueOffset);
+        Map<Queue, QueueOffset> queueOffsets = new HashMap<>();
+        queueOffsets.put(new Queue(), queueOffset);
+        when(session.getQueueOffset(any())).thenReturn(queueOffsets);
+
+        CompletableFuture<Long> maxIdResult = new CompletableFuture<>();
+        maxIdResult.complete(1L);
+        when(lmqQueueStore.queryQueueMaxOffset(any())).thenReturn(maxIdResult);
+        spySessionLoop.addSubscription("test", new HashSet<>(Arrays.asList(new Subscription())));
+        Assert.assertTrue(queueOffset.isInitialized());
+    }
+
+    @Test
+    public void testNotifyPullMessage() throws InterruptedException {
+        SessionLoopImpl spySessionLoop = spy(sessionLoop);
+
+        Session session = mock(Session.class);
+        when(session.getLoadStatusMap()).thenReturn(new ConcurrentHashMap<>());
+        QueueOffset queueOffset = new QueueOffset();
+        queueOffset.setInitialized();
+        when(session.getQueueOffset(any(), any())).thenReturn(queueOffset);
+        Map<Queue, QueueOffset> queueOffsets = new HashMap<>();
+        Queue queue = new Queue();
+        queueOffsets.put(queue, queueOffset);
+        when(session.sendingMessageIsEmpty(any(), any())).thenReturn(true);
+        NioSocketChannel channel = spy(new NioSocketChannel());
+        when(session.getChannel()).thenReturn(channel);
+        when(channel.isActive()).thenReturn(true);
+
+        CompletableFuture<Map<Queue, QueueOffset>> getOffsetResult = new CompletableFuture<>();
+        when(lmqOffsetStore.getOffset(any(), any())).thenReturn(getOffsetResult);
+
+        spySessionLoop.init();
+        spySessionLoop.notifyPullMessage(session, new Subscription(), queue);
+
+        getOffsetResult.complete(queueOffsets);
+
+        Thread.sleep(1000);
+
+        verify(queueCache, atLeastOnce()).pullMessage(any(), any(), any(), any(), anyInt(), any());
+
+    }
+}
diff --git a/mqtt-ds/pom.xml b/mqtt-ds/pom.xml
new file mode 100644
index 0000000..7aac121
--- /dev/null
+++ b/mqtt-ds/pom.xml
@@ -0,0 +1,63 @@
+<?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">
+    <parent>
+        <artifactId>rocketmq-mqtt</artifactId>
+        <groupId>org.apache.rocketmq</groupId>
+        <version>1.0.0-SNAPSHOT</version>
+    </parent>
+    <modelVersion>4.0.0</modelVersion>
+
+    <artifactId>mqtt-ds</artifactId>
+    <dependencies>
+        <dependency>
+            <groupId>org.apache.rocketmq</groupId>
+            <artifactId>mqtt-common</artifactId>
+        </dependency>
+        <dependency>
+            <groupId>org.apache.rocketmq</groupId>
+            <artifactId>rocketmq-client</artifactId>
+        </dependency>
+        <dependency>
+            <groupId>org.apache.rocketmq</groupId>
+            <artifactId>rocketmq-tools</artifactId>
+        </dependency>
+        <dependency>
+            <groupId>io.netty</groupId>
+            <artifactId>netty-all</artifactId>
+        </dependency>
+        <dependency>
+            <groupId>org.springframework</groupId>
+            <artifactId>spring-core</artifactId>
+        </dependency>
+        <dependency>
+            <groupId>org.springframework</groupId>
+            <artifactId>spring-context</artifactId>
+        </dependency>
+        <dependency>
+            <groupId>org.springframework</groupId>
+            <artifactId>spring-beans</artifactId>
+        </dependency>
+        <dependency>
+            <groupId>com.github.ben-manes.caffeine</groupId>
+            <artifactId>caffeine</artifactId>
+        </dependency>
+        <dependency>
+            <groupId>junit</groupId>
+            <artifactId>junit</artifactId>
+            <scope>test</scope>
+        </dependency>
+        <dependency>
+            <groupId>org.mockito</groupId>
+            <artifactId>mockito-core</artifactId>
+            <scope>test</scope>
+        </dependency>
+    </dependencies>
+
+    <properties>
+        <maven.compiler.source>8</maven.compiler.source>
+        <maven.compiler.target>8</maven.compiler.target>
+    </properties>
+
+</project>
\ No newline at end of file
diff --git a/mqtt-ds/src/main/java/org/apache/rocketmq/mqtt/ds/auth/AuthManagerSample.java b/mqtt-ds/src/main/java/org/apache/rocketmq/mqtt/ds/auth/AuthManagerSample.java
new file mode 100644
index 0000000..bde3d1c
--- /dev/null
+++ b/mqtt-ds/src/main/java/org/apache/rocketmq/mqtt/ds/auth/AuthManagerSample.java
@@ -0,0 +1,96 @@
+/*
+ *
+ *  * Licensed to the Apache Software Foundation (ASF) under one or more
+ *  * contributor license agreements.  See the NOTICE file distributed with
+ *  * this work for additional information regarding copyright ownership.
+ *  * The ASF licenses this file to You under the Apache License, Version 2.0
+ *  * (the "License"); you may not use this file except in compliance with
+ *  * the License.  You may obtain a copy of the License at
+ *  *
+ *  *     http://www.apache.org/licenses/LICENSE-2.0
+ *  *
+ *  * Unless required by applicable law or agreed to in writing, software
+ *  * distributed under the License is distributed on an "AS IS" BASIS,
+ *  * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ *  * See the License for the specific language governing permissions and
+ *  * limitations under the License.
+ *
+ */
+
+package org.apache.rocketmq.mqtt.ds.auth;
+
+import io.netty.handler.codec.mqtt.MqttConnectMessage;
+import io.netty.handler.codec.mqtt.MqttConnectPayload;
+import io.netty.handler.codec.mqtt.MqttConnectReturnCode;
+import io.netty.handler.codec.mqtt.MqttMessage;
+import org.apache.rocketmq.common.ThreadFactoryImpl;
+import org.apache.rocketmq.mqtt.common.facade.AuthManager;
+import org.apache.rocketmq.mqtt.common.hook.AbstractUpstreamHook;
+import org.apache.rocketmq.mqtt.common.hook.HookResult;
+import org.apache.rocketmq.mqtt.common.hook.UpstreamHookEnum;
+import org.apache.rocketmq.mqtt.common.hook.UpstreamHookManager;
+import org.apache.rocketmq.mqtt.common.model.MqttMessageUpContext;
+import org.apache.rocketmq.mqtt.common.model.Remark;
+import org.apache.rocketmq.mqtt.common.util.HmacSHA1Util;
+import org.apache.rocketmq.mqtt.ds.config.ServiceConf;
+
+import javax.annotation.Resource;
+import java.util.Objects;
+import java.util.concurrent.*;
+
+/**
+ * A Sample For Auth, Check sign
+ */
+public class AuthManagerSample extends AbstractUpstreamHook implements AuthManager {
+
+    @Resource
+    private UpstreamHookManager upstreamHookManager;
+
+    @Resource
+    private ServiceConf serviceConf;
+
+    public Executor executor;
+
+    public void init() {
+        executor = new ThreadPoolExecutor(
+                8,
+                16,
+                1,
+                TimeUnit.MINUTES,
+                new LinkedBlockingQueue<>(10000),
+                new ThreadFactoryImpl("AuthHook_"));
+        register();
+    }
+
+    @Override
+    public void register() {
+        upstreamHookManager.addHook(UpstreamHookEnum.AUTH.ordinal(), this);
+    }
+
+    @Override
+    public CompletableFuture<HookResult> processMqttMessage(MqttMessageUpContext context, MqttMessage message) {
+        return CompletableFuture.supplyAsync(() -> doAuth(message), executor);
+    }
+
+    @Override
+    public HookResult doAuth(MqttMessage message) {
+        if (message instanceof MqttConnectMessage) {
+            MqttConnectMessage mqttConnectMessage = (MqttConnectMessage) message;
+            MqttConnectPayload mqttConnectPayload = mqttConnectMessage.payload();
+            String clientId = mqttConnectPayload.clientIdentifier();
+            String username = mqttConnectPayload.userName();
+            byte[] password = mqttConnectPayload.passwordInBytes();
+            boolean validateSign = false;
+            try {
... 3607 lines suppressed ...