You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@rocketmq.apache.org by ji...@apache.org on 2022/03/14 06:52:21 UTC

[rocketmq] branch 5.0.0-beta-tmp updated (f45a030 -> 190617e)

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

jinrongtong pushed a change to branch 5.0.0-beta-tmp
in repository https://gitbox.apache.org/repos/asf/rocketmq.git.


 discard f45a030  feature(test):[RIP-31]Add ITs for BrokerContainer (#3980)
 discard 79f03c9  feature(store):[RIP-34]Support quorum write and adaptive degradation in master slave architecture (#3979)
 discard 503d141  feature(broker & acl & client):[RIP-32]Support Slave Acting Master mode (#3978)
 discard a8eb8a8  feature(logging&tools&distribution&docs):[RIP-31][PART-A]Support RocketMQ BrokerContainer (#3977)
 discard a48d286  feature(container&remoting):[RIP-31][PART-A]Support RocketMQ BrokerContainer (#3976)
 discard ecc9726  feature(namesrv&common):[RIP-29]Optimize RocketMQ NameServer (#3974)
     new 467b382  feature(namesrv&common):[RIP-29]Optimize RocketMQ NameServer (#3974)
     new 27ab1df  feature(container&remoting):[RIP-31][PART-A]Support RocketMQ BrokerContainer (#3976)
     new fb9e3f0  feature(logging&tools&distribution&docs):[RIP-31][PART-A]Support RocketMQ BrokerContainer (#3977)
     new cad0cfc  feature(broker & acl & client):[RIP-32]Support Slave Acting Master mode (#3978)
     new 5c350af  feature(store):[RIP-34]Support quorum write and adaptive degradation in master slave architecture (#3979)
     new 00d2d1c  feature(test):[RIP-31]Add ITs for BrokerContainer (#3980)
     new 190617e  Delete useless code file

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

 * -- * -- B -- O -- O -- O   (f45a030)
            \
             N -- N -- N   refs/heads/5.0.0-beta-tmp (190617e)

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

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

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


Summary of changes:
 .../broker/transaction/TransactionRecord.java      |  43 --
 .../broker/transaction/TransactionStore.java       |  42 --
 .../transaction/jdbc/JDBCTransactionStore.java     | 242 ------
 .../jdbc/JDBCTransactionStoreConfig.java           |  57 --
 .../rocketmq/broker/util/ServiceProvider.java      | 201 -----
 .../processor/DefaultRequestProcessorTest.java     | 523 -------------
 .../remoting/netty/AsyncNettyRequestProcessor.java |  29 -
 .../org/apache/rocketmq/store/MessageExtBatch.java |  51 --
 .../rocketmq/store/MessageExtBrokerInner.java      |  64 --
 .../schedule/DelayOffsetSerializeWrapper.java      |  34 -
 .../store/schedule/ScheduleMessageService.java     | 819 ---------------------
 .../rocketmq/store/ScheduleMessageServiceTest.java | 194 -----
 .../store/schedule/ScheduleMessageServiceTest.java | 235 ------
 13 files changed, 2534 deletions(-)
 delete mode 100644 broker/src/main/java/org/apache/rocketmq/broker/transaction/TransactionRecord.java
 delete mode 100644 broker/src/main/java/org/apache/rocketmq/broker/transaction/TransactionStore.java
 delete mode 100644 broker/src/main/java/org/apache/rocketmq/broker/transaction/jdbc/JDBCTransactionStore.java
 delete mode 100644 broker/src/main/java/org/apache/rocketmq/broker/transaction/jdbc/JDBCTransactionStoreConfig.java
 delete mode 100644 broker/src/main/java/org/apache/rocketmq/broker/util/ServiceProvider.java
 delete mode 100644 namesrv/src/test/java/org/apache/rocketmq/namesrv/processor/DefaultRequestProcessorTest.java
 delete mode 100644 remoting/src/main/java/org/apache/rocketmq/remoting/netty/AsyncNettyRequestProcessor.java
 delete mode 100644 store/src/main/java/org/apache/rocketmq/store/MessageExtBatch.java
 delete mode 100644 store/src/main/java/org/apache/rocketmq/store/MessageExtBrokerInner.java
 delete mode 100644 store/src/main/java/org/apache/rocketmq/store/schedule/DelayOffsetSerializeWrapper.java
 delete mode 100644 store/src/main/java/org/apache/rocketmq/store/schedule/ScheduleMessageService.java
 delete mode 100644 store/src/test/java/org/apache/rocketmq/store/ScheduleMessageServiceTest.java
 delete mode 100644 store/src/test/java/org/apache/rocketmq/store/schedule/ScheduleMessageServiceTest.java

[rocketmq] 04/07: feature(broker & acl & client):[RIP-32]Support Slave Acting Master mode (#3978)

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

jinrongtong pushed a commit to branch 5.0.0-beta-tmp
in repository https://gitbox.apache.org/repos/asf/rocketmq.git

commit cad0cfcecef88958b103059aea7b98395850226a
Author: rongtong <ji...@163.com>
AuthorDate: Mon Mar 14 14:06:45 2022 +0800

    feature(broker & acl & client):[RIP-32]Support Slave Acting Master mode (#3978)
---
 .../rocketmq/acl/common/AclClientRPCHook.java      |    4 +
 acl/src/test/resources/conf/plain_acl.yml          |   17 -
 broker/pom.xml                                     |   12 +-
 .../apache/rocketmq/broker/BrokerController.java   | 1407 +++++++++++++-------
 .../org/apache/rocketmq/broker/BrokerStartup.java  |   58 +-
 ...merIdsChangeListener.java => ShutdownHook.java} |   12 +-
 .../broker/client/ClientHousekeepingService.java   |   12 +-
 .../rocketmq/broker/client/ConsumerGroupInfo.java  |   25 +-
 .../broker/client/ConsumerIdsChangeListener.java   |    2 +
 .../rocketmq/broker/client/ConsumerManager.java    |   42 +-
 .../client/DefaultConsumerIdsChangeListener.java   |   68 +-
 .../rocketmq/broker/client/ProducerManager.java    |   23 +-
 .../rocketmq/broker/client/net/Broker2Client.java  |    8 +-
 .../client/rebalance/RebalanceLockManager.java     |   90 +-
 .../broker/dledger/DLedgerRoleChangeHandler.java   |   86 +-
 .../rocketmq/broker/failover/EscapeBridge.java     |  268 ++++
 .../broker/filtersrv/FilterServerManager.java      |    6 +-
 .../rocketmq/broker/latency/BrokerFastFailure.java |   44 +-
 .../broker/loadbalance/AssignmentManager.java      |   27 +-
 .../broker/longpolling/ManyPullRequest.java        |    4 +
 .../{PopRequest.java => NotificationRequest.java}  |   42 +-
 .../longpolling/NotifyMessageArrivingListener.java |   11 +-
 .../rocketmq/broker/longpolling/PopRequest.java    |    4 +-
 .../broker/longpolling/PullRequestHoldService.java |   36 +-
 .../broker/mqtrace/AbortProcessException.java      |   69 +
 .../broker/mqtrace/ConsumeMessageContext.java      |   77 +-
 .../broker/mqtrace/SendMessageContext.java         |  102 +-
 .../broker/offset/ConsumerOffsetManager.java       |   85 +-
 .../apache/rocketmq/broker/out/BrokerOuterAPI.java |  566 +++++++-
 .../broker/pagecache/OneMessageTransfer.java       |    1 +
 .../broker/plugin/AbstractPluginMessageStore.java  |  286 +++-
 .../broker/plugin/BrokerAttachedPlugin.java        |   74 +
 .../broker/plugin/MessageStoreFactory.java         |   15 +-
 .../broker/plugin/MessageStorePluginContext.java   |   16 +-
 .../broker/plugin/PullMessageResultHandler.java    |   53 +
 .../processor/AbstractSendMessageProcessor.java    |  378 +++++-
 .../broker/processor/AckMessageProcessor.java      |   74 +-
 .../broker/processor/AdminBrokerProcessor.java     |  635 +++++++--
 .../processor/ChangeInvisibleTimeProcessor.java    |   31 +-
 .../broker/processor/ClientManageProcessor.java    |   51 +-
 .../broker/processor/ConsumerManageProcessor.java  |   26 +-
 .../processor/DefaultPullMessageResultHandler.java |  244 ++++
 .../broker/processor/EndTransactionProcessor.java  |    9 +-
 .../broker/processor/ForwardRequestProcessor.java  |    3 +-
 .../broker/processor/NotificationProcessor.java    |  318 +++++
 .../broker/processor/PeekMessageProcessor.java     |  271 ++++
 .../broker/processor/PollingInfoProcessor.java     |  119 ++
 .../broker/processor/PopBufferMergeService.java    |  111 +-
 .../broker/processor/PopMessageProcessor.java      |  227 ++--
 .../broker/processor/PopReviveService.java         |  136 +-
 .../broker/processor/PullMessageProcessor.java     |  400 +++---
 .../broker/processor/QueryMessageProcessor.java    |   14 +-
 .../broker/processor/ReplyMessageProcessor.java    |   13 +-
 .../SendMessageCallback.java}                      |   15 +-
 .../broker/processor/SendMessageProcessor.java     |  593 +++------
 .../schedule/DelayOffsetSerializeWrapper.java      |   45 +
 .../broker/schedule/ScheduleMessageService.java    |  851 ++++++++++++
 .../rocketmq/broker/slave/SlaveSynchronize.java    |   28 +-
 .../subscription/SubscriptionGroupManager.java     |  112 +-
 .../rocketmq/broker/topic/TopicConfigManager.java  |  150 ++-
 .../topic/TopicQueueMappingCleanService.java       |    3 +
 .../broker/topic/TopicQueueMappingManager.java     |    4 +-
 .../AbstractTransactionalMessageCheckListener.java |   45 +-
 .../broker/transaction/OperationResult.java        |    8 +-
 .../TransactionalMessageCheckService.java          |    3 +
 .../transaction/TransactionalMessageService.java   |    2 +-
 .../DefaultTransactionalMessageCheckListener.java  |    2 +-
 .../queue/TransactionalMessageBridge.java          |    2 +-
 .../queue/TransactionalMessageServiceImpl.java     |    2 +-
 .../org/apache/rocketmq/broker/util/HookUtils.java |  164 +++
 .../org/apache/rocketmq/broker/util/MsgUtil.java   |    4 +-
 .../rocketmq/broker/BrokerControllerTest.java      |    3 +-
 .../apache/rocketmq/broker/BrokerOuterAPITest.java |   33 +-
 .../broker/filter/MessageStoreWithFilterTest.java  |   56 +-
 .../broker/offset/ConsumerOffsetManagerTest.java   |   38 +
 .../AbstractSendMessageProcessorTest.java          |    2 -
 .../broker/processor/AckMessageProcessorTest.java  |   15 +-
 .../broker/processor/AdminBrokerProcessorTest.java |   43 +-
 .../ChangeInvisibleTimeProcessorTest.java          |   11 +-
 .../processor/ClientManageProcessorTest.java       |    1 -
 .../processor/ConsumerManageProcessorTest.java     |   91 ++
 .../processor/EndTransactionProcessorTest.java     |    2 +-
 .../processor/PopBufferMergeServiceTest.java       |    8 +-
 .../broker/processor/PopMessageProcessorTest.java  |    9 +-
 .../broker/processor/PullMessageProcessorTest.java |   18 +-
 .../processor/QueryAssignmentProcessorTest.java    |    1 -
 .../processor/ReplyMessageProcessorTest.java       |    2 +-
 .../broker/processor/SendMessageProcessorTest.java |  206 ++-
 .../schedule/ScheduleMessageServiceTest.java       |  287 ++++
 .../broker/substription/ForbiddenTest.java         |   64 +
 .../broker/topic/TopicConfigManagerTest.java       |    7 +-
 ...faultTransactionalMessageCheckListenerTest.java |    2 +-
 .../queue/TransactionalMessageBridgeTest.java      |    2 +-
 .../queue/TransactionalMessageServiceImplTest.java |    2 +-
 .../rocketmq/broker/util/ServiceProviderTest.java  |    7 +-
 .../util/TransactionalMessageServiceImpl.java      |    2 +-
 .../rocketmq/client/impl/BaseInvokeCallback.java   |    1 +
 .../client/impl/ClientRemotingProcessor.java       |    3 +-
 .../apache/rocketmq/client/impl/MQAdminImpl.java   |   44 +-
 .../rocketmq/client/impl/MQClientAPIImpl.java      |  289 +++-
 .../client/impl/factory/MQClientInstance.java      |   16 +-
 .../impl/producer/DefaultMQProducerImpl.java       |    1 -
 .../client/producer/DefaultMQProducer.java         |    6 +-
 .../consumer/DefaultLitePullConsumerTest.java      |    2 +-
 .../rocketmq/client/impl/MQClientAPIImplTest.java  |    1 -
 .../consumer/DefaultMQPushConsumerImplTest.java    |    6 -
 .../client/producer/DefaultMQProducerTest.java     |    2 +-
 .../selector/SelectMessageQueueRetryTest.java      |    1 -
 .../trace/TransactionMQProducerWithTraceTest.java  |   11 +-
 109 files changed, 7872 insertions(+), 2168 deletions(-)

diff --git a/acl/src/main/java/org/apache/rocketmq/acl/common/AclClientRPCHook.java b/acl/src/main/java/org/apache/rocketmq/acl/common/AclClientRPCHook.java
index 9e5bf1f..50c73ca 100644
--- a/acl/src/main/java/org/apache/rocketmq/acl/common/AclClientRPCHook.java
+++ b/acl/src/main/java/org/apache/rocketmq/acl/common/AclClientRPCHook.java
@@ -56,6 +56,10 @@ public class AclClientRPCHook implements RPCHook {
 
     }
 
+    @Override public void doAfterRpcFailure(String remoteAddr, RemotingCommand request, Boolean remoteTimeout) {
+
+    }
+
     protected SortedMap<String, String> parseRequestContent(RemotingCommand request, String ak, String securityToken) {
         CommandCustomHeader header = request.readCustomHeader();
         // Sort property
diff --git a/acl/src/test/resources/conf/plain_acl.yml b/acl/src/test/resources/conf/plain_acl.yml
index 59bd6d4..40d66d9 100644
--- a/acl/src/test/resources/conf/plain_acl.yml
+++ b/acl/src/test/resources/conf/plain_acl.yml
@@ -1,20 +1,3 @@
-# Licensed to the Apache Software Foundation (ASF) under one or more
-# contributor license agreements.  See the NOTICE file distributed with
-# this work for additional information regarding copyright ownership.
-# The ASF licenses this file to You under the Apache License, Version 2.0
-# (the "License"); you may not use this file except in compliance with
-# the License.  You may obtain a copy of the License at
-#
-#     http://www.apache.org/licenses/LICENSE-2.0
-#
-#  Unless required by applicable law or agreed to in writing, software
-#  distributed under the License is distributed on an "AS IS" BASIS,
-#  WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
-#  See the License for the specific language governing permissions and
-#  limitations under the License.
-
-## suggested format
-
 globalWhiteRemoteAddresses:
 - 10.10.103.*
 - 192.168.0.*
diff --git a/broker/pom.xml b/broker/pom.xml
index 31a10cc..8e52c19 100644
--- a/broker/pom.xml
+++ b/broker/pom.xml
@@ -47,14 +47,18 @@
             <artifactId>rocketmq-filter</artifactId>
         </dependency>
         <dependency>
-            <groupId>${project.groupId}</groupId>
-            <artifactId>rocketmq-acl</artifactId>
+            <groupId>commons-io</groupId>
+            <artifactId>commons-io</artifactId>
         </dependency>
         <dependency>
             <groupId>ch.qos.logback</groupId>
             <artifactId>logback-classic</artifactId>
         </dependency>
         <dependency>
+            <groupId>ch.qos.logback</groupId>
+            <artifactId>logback-core</artifactId>
+        </dependency>
+        <dependency>
             <groupId>com.alibaba</groupId>
             <artifactId>fastjson</artifactId>
         </dependency>
@@ -74,6 +78,10 @@
             <groupId>com.googlecode.concurrentlinkedhashmap</groupId>
             <artifactId>concurrentlinkedhashmap-lru</artifactId>
         </dependency>
+        <dependency>
+            <groupId>org.apache.rocketmq</groupId>
+            <artifactId>rocketmq-acl</artifactId>
+        </dependency>
     </dependencies>
 
     <build>
diff --git a/broker/src/main/java/org/apache/rocketmq/broker/BrokerController.java b/broker/src/main/java/org/apache/rocketmq/broker/BrokerController.java
index 210e180..2d06f19 100644
--- a/broker/src/main/java/org/apache/rocketmq/broker/BrokerController.java
+++ b/broker/src/main/java/org/apache/rocketmq/broker/BrokerController.java
@@ -16,7 +16,29 @@
  */
 package org.apache.rocketmq.broker;
 
-import com.google.common.collect.Maps;
+import java.io.IOException;
+import java.net.InetSocketAddress;
+import java.util.AbstractMap;
+import java.util.ArrayList;
+import java.util.Collections;
+import java.util.HashMap;
+import java.util.List;
+import java.util.Map;
+import java.util.Objects;
+import java.util.Optional;
+import java.util.concurrent.BlockingQueue;
+import java.util.concurrent.ConcurrentHashMap;
+import java.util.concurrent.ConcurrentMap;
+import java.util.concurrent.CountDownLatch;
+import java.util.concurrent.ExecutorService;
+import java.util.concurrent.LinkedBlockingQueue;
+import java.util.concurrent.ScheduledExecutorService;
+import java.util.concurrent.ScheduledThreadPoolExecutor;
+import java.util.concurrent.TimeUnit;
+import java.util.function.Function;
+import java.util.stream.Collectors;
+
+import org.apache.commons.io.FilenameUtils;
 import org.apache.rocketmq.acl.AccessValidator;
 import org.apache.rocketmq.broker.client.ClientHousekeepingService;
 import org.apache.rocketmq.broker.client.ConsumerIdsChangeListener;
@@ -26,6 +48,7 @@ import org.apache.rocketmq.broker.client.ProducerManager;
 import org.apache.rocketmq.broker.client.net.Broker2Client;
 import org.apache.rocketmq.broker.client.rebalance.RebalanceLockManager;
 import org.apache.rocketmq.broker.dledger.DLedgerRoleChangeHandler;
+import org.apache.rocketmq.broker.failover.EscapeBridge;
 import org.apache.rocketmq.broker.filter.CommitLogDispatcherCalcBitMap;
 import org.apache.rocketmq.broker.filter.ConsumerFilterManager;
 import org.apache.rocketmq.broker.filtersrv.FilterServerManager;
@@ -41,6 +64,7 @@ import org.apache.rocketmq.broker.offset.ConsumerOffsetManager;
 import org.apache.rocketmq.broker.offset.ConsumerOrderInfoManager;
 import org.apache.rocketmq.broker.offset.LmqConsumerOffsetManager;
 import org.apache.rocketmq.broker.out.BrokerOuterAPI;
+import org.apache.rocketmq.broker.plugin.BrokerAttachedPlugin;
 import org.apache.rocketmq.broker.plugin.MessageStoreFactory;
 import org.apache.rocketmq.broker.plugin.MessageStorePluginContext;
 import org.apache.rocketmq.broker.processor.AckMessageProcessor;
@@ -49,12 +73,16 @@ import org.apache.rocketmq.broker.processor.ChangeInvisibleTimeProcessor;
 import org.apache.rocketmq.broker.processor.ClientManageProcessor;
 import org.apache.rocketmq.broker.processor.ConsumerManageProcessor;
 import org.apache.rocketmq.broker.processor.EndTransactionProcessor;
+import org.apache.rocketmq.broker.processor.NotificationProcessor;
+import org.apache.rocketmq.broker.processor.PeekMessageProcessor;
+import org.apache.rocketmq.broker.processor.PollingInfoProcessor;
 import org.apache.rocketmq.broker.processor.PopMessageProcessor;
 import org.apache.rocketmq.broker.processor.PullMessageProcessor;
 import org.apache.rocketmq.broker.processor.QueryAssignmentProcessor;
 import org.apache.rocketmq.broker.processor.QueryMessageProcessor;
 import org.apache.rocketmq.broker.processor.ReplyMessageProcessor;
 import org.apache.rocketmq.broker.processor.SendMessageProcessor;
+import org.apache.rocketmq.broker.schedule.ScheduleMessageService;
 import org.apache.rocketmq.broker.slave.SlaveSynchronize;
 import org.apache.rocketmq.broker.subscription.LmqSubscriptionGroupManager;
 import org.apache.rocketmq.broker.subscription.SubscriptionGroupManager;
@@ -68,22 +96,27 @@ import org.apache.rocketmq.broker.transaction.TransactionalMessageService;
 import org.apache.rocketmq.broker.transaction.queue.DefaultTransactionalMessageCheckListener;
 import org.apache.rocketmq.broker.transaction.queue.TransactionalMessageBridge;
 import org.apache.rocketmq.broker.transaction.queue.TransactionalMessageServiceImpl;
-import org.apache.rocketmq.broker.util.ServiceProvider;
+import org.apache.rocketmq.broker.util.HookUtils;
 import org.apache.rocketmq.common.BrokerConfig;
 import org.apache.rocketmq.common.Configuration;
 import org.apache.rocketmq.common.DataVersion;
+import org.apache.rocketmq.common.MixAll;
 import org.apache.rocketmq.common.ThreadFactoryImpl;
 import org.apache.rocketmq.common.TopicConfig;
 import org.apache.rocketmq.common.UtilAll;
 import org.apache.rocketmq.common.constant.LoggerName;
 import org.apache.rocketmq.common.constant.PermName;
+import org.apache.rocketmq.common.message.MessageExt;
 import org.apache.rocketmq.common.namesrv.RegisterBrokerResult;
+import org.apache.rocketmq.common.protocol.NamespaceUtil;
 import org.apache.rocketmq.common.protocol.RequestCode;
+import org.apache.rocketmq.common.protocol.body.BrokerMemberGroup;
 import org.apache.rocketmq.common.protocol.body.TopicConfigAndMappingSerializeWrapper;
 import org.apache.rocketmq.common.protocol.body.TopicConfigSerializeWrapper;
 import org.apache.rocketmq.common.statictopic.TopicQueueMappingDetail;
 import org.apache.rocketmq.common.statictopic.TopicQueueMappingInfo;
 import org.apache.rocketmq.common.stats.MomentStatsItem;
+import org.apache.rocketmq.common.utils.ServiceProvider;
 import org.apache.rocketmq.logging.InternalLogger;
 import org.apache.rocketmq.logging.InternalLoggerFactory;
 import org.apache.rocketmq.remoting.RPCHook;
@@ -99,116 +132,129 @@ import org.apache.rocketmq.remoting.protocol.RemotingCommand;
 import org.apache.rocketmq.srvutil.FileWatchService;
 import org.apache.rocketmq.store.DefaultMessageStore;
 import org.apache.rocketmq.store.MessageArrivingListener;
+import org.apache.rocketmq.common.message.MessageExtBrokerInner;
 import org.apache.rocketmq.store.MessageStore;
+import org.apache.rocketmq.store.PutMessageResult;
 import org.apache.rocketmq.store.config.BrokerRole;
 import org.apache.rocketmq.store.config.MessageStoreConfig;
 import org.apache.rocketmq.store.dledger.DLedgerCommitLog;
+import org.apache.rocketmq.store.hook.PutMessageHook;
+import org.apache.rocketmq.store.hook.SendMessageBackHook;
 import org.apache.rocketmq.store.stats.BrokerStats;
 import org.apache.rocketmq.store.stats.BrokerStatsManager;
-
-import java.io.IOException;
-import java.net.InetSocketAddress;
-import java.util.AbstractMap;
-import java.util.ArrayList;
-import java.util.Collections;
-import java.util.HashMap;
-import java.util.Iterator;
-import java.util.List;
-import java.util.Map;
-import java.util.Objects;
-import java.util.Optional;
-import java.util.concurrent.BlockingQueue;
-import java.util.concurrent.ConcurrentHashMap;
-import java.util.concurrent.ConcurrentMap;
-import java.util.concurrent.ExecutorService;
-import java.util.concurrent.Executors;
-import java.util.concurrent.Future;
-import java.util.concurrent.LinkedBlockingQueue;
-import java.util.concurrent.ScheduledExecutorService;
-import java.util.concurrent.ThreadPoolExecutor;
-import java.util.concurrent.TimeUnit;
-import java.util.function.Function;
-import java.util.stream.Collectors;
 import org.apache.rocketmq.store.stats.LmqBrokerStatsManager;
 
 public class BrokerController {
-    private static final InternalLogger log = InternalLoggerFactory.getLogger(LoggerName.BROKER_LOGGER_NAME);
+    protected static final InternalLogger LOG = InternalLoggerFactory.getLogger(LoggerName.BROKER_LOGGER_NAME);
     private static final InternalLogger LOG_PROTECTION = InternalLoggerFactory.getLogger(LoggerName.PROTECTION_LOGGER_NAME);
     private static final InternalLogger LOG_WATER_MARK = InternalLoggerFactory.getLogger(LoggerName.WATER_MARK_LOGGER_NAME);
-    private final BrokerConfig brokerConfig;
+    protected static final int HA_ADDRESS_MIN_LENGTH = 6;
+
+    protected final BrokerConfig brokerConfig;
     private final NettyServerConfig nettyServerConfig;
     private final NettyClientConfig nettyClientConfig;
-    private final MessageStoreConfig messageStoreConfig;
-    private final ConsumerOffsetManager consumerOffsetManager;
-    private final ConsumerManager consumerManager;
-    private final ConsumerFilterManager consumerFilterManager;
-    private final ConsumerOrderInfoManager consumerOrderInfoManager;
-    private final ProducerManager producerManager;
-    private final AssignmentManager assignmentManager;
-    private final ClientHousekeepingService clientHousekeepingService;
-
-    private final PullMessageProcessor pullMessageProcessor;
-    private final PopMessageProcessor popMessageProcessor;
-    private final AckMessageProcessor ackMessageProcessor;
-    private final ChangeInvisibleTimeProcessor changeInvisibleTimeProcessor;
-    private final QueryAssignmentProcessor queryAssignmentProcessor;
-    private final ClientManageProcessor clientManageProcessor;
-    private final SendMessageProcessor sendMessageProcessor;
-    private final PullRequestHoldService pullRequestHoldService;
-    private final MessageArrivingListener messageArrivingListener;
-    private final Broker2Client broker2Client;
-    private final SubscriptionGroupManager subscriptionGroupManager;
-    private final ConsumerIdsChangeListener consumerIdsChangeListener;
+    protected final MessageStoreConfig messageStoreConfig;
+    protected final ConsumerOffsetManager consumerOffsetManager;
+    protected final ConsumerManager consumerManager;
+    protected final ConsumerFilterManager consumerFilterManager;
+    protected final ConsumerOrderInfoManager consumerOrderInfoManager;
+    protected final ProducerManager producerManager;
+    protected final ScheduleMessageService scheduleMessageService;
+    protected final AssignmentManager assignmentManager;
+    protected final ClientHousekeepingService clientHousekeepingService;
+    protected final PullMessageProcessor pullMessageProcessor;
+    protected final PeekMessageProcessor peekMessageProcessor;
+    protected final PopMessageProcessor popMessageProcessor;
+    protected final AckMessageProcessor ackMessageProcessor;
+    protected final ChangeInvisibleTimeProcessor changeInvisibleTimeProcessor;
+    protected final NotificationProcessor notificationProcessor;
+    protected final PollingInfoProcessor pollingInfoProcessor;
+    protected final QueryAssignmentProcessor queryAssignmentProcessor;
+    protected final ClientManageProcessor clientManageProcessor;
+    protected final SendMessageProcessor sendMessageProcessor;
+    protected final ReplyMessageProcessor replyMessageProcessor;
+    protected final PullRequestHoldService pullRequestHoldService;
+    protected final MessageArrivingListener messageArrivingListener;
+    protected final Broker2Client broker2Client;
+    protected final SubscriptionGroupManager subscriptionGroupManager;
+    protected final ConsumerIdsChangeListener consumerIdsChangeListener;
     private final RebalanceLockManager rebalanceLockManager = new RebalanceLockManager();
-    private final BrokerOuterAPI brokerOuterAPI;
-    private final ScheduledExecutorService scheduledExecutorService = Executors.newSingleThreadScheduledExecutor(new ThreadFactoryImpl(
-        "BrokerControllerScheduledThread"));
-    private final SlaveSynchronize slaveSynchronize;
-    private final BlockingQueue<Runnable> sendThreadPoolQueue;
-    private final BlockingQueue<Runnable> putThreadPoolQueue;
-    private final BlockingQueue<Runnable> ackThreadPoolQueue;
-    private final BlockingQueue<Runnable> pullThreadPoolQueue;
-    private final BlockingQueue<Runnable> replyThreadPoolQueue;
-    private final BlockingQueue<Runnable> queryThreadPoolQueue;
-    private final BlockingQueue<Runnable> clientManagerThreadPoolQueue;
-    private final BlockingQueue<Runnable> heartbeatThreadPoolQueue;
-    private final BlockingQueue<Runnable> consumerManagerThreadPoolQueue;
-    private final BlockingQueue<Runnable> endTransactionThreadPoolQueue;
-    private final FilterServerManager filterServerManager;
-    private final BrokerStatsManager brokerStatsManager;
-    private final List<SendMessageHook> sendMessageHookList = new ArrayList<SendMessageHook>();
-    private final List<ConsumeMessageHook> consumeMessageHookList = new ArrayList<ConsumeMessageHook>();
-    private final ConcurrentMap<String, String> brokerName2AddrMap = Maps.newConcurrentMap();
-    private MessageStore messageStore;
-    private RemotingServer remotingServer;
-    private RemotingServer fastRemotingServer;
-    private TopicConfigManager topicConfigManager;
-    private TopicQueueMappingManager topicQueueMappingManager;
-    private ExecutorService sendMessageExecutor;
-    private ExecutorService putMessageFutureExecutor;
-    private ExecutorService pullMessageExecutor;
-    private ExecutorService ackMessageExecutor;
-    private ExecutorService replyMessageExecutor;
-    private ExecutorService queryMessageExecutor;
-    private ExecutorService adminBrokerExecutor;
-    private ExecutorService clientManageExecutor;
-    private ExecutorService heartbeatExecutor;
-    private ExecutorService consumerManageExecutor;
-    private ExecutorService loadBalanceExecutor;
-    private ExecutorService endTransactionExecutor;
-    private boolean updateMasterHAServerAddrPeriodically = false;
+    protected BrokerOuterAPI brokerOuterAPI;
+    protected ScheduledExecutorService scheduledExecutorService;
+    protected final SlaveSynchronize slaveSynchronize;
+    protected final BlockingQueue<Runnable> sendThreadPoolQueue;
+    protected final BlockingQueue<Runnable> putThreadPoolQueue;
+    protected final BlockingQueue<Runnable> ackThreadPoolQueue;
+    protected final BlockingQueue<Runnable> pullThreadPoolQueue;
+    protected final BlockingQueue<Runnable> litePullThreadPoolQueue;
+    protected final BlockingQueue<Runnable> replyThreadPoolQueue;
+    protected final BlockingQueue<Runnable> queryThreadPoolQueue;
+    protected final BlockingQueue<Runnable> clientManagerThreadPoolQueue;
+    protected final BlockingQueue<Runnable> heartbeatThreadPoolQueue;
+    protected final BlockingQueue<Runnable> consumerManagerThreadPoolQueue;
+    protected final BlockingQueue<Runnable> endTransactionThreadPoolQueue;
+    protected final BlockingQueue<Runnable> adminBrokerThreadPoolQueue;
+    protected final BlockingQueue<Runnable> loadBalanceThreadPoolQueue;
+    protected final FilterServerManager filterServerManager;
+    protected final BrokerStatsManager brokerStatsManager;
+    protected final List<SendMessageHook> sendMessageHookList = new ArrayList<SendMessageHook>();
+    protected final List<ConsumeMessageHook> consumeMessageHookList = new ArrayList<ConsumeMessageHook>();
+    protected MessageStore messageStore;
+    protected RemotingServer remotingServer;
+    protected CountDownLatch remotingServerStartLatch;
+    protected RemotingServer fastRemotingServer;
+    protected TopicConfigManager topicConfigManager;
+    protected TopicQueueMappingManager topicQueueMappingManager;
+    protected ExecutorService sendMessageExecutor;
+    protected ExecutorService pullMessageExecutor;
+    protected ExecutorService litePullMessageExecutor;
+    protected ExecutorService putMessageFutureExecutor;
+    protected ExecutorService ackMessageExecutor;
+    protected ExecutorService replyMessageExecutor;
+    protected ExecutorService queryMessageExecutor;
+    protected ExecutorService adminBrokerExecutor;
+    protected ExecutorService clientManageExecutor;
+    protected ExecutorService heartbeatExecutor;
+    protected ExecutorService consumerManageExecutor;
+    protected ExecutorService loadBalanceExecutor;
+    protected ExecutorService endTransactionExecutor;
+    protected boolean updateMasterHAServerAddrPeriodically = false;
     private BrokerStats brokerStats;
     private InetSocketAddress storeHost;
-    private BrokerFastFailure brokerFastFailure;
+    protected BrokerFastFailure brokerFastFailure;
     private Configuration configuration;
-    private TopicQueueMappingCleanService topicQueueMappingCleanService;
-    private FileWatchService fileWatchService;
-    private TransactionalMessageCheckService transactionalMessageCheckService;
-    private TransactionalMessageService transactionalMessageService;
-    private AbstractTransactionalMessageCheckListener transactionalMessageCheckListener;
-    private Future<?> slaveSyncFuture;
-    private Map<Class, AccessValidator> accessValidatorMap = new HashMap<Class, AccessValidator>();
-    private long shouldStartTime;
+    protected TopicQueueMappingCleanService topicQueueMappingCleanService;
+    protected FileWatchService fileWatchService;
+    protected TransactionalMessageCheckService transactionalMessageCheckService;
+    protected TransactionalMessageService transactionalMessageService;
+    protected AbstractTransactionalMessageCheckListener transactionalMessageCheckListener;
+    protected Map<Class, AccessValidator> accessValidatorMap = new HashMap<Class, AccessValidator>();
+    protected volatile boolean shutdown = false;
+    protected ShutdownHook shutdownHook;
+    private volatile boolean isScheduleServiceStart = false;
+    private volatile boolean isTransactionCheckServiceStart = false;
+    protected volatile BrokerMemberGroup brokerMemberGroup;
+    protected EscapeBridge escapeBridge;
+    protected List<BrokerAttachedPlugin> brokerAttachedPlugins = new ArrayList<>();
+    protected volatile long shouldStartTime;
+
+    public BrokerController(
+        final BrokerConfig brokerConfig,
+        final NettyServerConfig nettyServerConfig,
+        final NettyClientConfig nettyClientConfig,
+        final MessageStoreConfig messageStoreConfig,
+        final ShutdownHook shutdownHook
+    ) {
+        this(brokerConfig, nettyServerConfig, nettyClientConfig, messageStoreConfig);
+        this.shutdownHook = shutdownHook;
+    }
+
+    public BrokerController(
+        final BrokerConfig brokerConfig,
+        final MessageStoreConfig messageStoreConfig
+    ) {
+        this(brokerConfig, null, null, messageStoreConfig);
+    }
 
     public BrokerController(
         final BrokerConfig brokerConfig,
@@ -220,26 +266,36 @@ public class BrokerController {
         this.nettyServerConfig = nettyServerConfig;
         this.nettyClientConfig = nettyClientConfig;
         this.messageStoreConfig = messageStoreConfig;
+        this.setStoreHost(new InetSocketAddress(this.getBrokerConfig().getBrokerIP1(), getListenPort()));
+        this.brokerStatsManager = messageStoreConfig.isEnableLmq() ? new LmqBrokerStatsManager(this.brokerConfig.getBrokerClusterName(), this.brokerConfig.isEnableDetailStat()) : new BrokerStatsManager(this.brokerConfig.getBrokerClusterName(), this.brokerConfig.isEnableDetailStat());
         this.consumerOffsetManager = messageStoreConfig.isEnableLmq() ? new LmqConsumerOffsetManager(this) : new ConsumerOffsetManager(this);
         this.topicConfigManager = messageStoreConfig.isEnableLmq() ? new LmqTopicConfigManager(this) : new TopicConfigManager(this);
         this.topicQueueMappingManager = new TopicQueueMappingManager(this);
         this.pullMessageProcessor = new PullMessageProcessor(this);
+        this.peekMessageProcessor = new PeekMessageProcessor(this);
         this.pullRequestHoldService = messageStoreConfig.isEnableLmq() ? new LmqPullRequestHoldService(this) : new PullRequestHoldService(this);
         this.popMessageProcessor = new PopMessageProcessor(this);
+        this.notificationProcessor = new NotificationProcessor(this);
+        this.pollingInfoProcessor = new PollingInfoProcessor(this);
         this.ackMessageProcessor = new AckMessageProcessor(this);
         this.changeInvisibleTimeProcessor = new ChangeInvisibleTimeProcessor(this);
         this.sendMessageProcessor = new SendMessageProcessor(this);
-        this.messageArrivingListener = new NotifyMessageArrivingListener(this.pullRequestHoldService,
-            this.popMessageProcessor);
+        this.replyMessageProcessor = new ReplyMessageProcessor(this);
+        this.messageArrivingListener = new NotifyMessageArrivingListener(this.pullRequestHoldService, this.popMessageProcessor, this.notificationProcessor);
         this.consumerIdsChangeListener = new DefaultConsumerIdsChangeListener(this);
-        this.consumerManager = new ConsumerManager(this.consumerIdsChangeListener);
+        this.consumerManager = new ConsumerManager(this.consumerIdsChangeListener, this.brokerStatsManager);
+        this.producerManager = new ProducerManager(this.brokerStatsManager);
         this.consumerFilterManager = new ConsumerFilterManager(this);
         this.consumerOrderInfoManager = new ConsumerOrderInfoManager(this);
-        this.producerManager = new ProducerManager();
         this.clientHousekeepingService = new ClientHousekeepingService(this);
         this.broker2Client = new Broker2Client(this);
         this.subscriptionGroupManager = messageStoreConfig.isEnableLmq() ? new LmqSubscriptionGroupManager(this) : new SubscriptionGroupManager(this);
-        this.brokerOuterAPI = new BrokerOuterAPI(nettyClientConfig, this);
+        this.scheduleMessageService = new ScheduleMessageService(this);
+
+        if (nettyClientConfig != null) {
+            this.brokerOuterAPI = new BrokerOuterAPI(nettyClientConfig);
+        }
+
         this.filterServerManager = new FilterServerManager(this);
 
         this.assignmentManager = new AssignmentManager(this);
@@ -250,6 +306,8 @@ public class BrokerController {
         this.sendThreadPoolQueue = new LinkedBlockingQueue<Runnable>(this.brokerConfig.getSendThreadPoolQueueCapacity());
         this.putThreadPoolQueue = new LinkedBlockingQueue<Runnable>(this.brokerConfig.getPutThreadPoolQueueCapacity());
         this.pullThreadPoolQueue = new LinkedBlockingQueue<Runnable>(this.brokerConfig.getPullThreadPoolQueueCapacity());
+        this.litePullThreadPoolQueue = new LinkedBlockingQueue<Runnable>(this.brokerConfig.getLitePullThreadPoolQueueCapacity());
+
         this.ackThreadPoolQueue = new LinkedBlockingQueue<Runnable>(this.brokerConfig.getAckThreadPoolQueueCapacity());
         this.replyThreadPoolQueue = new LinkedBlockingQueue<Runnable>(this.brokerConfig.getReplyThreadPoolQueueCapacity());
         this.queryThreadPoolQueue = new LinkedBlockingQueue<Runnable>(this.brokerConfig.getQueryThreadPoolQueueCapacity());
@@ -257,25 +315,51 @@ public class BrokerController {
         this.consumerManagerThreadPoolQueue = new LinkedBlockingQueue<Runnable>(this.brokerConfig.getConsumerManagerThreadPoolQueueCapacity());
         this.heartbeatThreadPoolQueue = new LinkedBlockingQueue<Runnable>(this.brokerConfig.getHeartbeatThreadPoolQueueCapacity());
         this.endTransactionThreadPoolQueue = new LinkedBlockingQueue<Runnable>(this.brokerConfig.getEndTransactionPoolQueueCapacity());
-
-        this.brokerStatsManager = messageStoreConfig.isEnableLmq() ? new LmqBrokerStatsManager(this.brokerConfig.getBrokerClusterName(), this.brokerConfig.isEnableDetailStat()) : new BrokerStatsManager(this.brokerConfig.getBrokerClusterName(), this.brokerConfig.isEnableDetailStat());
-
-        this.setStoreHost(new InetSocketAddress(this.getBrokerConfig().getBrokerIP1(), this.getNettyServerConfig().getListenPort()));
+        this.adminBrokerThreadPoolQueue = new LinkedBlockingQueue<Runnable>(this.brokerConfig.getAdminBrokerThreadPoolQueueCapacity());
+        this.loadBalanceThreadPoolQueue = new LinkedBlockingQueue<Runnable>(this.brokerConfig.getLoadBalanceThreadPoolQueueCapacity());
 
         this.brokerFastFailure = new BrokerFastFailure(this);
+
+        String brokerConfigPath;
+        if (brokerConfig.getBrokerConfigPath() != null && !brokerConfig.getBrokerConfigPath().isEmpty()) {
+            brokerConfigPath = brokerConfig.getBrokerConfigPath();
+        } else {
+            brokerConfigPath = FilenameUtils.concat(
+                FilenameUtils.getFullPathNoEndSeparator(BrokerPathConfigHelper.getBrokerConfigPath()),
+                this.brokerConfig.getCanonicalName() + ".properties");
+        }
         this.configuration = new Configuration(
-            log,
-            BrokerPathConfigHelper.getBrokerConfigPath(),
+            LOG,
+            brokerConfigPath,
             this.brokerConfig, this.nettyServerConfig, this.nettyClientConfig, this.messageStoreConfig
         );
-    }
 
-    public ConsumerIdsChangeListener getConsumerIdsChangeListener() {
-        return consumerIdsChangeListener;
-    }
+        this.brokerStatsManager.setProduerStateGetter(new BrokerStatsManager.StateGetter() {
+            @Override
+            public boolean online(String instanceId, String group, String topic) {
+                if (getTopicConfigManager().getTopicConfigTable().containsKey(NamespaceUtil.wrapNamespace(instanceId, topic))) {
+                    return getProducerManager().groupOnline(NamespaceUtil.wrapNamespace(instanceId, group));
+                } else {
+                    return getProducerManager().groupOnline(group);
+                }
+            }
+        });
+        this.brokerStatsManager.setConsumerStateGetter(new BrokerStatsManager.StateGetter() {
+            @Override
+            public boolean online(String instanceId, String group, String topic) {
+                String topicFullName = NamespaceUtil.wrapNamespace(instanceId, topic);
+                if (getTopicConfigManager().getTopicConfigTable().containsKey(topicFullName)) {
+                    return getConsumerManager().findSubscriptionData(NamespaceUtil.wrapNamespace(instanceId, group), topicFullName) != null;
+                } else {
+                    return getConsumerManager().findSubscriptionData(group, topic) != null;
+                }
+            }
+        });
+
+        this.brokerMemberGroup = new BrokerMemberGroup(this.brokerConfig.getBrokerClusterName(), this.brokerConfig.getBrokerName());
+        this.brokerMemberGroup.getBrokerAddrs().put(this.brokerConfig.getBrokerId(), this.getBrokerAddr());
 
-    public ClientManageProcessor getClientManageProcessor() {
-        return clientManageProcessor;
+        this.escapeBridge = new EscapeBridge(this);
     }
 
     public BrokerConfig getBrokerConfig() {
@@ -286,6 +370,10 @@ public class BrokerController {
         return nettyServerConfig;
     }
 
+    public NettyClientConfig getNettyClientConfig() {
+        return nettyClientConfig;
+    }
+
     public BlockingQueue<Runnable> getPullThreadPoolQueue() {
         return pullThreadPoolQueue;
     }
@@ -294,245 +382,342 @@ public class BrokerController {
         return queryThreadPoolQueue;
     }
 
-    public boolean initialize() throws CloneNotSupportedException {
-        boolean result = this.topicConfigManager.load();
+    protected void initializeRemotingServer() throws CloneNotSupportedException {
+        this.remotingServer = new NettyRemotingServer(this.nettyServerConfig, this.clientHousekeepingService);
+        NettyServerConfig fastConfig = (NettyServerConfig) this.nettyServerConfig.clone();
+        fastConfig.setListenPort(nettyServerConfig.getListenPort() - 2);
+        this.fastRemotingServer = new NettyRemotingServer(fastConfig, this.clientHousekeepingService);
+    }
+
+    /**
+     * Initialize resources including remoting server and thread executors.
+     */
+    protected void initializeResources() {
+        this.scheduledExecutorService = new ScheduledThreadPoolExecutor(1,
+            new ThreadFactoryImpl("BrokerControllerScheduledThread", true, brokerConfig));
+
+        this.sendMessageExecutor = new BrokerFixedThreadPoolExecutor(
+            this.brokerConfig.getSendMessageThreadPoolNums(),
+            this.brokerConfig.getSendMessageThreadPoolNums(),
+            1000 * 60,
+            TimeUnit.MILLISECONDS,
+            this.sendThreadPoolQueue,
+            new ThreadFactoryImpl("SendMessageThread_", brokerConfig));
+
+        this.pullMessageExecutor = new BrokerFixedThreadPoolExecutor(
+            this.brokerConfig.getPullMessageThreadPoolNums(),
+            this.brokerConfig.getPullMessageThreadPoolNums(),
+            1000 * 60,
+            TimeUnit.MILLISECONDS,
+            this.pullThreadPoolQueue,
+            new ThreadFactoryImpl("PullMessageThread_", brokerConfig));
+
+        this.litePullMessageExecutor = new BrokerFixedThreadPoolExecutor(
+            this.brokerConfig.getLitePullMessageThreadPoolNums(),
+            this.brokerConfig.getLitePullMessageThreadPoolNums(),
+            1000 * 60,
+            TimeUnit.MILLISECONDS,
+            this.litePullThreadPoolQueue,
+            new ThreadFactoryImpl("LitePullMessageThread_", brokerConfig));
+
+        this.putMessageFutureExecutor = new BrokerFixedThreadPoolExecutor(
+            this.brokerConfig.getPutMessageFutureThreadPoolNums(),
+            this.brokerConfig.getPutMessageFutureThreadPoolNums(),
+            1000 * 60,
+            TimeUnit.MILLISECONDS,
+            this.putThreadPoolQueue,
+            new ThreadFactoryImpl("SendMessageThread_", brokerConfig));
+
+        this.ackMessageExecutor = new BrokerFixedThreadPoolExecutor(
+            this.brokerConfig.getAckMessageThreadPoolNums(),
+            this.brokerConfig.getAckMessageThreadPoolNums(),
+            1000 * 60,
+            TimeUnit.MILLISECONDS,
+            this.ackThreadPoolQueue,
+            new ThreadFactoryImpl("AckMessageThread_", brokerConfig));
+
+        this.queryMessageExecutor = new BrokerFixedThreadPoolExecutor(
+            this.brokerConfig.getQueryMessageThreadPoolNums(),
+            this.brokerConfig.getQueryMessageThreadPoolNums(),
+            1000 * 60,
+            TimeUnit.MILLISECONDS,
+            this.queryThreadPoolQueue,
+            new ThreadFactoryImpl("QueryMessageThread_", brokerConfig));
+
+        this.adminBrokerExecutor = new BrokerFixedThreadPoolExecutor(
+            this.brokerConfig.getAdminBrokerThreadPoolNums(),
+            this.brokerConfig.getAdminBrokerThreadPoolNums(),
+            1000 * 60,
+            TimeUnit.MILLISECONDS,
+            this.adminBrokerThreadPoolQueue,
+            new ThreadFactoryImpl("AdminBrokerThread_", brokerConfig));
+
+        this.clientManageExecutor = new BrokerFixedThreadPoolExecutor(
+            this.brokerConfig.getClientManageThreadPoolNums(),
+            this.brokerConfig.getClientManageThreadPoolNums(),
+            1000 * 60,
+            TimeUnit.MILLISECONDS,
+            this.clientManagerThreadPoolQueue,
+            new ThreadFactoryImpl("ClientManageThread_", brokerConfig));
+
+        this.heartbeatExecutor = new BrokerFixedThreadPoolExecutor(
+            this.brokerConfig.getHeartbeatThreadPoolNums(),
+            this.brokerConfig.getHeartbeatThreadPoolNums(),
+            1000 * 60,
+            TimeUnit.MILLISECONDS,
+            this.heartbeatThreadPoolQueue,
+            new ThreadFactoryImpl("HeartbeatThread_", true, brokerConfig));
+
+        this.consumerManageExecutor = new BrokerFixedThreadPoolExecutor(
+            this.brokerConfig.getConsumerManageThreadPoolNums(),
+            this.brokerConfig.getConsumerManageThreadPoolNums(),
+            1000 * 60,
+            TimeUnit.MILLISECONDS,
+            this.consumerManagerThreadPoolQueue,
+            new ThreadFactoryImpl("ConsumerManageThread_", true, brokerConfig));
+
+        this.replyMessageExecutor = new BrokerFixedThreadPoolExecutor(
+            this.brokerConfig.getProcessReplyMessageThreadPoolNums(),
+            this.brokerConfig.getProcessReplyMessageThreadPoolNums(),
+            1000 * 60,
+            TimeUnit.MILLISECONDS,
+            this.replyThreadPoolQueue,
+            new ThreadFactoryImpl("ProcessReplyMessageThread_", brokerConfig));
+
+        this.endTransactionExecutor = new BrokerFixedThreadPoolExecutor(
+            this.brokerConfig.getEndTransactionThreadPoolNums(),
+            this.brokerConfig.getEndTransactionThreadPoolNums(),
+            1000 * 60,
+            TimeUnit.MILLISECONDS,
+            this.endTransactionThreadPoolQueue,
+            new ThreadFactoryImpl("EndTransactionThread_", brokerConfig));
+
+        this.loadBalanceExecutor = new BrokerFixedThreadPoolExecutor(
+            this.brokerConfig.getLoadBalanceProcessorThreadPoolNums(),
+            this.brokerConfig.getLoadBalanceProcessorThreadPoolNums(),
+            1000 * 60,
+            TimeUnit.MILLISECONDS,
+            this.loadBalanceThreadPoolQueue,
+            new ThreadFactoryImpl("LoadBalanceProcessorThread_", brokerConfig));
+
+        this.topicQueueMappingCleanService = new TopicQueueMappingCleanService(this);
+    }
+
+    protected void initializeBrokerScheduledTasks() {
+        final long initialDelay = UtilAll.computeNextMorningTimeMillis() - System.currentTimeMillis();
+        final long period = 1000 * 60 * 60 * 24;
+        this.scheduledExecutorService.scheduleAtFixedRate(new Runnable() {
+            @Override
+            public void run() {
+                try {
+                    BrokerController.this.getBrokerStats().record();
+                } catch (Throwable e) {
+                    LOG.error("BrokerController: failed to record broker stats", e);
+                }
+            }
+        }, initialDelay, period, TimeUnit.MILLISECONDS);
 
-        result = result && this.topicQueueMappingManager.load();
+        this.scheduledExecutorService.scheduleAtFixedRate(new Runnable() {
+            @Override
+            public void run() {
+                try {
+                    BrokerController.this.consumerOffsetManager.persist();
+                } catch (Throwable e) {
+                    LOG.error(
+                        "BrokerController: failed to persist config file of consumerOffset", e);
+                }
+            }
+        }, 1000 * 10, this.brokerConfig.getFlushConsumerOffsetInterval(), TimeUnit.MILLISECONDS);
 
-        result = result && this.consumerOffsetManager.load();
-        result = result && this.subscriptionGroupManager.load();
-        result = result && this.consumerFilterManager.load();
+        this.scheduledExecutorService.scheduleAtFixedRate(new Runnable() {
+            @Override
+            public void run() {
+                try {
+                    BrokerController.this.consumerFilterManager.persist();
+                    BrokerController.this.consumerOrderInfoManager.persist();
+                } catch (Throwable e) {
+                    LOG.error(
+                        "BrokerController: failed to persist config file of consumerFilter or consumerOrderInfo",
+                        e);
+                }
+            }
+        }, 1000 * 10, 1000 * 10, TimeUnit.MILLISECONDS);
 
-        if (result) {
-            try {
-                this.messageStore = new DefaultMessageStore(this.messageStoreConfig, this.brokerStatsManager, this.messageArrivingListener, this.brokerConfig);
-                ((DefaultMessageStore) this.messageStore).setTopicConfigTable(topicConfigManager.getTopicConfigTable());
+        this.scheduledExecutorService.scheduleAtFixedRate(new Runnable() {
+            @Override
+            public void run() {
+                try {
+                    BrokerController.this.protectBroker();
+                } catch (Throwable e) {
+                    LOG.error("BrokerController: failed to protectBroker", e);
+                }
+            }
+        }, 3, 3, TimeUnit.MINUTES);
 
-                if (messageStoreConfig.isEnableDLegerCommitLog()) {
-                    DLedgerRoleChangeHandler roleChangeHandler = new DLedgerRoleChangeHandler(this, (DefaultMessageStore) messageStore);
-                    ((DLedgerCommitLog) messageStore.getCommitLog()).getdLedgerServer().getdLedgerLeaderElector().addRoleChangeHandler(roleChangeHandler);
+        this.scheduledExecutorService.scheduleAtFixedRate(new Runnable() {
+            @Override
+            public void run() {
+                try {
+                    BrokerController.this.printWaterMark();
+                } catch (Throwable e) {
+                    LOG.error("BrokerController: failed to print broker watermark", e);
                 }
-                this.brokerStats = new BrokerStats(this.messageStore);
-                //load plugin
-                MessageStorePluginContext context = new MessageStorePluginContext(messageStoreConfig, brokerStatsManager, messageArrivingListener, brokerConfig);
-                this.messageStore = MessageStoreFactory.build(context, this.messageStore);
-                this.messageStore.getDispatcherList().addFirst(new CommitLogDispatcherCalcBitMap(this.brokerConfig, this.consumerFilterManager));
-            } catch (IOException e) {
-                result = false;
-                log.error("Failed to initialize", e);
             }
-        }
+        }, 10, 1, TimeUnit.SECONDS);
 
-        result = result && this.messageStore.load();
+        this.scheduledExecutorService.scheduleAtFixedRate(new Runnable() {
 
-        if (result) {
-            this.remotingServer = new NettyRemotingServer(this.nettyServerConfig, this.clientHousekeepingService);
-            NettyServerConfig fastConfig = (NettyServerConfig) this.nettyServerConfig.clone();
-            fastConfig.setListenPort(nettyServerConfig.getListenPort() - 2);
-            this.fastRemotingServer = new NettyRemotingServer(fastConfig, this.clientHousekeepingService);
-            this.sendMessageExecutor = new BrokerFixedThreadPoolExecutor(
-                this.brokerConfig.getSendMessageThreadPoolNums(),
-                this.brokerConfig.getSendMessageThreadPoolNums(),
-                1000 * 60,
-                TimeUnit.MILLISECONDS,
-                this.sendThreadPoolQueue,
-                new ThreadFactoryImpl("SendMessageThread_"));
-
-            this.putMessageFutureExecutor = new BrokerFixedThreadPoolExecutor(
-                this.brokerConfig.getPutMessageFutureThreadPoolNums(),
-                this.brokerConfig.getPutMessageFutureThreadPoolNums(),
-                1000 * 60,
-                TimeUnit.MILLISECONDS,
-                this.putThreadPoolQueue,
-                new ThreadFactoryImpl("PutMessageThread_"));
-
-            this.pullMessageExecutor = new BrokerFixedThreadPoolExecutor(
-                this.brokerConfig.getPullMessageThreadPoolNums(),
-                this.brokerConfig.getPullMessageThreadPoolNums(),
-                1000 * 60,
-                TimeUnit.MILLISECONDS,
-                this.pullThreadPoolQueue,
-                new ThreadFactoryImpl("PullMessageThread_"));
-
-            this.ackMessageExecutor = new BrokerFixedThreadPoolExecutor(
-                this.brokerConfig.getAckMessageThreadPoolNums(),
-                this.brokerConfig.getAckMessageThreadPoolNums(),
-                1000 * 60,
-                TimeUnit.MILLISECONDS,
-                this.ackThreadPoolQueue,
-                new ThreadFactoryImpl("AckMessageThread_"));
-
-            this.replyMessageExecutor = new BrokerFixedThreadPoolExecutor(
-                this.brokerConfig.getProcessReplyMessageThreadPoolNums(),
-                this.brokerConfig.getProcessReplyMessageThreadPoolNums(),
-                1000 * 60,
-                TimeUnit.MILLISECONDS,
-                this.replyThreadPoolQueue,
-                new ThreadFactoryImpl("ProcessReplyMessageThread_"));
-
-            this.queryMessageExecutor = new BrokerFixedThreadPoolExecutor(
-                this.brokerConfig.getQueryMessageThreadPoolNums(),
-                this.brokerConfig.getQueryMessageThreadPoolNums(),
-                1000 * 60,
-                TimeUnit.MILLISECONDS,
-                this.queryThreadPoolQueue,
-                new ThreadFactoryImpl("QueryMessageThread_"));
-
-            this.adminBrokerExecutor =
-                Executors.newFixedThreadPool(this.brokerConfig.getAdminBrokerThreadPoolNums(), new ThreadFactoryImpl(
-                    "AdminBrokerThread_"));
-
-            this.clientManageExecutor = new ThreadPoolExecutor(
-                this.brokerConfig.getClientManageThreadPoolNums(),
-                this.brokerConfig.getClientManageThreadPoolNums(),
-                1000 * 60,
-                TimeUnit.MILLISECONDS,
-                this.clientManagerThreadPoolQueue,
-                new ThreadFactoryImpl("ClientManageThread_"));
-
-            this.heartbeatExecutor = new BrokerFixedThreadPoolExecutor(
-                this.brokerConfig.getHeartbeatThreadPoolNums(),
-                this.brokerConfig.getHeartbeatThreadPoolNums(),
-                1000 * 60,
-                TimeUnit.MILLISECONDS,
-                this.heartbeatThreadPoolQueue,
-                new ThreadFactoryImpl("HeartbeatThread_", true));
-
-            this.endTransactionExecutor = new BrokerFixedThreadPoolExecutor(
-                this.brokerConfig.getEndTransactionThreadPoolNums(),
-                this.brokerConfig.getEndTransactionThreadPoolNums(),
-                1000 * 60,
-                TimeUnit.MILLISECONDS,
-                this.endTransactionThreadPoolQueue,
-                new ThreadFactoryImpl("EndTransactionThread_"));
-
-            this.consumerManageExecutor =
-                Executors.newFixedThreadPool(this.brokerConfig.getConsumerManageThreadPoolNums(), new ThreadFactoryImpl(
-                    "ConsumerManageThread_"));
-
-            this.registerProcessor();
-
-            final long initialDelay = UtilAll.computeNextMorningTimeMillis() - System.currentTimeMillis();
-            final long period = 1000 * 60 * 60 * 24;
-            this.scheduledExecutorService.scheduleAtFixedRate(new Runnable() {
-                @Override
-                public void run() {
-                    try {
-                        BrokerController.this.getBrokerStats().record();
-                    } catch (Throwable e) {
-                        log.error("schedule record error.", e);
-                    }
+            @Override
+            public void run() {
+                try {
+                    LOG.info("Dispatch task fall behind commit log {}bytes",
+                        BrokerController.this.getMessageStore().dispatchBehindBytes());
+                } catch (Throwable e) {
+                    LOG.error("Failed to print dispatchBehindBytes", e);
                 }
-            }, initialDelay, period, TimeUnit.MILLISECONDS);
+            }
+        }, 1000 * 10, 1000 * 60, TimeUnit.MILLISECONDS);
 
-            this.scheduledExecutorService.scheduleAtFixedRate(new Runnable() {
-                @Override
-                public void run() {
-                    try {
-                        BrokerController.this.consumerOffsetManager.persist();
-                    } catch (Throwable e) {
-                        log.error("schedule persist consumerOffset error.", e);
-                    }
+        if (!messageStoreConfig.isEnableDLegerCommitLog() && !messageStoreConfig.isDuplicationEnable()) {
+            if (BrokerRole.SLAVE == this.messageStoreConfig.getBrokerRole()) {
+                if (this.messageStoreConfig.getHaMasterAddress() != null && this.messageStoreConfig.getHaMasterAddress().length() >= HA_ADDRESS_MIN_LENGTH) {
+                    this.messageStore.updateHaMasterAddress(this.messageStoreConfig.getHaMasterAddress());
+                    this.updateMasterHAServerAddrPeriodically = false;
+                } else {
+                    this.updateMasterHAServerAddrPeriodically = true;
                 }
-            }, 1000 * 10, this.brokerConfig.getFlushConsumerOffsetInterval(), TimeUnit.MILLISECONDS);
 
-            this.scheduledExecutorService.scheduleAtFixedRate(new Runnable() {
-                @Override
-                public void run() {
-                    try {
-                        BrokerController.this.consumerFilterManager.persist();
-                    } catch (Throwable e) {
-                        log.error("schedule persist consumer filter error.", e);
+                this.scheduledExecutorService.scheduleAtFixedRate(new Runnable() {
+
+                    @Override
+                    public void run() {
+                        try {
+                            BrokerController.this.slaveSynchronize.syncAll();
+                        } catch (Throwable e) {
+                            LOG.error("Failed to sync all config for slave.", e);
+                        }
                     }
-                }
-            }, 1000 * 10, 1000 * 10, TimeUnit.MILLISECONDS);
+                }, 1000 * 10, 1000 * 60, TimeUnit.MILLISECONDS);
 
-            this.scheduledExecutorService.scheduleAtFixedRate(new Runnable() {
-                @Override
-                public void run() {
-                    try {
-                        BrokerController.this.protectBroker();
-                    } catch (Throwable e) {
-                        log.error("protectBroker error.", e);
+            } else {
+                this.scheduledExecutorService.scheduleAtFixedRate(new Runnable() {
+
+                    @Override
+                    public void run() {
+                        try {
+                            BrokerController.this.printMasterAndSlaveDiff();
+                        } catch (Throwable e) {
+                            LOG.error("Failed to print diff of master and slave.", e);
+                        }
                     }
+                }, 1000 * 10, 1000 * 60, TimeUnit.MILLISECONDS);
+            }
+        }
+    }
+
+    protected void initializeScheduledTasks() {
+
+        initializeBrokerScheduledTasks();
+
+        this.scheduledExecutorService.scheduleAtFixedRate(new Runnable() {
+            @Override
+            public void run() {
+                try {
+                    BrokerController.this.brokerOuterAPI.refreshMetadata();
+                } catch (Exception e) {
+                    LOG.error("ScheduledTask refresh metadata exception", e);
                 }
-            }, 3, 3, TimeUnit.MINUTES);
+            }
+        }, 1, 5, TimeUnit.SECONDS);
 
+        if (this.brokerConfig.getNamesrvAddr() != null) {
+            this.brokerOuterAPI.updateNameServerAddressList(this.brokerConfig.getNamesrvAddr());
+            LOG.info("Set user specified name server address: {}", this.brokerConfig.getNamesrvAddr());
+            // also auto update namesrv if specify
             this.scheduledExecutorService.scheduleAtFixedRate(new Runnable() {
                 @Override
                 public void run() {
                     try {
-                        BrokerController.this.printWaterMark();
+                        BrokerController.this.brokerOuterAPI.updateNameServerAddressList(BrokerController.this.brokerConfig.getNamesrvAddr());
                     } catch (Throwable e) {
-                        log.error("printWaterMark error.", e);
+                        LOG.error("Failed to update nameServer address list", e);
                     }
                 }
-            }, 10, 1, TimeUnit.SECONDS);
-
+            }, 1000 * 10, 1000 * 60 * 2, TimeUnit.MILLISECONDS);
+        } else if (this.brokerConfig.isFetchNamesrvAddrByAddressServer()) {
             this.scheduledExecutorService.scheduleAtFixedRate(new Runnable() {
 
                 @Override
                 public void run() {
                     try {
-                        log.info("dispatch behind commit log {} bytes", BrokerController.this.getMessageStore().dispatchBehindBytes());
+                        BrokerController.this.brokerOuterAPI.fetchNameServerAddr();
                     } catch (Throwable e) {
-                        log.error("schedule dispatchBehindBytes error.", e);
+                        LOG.error("Failed to fetch nameServer address", e);
                     }
                 }
-            }, 1000 * 10, 1000 * 60, TimeUnit.MILLISECONDS);
+            }, 1000 * 10, 1000 * 60 * 2, TimeUnit.MILLISECONDS);
+        }
+    }
 
-            this.loadBalanceExecutor =
-                Executors.newFixedThreadPool(this.brokerConfig.getLoadBalanceProcessorThreadPoolNums(), new ThreadFactoryImpl(
-                    "LoadBalanceProcessorThread_"));
+    public boolean initialize() throws CloneNotSupportedException {
 
-            if (this.brokerConfig.getNamesrvAddr() != null) {
-                this.brokerOuterAPI.updateNameServerAddressList(this.brokerConfig.getNamesrvAddr());
-                log.info("Set user specified name server address: {}", this.brokerConfig.getNamesrvAddr());
-            } else if (this.brokerConfig.isFetchNamesrvAddrByAddressServer()) {
-                this.scheduledExecutorService.scheduleAtFixedRate(new Runnable() {
+        boolean result = this.topicConfigManager.load();
+        result = result && this.topicQueueMappingManager.load();
+        result = result && this.consumerOffsetManager.load();
+        result = result && this.subscriptionGroupManager.load();
+        result = result && this.consumerFilterManager.load();
+        result = result && this.consumerOrderInfoManager.load();
 
-                    @Override
-                    public void run() {
-                        try {
-                            BrokerController.this.brokerOuterAPI.fetchNameServerAddr();
-                        } catch (Throwable e) {
-                            log.error("ScheduledTask fetchNameServerAddr exception", e);
-                        }
-                    }
-                }, 1000 * 10, 1000 * 60 * 2, TimeUnit.MILLISECONDS);
-            }
+        if (result) {
+            try {
+                DefaultMessageStore defaultMessageStore = new DefaultMessageStore(this.messageStoreConfig, this.brokerStatsManager, this.messageArrivingListener, this.brokerConfig);
+                defaultMessageStore.setTopicConfigTable(topicConfigManager.getTopicConfigTable());
 
-            this.scheduledExecutorService.scheduleAtFixedRate(() -> {
-                try {
-                    BrokerController.this.brokerOuterAPI.refreshMetadata();
-                } catch (Exception e) {
-                    log.error("ScheduledTask refresh metadata exception", e);
-                }
-            }, 1, 5, TimeUnit.SECONDS);
-
-            if (!messageStoreConfig.isEnableDLegerCommitLog()) {
-                if (BrokerRole.SLAVE == this.messageStoreConfig.getBrokerRole()) {
-                    if (this.messageStoreConfig.getHaMasterAddress() != null && this.messageStoreConfig.getHaMasterAddress().length() >= 6) {
-                        this.messageStore.updateHaMasterAddress(this.messageStoreConfig.getHaMasterAddress());
-                        this.updateMasterHAServerAddrPeriodically = false;
-                    } else {
-                        this.updateMasterHAServerAddrPeriodically = true;
-                    }
-                } else {
-                    this.scheduledExecutorService.scheduleAtFixedRate(new Runnable() {
-                        @Override
-                        public void run() {
-                            try {
-                                BrokerController.this.printMasterAndSlaveDiff();
-                            } catch (Throwable e) {
-                                log.error("schedule printMasterAndSlaveDiff error.", e);
-                            }
-                        }
-                    }, 1000 * 10, 1000 * 60, TimeUnit.MILLISECONDS);
+                if (messageStoreConfig.isEnableDLegerCommitLog()) {
+                    DLedgerRoleChangeHandler roleChangeHandler = new DLedgerRoleChangeHandler(this, defaultMessageStore);
+                    ((DLedgerCommitLog) defaultMessageStore.getCommitLog()).getdLedgerServer().getdLedgerLeaderElector().addRoleChangeHandler(roleChangeHandler);
                 }
+                this.brokerStats = new BrokerStats(defaultMessageStore);
+                //load plugin
+                MessageStorePluginContext context = new MessageStorePluginContext(this, messageStoreConfig, brokerStatsManager, messageArrivingListener);
+                this.messageStore = MessageStoreFactory.build(context, defaultMessageStore);
+                this.messageStore.getDispatcherList().addFirst(new CommitLogDispatcherCalcBitMap(this.brokerConfig, this.consumerFilterManager));
+
+            } catch (IOException e) {
+                result = false;
+                LOG.error("BrokerController#initialize: unexpected error occurs", e);
+            }
+        }
+        if (messageStore != null) {
+            registerMessageStoreHook();
+        }
+
+        result = result && this.messageStore.load();
+
+        //scheduleMessageService load after messageStore load success
+        result = result && this.scheduleMessageService.load();
+
+        for (BrokerAttachedPlugin brokerAttachedPlugin : brokerAttachedPlugins) {
+            if (brokerAttachedPlugin != null) {
+                result = result && brokerAttachedPlugin.load();
             }
+        }
+
+        if (result) {
+
+            initializeRemotingServer();
+
+            initializeResources();
+
+            registerProcessor();
+
+            initializeScheduledTasks();
+
+            initialTransaction();
+
+            initialAcl();
 
-            this.topicQueueMappingCleanService = new TopicQueueMappingCleanService(this);
+            initialRpcHooks();
 
             if (TlsSystemConfig.tlsMode != TlsMode.DISABLED) {
                 // Register a listener to reload SslContext
@@ -549,7 +734,7 @@ public class BrokerController {
                             @Override
                             public void onChanged(String path) {
                                 if (path.equals(TlsSystemConfig.tlsServerTrustCertPath)) {
-                                    log.info("The trust certificate changed, reload the ssl context");
+                                    LOG.info("The trust certificate changed, reload the ssl context");
                                     reloadServerSslContext();
                                 }
                                 if (path.equals(TlsSystemConfig.tlsServerCertPath)) {
@@ -559,7 +744,7 @@ public class BrokerController {
                                     keyChanged = true;
                                 }
                                 if (certChanged && keyChanged) {
-                                    log.info("The certificate and private key changed, reload the ssl context");
+                                    LOG.info("The certificate and private key changed, reload the ssl context");
                                     certChanged = keyChanged = false;
                                     reloadServerSslContext();
                                 }
@@ -571,26 +756,76 @@ public class BrokerController {
                             }
                         });
                 } catch (Exception e) {
-                    log.warn("FileWatchService created error, can't load the certificate dynamically");
+                    result = false;
+                    LOG.warn("FileWatchService created error, can't load the certificate dynamically");
                 }
             }
-            initialTransaction();
-            initialAcl();
-            initialRpcHooks();
         }
+
         return result;
     }
 
+    public void registerMessageStoreHook() {
+        List<PutMessageHook> putMessageHookList = messageStore.getPutMessageHookList();
+
+        putMessageHookList.add(new PutMessageHook() {
+            @Override public String hookName() {
+                return "checkBeforePutMessage";
+            }
+
+            @Override public PutMessageResult executeBeforePutMessage(MessageExt msg) {
+                return HookUtils.checkBeforePutMessage(BrokerController.this, msg);
+            }
+        });
+
+        putMessageHookList.add(new PutMessageHook() {
+            @Override public String hookName() {
+                return "innerBatchChecker";
+            }
+
+            @Override public PutMessageResult executeBeforePutMessage(MessageExt msg) {
+                if (msg instanceof MessageExtBrokerInner) {
+                    return HookUtils.checkInnerBatch(BrokerController.this, msg);
+                }
+                return null;
+            }
+        });
+
+        putMessageHookList.add(new PutMessageHook() {
+            @Override public String hookName() {
+                return "handleScheduleMessage";
+            }
+
+            @Override public PutMessageResult executeBeforePutMessage(MessageExt msg) {
+                if (msg instanceof MessageExtBrokerInner) {
+                    return HookUtils.handleScheduleMessage(BrokerController.this, (MessageExtBrokerInner) msg);
+                }
+                return null;
+            }
+        });
+
+        SendMessageBackHook sendMessageBackHook = new SendMessageBackHook() {
+            @Override
+            public boolean executeSendMessageBack(List<MessageExt> msgList, String brokerName, String brokerAddr) {
+                return HookUtils.sendMessageBack(BrokerController.this, msgList, brokerName, brokerAddr);
+            }
+        };
+
+        if (messageStore != null) {
+            messageStore.setSendMessageBackHook(sendMessageBackHook);
+        }
+    }
+
     private void initialTransaction() {
         this.transactionalMessageService = ServiceProvider.loadClass(ServiceProvider.TRANSACTION_SERVICE_ID, TransactionalMessageService.class);
         if (null == this.transactionalMessageService) {
             this.transactionalMessageService = new TransactionalMessageServiceImpl(new TransactionalMessageBridge(this, this.getMessageStore()));
-            log.warn("Load default transaction message hook service: {}", TransactionalMessageServiceImpl.class.getSimpleName());
+            LOG.warn("Load default transaction message hook service: {}", TransactionalMessageServiceImpl.class.getSimpleName());
         }
         this.transactionalMessageCheckListener = ServiceProvider.loadClass(ServiceProvider.TRANSACTION_LISTENER_ID, AbstractTransactionalMessageCheckListener.class);
         if (null == this.transactionalMessageCheckListener) {
             this.transactionalMessageCheckListener = new DefaultTransactionalMessageCheckListener();
-            log.warn("Load default discard message hook service: {}", DefaultTransactionalMessageCheckListener.class.getSimpleName());
+            LOG.warn("Load default discard message hook service: {}", DefaultTransactionalMessageCheckListener.class.getSimpleName());
         }
         this.transactionalMessageCheckListener.setBrokerController(this);
         this.transactionalMessageCheckService = new TransactionalMessageCheckService(this);
@@ -598,13 +833,13 @@ public class BrokerController {
 
     private void initialAcl() {
         if (!this.brokerConfig.isAclEnable()) {
-            log.info("The broker dose not enable acl");
+            LOG.info("The broker dose not enable acl");
             return;
         }
 
         List<AccessValidator> accessValidators = ServiceProvider.load(ServiceProvider.ACL_VALIDATOR_ID, AccessValidator.class);
-        if (accessValidators == null || accessValidators.isEmpty()) {
-            log.info("The broker dose not load the AccessValidator");
+        if (accessValidators.isEmpty()) {
+            LOG.info("The broker dose not load the AccessValidator");
             return;
         }
 
@@ -622,6 +857,11 @@ public class BrokerController {
                 @Override
                 public void doAfterResponse(String remoteAddr, RemotingCommand request, RemotingCommand response) {
                 }
+
+                @Override
+                public void doAfterRpcFailure(String remoteAddr, RemotingCommand request, Boolean remoteTimeout) {
+
+                }
             });
         }
     }
@@ -637,15 +877,10 @@ public class BrokerController {
         }
     }
 
-    public String getBrokerAddrByName(String brokerName) {
-        return this.brokerName2AddrMap.get(brokerName);
-    }
-
     public void registerProcessor() {
-        /**
+        /*
          * SendMessageProcessor
          */
-
         sendMessageProcessor.registerSendMessageHook(sendMessageHookList);
         sendMessageProcessor.registerConsumeMessageHook(consumeMessageHookList);
 
@@ -661,8 +896,13 @@ public class BrokerController {
          * PullMessageProcessor
          */
         this.remotingServer.registerProcessor(RequestCode.PULL_MESSAGE, this.pullMessageProcessor, this.pullMessageExecutor);
+        this.remotingServer.registerProcessor(RequestCode.LITE_PULL_MESSAGE, this.pullMessageProcessor, this.litePullMessageExecutor);
         this.pullMessageProcessor.registerConsumeMessageHook(consumeMessageHookList);
         /**
+         * PeekMessageProcessor
+         */
+        this.remotingServer.registerProcessor(RequestCode.PEEK_MESSAGE, this.peekMessageProcessor, this.pullMessageExecutor);
+        /**
          * PopMessageProcessor
          */
         this.remotingServer.registerProcessor(RequestCode.POP_MESSAGE, this.popMessageProcessor, this.pullMessageExecutor);
@@ -677,11 +917,20 @@ public class BrokerController {
          */
         this.remotingServer.registerProcessor(RequestCode.CHANGE_MESSAGE_INVISIBLETIME, this.changeInvisibleTimeProcessor, this.ackMessageExecutor);
         this.fastRemotingServer.registerProcessor(RequestCode.CHANGE_MESSAGE_INVISIBLETIME, this.changeInvisibleTimeProcessor, this.ackMessageExecutor);
+        /**
+         * notificationProcessor
+         */
+        this.remotingServer.registerProcessor(RequestCode.NOTIFICATION, this.notificationProcessor, this.pullMessageExecutor);
+
+        /**
+         * pollingInfoProcessor
+         */
+        this.remotingServer.registerProcessor(RequestCode.POLLING_INFO, this.pollingInfoProcessor, this.pullMessageExecutor);
 
         /**
          * ReplyMessageProcessor
          */
-        ReplyMessageProcessor replyMessageProcessor = new ReplyMessageProcessor(this);
+
         replyMessageProcessor.registerSendMessageHook(sendMessageHookList);
 
         this.remotingServer.registerProcessor(RequestCode.SEND_REPLY_MESSAGE, replyMessageProcessor, replyMessageExecutor);
@@ -733,10 +982,10 @@ public class BrokerController {
         /**
          * EndTransactionProcessor
          */
-        this.remotingServer.registerProcessor(RequestCode.END_TRANSACTION, new EndTransactionProcessor(this), this.endTransactionExecutor);
-        this.fastRemotingServer.registerProcessor(RequestCode.END_TRANSACTION, new EndTransactionProcessor(this), this.endTransactionExecutor);
+        this.remotingServer.registerProcessor(RequestCode.END_TRANSACTION, new EndTransactionProcessor(this), this.sendMessageExecutor);
+        this.fastRemotingServer.registerProcessor(RequestCode.END_TRANSACTION, new EndTransactionProcessor(this), this.sendMessageExecutor);
 
-        /**
+        /*
          * Default
          */
         AdminBrokerProcessor adminProcessor = new AdminBrokerProcessor(this);
@@ -754,9 +1003,7 @@ public class BrokerController {
 
     public void protectBroker() {
         if (this.brokerConfig.isDisableConsumeIfConsumerReadSlowly()) {
-            final Iterator<Map.Entry<String, MomentStatsItem>> it = this.brokerStatsManager.getMomentStatsItemSetFallSize().getStatsItemTable().entrySet().iterator();
-            while (it.hasNext()) {
-                final Map.Entry<String, MomentStatsItem> next = it.next();
+            for (Map.Entry<String, MomentStatsItem> next : this.brokerStatsManager.getMomentStatsItemSetFallSize().getStatsItemTable().entrySet()) {
                 final long fallBehindBytes = next.getValue().getValue().get();
                 if (fallBehindBytes > this.brokerConfig.getConsumerFallbehindThreshold()) {
                     final String[] split = next.getValue().getStatsKey().split("@");
@@ -791,19 +1038,23 @@ public class BrokerController {
         return this.headSlowTimeMills(this.pullThreadPoolQueue);
     }
 
-    public long headSlowTimeMills4QueryThreadPoolQueue() {
-        return this.headSlowTimeMills(this.queryThreadPoolQueue);
+    public long headSlowTimeMills4LitePullThreadPoolQueue() {
+        return this.headSlowTimeMills(this.litePullThreadPoolQueue);
     }
 
-    public long headSlowTimeMills4EndTransactionThreadPoolQueue() {
-        return this.headSlowTimeMills(this.endTransactionThreadPoolQueue);
+    public long headSlowTimeMills4QueryThreadPoolQueue() {
+        return this.headSlowTimeMills(this.queryThreadPoolQueue);
     }
 
     public void printWaterMark() {
         LOG_WATER_MARK.info("[WATERMARK] Send Queue Size: {} SlowTimeMills: {}", this.sendThreadPoolQueue.size(), headSlowTimeMills4SendThreadPoolQueue());
         LOG_WATER_MARK.info("[WATERMARK] Pull Queue Size: {} SlowTimeMills: {}", this.pullThreadPoolQueue.size(), headSlowTimeMills4PullThreadPoolQueue());
         LOG_WATER_MARK.info("[WATERMARK] Query Queue Size: {} SlowTimeMills: {}", this.queryThreadPoolQueue.size(), headSlowTimeMills4QueryThreadPoolQueue());
-        LOG_WATER_MARK.info("[WATERMARK] Transaction Queue Size: {} SlowTimeMills: {}", this.endTransactionThreadPoolQueue.size(), headSlowTimeMills4EndTransactionThreadPoolQueue());
+        LOG_WATER_MARK.info("[WATERMARK] Lite Pull Queue Size: {} SlowTimeMills: {}", this.litePullThreadPoolQueue.size(), headSlowTimeMills4LitePullThreadPoolQueue());
+        LOG_WATER_MARK.info("[WATERMARK] Transaction Queue Size: {} SlowTimeMills: {}", this.endTransactionThreadPoolQueue.size(), headSlowTimeMills(this.endTransactionThreadPoolQueue));
+        LOG_WATER_MARK.info("[WATERMARK] ClientManager Queue Size: {} SlowTimeMills: {}", this.clientManagerThreadPoolQueue.size(), this.headSlowTimeMills(this.clientManagerThreadPoolQueue));
+        LOG_WATER_MARK.info("[WATERMARK] Heartbeat Queue Size: {} SlowTimeMills: {}", this.heartbeatThreadPoolQueue.size(), this.headSlowTimeMills(this.heartbeatThreadPoolQueue));
+        LOG_WATER_MARK.info("[WATERMARK] Ack Queue Size: {} SlowTimeMills: {}", this.ackThreadPoolQueue.size(), headSlowTimeMills(this.ackThreadPoolQueue));
     }
 
     public MessageStore getMessageStore() {
@@ -814,11 +1065,11 @@ public class BrokerController {
         this.messageStore = messageStore;
     }
 
-    private void printMasterAndSlaveDiff() {
-        long diff = this.messageStore.slaveFallBehindMuch();
-
-        // XXX: warn and notify me
-        log.info("Slave fall behind master: {} bytes", diff);
+    protected void printMasterAndSlaveDiff() {
+        if (messageStore.getHaService() != null && messageStore.getHaService().getConnectionCount().get() > 0) {
+            long diff = this.messageStore.slaveFallBehindMuch();
+            LOG.info("CommitLog: slave fall behind master {}bytes", diff);
+        }
     }
 
     public Broker2Client getBroker2Client() {
@@ -853,6 +1104,10 @@ public class BrokerController {
         this.fastRemotingServer = fastRemotingServer;
     }
 
+    public RemotingServer getFastRemotingServer() {
+        return fastRemotingServer;
+    }
+
     public PullMessageProcessor getPullMessageProcessor() {
         return pullMessageProcessor;
     }
@@ -869,7 +1124,24 @@ public class BrokerController {
         return popMessageProcessor;
     }
 
-    public void shutdown() {
+    protected void shutdownBasicService() {
+
+        shutdown = true;
+
+        this.unregisterBrokerAll();
+
+        if (this.shutdownHook != null) {
+            this.shutdownHook.beforeShutdown(this);
+        }
+
+        if (this.remotingServer != null) {
+            this.remotingServer.shutdown();
+        }
+
+        if (this.fastRemotingServer != null) {
+            this.fastRemotingServer.shutdown();
+        }
+
         if (this.brokerStatsManager != null) {
             this.brokerStatsManager.shutdown();
         }
@@ -882,12 +1154,26 @@ public class BrokerController {
             this.pullRequestHoldService.shutdown();
         }
 
-        if (this.remotingServer != null) {
-            this.remotingServer.shutdown();
+        {
+            this.popMessageProcessor.getPopLongPollingService().shutdown();
+            this.popMessageProcessor.getQueueLockManager().shutdown();
         }
 
-        if (this.fastRemotingServer != null) {
-            this.fastRemotingServer.shutdown();
+        {
+            this.popMessageProcessor.getPopBufferMergeService().shutdown();
+            this.ackMessageProcessor.shutdownPopReviveService();
+        }
+
+        if (this.assignmentManager != null) {
+            this.assignmentManager.shutdown();
+        }
+
+        if (this.notificationProcessor != null) {
+            this.notificationProcessor.shutdown();
+        }
+
+        if (this.consumerIdsChangeListener != null) {
+            this.consumerIdsChangeListener.shutdown();
         }
 
         if (this.topicQueueMappingCleanService != null) {
@@ -902,20 +1188,14 @@ public class BrokerController {
             this.messageStore.shutdown();
         }
 
-        this.scheduledExecutorService.shutdown();
-        try {
-            this.scheduledExecutorService.awaitTermination(5000, TimeUnit.MILLISECONDS);
-        } catch (InterruptedException e) {
-        }
-
-        this.unregisterBrokerAll();
+        shutdownScheduledExecutorService(this.scheduledExecutorService);
 
         if (this.sendMessageExecutor != null) {
             this.sendMessageExecutor.shutdown();
         }
 
-        if (this.putMessageFutureExecutor != null) {
-            this.putMessageFutureExecutor.shutdown();
+        if (this.litePullMessageExecutor != null) {
+            this.litePullMessageExecutor.shutdown();
         }
 
         if (this.pullMessageExecutor != null) {
@@ -926,12 +1206,16 @@ public class BrokerController {
             this.replyMessageExecutor.shutdown();
         }
 
-        if (this.adminBrokerExecutor != null) {
-            this.adminBrokerExecutor.shutdown();
+        if (this.putMessageFutureExecutor != null) {
+            this.putMessageFutureExecutor.shutdown();
         }
 
-        if (this.brokerOuterAPI != null) {
-            this.brokerOuterAPI.shutdown();
+        if (this.ackMessageExecutor != null) {
+            this.ackMessageExecutor.shutdown();
+        }
+
+        if (this.adminBrokerExecutor != null) {
+            this.adminBrokerExecutor.shutdown();
         }
 
         this.consumerOffsetManager.persist();
@@ -948,6 +1232,15 @@ public class BrokerController {
             this.consumerFilterManager.persist();
         }
 
+        if (this.consumerOrderInfoManager != null) {
+            this.consumerOrderInfoManager.persist();
+        }
+
+        if (this.scheduleMessageService != null) {
+            this.scheduleMessageService.persist();
+            this.scheduleMessageService.shutdown();
+        }
+
         if (this.clientManageExecutor != null) {
             this.clientManageExecutor.shutdown();
         }
@@ -956,13 +1249,12 @@ public class BrokerController {
             this.queryMessageExecutor.shutdown();
         }
 
-        if (this.consumerManageExecutor != null) {
-            this.consumerManageExecutor.shutdown();
+        if (this.heartbeatExecutor != null) {
+            this.heartbeatExecutor.shutdown();
         }
 
-        {
-            this.popMessageProcessor.getPopBufferMergeService().shutdown();
-            this.ackMessageProcessor.shutdownPopReviveService();
+        if (this.consumerManageExecutor != null) {
+            this.consumerManageExecutor.shutdown();
         }
 
         if (this.fileWatchService != null) {
@@ -975,9 +1267,42 @@ public class BrokerController {
         if (this.endTransactionExecutor != null) {
             this.endTransactionExecutor.shutdown();
         }
+
+        if (this.escapeBridge != null) {
+            escapeBridge.shutdown();
+        }
+
+        this.topicConfigManager.persist();
+        this.subscriptionGroupManager.persist();
+
+        for (BrokerAttachedPlugin brokerAttachedPlugin : brokerAttachedPlugins) {
+            if (brokerAttachedPlugin != null) {
+                brokerAttachedPlugin.shutdown();
+            }
+        }
+    }
+
+    public void shutdown() {
+
+        shutdownBasicService();
+
+        if (this.brokerOuterAPI != null) {
+            this.brokerOuterAPI.shutdown();
+        }
+    }
+
+    protected void shutdownScheduledExecutorService(ScheduledExecutorService scheduledExecutorService) {
+        if (scheduledExecutorService == null) {
+            return;
+        }
+        scheduledExecutorService.shutdown();
+        try {
+            scheduledExecutorService.awaitTermination(5000, TimeUnit.MILLISECONDS);
+        } catch (InterruptedException ignore) {
+        }
     }
 
-    private void unregisterBrokerAll() {
+    protected void unregisterBrokerAll() {
         this.brokerOuterAPI.unregisterBrokerAll(
             this.brokerConfig.getBrokerClusterName(),
             this.getBrokerAddr(),
@@ -989,30 +1314,42 @@ public class BrokerController {
         return this.brokerConfig.getBrokerIP1() + ":" + this.nettyServerConfig.getListenPort();
     }
 
-    public void start() throws Exception {
-        this.shouldStartTime = System.currentTimeMillis();
+    protected void startBasicService() throws Exception {
 
         if (this.messageStore != null) {
             this.messageStore.start();
         }
 
+        if (remotingServerStartLatch != null) {
+            remotingServerStartLatch.await();
+        }
+
         if (this.remotingServer != null) {
             this.remotingServer.start();
         }
 
-        {
+        if (this.fastRemotingServer != null) {
+            this.fastRemotingServer.start();
+        }
+
+        for (BrokerAttachedPlugin brokerAttachedPlugin : brokerAttachedPlugins) {
+            if (brokerAttachedPlugin != null) {
+                brokerAttachedPlugin.start();
+            }
+        }
+
+        if (this.popMessageProcessor != null) {
             this.popMessageProcessor.getPopLongPollingService().start();
             this.popMessageProcessor.getPopBufferMergeService().start();
             this.popMessageProcessor.getQueueLockManager().start();
-            this.ackMessageProcessor.startPopReviveService();
         }
 
-        {
-            assignmentManager.start();
+        if (this.ackMessageProcessor != null) {
+            this.ackMessageProcessor.startPopReviveService();
         }
 
-        if (this.fastRemotingServer != null) {
-            this.fastRemotingServer.start();
+        if (this.assignmentManager != null) {
+            this.assignmentManager.start();
         }
 
         if (this.topicQueueMappingCleanService != null) {
@@ -1023,10 +1360,6 @@ public class BrokerController {
             this.fileWatchService.start();
         }
 
-        if (this.brokerOuterAPI != null) {
-            this.brokerOuterAPI.start();
-        }
-
         if (this.pullRequestHoldService != null) {
             this.pullRequestHoldService.start();
         }
@@ -1039,9 +1372,34 @@ public class BrokerController {
             this.filterServerManager.start();
         }
 
-        if (!messageStoreConfig.isEnableDLegerCommitLog()) {
-            startProcessorByHa(messageStoreConfig.getBrokerRole());
-            handleSlaveSynchronize(messageStoreConfig.getBrokerRole());
+        if (this.brokerStatsManager != null) {
+            this.brokerStatsManager.start();
+        }
+
+        if (this.brokerFastFailure != null) {
+            this.brokerFastFailure.start();
+        }
+
+        if (this.escapeBridge != null) {
+            this.escapeBridge.start();
+        }
+
+        //Init state version after messageStore initialized.
+        this.topicConfigManager.initStateVersion();
+    }
+
+    public void start() throws Exception {
+
+        this.shouldStartTime = System.currentTimeMillis() + messageStoreConfig.getDisappearTimeAfterStart();
+
+        if (this.brokerOuterAPI != null) {
+            this.brokerOuterAPI.start();
+        }
+
+        startBasicService();
+
+        if (!this.messageStoreConfig.isEnableDLegerCommitLog() && !this.messageStoreConfig.isDuplicationEnable()) {
+            changeSpecialServiceStatus(this.brokerConfig.getBrokerId() == MixAll.MASTER_ID);
             this.registerBrokerAll(true, false, true);
         }
 
@@ -1050,21 +1408,17 @@ public class BrokerController {
             @Override
             public void run() {
                 try {
+                    if (System.currentTimeMillis() < shouldStartTime) {
+                        LOG.info("Register to namesrv after {}", shouldStartTime);
+                        return;
+                    }
                     BrokerController.this.registerBrokerAll(true, false, brokerConfig.isForceRegister());
                 } catch (Throwable e) {
-                    log.error("registerBrokerAll Exception", e);
+                    LOG.error("BrokerController#registerBrokerAll: unexpected error occurs.", e);
                 }
             }
         }, 1000 * 10, Math.max(10000, Math.min(brokerConfig.getRegisterNameServerPeriod(), 60000)), TimeUnit.MILLISECONDS);
 
-        if (this.brokerStatsManager != null) {
-            this.brokerStatsManager.start();
-        }
-
-        if (this.brokerFastFailure != null) {
-            this.brokerFastFailure.start();
-        }
-
     }
 
     public synchronized void registerIncrementBrokerData(TopicConfig topicConfig, DataVersion dataVersion) {
@@ -1088,7 +1442,7 @@ public class BrokerController {
                         new TopicConfig(topicConfig.getTopicName(),
                             topicConfig.getReadQueueNums(),
                             topicConfig.getWriteQueueNums(),
-                            this.brokerConfig.getBrokerPermission());
+                            this.brokerConfig.getBrokerPermission(), topicConfig.getTopicSysFlag());
                 } else {
                     registerTopicConfig = new TopicConfig(topicConfig);
                 }
@@ -1128,7 +1482,7 @@ public class BrokerController {
             for (TopicConfig topicConfig : topicConfigWrapper.getTopicConfigTable().values()) {
                 TopicConfig tmp =
                     new TopicConfig(topicConfig.getTopicName(), topicConfig.getReadQueueNums(), topicConfig.getWriteQueueNums(),
-                        this.brokerConfig.getBrokerPermission());
+                        topicConfig.getPerm() & this.brokerConfig.getBrokerPermission(), topicConfig.getTopicSysFlag());
                 topicConfigTable.put(topicConfig.getTopicName(), tmp);
             }
             topicConfigWrapper.setTopicConfigTable(topicConfigTable);
@@ -1138,14 +1492,20 @@ public class BrokerController {
             this.getBrokerAddr(),
             this.brokerConfig.getBrokerName(),
             this.brokerConfig.getBrokerId(),
-            this.brokerConfig.getRegisterBrokerTimeoutMills())) {
+            this.brokerConfig.getRegisterBrokerTimeoutMills(),
+            this.brokerConfig.isInBrokerContainer())) {
             doRegisterBrokerAll(checkOrderConfig, oneway, topicConfigWrapper);
         }
     }
 
-    private void doRegisterBrokerAll(boolean checkOrderConfig, boolean oneway,
-        TopicConfigAndMappingSerializeWrapper topicConfigWrapper) {
-        List<RegisterBrokerResult> registerBrokerResultList = this.brokerOuterAPI.registerBrokerAll(
+    protected void doRegisterBrokerAll(boolean checkOrderConfig, boolean oneway,
+        TopicConfigSerializeWrapper topicConfigWrapper) {
+
+        if (shutdown) {
+            LOG.info("BrokerController#doResterBrokerAll: broker has shutdown, no need to register any more.");
+            return;
+        }
+        List<RegisterBrokerResult> registerBrokerResultList = this.getBrokerOuterAPI().registerBrokerAll(
             this.brokerConfig.getBrokerClusterName(),
             this.getBrokerAddr(),
             this.brokerConfig.getBrokerName(),
@@ -1155,10 +1515,16 @@ public class BrokerController {
             this.filterServerManager.buildNewFilterServerList(),
             oneway,
             this.brokerConfig.getRegisterBrokerTimeoutMills(),
-            this.brokerConfig.isCompressedRegister());
+            this.brokerConfig.isEnableSlaveActingMaster(),
+            this.brokerConfig.isCompressedRegister(),
+            this.brokerConfig.isInBrokerContainer());
+
+        handleRegisterBrokerResult(registerBrokerResultList, checkOrderConfig);
+    }
 
-        if (registerBrokerResultList.size() > 0) {
-            RegisterBrokerResult registerBrokerResult = registerBrokerResultList.get(0);
+    protected void handleRegisterBrokerResult(List<RegisterBrokerResult> registerBrokerResultList,
+        boolean checkOrderConfig) {
+        for (RegisterBrokerResult registerBrokerResult : registerBrokerResultList) {
             if (registerBrokerResult != null) {
                 if (this.updateMasterHAServerAddrPeriodically && registerBrokerResult.getHaServerAddr() != null) {
                     this.messageStore.updateHaMasterAddress(registerBrokerResult.getHaServerAddr());
@@ -1169,6 +1535,7 @@ public class BrokerController {
                 if (checkOrderConfig) {
                     this.getTopicConfigManager().updateOrderTopicConfig(registerBrokerResult.getKvTable());
                 }
+                break;
             }
         }
     }
@@ -1177,10 +1544,11 @@ public class BrokerController {
         final String brokerAddr,
         final String brokerName,
         final long brokerId,
-        final int timeoutMills) {
+        final int timeoutMills,
+        final boolean isInBrokerContainer) {
 
         TopicConfigSerializeWrapper topicConfigWrapper = this.getTopicConfigManager().buildTopicConfigSerializeWrapper();
-        List<Boolean> changeList = brokerOuterAPI.needRegister(clusterName, brokerAddr, brokerName, brokerId, topicConfigWrapper, timeoutMills);
+        List<Boolean> changeList = brokerOuterAPI.needRegister(clusterName, brokerAddr, brokerName, brokerId, topicConfigWrapper, timeoutMills, isInBrokerContainer);
         boolean needRegister = false;
         for (Boolean changed : changeList) {
             if (changed) {
@@ -1191,6 +1559,16 @@ public class BrokerController {
         return needRegister;
     }
 
+    public String getNameServerList() {
+        if (this.brokerConfig.getNamesrvAddr() != null) {
+            this.brokerOuterAPI.updateNameServerAddressList(this.brokerConfig.getNamesrvAddr());
+            return this.brokerConfig.getNamesrvAddr();
+        } else if (this.brokerConfig.isFetchNamesrvAddrByAddressServer()) {
+            return this.brokerOuterAPI.fetchNameServerAddr();
+        }
+        return null;
+    }
+
     public TopicConfigManager getTopicConfigManager() {
         return topicConfigManager;
     }
@@ -1215,10 +1593,18 @@ public class BrokerController {
         return slaveSynchronize;
     }
 
+    public ScheduledExecutorService getScheduledExecutorService() {
+        return scheduledExecutorService;
+    }
+
     public ExecutorService getPullMessageExecutor() {
         return pullMessageExecutor;
     }
 
+    public ExecutorService getPutMessageFutureExecutor() {
+        return putMessageFutureExecutor;
+    }
+
     public void setPullMessageExecutor(ExecutorService pullMessageExecutor) {
         this.pullMessageExecutor = pullMessageExecutor;
     }
@@ -1227,6 +1613,10 @@ public class BrokerController {
         return sendThreadPoolQueue;
     }
 
+    public BlockingQueue<Runnable> getAckThreadPoolQueue() {
+        return ackThreadPoolQueue;
+    }
+
     public FilterServerManager getFilterServerManager() {
         return filterServerManager;
     }
@@ -1241,7 +1631,7 @@ public class BrokerController {
 
     public void registerSendMessageHook(final SendMessageHook hook) {
         this.sendMessageHookList.add(hook);
-        log.info("register SendMessageHook Hook, {}", hook.hookName());
+        LOG.info("register SendMessageHook Hook, {}", hook.hookName());
     }
 
     public List<ConsumeMessageHook> getConsumeMessageHookList() {
@@ -1250,7 +1640,7 @@ public class BrokerController {
 
     public void registerConsumeMessageHook(final ConsumeMessageHook hook) {
         this.consumeMessageHookList.add(hook);
-        log.info("register ConsumeMessageHook Hook, {}", hook.hookName());
+        LOG.info("register ConsumeMessageHook Hook, {}", hook.hookName());
     }
 
     public void registerServerRPCHook(RPCHook rpcHook) {
@@ -1266,6 +1656,14 @@ public class BrokerController {
         this.remotingServer = remotingServer;
     }
 
+    public CountDownLatch getRemotingServerStartLatch() {
+        return remotingServerStartLatch;
+    }
+
+    public void setRemotingServerStartLatch(CountDownLatch remotingServerStartLatch) {
+        this.remotingServerStartLatch = remotingServerStartLatch;
+    }
+
     public void registerClientRPCHook(RPCHook rpcHook) {
         this.getBrokerOuterAPI().registerRPCHook(rpcHook);
     }
@@ -1325,133 +1723,140 @@ public class BrokerController {
         return accessValidatorMap;
     }
 
-    private void handleSlaveSynchronize(BrokerRole role) {
-        if (role == BrokerRole.SLAVE) {
-            if (null != slaveSyncFuture) {
-                slaveSyncFuture.cancel(false);
-            }
-            this.slaveSynchronize.setMasterAddr(null);
-            slaveSyncFuture = this.scheduledExecutorService.scheduleAtFixedRate(new Runnable() {
-                @Override
-                public void run() {
-                    try {
-                        BrokerController.this.slaveSynchronize.syncAll();
-                    } catch (Throwable e) {
-                        log.error("ScheduledTask SlaveSynchronize syncAll error.", e);
-                    }
-                }
-            }, 1000 * 3, 1000 * 10, TimeUnit.MILLISECONDS);
-        } else {
-            //handle the slave synchronise
-            if (null != slaveSyncFuture) {
-                slaveSyncFuture.cancel(false);
-            }
-            this.slaveSynchronize.setMasterAddr(null);
-        }
+    public ExecutorService getSendMessageExecutor() {
+        return sendMessageExecutor;
     }
 
-    public void changeToSlave(int brokerId) {
-        log.info("Begin to change to slave brokerName={} brokerId={}", brokerConfig.getBrokerName(), brokerId);
+    public AssignmentManager getAssignmentManager() {
+        return assignmentManager;
+    }
 
-        //change the role
-        brokerConfig.setBrokerId(brokerId == 0 ? 1 : brokerId); //TO DO check
-        messageStoreConfig.setBrokerRole(BrokerRole.SLAVE);
+    public SendMessageProcessor getSendMessageProcessor() {
+        return sendMessageProcessor;
+    }
 
-        //handle the scheduled service
-        try {
-            this.messageStore.handleScheduleMessageService(BrokerRole.SLAVE);
-        } catch (Throwable t) {
-            log.error("[MONITOR] handleScheduleMessageService failed when changing to slave", t);
-        }
+    public QueryAssignmentProcessor getQueryAssignmentProcessor() {
+        return queryAssignmentProcessor;
+    }
 
-        //handle the transactional service
-        try {
-            this.shutdownProcessorByHa();
-        } catch (Throwable t) {
-            log.error("[MONITOR] shutdownProcessorByHa failed when changing to slave", t);
-        }
+    public TopicQueueMappingCleanService getTopicQueueMappingCleanService() {
+        return topicQueueMappingCleanService;
+    }
 
-        //handle the slave synchronise
-        handleSlaveSynchronize(BrokerRole.SLAVE);
+    public ExecutorService getAdminBrokerExecutor() {
+        return adminBrokerExecutor;
+    }
 
-        try {
-            this.registerBrokerAll(true, true, true);
-        } catch (Throwable ignored) {
+    public BlockingQueue<Runnable> getLitePullThreadPoolQueue() {
+        return litePullThreadPoolQueue;
+    }
 
-        }
-        log.info("Finish to change to slave brokerName={} brokerId={}", brokerConfig.getBrokerName(), brokerId);
+    public ShutdownHook getShutdownHook() {
+        return shutdownHook;
     }
 
-    public void changeToMaster(BrokerRole role) {
-        if (role == BrokerRole.SLAVE) {
-            return;
-        }
-        log.info("Begin to change to master brokerName={}", brokerConfig.getBrokerName());
+    public void setShutdownHook(ShutdownHook shutdownHook) {
+        this.shutdownHook = shutdownHook;
+    }
 
-        //handle the slave synchronise
-        handleSlaveSynchronize(role);
+    public long getMinBrokerIdInGroup() {
+        return this.brokerConfig.getBrokerId();
+    }
 
-        //handle the scheduled service
-        try {
-            this.messageStore.handleScheduleMessageService(role);
-        } catch (Throwable t) {
-            log.error("[MONITOR] handleScheduleMessageService failed when changing to master", t);
-        }
+    public BrokerController peekMasterBroker() {
+        return brokerConfig.getBrokerId() == MixAll.MASTER_ID ? this : null;
+    }
 
-        //handle the transactional service
-        try {
-            this.startProcessorByHa(BrokerRole.SYNC_MASTER);
-        } catch (Throwable t) {
-            log.error("[MONITOR] startProcessorByHa failed when changing to master", t);
-        }
+    public BrokerMemberGroup getBrokerMemberGroup() {
+        return this.brokerMemberGroup;
+    }
 
-        //if the operations above are totally successful, we change to master
-        brokerConfig.setBrokerId(0); //TO DO check
-        messageStoreConfig.setBrokerRole(role);
+    public boolean isSpecialServiceRunning() {
+        return this.brokerConfig.getBrokerId() == MixAll.MASTER_ID;
+    }
 
-        try {
-            this.registerBrokerAll(true, true, true);
-        } catch (Throwable ignored) {
+    public void updateMinBroker(long minBrokerId, String minBrokerAddr) {
+        //do nothing
+    }
 
-        }
-        log.info("Finish to change to master brokerName={}", brokerConfig.getBrokerName());
+    public void updateMinBroker(long minBrokerId, String minBrokerAddr, String offlineBrokerAddr, String masterHaAddr) {
+        //do nothing
     }
 
-    private void startProcessorByHa(BrokerRole role) {
-        if (BrokerRole.SLAVE != role) {
-            if (this.transactionalMessageCheckService != null) {
-                this.transactionalMessageCheckService.start();
-            }
-        }
+    public int getListenPort() {
+        return this.nettyServerConfig.getListenPort();
     }
 
-    private void shutdownProcessorByHa() {
-        if (this.transactionalMessageCheckService != null) {
-            this.transactionalMessageCheckService.shutdown(true);
-        }
+    public List<BrokerAttachedPlugin> getBrokerAttachedPlugins() {
+        return brokerAttachedPlugins;
     }
 
-    public ExecutorService getPutMessageFutureExecutor() {
-        return putMessageFutureExecutor;
+    public EscapeBridge getEscapeBridge() {
+        return escapeBridge;
     }
 
     public long getShouldStartTime() {
         return shouldStartTime;
     }
 
-    public AssignmentManager getAssignmentManager() {
-        return assignmentManager;
+    public void changeSpecialServiceStatus(boolean shouldStart) {
+
+        for (BrokerAttachedPlugin brokerAttachedPlugin : brokerAttachedPlugins) {
+            if (brokerAttachedPlugin != null) {
+                brokerAttachedPlugin.statusChanged(shouldStart);
+            }
+        }
+
+        changeScheduleServiceStatus(shouldStart);
+
+        changeTransactionCheckServiceStatus(shouldStart);
+
+        if (this.ackMessageProcessor != null) {
+            LOG.info("Set PopReviveService Status to {}", shouldStart);
+            this.ackMessageProcessor.setPopReviveServiceStatus(shouldStart);
+        }
     }
 
-    public SendMessageProcessor getSendMessageProcessor() {
-        return sendMessageProcessor;
+    private synchronized void changeTransactionCheckServiceStatus(boolean shouldStart) {
+        if (isTransactionCheckServiceStart != shouldStart) {
+            LOG.info("TransactionCheckService status changed to {}", shouldStart);
+            if (shouldStart) {
+                this.transactionalMessageCheckService.start();
+            } else {
+                this.transactionalMessageCheckService.shutdown(true);
+            }
+            isTransactionCheckServiceStart = shouldStart;
+        }
     }
 
-    public QueryAssignmentProcessor getQueryAssignmentProcessor() {
-        return queryAssignmentProcessor;
+    public synchronized void changeScheduleServiceStatus(boolean shouldStart) {
+        if (isScheduleServiceStart != shouldStart) {
+            LOG.info("ScheduleServiceStatus changed to {}", shouldStart);
+            if (shouldStart) {
+                this.scheduleMessageService.start();
+            } else {
+                this.scheduleMessageService.stop();
+            }
+            isScheduleServiceStart = shouldStart;
+        }
     }
 
-    public TopicQueueMappingCleanService getTopicQueueMappingCleanService() {
-        return topicQueueMappingCleanService;
+    public boolean isScheduleServiceStart() {
+        return isScheduleServiceStart;
+    }
+
+    public boolean isTransactionCheckServiceStart() {
+        return isTransactionCheckServiceStart;
+    }
+
+    public ScheduleMessageService getScheduleMessageService() {
+        return scheduleMessageService;
+    }
+
+    public MessageStore getMessageStoreByBrokerName(String brokerName) {
+        if (this.brokerConfig.getBrokerName().equals(brokerName)) {
+            return this.getMessageStore();
+        }
+        return null;
     }
 }
diff --git a/broker/src/main/java/org/apache/rocketmq/broker/BrokerStartup.java b/broker/src/main/java/org/apache/rocketmq/broker/BrokerStartup.java
index 19e618b..ca388b6 100644
--- a/broker/src/main/java/org/apache/rocketmq/broker/BrokerStartup.java
+++ b/broker/src/main/java/org/apache/rocketmq/broker/BrokerStartup.java
@@ -19,9 +19,9 @@ package org.apache.rocketmq.broker;
 import ch.qos.logback.classic.LoggerContext;
 import ch.qos.logback.classic.joran.JoranConfigurator;
 import org.apache.commons.cli.CommandLine;
+import org.apache.commons.cli.DefaultParser;
 import org.apache.commons.cli.Option;
 import org.apache.commons.cli.Options;
-import org.apache.commons.cli.PosixParser;
 import org.apache.rocketmq.common.BrokerConfig;
 import org.apache.rocketmq.common.MQVersion;
 import org.apache.rocketmq.common.MixAll;
@@ -37,6 +37,7 @@ import org.apache.rocketmq.remoting.protocol.RemotingCommand;
 import org.apache.rocketmq.srvutil.ServerUtil;
 import org.apache.rocketmq.store.config.BrokerRole;
 import org.apache.rocketmq.store.config.MessageStoreConfig;
+import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 
 import java.io.BufferedInputStream;
@@ -52,6 +53,7 @@ public class BrokerStartup {
     public static CommandLine commandLine = null;
     public static String configFile = null;
     public static InternalLogger log;
+    public static SystemConfigFileHelper configFileHelper = new SystemConfigFileHelper();
 
     public static void main(String[] args) {
         start(createBrokerController(args));
@@ -93,7 +95,7 @@ public class BrokerStartup {
             //PackageConflictDetect.detectFastjson();
             Options options = ServerUtil.buildCommandlineOptions(new Options());
             commandLine = ServerUtil.parseCmdLine("mqbroker", args, buildCommandlineOptions(options),
-                new PosixParser());
+                new DefaultParser());
             if (null == commandLine) {
                 System.exit(-1);
             }
@@ -115,22 +117,21 @@ public class BrokerStartup {
             if (commandLine.hasOption('c')) {
                 String file = commandLine.getOptionValue('c');
                 if (file != null) {
+                    configFileHelper.setFile(file);
                     configFile = file;
-                    InputStream in = new BufferedInputStream(new FileInputStream(file));
-                    properties = new Properties();
-                    properties.load(in);
-
-                    properties2SystemEnv(properties);
-                    MixAll.properties2Object(properties, brokerConfig);
-                    MixAll.properties2Object(properties, nettyServerConfig);
-                    MixAll.properties2Object(properties, nettyClientConfig);
-                    MixAll.properties2Object(properties, messageStoreConfig);
-
                     BrokerPathConfigHelper.setBrokerConfigPath(file);
-                    in.close();
                 }
             }
 
+            properties = configFileHelper.loadConfig();
+            if (properties != null) {
+                properties2SystemEnv(properties);
+                MixAll.properties2Object(properties, brokerConfig);
+                MixAll.properties2Object(properties, nettyServerConfig);
+                MixAll.properties2Object(properties, nettyClientConfig);
+                MixAll.properties2Object(properties, messageStoreConfig);
+            }
+
             MixAll.properties2Object(ServerUtil.commandLine2Properties(commandLine), brokerConfig);
 
             if (null == brokerConfig.getRocketmqHome()) {
@@ -209,6 +210,8 @@ public class BrokerStartup {
             MixAll.printObjectProperties(log, nettyClientConfig);
             MixAll.printObjectProperties(log, messageStoreConfig);
 
+            brokerConfig.setInBrokerContainer(false);
+
             final BrokerController controller = new BrokerController(
                 brokerConfig,
                 nettyServerConfig,
@@ -276,4 +279,33 @@ public class BrokerStartup {
 
         return options;
     }
+
+    public static class SystemConfigFileHelper {
+        private static final Logger LOGGER = LoggerFactory.getLogger(SystemConfigFileHelper.class);
+
+        private String file;
+
+        public SystemConfigFileHelper() {
+        }
+
+        public Properties loadConfig() throws Exception {
+            InputStream in = new BufferedInputStream(new FileInputStream(file));
+            Properties properties = new Properties();
+            properties.load(in);
+            in.close();
+            return properties;
+        }
+
+        public void update(Properties properties) throws Exception {
+            LOGGER.error("[SystemConfigFileHelper] update no thing.");
+        }
+
+        public void setFile(String file) {
+            this.file = file;
+        }
+
+        public String getFile() {
+            return file;
+        }
+    }
 }
diff --git a/broker/src/main/java/org/apache/rocketmq/broker/client/ConsumerIdsChangeListener.java b/broker/src/main/java/org/apache/rocketmq/broker/ShutdownHook.java
similarity index 77%
copy from broker/src/main/java/org/apache/rocketmq/broker/client/ConsumerIdsChangeListener.java
copy to broker/src/main/java/org/apache/rocketmq/broker/ShutdownHook.java
index 831e293..63567f8 100644
--- a/broker/src/main/java/org/apache/rocketmq/broker/client/ConsumerIdsChangeListener.java
+++ b/broker/src/main/java/org/apache/rocketmq/broker/ShutdownHook.java
@@ -14,9 +14,13 @@
  * See the License for the specific language governing permissions and
  * limitations under the License.
  */
-package org.apache.rocketmq.broker.client;
+package org.apache.rocketmq.broker;
 
-public interface ConsumerIdsChangeListener {
-
-    void handle(ConsumerGroupEvent event, String group, Object... args);
+public interface ShutdownHook {
+    /**
+     * Code to execute before broker shutdown.
+     *
+     * @param controller broker to shutdown
+     */
+    void beforeShutdown(BrokerController controller);
 }
diff --git a/broker/src/main/java/org/apache/rocketmq/broker/client/ClientHousekeepingService.java b/broker/src/main/java/org/apache/rocketmq/broker/client/ClientHousekeepingService.java
index d536db5..e7890a4 100644
--- a/broker/src/main/java/org/apache/rocketmq/broker/client/ClientHousekeepingService.java
+++ b/broker/src/main/java/org/apache/rocketmq/broker/client/ClientHousekeepingService.java
@@ -17,8 +17,8 @@
 package org.apache.rocketmq.broker.client;
 
 import io.netty.channel.Channel;
-import java.util.concurrent.Executors;
 import java.util.concurrent.ScheduledExecutorService;
+import java.util.concurrent.ScheduledThreadPoolExecutor;
 import java.util.concurrent.TimeUnit;
 import org.apache.rocketmq.broker.BrokerController;
 import org.apache.rocketmq.common.ThreadFactoryImpl;
@@ -31,11 +31,12 @@ public class ClientHousekeepingService implements ChannelEventListener {
     private static final InternalLogger log = InternalLoggerFactory.getLogger(LoggerName.BROKER_LOGGER_NAME);
     private final BrokerController brokerController;
 
-    private ScheduledExecutorService scheduledExecutorService = Executors
-        .newSingleThreadScheduledExecutor(new ThreadFactoryImpl("ClientHousekeepingScheduledThread"));
+    private ScheduledExecutorService scheduledExecutorService;
 
     public ClientHousekeepingService(final BrokerController brokerController) {
         this.brokerController = brokerController;
+        scheduledExecutorService = new ScheduledThreadPoolExecutor(1,
+            new ThreadFactoryImpl("ClientHousekeepingScheduledThread", brokerController.getBrokerConfig()));
     }
 
     public void start() {
@@ -64,7 +65,7 @@ public class ClientHousekeepingService implements ChannelEventListener {
 
     @Override
     public void onChannelConnect(String remoteAddr, Channel channel) {
-
+        this.brokerController.getBrokerStatsManager().incChannelConnectNum();
     }
 
     @Override
@@ -72,6 +73,7 @@ public class ClientHousekeepingService implements ChannelEventListener {
         this.brokerController.getProducerManager().doChannelCloseEvent(remoteAddr, channel);
         this.brokerController.getConsumerManager().doChannelCloseEvent(remoteAddr, channel);
         this.brokerController.getFilterServerManager().doChannelCloseEvent(remoteAddr, channel);
+        this.brokerController.getBrokerStatsManager().incChannelCloseNum();
     }
 
     @Override
@@ -79,6 +81,7 @@ public class ClientHousekeepingService implements ChannelEventListener {
         this.brokerController.getProducerManager().doChannelCloseEvent(remoteAddr, channel);
         this.brokerController.getConsumerManager().doChannelCloseEvent(remoteAddr, channel);
         this.brokerController.getFilterServerManager().doChannelCloseEvent(remoteAddr, channel);
+        this.brokerController.getBrokerStatsManager().incChannelExceptionNum();
     }
 
     @Override
@@ -86,5 +89,6 @@ public class ClientHousekeepingService implements ChannelEventListener {
         this.brokerController.getProducerManager().doChannelCloseEvent(remoteAddr, channel);
         this.brokerController.getConsumerManager().doChannelCloseEvent(remoteAddr, channel);
         this.brokerController.getFilterServerManager().doChannelCloseEvent(remoteAddr, channel);
+        this.brokerController.getBrokerStatsManager().incChannelIdleNum();
     }
 }
diff --git a/broker/src/main/java/org/apache/rocketmq/broker/client/ConsumerGroupInfo.java b/broker/src/main/java/org/apache/rocketmq/broker/client/ConsumerGroupInfo.java
index c90d494..09e1241 100644
--- a/broker/src/main/java/org/apache/rocketmq/broker/client/ConsumerGroupInfo.java
+++ b/broker/src/main/java/org/apache/rocketmq/broker/client/ConsumerGroupInfo.java
@@ -68,6 +68,10 @@ public class ConsumerGroupInfo {
         return subscriptionTable;
     }
 
+    public ClientChannelInfo findChannel(final Channel channel) {
+        return this.channelInfoTable.get(channel);
+    }
+
     public ConcurrentMap<Channel, ClientChannelInfo> getChannelInfoTable() {
         return channelInfoTable;
     }
@@ -113,6 +117,15 @@ public class ConsumerGroupInfo {
         return false;
     }
 
+    /**
+     * Update {@link #channelInfoTable} in {@link ConsumerGroupInfo}
+     *
+     * @param infoNew Channel info of new client.
+     * @param consumeType consume type of new client.
+     * @param messageModel message consuming model (CLUSTERING/BROADCASTING) of new client.
+     * @param consumeFromWhere indicate the position when the client consume message firstly.
+     * @return the result that if new connector is connected or not.
+     */
     public boolean updateChannel(final ClientChannelInfo infoNew, ConsumeType consumeType,
         MessageModel messageModel, ConsumeFromWhere consumeFromWhere) {
         boolean updated = false;
@@ -132,9 +145,9 @@ public class ConsumerGroupInfo {
             infoOld = infoNew;
         } else {
             if (!infoOld.getClientId().equals(infoNew.getClientId())) {
-                log.error("[BUG] consumer channel exist in broker, but clientId not equal. GROUP: {} OLD: {} NEW: {} ",
-                    this.groupName,
-                    infoOld.toString(),
+                log.error(
+                    "ConsumerGroupInfo: consumer channel exists in broker, but clientId is not the same one, "
+                        + "group={}, old clientChannelInfo={}, new clientChannelInfo={}", groupName, infoOld.toString(),
                     infoNew.toString());
                 this.channelInfoTable.put(infoNew.getChannel(), infoNew);
             }
@@ -146,6 +159,12 @@ public class ConsumerGroupInfo {
         return updated;
     }
 
+    /**
+     * Update subscription.
+     *
+     * @param subList set of {@link SubscriptionData}
+     * @return the boolean indicates the subscription has changed or not.
+     */
     public boolean updateSubscription(final Set<SubscriptionData> subList) {
         boolean updated = false;
 
diff --git a/broker/src/main/java/org/apache/rocketmq/broker/client/ConsumerIdsChangeListener.java b/broker/src/main/java/org/apache/rocketmq/broker/client/ConsumerIdsChangeListener.java
index 831e293..144092c 100644
--- a/broker/src/main/java/org/apache/rocketmq/broker/client/ConsumerIdsChangeListener.java
+++ b/broker/src/main/java/org/apache/rocketmq/broker/client/ConsumerIdsChangeListener.java
@@ -19,4 +19,6 @@ package org.apache.rocketmq.broker.client;
 public interface ConsumerIdsChangeListener {
 
     void handle(ConsumerGroupEvent event, String group, Object... args);
+
+    void shutdown();
 }
diff --git a/broker/src/main/java/org/apache/rocketmq/broker/client/ConsumerManager.java b/broker/src/main/java/org/apache/rocketmq/broker/client/ConsumerManager.java
index cb60655..b3bee7c 100644
--- a/broker/src/main/java/org/apache/rocketmq/broker/client/ConsumerManager.java
+++ b/broker/src/main/java/org/apache/rocketmq/broker/client/ConsumerManager.java
@@ -16,32 +16,41 @@
  */
 package org.apache.rocketmq.broker.client;
 
-import io.netty.channel.Channel;
 import java.util.HashSet;
 import java.util.Iterator;
 import java.util.Map.Entry;
 import java.util.Set;
 import java.util.concurrent.ConcurrentHashMap;
 import java.util.concurrent.ConcurrentMap;
+
+import io.netty.channel.Channel;
 import org.apache.rocketmq.common.constant.LoggerName;
 import org.apache.rocketmq.common.consumer.ConsumeFromWhere;
-import org.apache.rocketmq.logging.InternalLogger;
-import org.apache.rocketmq.logging.InternalLoggerFactory;
 import org.apache.rocketmq.common.protocol.heartbeat.ConsumeType;
 import org.apache.rocketmq.common.protocol.heartbeat.MessageModel;
 import org.apache.rocketmq.common.protocol.heartbeat.SubscriptionData;
+import org.apache.rocketmq.logging.InternalLogger;
+import org.apache.rocketmq.logging.InternalLoggerFactory;
 import org.apache.rocketmq.remoting.common.RemotingHelper;
 import org.apache.rocketmq.remoting.common.RemotingUtil;
+import org.apache.rocketmq.store.stats.BrokerStatsManager;
 
 public class ConsumerManager {
-    private static final InternalLogger log = InternalLoggerFactory.getLogger(LoggerName.BROKER_LOGGER_NAME);
+    private static final InternalLogger LOGGER = InternalLoggerFactory.getLogger(LoggerName.BROKER_LOGGER_NAME);
     private static final long CHANNEL_EXPIRED_TIMEOUT = 1000 * 120;
-    private final ConcurrentMap<String/* Group */, ConsumerGroupInfo> consumerTable =
+    private final ConcurrentMap<String, ConsumerGroupInfo> consumerTable =
         new ConcurrentHashMap<String, ConsumerGroupInfo>(1024);
     private final ConsumerIdsChangeListener consumerIdsChangeListener;
+    protected final BrokerStatsManager brokerStatsManager;
 
     public ConsumerManager(final ConsumerIdsChangeListener consumerIdsChangeListener) {
         this.consumerIdsChangeListener = consumerIdsChangeListener;
+        this.brokerStatsManager = null;
+    }
+
+    public ConsumerManager(final ConsumerIdsChangeListener consumerIdsChangeListener, final BrokerStatsManager brokerStatsManager) {
+        this.consumerIdsChangeListener = consumerIdsChangeListener;
+        this.brokerStatsManager = brokerStatsManager;
     }
 
     public ClientChannelInfo findChannel(final String group, final String clientId) {
@@ -61,6 +70,10 @@ public class ConsumerManager {
         return null;
     }
 
+    public ConcurrentMap<String, ConsumerGroupInfo> getConsumerTable() {
+        return this.consumerTable;
+    }
+
     public ConsumerGroupInfo getConsumerGroupInfo(final String group) {
         return this.consumerTable.get(group);
     }
@@ -74,17 +87,18 @@ public class ConsumerManager {
         return 0;
     }
 
-    public void doChannelCloseEvent(final String remoteAddr, final Channel channel) {
+    public boolean doChannelCloseEvent(final String remoteAddr, final Channel channel) {
+        boolean removed = false;
         Iterator<Entry<String, ConsumerGroupInfo>> it = this.consumerTable.entrySet().iterator();
         while (it.hasNext()) {
             Entry<String, ConsumerGroupInfo> next = it.next();
             ConsumerGroupInfo info = next.getValue();
-            boolean removed = info.doChannelCloseEvent(remoteAddr, channel);
+            removed = info.doChannelCloseEvent(remoteAddr, channel);
             if (removed) {
                 if (info.getChannelInfoTable().isEmpty()) {
                     ConsumerGroupInfo remove = this.consumerTable.remove(next.getKey());
                     if (remove != null) {
-                        log.info("unregister consumer ok, no any connection, and remove consumer group, {}",
+                        LOGGER.info("unregister consumer ok, no any connection, and remove consumer group, {}",
                             next.getKey());
                         this.consumerIdsChangeListener.handle(ConsumerGroupEvent.UNREGISTER, next.getKey());
                     }
@@ -93,12 +107,13 @@ public class ConsumerManager {
                 this.consumerIdsChangeListener.handle(ConsumerGroupEvent.CHANGE, next.getKey(), info.getAllChannel());
             }
         }
+        return removed;
     }
 
     public boolean registerConsumer(final String group, final ClientChannelInfo clientChannelInfo,
         ConsumeType consumeType, MessageModel messageModel, ConsumeFromWhere consumeFromWhere,
         final Set<SubscriptionData> subList, boolean isNotifyConsumerIdsChangedEnable) {
-
+        long start = System.currentTimeMillis();
         ConsumerGroupInfo consumerGroupInfo = this.consumerTable.get(group);
         if (null == consumerGroupInfo) {
             ConsumerGroupInfo tmp = new ConsumerGroupInfo(group, consumeType, messageModel, consumeFromWhere);
@@ -116,6 +131,9 @@ public class ConsumerManager {
                 this.consumerIdsChangeListener.handle(ConsumerGroupEvent.CHANGE, group, consumerGroupInfo.getAllChannel());
             }
         }
+        if (null != this.brokerStatsManager) {
+            this.brokerStatsManager.incConsumerRegisterTime((int) (System.currentTimeMillis() - start));
+        }
 
         this.consumerIdsChangeListener.handle(ConsumerGroupEvent.REGISTER, group, subList);
 
@@ -130,7 +148,7 @@ public class ConsumerManager {
             if (consumerGroupInfo.getChannelInfoTable().isEmpty()) {
                 ConsumerGroupInfo remove = this.consumerTable.remove(group);
                 if (remove != null) {
-                    log.info("unregister consumer ok, no any connection, and remove consumer group, {}", group);
+                    LOGGER.info("unregister consumer ok, no any connection, and remove consumer group, {}", group);
 
                     this.consumerIdsChangeListener.handle(ConsumerGroupEvent.UNREGISTER, group);
                 }
@@ -156,7 +174,7 @@ public class ConsumerManager {
                 ClientChannelInfo clientChannelInfo = nextChannel.getValue();
                 long diff = System.currentTimeMillis() - clientChannelInfo.getLastUpdateTimestamp();
                 if (diff > CHANNEL_EXPIRED_TIMEOUT) {
-                    log.warn(
+                    LOGGER.warn(
                         "SCAN: remove expired channel from ConsumerManager consumerTable. channel={}, consumerGroup={}",
                         RemotingHelper.parseChannelRemoteAddr(clientChannelInfo.getChannel()), group);
                     RemotingUtil.closeChannel(clientChannelInfo.getChannel());
@@ -165,7 +183,7 @@ public class ConsumerManager {
             }
 
             if (channelInfoTable.isEmpty()) {
-                log.warn(
+                LOGGER.warn(
                     "SCAN: remove expired channel from ConsumerManager consumerTable, all clear, consumerGroup={}",
                     group);
                 it.remove();
diff --git a/broker/src/main/java/org/apache/rocketmq/broker/client/DefaultConsumerIdsChangeListener.java b/broker/src/main/java/org/apache/rocketmq/broker/client/DefaultConsumerIdsChangeListener.java
index d716a33..8e6e667 100644
--- a/broker/src/main/java/org/apache/rocketmq/broker/client/DefaultConsumerIdsChangeListener.java
+++ b/broker/src/main/java/org/apache/rocketmq/broker/client/DefaultConsumerIdsChangeListener.java
@@ -17,18 +17,46 @@
 package org.apache.rocketmq.broker.client;
 
 import io.netty.channel.Channel;
-
 import java.util.Collection;
 import java.util.List;
+import java.util.Map;
+import java.util.concurrent.ConcurrentHashMap;
+import java.util.concurrent.ScheduledExecutorService;
+import java.util.concurrent.ScheduledThreadPoolExecutor;
+import java.util.concurrent.TimeUnit;
 
 import org.apache.rocketmq.broker.BrokerController;
+import org.apache.rocketmq.common.AbstractBrokerRunnable;
+import org.apache.rocketmq.common.constant.LoggerName;
 import org.apache.rocketmq.common.protocol.heartbeat.SubscriptionData;
+import org.apache.rocketmq.common.utils.ThreadUtils;
+import org.apache.rocketmq.logging.InternalLogger;
+import org.apache.rocketmq.logging.InternalLoggerFactory;
 
 public class DefaultConsumerIdsChangeListener implements ConsumerIdsChangeListener {
+    private static final InternalLogger log = InternalLoggerFactory.getLogger(LoggerName.BROKER_LOGGER_NAME);
     private final BrokerController brokerController;
+    private final int cacheSize = 8096;
+
+    private final ScheduledExecutorService scheduledExecutorService =  new ScheduledThreadPoolExecutor(1,
+        ThreadUtils.newGenericThreadFactory("DefaultConsumerIdsChangeListener", true));
+
+    private ConcurrentHashMap<String,List<Channel>> consumerChannelMap = new ConcurrentHashMap<>(cacheSize);
 
     public DefaultConsumerIdsChangeListener(BrokerController brokerController) {
         this.brokerController = brokerController;
+
+        scheduledExecutorService.scheduleAtFixedRate(new AbstractBrokerRunnable(brokerController.getBrokerConfig()) {
+            @Override
+            public void run2() {
+                try {
+                    notifyConsumerChange();
+                } catch (Exception e) {
+                    log.error(
+                        "DefaultConsumerIdsChangeListen#notifyConsumerChange: unexpected error occurs", e);
+                }
+            }
+        }, 30, 15, TimeUnit.SECONDS);
     }
 
     @Override
@@ -43,8 +71,12 @@ public class DefaultConsumerIdsChangeListener implements ConsumerIdsChangeListen
                 }
                 List<Channel> channels = (List<Channel>) args[0];
                 if (channels != null && brokerController.getBrokerConfig().isNotifyConsumerIdsChangedEnable()) {
-                    for (Channel chl : channels) {
-                        this.brokerController.getBroker2Client().notifyConsumerIdsChanged(chl, group);
+                    if (this.brokerController.getBrokerConfig().isRealTimeNotifyConsumerChange()) {
+                        for (Channel chl : channels) {
+                            this.brokerController.getBroker2Client().notifyConsumerIdsChanged(chl, group);
+                        }
+                    } else {
+                        consumerChannelMap.put(group, channels);
                     }
                 }
                 break;
@@ -62,4 +94,34 @@ public class DefaultConsumerIdsChangeListener implements ConsumerIdsChangeListen
                 throw new RuntimeException("Unknown event " + event);
         }
     }
+
+    private void notifyConsumerChange() {
+
+        if (consumerChannelMap.isEmpty()) {
+            return;
+        }
+
+        ConcurrentHashMap<String, List<Channel>> processMap = new ConcurrentHashMap<>(consumerChannelMap);
+        consumerChannelMap = new ConcurrentHashMap<>(cacheSize);
+
+        for (Map.Entry<String, List<Channel>> entry : processMap.entrySet()) {
+            String consumerId = entry.getKey();
+            List<Channel> channelList = entry.getValue();
+            try {
+                if (channelList != null && brokerController.getBrokerConfig().isNotifyConsumerIdsChangedEnable()) {
+                    for (Channel chl : channelList) {
+                        this.brokerController.getBroker2Client().notifyConsumerIdsChanged(chl, consumerId);
+                    }
+                }
+            } catch (Exception e) {
+                log.error("Failed to notify consumer when some consumers changed, consumerId to notify: {}",
+                    consumerId, e);
+            }
+        }
+    }
+
+    @Override
+    public void shutdown() {
+        this.scheduledExecutorService.shutdown();
+    }
 }
diff --git a/broker/src/main/java/org/apache/rocketmq/broker/client/ProducerManager.java b/broker/src/main/java/org/apache/rocketmq/broker/client/ProducerManager.java
index 4bd00ef..11b9088 100644
--- a/broker/src/main/java/org/apache/rocketmq/broker/client/ProducerManager.java
+++ b/broker/src/main/java/org/apache/rocketmq/broker/client/ProducerManager.java
@@ -29,6 +29,7 @@ import org.apache.rocketmq.logging.InternalLogger;
 import org.apache.rocketmq.logging.InternalLoggerFactory;
 import org.apache.rocketmq.remoting.common.RemotingHelper;
 import org.apache.rocketmq.remoting.common.RemotingUtil;
+import org.apache.rocketmq.store.stats.BrokerStatsManager;
 
 public class ProducerManager {
     private static final InternalLogger log = InternalLoggerFactory.getLogger(LoggerName.BROKER_LOGGER_NAME);
@@ -37,9 +38,24 @@ public class ProducerManager {
     private final ConcurrentHashMap<String /* group name */, ConcurrentHashMap<Channel, ClientChannelInfo>> groupChannelTable =
         new ConcurrentHashMap<>();
     private final ConcurrentHashMap<String, Channel> clientChannelTable = new ConcurrentHashMap<>();
+    protected final BrokerStatsManager brokerStatsManager;
     private PositiveAtomicCounter positiveAtomicCounter = new PositiveAtomicCounter();
 
     public ProducerManager() {
+        this.brokerStatsManager = null;
+    }
+
+    public ProducerManager(final BrokerStatsManager brokerStatsManager) {
+        this.brokerStatsManager = brokerStatsManager;
+    }
+
+    public int groupSize() {
+        return this.groupChannelTable.size();
+    }
+
+    public boolean groupOnline(String group) {
+        Map<Channel, ClientChannelInfo> channels = this.groupChannelTable.get(group);
+        return channels != null && !channels.isEmpty();
     }
 
     public ConcurrentHashMap<String, ConcurrentHashMap<Channel, ClientChannelInfo>> getGroupChannelTable() {
@@ -63,7 +79,7 @@ public class ProducerManager {
                     it.remove();
                     clientChannelTable.remove(info.getClientId());
                     log.warn(
-                            "SCAN: remove expired channel[{}] from ProducerManager groupChannelTable, producer group name: {}",
+                            "ProducerManager#scanNotActiveChannel: remove expired channel[{}] from ProducerManager groupChannelTable, producer group name: {}",
                             RemotingHelper.parseChannelRemoteAddr(info.getChannel()), group);
                     RemotingUtil.closeChannel(info.getChannel());
                 }
@@ -71,7 +87,8 @@ public class ProducerManager {
         }
     }
 
-    public synchronized void doChannelCloseEvent(final String remoteAddr, final Channel channel) {
+    public synchronized boolean doChannelCloseEvent(final String remoteAddr, final Channel channel) {
+        boolean removed = false;
         if (channel != null) {
             for (final Map.Entry<String, ConcurrentHashMap<Channel, ClientChannelInfo>> entry : this.groupChannelTable
                     .entrySet()) {
@@ -82,6 +99,7 @@ public class ProducerManager {
                         clientChannelInfoTable.remove(channel);
                 if (clientChannelInfo != null) {
                     clientChannelTable.remove(clientChannelInfo.getClientId());
+                    removed = true;
                     log.info(
                             "NETTY EVENT: remove channel[{}][{}] from ProducerManager groupChannelTable, producer group: {}",
                             clientChannelInfo.toString(), remoteAddr, group);
@@ -89,6 +107,7 @@ public class ProducerManager {
 
             }
         }
+        return removed;
     }
 
     public synchronized void registerProducer(final String group, final ClientChannelInfo clientChannelInfo) {
diff --git a/broker/src/main/java/org/apache/rocketmq/broker/client/net/Broker2Client.java b/broker/src/main/java/org/apache/rocketmq/broker/client/net/Broker2Client.java
index 4d28cd8..b78129c 100644
--- a/broker/src/main/java/org/apache/rocketmq/broker/client/net/Broker2Client.java
+++ b/broker/src/main/java/org/apache/rocketmq/broker/client/net/Broker2Client.java
@@ -76,7 +76,7 @@ public class Broker2Client {
     }
 
     public RemotingCommand callClient(final Channel channel,
-                                      final RemotingCommand request
+        final RemotingCommand request
     ) throws RemotingSendRequestException, RemotingTimeoutException, InterruptedException {
         return this.brokerController.getRemotingServer().invokeSync(channel, request, 10000);
     }
@@ -101,12 +101,13 @@ public class Broker2Client {
         }
     }
 
+
     public RemotingCommand resetOffset(String topic, String group, long timeStamp, boolean isForce) {
         return resetOffset(topic, group, timeStamp, isForce, false);
     }
 
     public RemotingCommand resetOffset(String topic, String group, long timeStamp, boolean isForce,
-                                       boolean isC) {
+        boolean isC) {
         final RemotingCommand response = RemotingCommand.createResponseCommand(null);
 
         TopicConfig topicConfig = this.brokerController.getTopicConfigManager().selectTopicConfig(topic);
@@ -237,8 +238,7 @@ public class Broker2Client {
             RemotingCommand.createRequestCommand(RequestCode.GET_CONSUMER_STATUS_FROM_CLIENT,
                 requestHeader);
 
-        Map<String, Map<MessageQueue, Long>> consumerStatusTable =
-            new HashMap<String, Map<MessageQueue, Long>>();
+        Map<String, Map<MessageQueue, Long>> consumerStatusTable = new HashMap<String, Map<MessageQueue, Long>>();
         ConcurrentMap<Channel, ClientChannelInfo> channelInfoTable =
             this.brokerController.getConsumerManager().getConsumerGroupInfo(group).getChannelInfoTable();
         if (null == channelInfoTable || channelInfoTable.isEmpty()) {
diff --git a/broker/src/main/java/org/apache/rocketmq/broker/client/rebalance/RebalanceLockManager.java b/broker/src/main/java/org/apache/rocketmq/broker/client/rebalance/RebalanceLockManager.java
index 9056998..a0abd70 100644
--- a/broker/src/main/java/org/apache/rocketmq/broker/client/rebalance/RebalanceLockManager.java
+++ b/broker/src/main/java/org/apache/rocketmq/broker/client/rebalance/RebalanceLockManager.java
@@ -36,6 +36,19 @@ public class RebalanceLockManager {
     private final ConcurrentMap<String/* group */, ConcurrentHashMap<MessageQueue, LockEntry>> mqLockTable =
         new ConcurrentHashMap<String, ConcurrentHashMap<MessageQueue, LockEntry>>(1024);
 
+    public boolean isLockAllExpired(final String group) {
+        final ConcurrentHashMap<MessageQueue, LockEntry> lockEntryMap = mqLockTable.get(group);
+        if (null == lockEntryMap) {
+            return true;
+        }
+        for (LockEntry entry : lockEntryMap.values()) {
+            if (!entry.isExpired()) {
+                return false;
+            }
+        }
+        return true;
+    }
+
     public boolean tryLock(final String group, final MessageQueue mq, final String clientId) {
 
         if (!this.isLocked(group, mq, clientId)) {
@@ -53,10 +66,9 @@ public class RebalanceLockManager {
                         lockEntry = new LockEntry();
                         lockEntry.setClientId(clientId);
                         groupValue.put(mq, lockEntry);
-                        log.info("tryLock, message queue not locked, I got it. Group: {} NewClientId: {} {}",
-                            group,
-                            clientId,
-                            mq);
+                        log.info(
+                            "RebalanceLockManager#tryLock: lock a message queue which has not been locked yet, "
+                                + "group={}, clientId={}, mq={}", group, clientId, mq);
                     }
 
                     if (lockEntry.isLocked(clientId)) {
@@ -70,26 +82,21 @@ public class RebalanceLockManager {
                         lockEntry.setClientId(clientId);
                         lockEntry.setLastUpdateTimestamp(System.currentTimeMillis());
                         log.warn(
-                            "tryLock, message queue lock expired, I got it. Group: {} OldClientId: {} NewClientId: {} {}",
-                            group,
-                            oldClientId,
-                            clientId,
-                            mq);
+                            "RebalanceLockManager#tryLock: try to lock a expired message queue, group={}, mq={}, old "
+                                + "client id={}, new client id={}", group, mq, oldClientId, clientId);
                         return true;
                     }
 
                     log.warn(
-                        "tryLock, message queue locked by other client. Group: {} OtherClientId: {} NewClientId: {} {}",
-                        group,
-                        oldClientId,
-                        clientId,
-                        mq);
+                        "RebalanceLockManager#tryLock: message queue has been locked by other client, group={}, "
+                            + "mq={}, locked client id={}, current client id={}", group, mq, oldClientId, clientId);
                     return false;
                 } finally {
                     this.lock.unlock();
                 }
             } catch (InterruptedException e) {
-                log.error("putMessage exception", e);
+                log.error("RebalanceLockManager#tryLock: unexpected error, group={}, mq={}, clientId={}", group, mq,
+                    clientId, e);
             }
         } else {
 
@@ -145,10 +152,8 @@ public class RebalanceLockManager {
                             lockEntry.setClientId(clientId);
                             groupValue.put(mq, lockEntry);
                             log.info(
-                                "tryLockBatch, message queue not locked, I got it. Group: {} NewClientId: {} {}",
-                                group,
-                                clientId,
-                                mq);
+                                "RebalanceLockManager#tryLockBatch: lock a message which has not been locked yet, "
+                                    + "group={}, clientId={}, mq={}", group, clientId, mq);
                         }
 
                         if (lockEntry.isLocked(clientId)) {
@@ -163,27 +168,23 @@ public class RebalanceLockManager {
                             lockEntry.setClientId(clientId);
                             lockEntry.setLastUpdateTimestamp(System.currentTimeMillis());
                             log.warn(
-                                "tryLockBatch, message queue lock expired, I got it. Group: {} OldClientId: {} NewClientId: {} {}",
-                                group,
-                                oldClientId,
-                                clientId,
-                                mq);
+                                "RebalanceLockManager#tryLockBatch: try to lock a expired message queue, group={}, "
+                                    + "mq={}, old client id={}, new client id={}", group, mq, oldClientId, clientId);
                             lockedMqs.add(mq);
                             continue;
                         }
 
                         log.warn(
-                            "tryLockBatch, message queue locked by other client. Group: {} OtherClientId: {} NewClientId: {} {}",
-                            group,
-                            oldClientId,
-                            clientId,
-                            mq);
+                            "RebalanceLockManager#tryLockBatch: message queue has been locked by other client, "
+                                + "group={}, mq={}, locked client id={}, current client id={}", group, mq, oldClientId,
+                            clientId);
                     }
                 } finally {
                     this.lock.unlock();
                 }
             } catch (InterruptedException e) {
-                log.error("putMessage exception", e);
+                log.error("RebalanceLockManager#tryBatch: unexpected error, group={}, mqs={}, clientId={}", group, mqs,
+                    clientId, e);
             }
         }
 
@@ -201,34 +202,29 @@ public class RebalanceLockManager {
                         if (null != lockEntry) {
                             if (lockEntry.getClientId().equals(clientId)) {
                                 groupValue.remove(mq);
-                                log.info("unlockBatch, Group: {} {} {}",
-                                    group,
-                                    mq,
-                                    clientId);
+                                log.info("RebalanceLockManager#unlockBatch: unlock mq, group={}, clientId={}, mqs={}",
+                                    group, clientId, mq);
                             } else {
-                                log.warn("unlockBatch, but mq locked by other client: {}, Group: {} {} {}",
-                                    lockEntry.getClientId(),
-                                    group,
-                                    mq,
-                                    clientId);
+                                log.warn(
+                                    "RebalanceLockManager#unlockBatch: mq locked by other client, group={}, locked "
+                                        + "clientId={}, current clientId={}, mqs={}", group, lockEntry.getClientId(),
+                                    clientId, mq);
                             }
                         } else {
-                            log.warn("unlockBatch, but mq not locked, Group: {} {} {}",
-                                group,
-                                mq,
-                                clientId);
+                            log.warn("RebalanceLockManager#unlockBatch: mq not locked, group={}, clientId={}, mq={}",
+                                group, clientId, mq);
                         }
                     }
                 } else {
-                    log.warn("unlockBatch, group not exist, Group: {} {}",
-                        group,
-                        clientId);
+                    log.warn("RebalanceLockManager#unlockBatch: group not exist, group={}, clientId={}, mqs={}", group,
+                        clientId, mqs);
                 }
             } finally {
                 this.lock.unlock();
             }
         } catch (InterruptedException e) {
-            log.error("putMessage exception", e);
+            log.error("RebalanceLockManager#unlockBatch: unexpected error, group={}, mqs={}, clientId={}", group, mqs,
+                clientId);
         }
     }
 
diff --git a/broker/src/main/java/org/apache/rocketmq/broker/dledger/DLedgerRoleChangeHandler.java b/broker/src/main/java/org/apache/rocketmq/broker/dledger/DLedgerRoleChangeHandler.java
index 09bf10c..3f5f024 100644
--- a/broker/src/main/java/org/apache/rocketmq/broker/dledger/DLedgerRoleChangeHandler.java
+++ b/broker/src/main/java/org/apache/rocketmq/broker/dledger/DLedgerRoleChangeHandler.java
@@ -22,6 +22,8 @@ import io.openmessaging.storage.dledger.MemberState;
 import io.openmessaging.storage.dledger.utils.DLedgerUtils;
 import java.util.concurrent.ExecutorService;
 import java.util.concurrent.Executors;
+import java.util.concurrent.Future;
+import java.util.concurrent.TimeUnit;
 import org.apache.rocketmq.broker.BrokerController;
 import org.apache.rocketmq.common.ThreadFactoryImpl;
 import org.apache.rocketmq.common.constant.LoggerName;
@@ -33,12 +35,14 @@ import org.apache.rocketmq.store.dledger.DLedgerCommitLog;
 
 public class DLedgerRoleChangeHandler implements DLedgerLeaderElector.RoleChangeHandler {
 
-    private static final InternalLogger log = InternalLoggerFactory.getLogger(LoggerName.BROKER_LOGGER_NAME);
+    private static final InternalLogger LOGGER = InternalLoggerFactory.getLogger(LoggerName.BROKER_LOGGER_NAME);
     private ExecutorService executorService = Executors.newSingleThreadExecutor(new ThreadFactoryImpl("DLegerRoleChangeHandler_"));
     private BrokerController brokerController;
     private DefaultMessageStore messageStore;
     private DLedgerCommitLog dLedgerCommitLog;
     private DLedgerServer dLegerServer;
+    private Future<?> slaveSyncFuture;
+
     public DLedgerRoleChangeHandler(BrokerController brokerController, DefaultMessageStore messageStore) {
         this.brokerController = brokerController;
         this.messageStore = messageStore;
@@ -52,15 +56,15 @@ public class DLedgerRoleChangeHandler implements DLedgerLeaderElector.RoleChange
                 long start = System.currentTimeMillis();
                 try {
                     boolean succ = true;
-                    log.info("Begin handling broker role change term={} role={} currStoreRole={}", term, role, messageStore.getMessageStoreConfig().getBrokerRole());
+                    LOGGER.info("Begin handling broker role change term={} role={} currStoreRole={}", term, role, messageStore.getMessageStoreConfig().getBrokerRole());
                     switch (role) {
                         case CANDIDATE:
                             if (messageStore.getMessageStoreConfig().getBrokerRole() != BrokerRole.SLAVE) {
-                                brokerController.changeToSlave(dLedgerCommitLog.getId());
+                                changeToSlave(dLedgerCommitLog.getId());
                             }
                             break;
                         case FOLLOWER:
-                            brokerController.changeToSlave(dLedgerCommitLog.getId());
+                            changeToSlave(dLedgerCommitLog.getId());
                             break;
                         case LEADER:
                             while (true) {
@@ -79,21 +83,89 @@ public class DLedgerRoleChangeHandler implements DLedgerLeaderElector.RoleChange
                             }
                             if (succ) {
                                 messageStore.recoverTopicQueueTable();
-                                brokerController.changeToMaster(BrokerRole.SYNC_MASTER);
+                                changeToMaster(BrokerRole.SYNC_MASTER);
                             }
                             break;
                         default:
                             break;
                     }
-                    log.info("Finish handling broker role change succ={} term={} role={} currStoreRole={} cost={}", succ, term, role, messageStore.getMessageStoreConfig().getBrokerRole(), DLedgerUtils.elapsed(start));
+                    LOGGER.info("Finish handling broker role change succ={} term={} role={} currStoreRole={} cost={}", succ, term, role, messageStore.getMessageStoreConfig().getBrokerRole(), DLedgerUtils.elapsed(start));
                 } catch (Throwable t) {
-                    log.info("[MONITOR]Failed handling broker role change term={} role={} currStoreRole={} cost={}", term, role, messageStore.getMessageStoreConfig().getBrokerRole(), DLedgerUtils.elapsed(start), t);
+                    LOGGER.info("[MONITOR]Failed handling broker role change term={} role={} currStoreRole={} cost={}", term, role, messageStore.getMessageStoreConfig().getBrokerRole(), DLedgerUtils.elapsed(start), t);
                 }
             }
         };
         executorService.submit(runnable);
     }
 
+    private void handleSlaveSynchronize(BrokerRole role) {
+        if (role == BrokerRole.SLAVE) {
+            if (null != slaveSyncFuture) {
+                slaveSyncFuture.cancel(false);
+            }
+            this.brokerController.getSlaveSynchronize().setMasterAddr(null);
+            slaveSyncFuture = this.brokerController.getScheduledExecutorService().scheduleAtFixedRate(new Runnable() {
+                @Override
+                public void run() {
+                    try {
+                        brokerController.getSlaveSynchronize().syncAll();
+                    } catch (Throwable e) {
+                        LOGGER.error("ScheduledTask SlaveSynchronize syncAll error.", e);
+                    }
+                }
+            }, 1000 * 3, 1000 * 10, TimeUnit.MILLISECONDS);
+        } else {
+            //handle the slave synchronise
+            if (null != slaveSyncFuture) {
+                slaveSyncFuture.cancel(false);
+            }
+            this.brokerController.getSlaveSynchronize().setMasterAddr(null);
+        }
+    }
+
+    public void changeToSlave(int brokerId) {
+        LOGGER.info("Begin to change to slave brokerName={} brokerId={}", this.brokerController.getBrokerConfig().getBrokerName(), brokerId);
+
+        //change the role
+        this.brokerController.getBrokerConfig().setBrokerId(brokerId == 0 ? 1 : brokerId); //TO DO check
+        this.brokerController.getMessageStoreConfig().setBrokerRole(BrokerRole.SLAVE);
+
+        this.brokerController.changeSpecialServiceStatus(false);
+
+        //handle the slave synchronise
+        handleSlaveSynchronize(BrokerRole.SLAVE);
+
+        try {
+            this.brokerController.registerBrokerAll(true, true, this.brokerController.getBrokerConfig().isForceRegister());
+        } catch (Throwable ignored) {
+
+        }
+        LOGGER.info("Finish to change to slave brokerName={} brokerId={}", this.brokerController.getBrokerConfig().getBrokerName(), brokerId);
+    }
+
+    public void changeToMaster(BrokerRole role) {
+        if (role == BrokerRole.SLAVE) {
+            return;
+        }
+        LOGGER.info("Begin to change to master brokerName={}", this.brokerController.getBrokerConfig().getBrokerName());
+
+        //handle the slave synchronise
+        handleSlaveSynchronize(role);
+
+        this.brokerController.changeSpecialServiceStatus(true);
+
+        //if the operations above are totally successful, we change to master
+        this.brokerController.getBrokerConfig().setBrokerId(0); //TO DO check
+        this.brokerController.getMessageStoreConfig().setBrokerRole(role);
+
+        try {
+            this.brokerController.registerBrokerAll(true, true, this.brokerController.getBrokerConfig().isForceRegister());
+        } catch (Throwable ignored) {
+
+        }
+        LOGGER.info("Finish to change to master brokerName={}", this.brokerController.getBrokerConfig().getBrokerName());
+    }
+
     @Override public void startup() {
 
     }
diff --git a/broker/src/main/java/org/apache/rocketmq/broker/failover/EscapeBridge.java b/broker/src/main/java/org/apache/rocketmq/broker/failover/EscapeBridge.java
new file mode 100644
index 0000000..c7f10a2
--- /dev/null
+++ b/broker/src/main/java/org/apache/rocketmq/broker/failover/EscapeBridge.java
@@ -0,0 +1,268 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.rocketmq.broker.failover;
+
+import java.nio.ByteBuffer;
+import java.util.ArrayList;
+import java.util.List;
+import java.util.concurrent.CompletableFuture;
+import org.apache.rocketmq.broker.BrokerController;
+import org.apache.rocketmq.client.consumer.DefaultMQPullConsumer;
+import org.apache.rocketmq.client.consumer.PullResult;
+import org.apache.rocketmq.client.consumer.PullStatus;
+import org.apache.rocketmq.client.exception.MQClientException;
+import org.apache.rocketmq.client.producer.DefaultMQProducer;
+import org.apache.rocketmq.client.producer.MessageQueueSelector;
+import org.apache.rocketmq.client.producer.SendCallback;
+import org.apache.rocketmq.client.producer.SendResult;
+import org.apache.rocketmq.common.constant.LoggerName;
+import org.apache.rocketmq.common.message.Message;
+import org.apache.rocketmq.common.message.MessageDecoder;
+import org.apache.rocketmq.common.message.MessageExt;
+import org.apache.rocketmq.common.message.MessageQueue;
+import org.apache.rocketmq.logging.InternalLogger;
+import org.apache.rocketmq.logging.InternalLoggerFactory;
+import org.apache.rocketmq.store.GetMessageResult;
+import org.apache.rocketmq.common.message.MessageExtBrokerInner;
+import org.apache.rocketmq.store.MessageStore;
+import org.apache.rocketmq.store.PutMessageResult;
+import org.apache.rocketmq.store.PutMessageStatus;
+
+public class EscapeBridge {
+    protected static final InternalLogger LOG = InternalLoggerFactory.getLogger(LoggerName.BROKER_LOGGER_NAME);
+    private final String innerProducerGroupName;
+    private final String innerConsumerGroupName;
+
+    private final BrokerController brokerController;
+
+    private DefaultMQProducer innerProducer;
+    private DefaultMQPullConsumer innerConsumer;
+
+    public EscapeBridge(BrokerController brokerController) {
+        this.brokerController = brokerController;
+        this.innerProducerGroupName = "InnerProducerGroup_" + brokerController.getBrokerConfig().getBrokerName() + "_" + brokerController.getBrokerConfig().getBrokerId();
+        this.innerConsumerGroupName = "InnerConsumerGroup_" + brokerController.getBrokerConfig().getBrokerName() + "_" + brokerController.getBrokerConfig().getBrokerId();
+    }
+
+    public void start() throws Exception {
+        if (brokerController.getBrokerConfig().isEnableSlaveActingMaster() && brokerController.getBrokerConfig().isEnableRemoteEscape()) {
+            String nameserver = this.brokerController.getNameServerList();
+            if (nameserver != null && !nameserver.isEmpty()) {
+                startInnerProducer(nameserver);
+                startInnerConsumer(nameserver);
+                LOG.info("start inner producer and consumer success.");
+            } else {
+                throw new RuntimeException("nameserver address is null or empty");
+            }
+        }
+    }
+
+    public void shutdown() {
+        if (this.innerProducer != null) {
+            this.innerProducer.shutdown();
+        }
+
+        if (this.innerConsumer != null) {
+            this.innerConsumer.shutdown();
+        }
+    }
+
+    private void startInnerProducer(String nameServer) throws MQClientException {
+        try {
+            innerProducer = new DefaultMQProducer(innerProducerGroupName);
+            innerProducer.setNamesrvAddr(nameServer);
+            innerProducer.start();
+        } catch (MQClientException e) {
+            LOG.error("start inner producer failed, nameserver address: {}", nameServer, e);
+            throw e;
+        }
+    }
+
+    private void startInnerConsumer(String nameServer) throws MQClientException {
+        try {
+            innerConsumer = new DefaultMQPullConsumer(innerConsumerGroupName);
+            innerConsumer.setNamesrvAddr(nameServer);
+            innerConsumer.start();
+        } catch (MQClientException e) {
+            LOG.error("start inner consumer failed, nameserver address: {}", nameServer, e);
+            throw e;
+        }
+    }
+
+    public PutMessageResult putMessage(MessageExtBrokerInner messageExt) {
+        BrokerController masterBroker = this.brokerController.peekMasterBroker();
+        if (masterBroker != null) {
+            return masterBroker.getMessageStore().putMessage(messageExt);
+        } else if (this.brokerController.getBrokerConfig().isEnableSlaveActingMaster()
+            && this.brokerController.getBrokerConfig().isEnableRemoteEscape()
+            && innerProducer != null) {
+            // Remote Acting lead to born timestamp, msgId changed, it need to polish.
+            try {
+                messageExt.setWaitStoreMsgOK(false);
+                SendResult sendResult = innerProducer.send(messageExt);
+                return transformSendResult2PutResult(sendResult);
+            } catch (Exception e) {
+                LOG.error("sendMessageInFailover to remote failed", e);
+                return new PutMessageResult(PutMessageStatus.PUT_TO_REMOTE_BROKER_FAIL, null, true);
+            }
+        } else {
+            LOG.warn("Put message failed, enableSlaveActingMaster={}, enableRemoteEscape={}.",
+                this.brokerController.getBrokerConfig().isEnableSlaveActingMaster(), this.brokerController.getBrokerConfig().isEnableRemoteEscape());
+            return new PutMessageResult(PutMessageStatus.SERVICE_NOT_AVAILABLE, null);
+        }
+    }
+
+    public CompletableFuture<PutMessageResult> asyncPutMessage(MessageExtBrokerInner messageExt) {
+        BrokerController masterBroker = this.brokerController.peekMasterBroker();
+        CompletableFuture<PutMessageResult> completableFuture = new CompletableFuture<>();
+        if (masterBroker != null) {
+            return masterBroker.getMessageStore().asyncPutMessage(messageExt);
+        } else if (this.brokerController.getBrokerConfig().isEnableSlaveActingMaster()
+            && this.brokerController.getBrokerConfig().isEnableRemoteEscape()
+            && innerProducer != null) {
+            // Remote Acting lead to born timestamp, msgId changed, it need to polish.
+            try {
+                messageExt.setWaitStoreMsgOK(false);
+                innerProducer.send(messageExt, new SendCallback() {
+                    @Override public void onSuccess(SendResult sendResult) {
+                        completableFuture.complete(transformSendResult2PutResult(sendResult));
+                    }
+
+                    @Override public void onException(Throwable e) {
+                        completableFuture.complete(new PutMessageResult(PutMessageStatus.PUT_TO_REMOTE_BROKER_FAIL, null, true));
+                    }
+                });
+                return completableFuture;
+            } catch (Exception e) {
+                LOG.error("sendMessageInFailover to remote failed", e);
+                return CompletableFuture.completedFuture(new PutMessageResult(PutMessageStatus.PUT_TO_REMOTE_BROKER_FAIL, null, true));
+            }
+        } else {
+            LOG.warn("Put message failed, enableSlaveActingMaster={}, enableRemoteEscape={}.",
+                this.brokerController.getBrokerConfig().isEnableSlaveActingMaster(), this.brokerController.getBrokerConfig().isEnableRemoteEscape());
+            return CompletableFuture.completedFuture(new PutMessageResult(PutMessageStatus.SERVICE_NOT_AVAILABLE, null));
+        }
+    }
+
+    public PutMessageResult putMessageToSpecificQueue(MessageExtBrokerInner messageExt) {
+        BrokerController masterBroker = this.brokerController.peekMasterBroker();
+        if (masterBroker != null) {
+            return masterBroker.getMessageStore().putMessage(messageExt);
+        } else if (this.brokerController.getBrokerConfig().isEnableSlaveActingMaster()
+            && this.brokerController.getBrokerConfig().isEnableRemoteEscape()
+            && this.innerProducer != null) {
+            try {
+                messageExt.setWaitStoreMsgOK(false);
+                // Remote Acting lead to born timestamp, msgId changed, it need to polish.
+                SendResult sendResult = innerProducer.send(messageExt, new MessageQueueSelector() {
+                    @Override
+                    public MessageQueue select(List<MessageQueue> mqs, Message msg, Object arg) {
+                        String id = (String) arg;
+                        int index = Math.abs(id.hashCode()) % mqs.size();
+                        return mqs.get(index);
+                    }
+                }, messageExt.getTopic() + messageExt.getStoreHost());
+                return transformSendResult2PutResult(sendResult);
+            } catch (Exception e) {
+                LOG.error("sendMessageInFailover to remote failed", e);
+                return new PutMessageResult(PutMessageStatus.PUT_TO_REMOTE_BROKER_FAIL, null, true);
+            }
+        } else {
+            LOG.warn("Put message to specific queue failed, enableSlaveActingMaster={}, enableRemoteEscape={}.",
+                this.brokerController.getBrokerConfig().isEnableSlaveActingMaster(), this.brokerController.getBrokerConfig().isEnableRemoteEscape());
+            return new PutMessageResult(PutMessageStatus.SERVICE_NOT_AVAILABLE, null);
+        }
+    }
+
+    private PutMessageResult transformSendResult2PutResult(SendResult sendResult) {
+        if (sendResult == null) {
+            return new PutMessageResult(PutMessageStatus.PUT_TO_REMOTE_BROKER_FAIL, null, true);
+        }
+        switch (sendResult.getSendStatus()) {
+            case SEND_OK:
+                return new PutMessageResult(PutMessageStatus.PUT_OK, null, true);
+            case SLAVE_NOT_AVAILABLE:
+                return new PutMessageResult(PutMessageStatus.SLAVE_NOT_AVAILABLE, null, true);
+            case FLUSH_DISK_TIMEOUT:
+                return new PutMessageResult(PutMessageStatus.FLUSH_DISK_TIMEOUT, null, true);
+            case FLUSH_SLAVE_TIMEOUT:
+                return new PutMessageResult(PutMessageStatus.FLUSH_SLAVE_TIMEOUT, null, true);
+            default:
+                return new PutMessageResult(PutMessageStatus.PUT_TO_REMOTE_BROKER_FAIL, null, true);
+        }
+    }
+
+    public MessageExt getMessage(String topic, long offset, int queueId, String brokerName) {
+        MessageStore messageStore = brokerController.getMessageStoreByBrokerName(brokerName);
+        if (messageStore != null) {
+            final GetMessageResult getMessageTmpResult = messageStore.getMessage(innerConsumerGroupName, topic, queueId, offset, 1, null);
+            List<MessageExt> list = decodeMsgList(getMessageTmpResult);
+            if (list == null || list.isEmpty()) {
+                LOG.warn("Can not get msg , topic {}, offset {}, queueId {}, result is {}", topic, offset, queueId, getMessageTmpResult);
+                return null;
+            } else {
+                return list.get(0);
+            }
+        } else if (innerConsumer != null) {
+            return getMessageFromRemote(topic, offset, queueId, brokerName);
+        } else {
+            return null;
+        }
+    }
+
+    protected List<MessageExt> decodeMsgList(GetMessageResult getMessageResult) {
+        List<MessageExt> foundList = new ArrayList<>();
+        try {
+            List<ByteBuffer> messageBufferList = getMessageResult.getMessageBufferList();
+            if (messageBufferList != null) {
+                for (int i = 0; i < messageBufferList.size(); i++) {
+                    ByteBuffer bb = messageBufferList.get(i);
+                    if (bb == null) {
+                        LOG.error("bb is null {}", getMessageResult);
+                        continue;
+                    }
+                    MessageExt msgExt = MessageDecoder.decode(bb);
+                    if (msgExt == null) {
+                        LOG.error("decode msgExt is null {}", getMessageResult);
+                        continue;
+                    }
+                    // use CQ offset, not offset in Message
+                    msgExt.setQueueOffset(getMessageResult.getMessageQueueOffset().get(i));
+                    foundList.add(msgExt);
+                }
+            }
+        } finally {
+            getMessageResult.release();
+        }
+
+        return foundList;
+    }
+
+    protected MessageExt getMessageFromRemote(String topic, long offset, int queueId, String brokerName) {
+        try {
+            PullResult pullResult = innerConsumer.pull(new MessageQueue(topic, brokerName, queueId), "*", offset, 1);
+            if (pullResult.getPullStatus().equals(PullStatus.FOUND)) {
+                return pullResult.getMsgFoundList().get(0);
+            }
+        } catch (Exception e) {
+            LOG.error("Get message from remote failed.", e);
+        }
+
+        return null;
+    }
+}
diff --git a/broker/src/main/java/org/apache/rocketmq/broker/filtersrv/FilterServerManager.java b/broker/src/main/java/org/apache/rocketmq/broker/filtersrv/FilterServerManager.java
index c1a860a..fa01d54 100644
--- a/broker/src/main/java/org/apache/rocketmq/broker/filtersrv/FilterServerManager.java
+++ b/broker/src/main/java/org/apache/rocketmq/broker/filtersrv/FilterServerManager.java
@@ -29,6 +29,7 @@ import java.util.concurrent.ScheduledExecutorService;
 import java.util.concurrent.TimeUnit;
 import org.apache.rocketmq.broker.BrokerController;
 import org.apache.rocketmq.broker.BrokerStartup;
+import org.apache.rocketmq.common.AbstractBrokerRunnable;
 import org.apache.rocketmq.common.ThreadFactoryImpl;
 import org.apache.rocketmq.common.constant.LoggerName;
 import org.apache.rocketmq.logging.InternalLogger;
@@ -52,9 +53,8 @@ public class FilterServerManager {
 
     public void start() {
 
-        this.scheduledExecutorService.scheduleAtFixedRate(new Runnable() {
-            @Override
-            public void run() {
+        this.scheduledExecutorService.scheduleAtFixedRate(new AbstractBrokerRunnable(brokerController.getBrokerConfig()) {
+            @Override public void run2() {
                 try {
                     FilterServerManager.this.createFilterServer();
                 } catch (Exception e) {
diff --git a/broker/src/main/java/org/apache/rocketmq/broker/latency/BrokerFastFailure.java b/broker/src/main/java/org/apache/rocketmq/broker/latency/BrokerFastFailure.java
index d176b86..b3184e2 100644
--- a/broker/src/main/java/org/apache/rocketmq/broker/latency/BrokerFastFailure.java
+++ b/broker/src/main/java/org/apache/rocketmq/broker/latency/BrokerFastFailure.java
@@ -17,25 +17,35 @@
 package org.apache.rocketmq.broker.latency;
 
 import java.util.concurrent.BlockingQueue;
-import java.util.concurrent.Executors;
 import java.util.concurrent.ScheduledExecutorService;
+import java.util.concurrent.ScheduledThreadPoolExecutor;
 import java.util.concurrent.TimeUnit;
 import org.apache.rocketmq.broker.BrokerController;
+import org.apache.rocketmq.common.AbstractBrokerRunnable;
 import org.apache.rocketmq.common.ThreadFactoryImpl;
+import org.apache.rocketmq.common.UtilAll;
 import org.apache.rocketmq.common.constant.LoggerName;
 import org.apache.rocketmq.logging.InternalLogger;
 import org.apache.rocketmq.logging.InternalLoggerFactory;
 import org.apache.rocketmq.remoting.netty.RequestTask;
 import org.apache.rocketmq.remoting.protocol.RemotingSysResponseCode;
 
+/**
+ * BrokerFastFailure will cover {@link BrokerController#getSendThreadPoolQueue()} and {@link
+ * BrokerController#getPullThreadPoolQueue()}
+ */
 public class BrokerFastFailure {
-    private static final InternalLogger log = InternalLoggerFactory.getLogger(LoggerName.BROKER_LOGGER_NAME);
-    private final ScheduledExecutorService scheduledExecutorService = Executors.newSingleThreadScheduledExecutor(new ThreadFactoryImpl(
-        "BrokerFastFailureScheduledThread"));
+    private static final InternalLogger LOGGER = InternalLoggerFactory.getLogger(LoggerName.BROKER_LOGGER_NAME);
+    private final ScheduledExecutorService scheduledExecutorService;
     private final BrokerController brokerController;
 
+    private volatile long jstackTime = System.currentTimeMillis();
+
     public BrokerFastFailure(final BrokerController brokerController) {
         this.brokerController = brokerController;
+        this.scheduledExecutorService = new ScheduledThreadPoolExecutor(1,
+            new ThreadFactoryImpl("BrokerFastFailureScheduledThread", true,
+                brokerController == null ? null : brokerController.getBrokerConfig()));
     }
 
     public static RequestTask castRunnable(final Runnable runnable) {
@@ -45,16 +55,15 @@ public class BrokerFastFailure {
                 return (RequestTask) object.getRunnable();
             }
         } catch (Throwable e) {
-            log.error(String.format("castRunnable exception, %s", runnable.getClass().getName()), e);
+            LOGGER.error(String.format("castRunnable exception, %s", runnable.getClass().getName()), e);
         }
 
         return null;
     }
 
     public void start() {
-        this.scheduledExecutorService.scheduleAtFixedRate(new Runnable() {
-            @Override
-            public void run() {
+        this.scheduledExecutorService.scheduleAtFixedRate(new AbstractBrokerRunnable(this.brokerController.getBrokerConfig()) {
+            @Override public void run2() {
                 if (brokerController.getBrokerConfig().isBrokerFastFailureEnable()) {
                     cleanExpiredRequest();
                 }
@@ -63,6 +72,7 @@ public class BrokerFastFailure {
     }
 
     private void cleanExpiredRequest() {
+
         while (this.brokerController.getMessageStore().isOSPageCacheBusy()) {
             try {
                 if (!this.brokerController.getSendThreadPoolQueue().isEmpty()) {
@@ -72,7 +82,13 @@ public class BrokerFastFailure {
                     }
 
                     final RequestTask rt = castRunnable(runnable);
-                    rt.returnResponse(RemotingSysResponseCode.SYSTEM_BUSY, String.format("[PCBUSY_CLEAN_QUEUE]broker busy, start flow control for a while, period in queue: %sms, size of queue: %d", System.currentTimeMillis() - rt.getCreateTimestamp(), this.brokerController.getSendThreadPoolQueue().size()));
+                    if (rt != null) {
+                        rt.returnResponse(RemotingSysResponseCode.SYSTEM_BUSY, String.format(
+                            "[PCBUSY_CLEAN_QUEUE]broker busy, start flow control for a while, period in queue: %sms, "
+                                + "size of queue: %d",
+                            System.currentTimeMillis() - rt.getCreateTimestamp(),
+                            this.brokerController.getSendThreadPoolQueue().size()));
+                    }
                 } else {
                     break;
                 }
@@ -86,11 +102,17 @@ public class BrokerFastFailure {
         cleanExpiredRequestInQueue(this.brokerController.getPullThreadPoolQueue(),
             this.brokerController.getBrokerConfig().getWaitTimeMillsInPullQueue());
 
+        cleanExpiredRequestInQueue(this.brokerController.getLitePullThreadPoolQueue(),
+            this.brokerController.getBrokerConfig().getWaitTimeMillsInLitePullQueue());
+
         cleanExpiredRequestInQueue(this.brokerController.getHeartbeatThreadPoolQueue(),
             this.brokerController.getBrokerConfig().getWaitTimeMillsInHeartbeatQueue());
 
         cleanExpiredRequestInQueue(this.brokerController.getEndTransactionThreadPoolQueue(), this
             .brokerController.getBrokerConfig().getWaitTimeMillsInTransactionQueue());
+
+        cleanExpiredRequestInQueue(this.brokerController.getAckThreadPoolQueue(),
+            brokerController.getBrokerConfig().getWaitTimeMillsInAckQueue());
     }
 
     void cleanExpiredRequestInQueue(final BlockingQueue<Runnable> blockingQueue, final long maxWaitTimeMillsInQueue) {
@@ -111,6 +133,10 @@ public class BrokerFastFailure {
                         if (blockingQueue.remove(runnable)) {
                             rt.setStopRun(true);
                             rt.returnResponse(RemotingSysResponseCode.SYSTEM_BUSY, String.format("[TIMEOUT_CLEAN_QUEUE]broker busy, start flow control for a while, period in queue: %sms, size of queue: %d", behind, blockingQueue.size()));
+                            if (System.currentTimeMillis() - jstackTime > 15000) {
+                                jstackTime = System.currentTimeMillis();
+                                LOGGER.warn("broker jstack \n " + UtilAll.jstack());
+                            }
                         }
                     } else {
                         break;
diff --git a/broker/src/main/java/org/apache/rocketmq/broker/loadbalance/AssignmentManager.java b/broker/src/main/java/org/apache/rocketmq/broker/loadbalance/AssignmentManager.java
index 877ddd8..fe5d5d0 100644
--- a/broker/src/main/java/org/apache/rocketmq/broker/loadbalance/AssignmentManager.java
+++ b/broker/src/main/java/org/apache/rocketmq/broker/loadbalance/AssignmentManager.java
@@ -24,10 +24,7 @@ import java.util.concurrent.ConcurrentHashMap;
 import java.util.concurrent.Executors;
 import java.util.concurrent.ScheduledExecutorService;
 import java.util.concurrent.TimeUnit;
-import java.util.concurrent.locks.Lock;
-import java.util.concurrent.locks.ReentrantLock;
 import org.apache.rocketmq.broker.BrokerController;
-import org.apache.rocketmq.broker.out.BrokerOuterAPI;
 import org.apache.rocketmq.client.exception.MQBrokerException;
 import org.apache.rocketmq.client.impl.factory.MQClientInstance;
 import org.apache.rocketmq.common.MixAll;
@@ -40,25 +37,16 @@ import org.apache.rocketmq.common.topic.TopicValidator;
 import org.apache.rocketmq.logging.InternalLogger;
 import org.apache.rocketmq.logging.InternalLoggerFactory;
 
-
 public class AssignmentManager {
     private static final InternalLogger log = InternalLoggerFactory.getLogger(LoggerName.BROKER_LOGGER_NAME);
 
     private transient BrokerController brokerController;
 
-    private final static long LOCK_TIMEOUT_MILLIS = 3000;
-
-    private final Lock lockNamesrv = new ReentrantLock();
-
-    private final BrokerOuterAPI mQClientAPIImpl;
-
     private final ConcurrentHashMap<String, Set<MessageQueue>> topicSubscribeInfoTable = new ConcurrentHashMap<String, Set<MessageQueue>>();
 
-    private ScheduledExecutorService scheduledExecutorService = Executors
-        .newSingleThreadScheduledExecutor(new ThreadFactoryImpl("LoadBalanceManagerScheduledThread"));
+    private ScheduledExecutorService scheduledExecutorService;
 
     private static final List<String> IGNORE_ROUTE_TOPICS = Lists.newArrayList(
-        TopicValidator.SYSTEM_TOPIC_PREFIX,
         MixAll.CID_RMQ_SYS_PREFIX,
         MixAll.DEFAULT_CONSUMER_GROUP,
         MixAll.TOOLS_CONSUMER_GROUP,
@@ -74,9 +62,10 @@ public class AssignmentManager {
 
     public AssignmentManager(BrokerController brokerController) {
         this.brokerController = brokerController;
-        this.mQClientAPIImpl = brokerController.getBrokerOuterAPI();
         ignoreRouteTopics.add(brokerController.getBrokerConfig().getBrokerClusterName());
         ignoreRouteTopics.add(brokerController.getBrokerConfig().getBrokerName());
+        scheduledExecutorService = Executors
+            .newSingleThreadScheduledExecutor(new ThreadFactoryImpl("LoadBalanceManagerScheduledThread", brokerController.getBrokerConfig()));
     }
 
     public void start() {
@@ -90,9 +79,12 @@ public class AssignmentManager {
                     log.error("ScheduledTask: failed to pull TopicRouteData from NameServer", e);
                 }
             }
-        }, 200, this.brokerController.getBrokerConfig().getLoadBalancePollNameServerInterval(), TimeUnit.MILLISECONDS);
+        }, 1000, this.brokerController.getBrokerConfig().getLoadBalancePollNameServerInterval(), TimeUnit.MILLISECONDS);
     }
 
+    public void shutdown() {
+        this.scheduledExecutorService.shutdown();
+    }
 
     public void updateTopicRouteInfoFromNameServer() {
         Set<String> topicList = new HashSet<>(brokerController.getTopicConfigManager().getTopicConfigTable().keySet());
@@ -100,7 +92,7 @@ public class AssignmentManager {
         LOOP:
         for (String topic : topicList) {
             for (String keyword : ignoreRouteTopics) {
-                if (topic.contains(keyword)) {
+                if (topic.contains(keyword) || TopicValidator.isSystemTopic(topic)) {
                     continue LOOP;
                 }
             }
@@ -111,7 +103,7 @@ public class AssignmentManager {
 
     public boolean updateTopicRouteInfoFromNameServer(final String topic) {
         try {
-            TopicRouteData topicRouteData = this.mQClientAPIImpl.getTopicRouteInfoFromNameServer(topic, 1000 * 3);
+            TopicRouteData topicRouteData = this.brokerController.getBrokerOuterAPI().getTopicRouteInfoFromNameServer(topic, 1000 * 3);
             if (topicRouteData != null) {
                 Set<MessageQueue> newSubscribeInfo = MQClientInstance.topicRouteData2TopicSubscribeInfo(topic, topicRouteData);
                 Set<MessageQueue> oldSubscribeInfo = topicSubscribeInfoTable.get(topic);
@@ -142,7 +134,6 @@ public class AssignmentManager {
         topicSubscribeInfoTable.remove(topic);
     }
 
-
     public Set<MessageQueue> getTopicSubscribeInfo(String topic) {
         return topicSubscribeInfoTable.get(topic);
     }
diff --git a/broker/src/main/java/org/apache/rocketmq/broker/longpolling/ManyPullRequest.java b/broker/src/main/java/org/apache/rocketmq/broker/longpolling/ManyPullRequest.java
index 170dae2..08703de 100644
--- a/broker/src/main/java/org/apache/rocketmq/broker/longpolling/ManyPullRequest.java
+++ b/broker/src/main/java/org/apache/rocketmq/broker/longpolling/ManyPullRequest.java
@@ -43,4 +43,8 @@ public class ManyPullRequest {
     public ArrayList<PullRequest> getPullRequestList() {
         return pullRequestList;
     }
+
+    public synchronized boolean isEmpty() {
+        return this.pullRequestList.isEmpty();
+    }
 }
diff --git a/broker/src/main/java/org/apache/rocketmq/broker/longpolling/PopRequest.java b/broker/src/main/java/org/apache/rocketmq/broker/longpolling/NotificationRequest.java
similarity index 56%
copy from broker/src/main/java/org/apache/rocketmq/broker/longpolling/PopRequest.java
copy to broker/src/main/java/org/apache/rocketmq/broker/longpolling/NotificationRequest.java
index e8e4567..fdae881 100644
--- a/broker/src/main/java/org/apache/rocketmq/broker/longpolling/PopRequest.java
+++ b/broker/src/main/java/org/apache/rocketmq/broker/longpolling/NotificationRequest.java
@@ -16,22 +16,19 @@
  */
 package org.apache.rocketmq.broker.longpolling;
 
-import io.netty.channel.Channel;
-import java.util.Comparator;
 import java.util.concurrent.atomic.AtomicBoolean;
-import java.util.concurrent.atomic.AtomicLong;
+
 import org.apache.rocketmq.remoting.protocol.RemotingCommand;
 
-public class PopRequest {
-    private static final AtomicLong COUNTER = new AtomicLong(Long.MIN_VALUE);
+import io.netty.channel.Channel;
 
+public class NotificationRequest {
     private RemotingCommand remotingCommand;
     private Channel channel;
     private long expired;
     private AtomicBoolean complete = new AtomicBoolean(false);
-    private final long op = COUNTER.getAndIncrement();
 
-    public PopRequest(RemotingCommand remotingCommand, Channel channel, long expired) {
+    public NotificationRequest(RemotingCommand remotingCommand, Channel channel, long expired) {
         this.channel = channel;
         this.remotingCommand = remotingCommand;
         this.expired = expired;
@@ -46,42 +43,15 @@ public class PopRequest {
     }
 
     public boolean isTimeout() {
-        return System.currentTimeMillis() > (expired - 50);
+        return System.currentTimeMillis() > (expired - 3000);
     }
 
     public boolean complete() {
         return complete.compareAndSet(false, true);
     }
 
-    public long getExpired() {
-        return expired;
-    }
-
     @Override
     public String toString() {
-        final StringBuilder sb = new StringBuilder("PopRequest{");
-        sb.append("cmd=").append(remotingCommand);
-        sb.append(", channel=").append(channel);
-        sb.append(", expired=").append(expired);
-        sb.append(", complete=").append(complete);
-        sb.append(", op=").append(op);
-        sb.append('}');
-        return sb.toString();
+        return remotingCommand.toString();
     }
-
-    public static final Comparator<PopRequest> COMPARATOR = new Comparator<PopRequest>() {
-        @Override
-        public int compare(PopRequest o1, PopRequest o2) {
-            int ret = (int) (o1.getExpired() - o2.getExpired());
-
-            if (ret != 0) {
-                return ret;
-            }
-            ret = (int) (o1.op - o2.op);
-            if (ret != 0) {
-                return ret;
-            }
-            return -1;
-        }
-    };
 }
diff --git a/broker/src/main/java/org/apache/rocketmq/broker/longpolling/NotifyMessageArrivingListener.java b/broker/src/main/java/org/apache/rocketmq/broker/longpolling/NotifyMessageArrivingListener.java
index 7803462..3c099fe 100644
--- a/broker/src/main/java/org/apache/rocketmq/broker/longpolling/NotifyMessageArrivingListener.java
+++ b/broker/src/main/java/org/apache/rocketmq/broker/longpolling/NotifyMessageArrivingListener.java
@@ -17,24 +17,29 @@
 
 package org.apache.rocketmq.broker.longpolling;
 
-import java.util.Map;
+import org.apache.rocketmq.broker.processor.NotificationProcessor;
 import org.apache.rocketmq.broker.processor.PopMessageProcessor;
 import org.apache.rocketmq.store.MessageArrivingListener;
 
+import java.util.Map;
+
 public class NotifyMessageArrivingListener implements MessageArrivingListener {
     private final PullRequestHoldService pullRequestHoldService;
     private final PopMessageProcessor popMessageProcessor;
+    private final NotificationProcessor notificationProcessor;
 
-    public NotifyMessageArrivingListener(final PullRequestHoldService pullRequestHoldService, final PopMessageProcessor popMessageProcessor) {
+    public NotifyMessageArrivingListener(final PullRequestHoldService pullRequestHoldService, final PopMessageProcessor popMessageProcessor, final NotificationProcessor notificationProcessor) {
         this.pullRequestHoldService = pullRequestHoldService;
         this.popMessageProcessor = popMessageProcessor;
+        this.notificationProcessor = notificationProcessor;
     }
 
     @Override
     public void arriving(String topic, int queueId, long logicOffset, long tagsCode,
-        long msgStoreTime, byte[] filterBitMap, Map<String, String> properties) {
+                         long msgStoreTime, byte[] filterBitMap, Map<String, String> properties) {
         this.pullRequestHoldService.notifyMessageArriving(topic, queueId, logicOffset, tagsCode,
             msgStoreTime, filterBitMap, properties);
         this.popMessageProcessor.notifyMessageArriving(topic, queueId);
+        this.notificationProcessor.notifyMessageArriving(topic, queueId);
     }
 }
diff --git a/broker/src/main/java/org/apache/rocketmq/broker/longpolling/PopRequest.java b/broker/src/main/java/org/apache/rocketmq/broker/longpolling/PopRequest.java
index e8e4567..2eccf77 100644
--- a/broker/src/main/java/org/apache/rocketmq/broker/longpolling/PopRequest.java
+++ b/broker/src/main/java/org/apache/rocketmq/broker/longpolling/PopRequest.java
@@ -16,12 +16,14 @@
  */
 package org.apache.rocketmq.broker.longpolling;
 
-import io.netty.channel.Channel;
 import java.util.Comparator;
 import java.util.concurrent.atomic.AtomicBoolean;
 import java.util.concurrent.atomic.AtomicLong;
+
 import org.apache.rocketmq.remoting.protocol.RemotingCommand;
 
+import io.netty.channel.Channel;
+
 public class PopRequest {
     private static final AtomicLong COUNTER = new AtomicLong(Long.MIN_VALUE);
 
diff --git a/broker/src/main/java/org/apache/rocketmq/broker/longpolling/PullRequestHoldService.java b/broker/src/main/java/org/apache/rocketmq/broker/longpolling/PullRequestHoldService.java
index 85ca9f7..0a05dba 100644
--- a/broker/src/main/java/org/apache/rocketmq/broker/longpolling/PullRequestHoldService.java
+++ b/broker/src/main/java/org/apache/rocketmq/broker/longpolling/PullRequestHoldService.java
@@ -21,6 +21,7 @@ import java.util.List;
 import java.util.Map;
 import java.util.concurrent.ConcurrentHashMap;
 import java.util.concurrent.ConcurrentMap;
+
 import org.apache.rocketmq.broker.BrokerController;
 import org.apache.rocketmq.common.ServiceThread;
 import org.apache.rocketmq.common.SystemClock;
@@ -78,7 +79,7 @@ public class PullRequestHoldService extends ServiceThread {
                 this.checkHoldRequest();
                 long costTime = this.systemClock.now() - beginLockTimestamp;
                 if (costTime > 5 * 1000) {
-                    log.info("[NOTIFYME] check hold request cost {} ms.", costTime);
+                    log.warn("PullRequestHoldService: check hold pull request cost {}ms", costTime);
                 }
             } catch (Throwable e) {
                 log.warn(this.getServiceName() + " service has exception. ", e);
@@ -90,6 +91,9 @@ public class PullRequestHoldService extends ServiceThread {
 
     @Override
     public String getServiceName() {
+        if (brokerController != null && brokerController.getBrokerConfig().isInBrokerContainer()) {
+            return this.brokerController.getBrokerConfig().getLoggerIdentifier() + PullRequestHoldService.class.getSimpleName();
+        }
         return PullRequestHoldService.class.getSimpleName();
     }
 
@@ -103,7 +107,9 @@ public class PullRequestHoldService extends ServiceThread {
                 try {
                     this.notifyMessageArriving(topic, queueId, offset);
                 } catch (Throwable e) {
-                    log.error("check hold request failed. topic={}, queueId={}", topic, queueId, e);
+                    log.error(
+                        "PullRequestHoldService: failed to check hold request failed, topic={}, queueId={}", topic,
+                        queueId, e);
                 }
             }
         }
@@ -141,7 +147,9 @@ public class PullRequestHoldService extends ServiceThread {
                                 this.brokerController.getPullMessageProcessor().executeRequestWhenWakeup(request.getClientChannel(),
                                     request.getRequestCommand());
                             } catch (Throwable e) {
-                                log.error("execute request when wakeup failed.", e);
+                                log.error(
+                                    "PullRequestHoldService#notifyMessageArriving: failed to execute request when "
+                                        + "message matched, topic={}, queueId={}", topic, queueId, e);
                             }
                             continue;
                         }
@@ -152,7 +160,9 @@ public class PullRequestHoldService extends ServiceThread {
                             this.brokerController.getPullMessageProcessor().executeRequestWhenWakeup(request.getClientChannel(),
                                 request.getRequestCommand());
                         } catch (Throwable e) {
-                            log.error("execute request when wakeup failed.", e);
+                            log.error(
+                                "PullRequestHoldService#notifyMessageArriving: failed to execute request when time's "
+                                    + "up, topic={}, queueId={}", topic, queueId, e);
                         }
                         continue;
                     }
@@ -166,4 +176,22 @@ public class PullRequestHoldService extends ServiceThread {
             }
         }
     }
+
+    public void notifyMasterOnline() {
+        for (ManyPullRequest mpr : this.pullRequestTable.values()) {
+            if (mpr == null || mpr.isEmpty()) {
+                continue;
+            }
+            for (PullRequest request : mpr.cloneListAndClear()) {
+                try {
+                    log.info("notify master online, wakeup {} {}", request.getClientChannel(), request.getRequestCommand());
+                    this.brokerController.getPullMessageProcessor().executeRequestWhenWakeup(request.getClientChannel(),
+                        request.getRequestCommand());
+                } catch (Throwable e) {
+                    log.error("execute request when master online failed.", e);
+                }
+            }
+        }
+
+    }
 }
diff --git a/broker/src/main/java/org/apache/rocketmq/broker/mqtrace/AbortProcessException.java b/broker/src/main/java/org/apache/rocketmq/broker/mqtrace/AbortProcessException.java
new file mode 100644
index 0000000..c81a29a
--- /dev/null
+++ b/broker/src/main/java/org/apache/rocketmq/broker/mqtrace/AbortProcessException.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.broker.mqtrace;
+
+import org.apache.rocketmq.common.UtilAll;
+import org.apache.rocketmq.common.help.FAQUrl;
+
+/**
+ *
+ * This exception is used for broker hooks only : SendMessageHook, ConsumeMessageHook, pullMessageHook
+ * This exception is not ignored while executing hooks and it means that
+ * certain processor should return an immediate error response to the client. The
+ * error response code is included in AbortProcessException.  it's naming might
+ * be confusing, so feel free to refactor this class. Also when any class implements
+ * the 3 hook interface mentioned above we should be careful if we want to throw
+ * an AbortProcessException, because it will change the control flow of broker
+ * and cause a RemotingCommand return error immediately. So be aware of the side
+ * effect before throw AbortProcessException in your implementation.
+ *
+ */
+public class AbortProcessException extends RuntimeException {
+    private static final long serialVersionUID = -5728810933841185841L;
+    private int responseCode;
+    private String errorMessage;
+
+    public AbortProcessException(String errorMessage, Throwable cause) {
+        super(FAQUrl.attachDefaultURL(errorMessage), cause);
+        this.responseCode = -1;
+        this.errorMessage = errorMessage;
+    }
+
+    public AbortProcessException(int responseCode, String errorMessage) {
+        super(FAQUrl.attachDefaultURL("CODE: " + UtilAll.responseCode2String(responseCode) + "  DESC: "
+            + errorMessage));
+        this.responseCode = responseCode;
+        this.errorMessage = errorMessage;
+    }
+
+    public int getResponseCode() {
+        return responseCode;
+    }
+
+    public AbortProcessException setResponseCode(final int responseCode) {
+        this.responseCode = responseCode;
+        return this;
+    }
+
+    public String getErrorMessage() {
+        return errorMessage;
+    }
+
+    public void setErrorMessage(final String errorMessage) {
+        this.errorMessage = errorMessage;
+    }
+}
\ No newline at end of file
diff --git a/broker/src/main/java/org/apache/rocketmq/broker/mqtrace/ConsumeMessageContext.java b/broker/src/main/java/org/apache/rocketmq/broker/mqtrace/ConsumeMessageContext.java
index ae5d077..ed7bfba 100644
--- a/broker/src/main/java/org/apache/rocketmq/broker/mqtrace/ConsumeMessageContext.java
+++ b/broker/src/main/java/org/apache/rocketmq/broker/mqtrace/ConsumeMessageContext.java
@@ -17,6 +17,8 @@
 package org.apache.rocketmq.broker.mqtrace;
 
 import java.util.Map;
+
+import org.apache.rocketmq.common.TopicConfig;
 import org.apache.rocketmq.store.stats.BrokerStatsManager;
 
 public class ConsumeMessageContext {
@@ -30,13 +32,22 @@ public class ConsumeMessageContext {
     private boolean success;
     private String status;
     private Object mqTraceContext;
+    private TopicConfig topicConfig;
+
+    private String accountAuthType;
+    private String accountOwnerParent;
+    private String accountOwnerSelf;
+    private int rcvMsgNum;
+    private int rcvMsgSize;
+    private BrokerStatsManager.StatsType rcvStat;
+    private int commercialRcvMsgNum;
 
     private String commercialOwner;
     private BrokerStatsManager.StatsType commercialRcvStats;
     private int commercialRcvTimes;
     private int commercialRcvSize;
-    private String namespace;
 
+    private String namespace;
     public String getConsumerGroup() {
         return consumerGroup;
     }
@@ -109,6 +120,14 @@ public class ConsumeMessageContext {
         this.mqTraceContext = mqTraceContext;
     }
 
+    public TopicConfig getTopicConfig() {
+        return topicConfig;
+    }
+
+    public void setTopicConfig(TopicConfig topicConfig) {
+        this.topicConfig = topicConfig;
+    }
+
     public int getBodyLength() {
         return bodyLength;
     }
@@ -117,6 +136,62 @@ public class ConsumeMessageContext {
         this.bodyLength = bodyLength;
     }
 
+    public String getAccountAuthType() {
+        return accountAuthType;
+    }
+
+    public void setAccountAuthType(String accountAuthType) {
+        this.accountAuthType = accountAuthType;
+    }
+
+    public String getAccountOwnerParent() {
+        return accountOwnerParent;
+    }
+
+    public void setAccountOwnerParent(String accountOwnerParent) {
+        this.accountOwnerParent = accountOwnerParent;
+    }
+
+    public String getAccountOwnerSelf() {
+        return accountOwnerSelf;
+    }
+
+    public void setAccountOwnerSelf(String accountOwnerSelf) {
+        this.accountOwnerSelf = accountOwnerSelf;
+    }
+
+    public int getRcvMsgNum() {
+        return rcvMsgNum;
+    }
+
+    public void setRcvMsgNum(int rcvMsgNum) {
+        this.rcvMsgNum = rcvMsgNum;
+    }
+
+    public int getRcvMsgSize() {
+        return rcvMsgSize;
+    }
+
+    public void setRcvMsgSize(int rcvMsgSize) {
+        this.rcvMsgSize = rcvMsgSize;
+    }
+
+    public BrokerStatsManager.StatsType getRcvStat() {
+        return rcvStat;
+    }
+
+    public void setRcvStat(BrokerStatsManager.StatsType rcvStat) {
+        this.rcvStat = rcvStat;
+    }
+
+    public int getCommercialRcvMsgNum() {
+        return commercialRcvMsgNum;
+    }
+
+    public void setCommercialRcvMsgNum(int commercialRcvMsgNum) {
+        this.commercialRcvMsgNum = commercialRcvMsgNum;
+    }
+
     public String getCommercialOwner() {
         return commercialOwner;
     }
diff --git a/broker/src/main/java/org/apache/rocketmq/broker/mqtrace/SendMessageContext.java b/broker/src/main/java/org/apache/rocketmq/broker/mqtrace/SendMessageContext.java
index ab6452e..aaa84b7 100644
--- a/broker/src/main/java/org/apache/rocketmq/broker/mqtrace/SendMessageContext.java
+++ b/broker/src/main/java/org/apache/rocketmq/broker/mqtrace/SendMessageContext.java
@@ -17,11 +17,16 @@
 package org.apache.rocketmq.broker.mqtrace;
 
 import java.util.Properties;
+
 import org.apache.rocketmq.common.message.MessageType;
 import org.apache.rocketmq.store.stats.BrokerStatsManager;
 
 public class SendMessageContext {
+    /** namespace */
+    private String namespace;
+    /** producer group without namespace. */
     private String producerGroup;
+    /** topic without namespace. */
     private String topic;
     private String msgId;
     private String originMsgId;
@@ -38,14 +43,37 @@ public class SendMessageContext {
     private String brokerRegionId;
     private String msgUniqueKey;
     private long bornTimeStamp;
+    private long requestTimeStamp;
     private MessageType msgType = MessageType.Trans_msg_Commit;
+
     private boolean isSuccess = false;
 
+    /**
+     * Account Statistics
+     */
+    private String accountAuthType;
+    private String accountOwnerParent;
+    private String accountOwnerSelf;
+    private int sendMsgNum;
+    private int sendMsgSize;
+    private BrokerStatsManager.StatsType sendStat;
+    private int commercialSendMsgNum;
+
+    /**
+     * For Commercial
+     */
     private String commercialOwner;
     private BrokerStatsManager.StatsType commercialSendStats;
     private int commercialSendSize;
     private int commercialSendTimes;
-    private String namespace;
+
+    public String getNamespace() {
+        return namespace;
+    }
+
+    public void setNamespace(String namespace) {
+        this.namespace = namespace;
+    }
 
     public boolean isSuccess() {
         return isSuccess;
@@ -79,6 +107,14 @@ public class SendMessageContext {
         this.bornTimeStamp = bornTimeStamp;
     }
 
+    public long getRequestTimeStamp() {
+        return requestTimeStamp;
+    }
+
+    public void setRequestTimeStamp(long requestTimeStamp) {
+        this.requestTimeStamp = requestTimeStamp;
+    }
+
     public String getBrokerRegionId() {
         return brokerRegionId;
     }
@@ -207,10 +243,66 @@ public class SendMessageContext {
         this.commercialOwner = commercialOwner;
     }
 
+    public String getAccountAuthType() {
+        return accountAuthType;
+    }
+
+    public void setAccountAuthType(String accountAuthType) {
+        this.accountAuthType = accountAuthType;
+    }
+
+    public String getAccountOwnerParent() {
+        return accountOwnerParent;
+    }
+
+    public void setAccountOwnerParent(String accountOwnerParent) {
+        this.accountOwnerParent = accountOwnerParent;
+    }
+
+    public String getAccountOwnerSelf() {
+        return accountOwnerSelf;
+    }
+
+    public void setAccountOwnerSelf(String accountOwnerSelf) {
+        this.accountOwnerSelf = accountOwnerSelf;
+    }
+
+    public int getSendMsgNum() {
+        return sendMsgNum;
+    }
+
+    public void setSendMsgNum(int sendMsgNum) {
+        this.sendMsgNum = sendMsgNum;
+    }
+
+    public int getSendMsgSize() {
+        return sendMsgSize;
+    }
+
+    public void setSendMsgSize(int sendMsgSize) {
+        this.sendMsgSize = sendMsgSize;
+    }
+
+    public BrokerStatsManager.StatsType getSendStat() {
+        return sendStat;
+    }
+
+    public void setSendStat(BrokerStatsManager.StatsType sendStat) {
+        this.sendStat = sendStat;
+    }
+
     public BrokerStatsManager.StatsType getCommercialSendStats() {
         return commercialSendStats;
     }
 
+    public int getCommercialSendMsgNum() {
+        return commercialSendMsgNum;
+    }
+
+    public void setCommercialSendMsgNum(int commercialSendMsgNum) {
+        this.commercialSendMsgNum = commercialSendMsgNum;
+    }
+
     public void setCommercialSendStats(final BrokerStatsManager.StatsType commercialSendStats) {
         this.commercialSendStats = commercialSendStats;
     }
@@ -230,12 +322,4 @@ public class SendMessageContext {
     public void setCommercialSendTimes(final int commercialSendTimes) {
         this.commercialSendTimes = commercialSendTimes;
     }
-
-    public String getNamespace() {
-        return namespace;
-    }
-
-    public void setNamespace(String namespace) {
-        this.namespace = namespace;
-    }
 }
diff --git a/broker/src/main/java/org/apache/rocketmq/broker/offset/ConsumerOffsetManager.java b/broker/src/main/java/org/apache/rocketmq/broker/offset/ConsumerOffsetManager.java
index f09522a..4eadaa8 100644
--- a/broker/src/main/java/org/apache/rocketmq/broker/offset/ConsumerOffsetManager.java
+++ b/broker/src/main/java/org/apache/rocketmq/broker/offset/ConsumerOffsetManager.java
@@ -24,9 +24,11 @@ import java.util.Map.Entry;
 import java.util.Set;
 import java.util.concurrent.ConcurrentHashMap;
 import java.util.concurrent.ConcurrentMap;
+import java.util.concurrent.atomic.AtomicLong;
 import org.apache.rocketmq.broker.BrokerController;
 import org.apache.rocketmq.broker.BrokerPathConfigHelper;
 import org.apache.rocketmq.common.ConfigManager;
+import org.apache.rocketmq.common.DataVersion;
 import org.apache.rocketmq.common.UtilAll;
 import org.apache.rocketmq.common.constant.LoggerName;
 import org.apache.rocketmq.logging.InternalLogger;
@@ -34,14 +36,18 @@ import org.apache.rocketmq.logging.InternalLoggerFactory;
 import org.apache.rocketmq.remoting.protocol.RemotingSerializable;
 
 public class ConsumerOffsetManager extends ConfigManager {
-    private static final InternalLogger log = InternalLoggerFactory.getLogger(LoggerName.BROKER_LOGGER_NAME);
-    protected static final String TOPIC_GROUP_SEPARATOR = "@";
+    private static final InternalLogger LOG = InternalLoggerFactory.getLogger(LoggerName.BROKER_LOGGER_NAME);
+    public static final String TOPIC_GROUP_SEPARATOR = "@";
+
+    private DataVersion dataVersion = new DataVersion();
 
     protected ConcurrentMap<String/* topic@group */, ConcurrentMap<Integer, Long>> offsetTable =
         new ConcurrentHashMap<String, ConcurrentMap<Integer, Long>>(512);
 
     protected transient BrokerController brokerController;
 
+    private transient AtomicLong versionChangeCounter = new AtomicLong(0);
+
     public ConsumerOffsetManager() {
     }
 
@@ -49,6 +55,36 @@ public class ConsumerOffsetManager extends ConfigManager {
         this.brokerController = brokerController;
     }
 
+    public void cleanOffset(String group) {
+        Iterator<Entry<String, ConcurrentMap<Integer, Long>>> it = this.offsetTable.entrySet().iterator();
+        while (it.hasNext()) {
+            Entry<String, ConcurrentMap<Integer, Long>> next = it.next();
+            String topicAtGroup = next.getKey();
+            if (topicAtGroup.contains(group)) {
+                String[] arrays = topicAtGroup.split(TOPIC_GROUP_SEPARATOR);
+                if (arrays.length == 2 && group.equals(arrays[1])) {
+                    it.remove();
+                    LOG.warn("Clean group's offset, {}, {}", topicAtGroup, next.getValue());
+                }
+            }
+        }
+    }
+
+    public void cleanOffsetByTopic(String topic) {
+        Iterator<Entry<String, ConcurrentMap<Integer, Long>>> it = this.offsetTable.entrySet().iterator();
+        while (it.hasNext()) {
+            Entry<String, ConcurrentMap<Integer, Long>> next = it.next();
+            String topicAtGroup = next.getKey();
+            if (topicAtGroup.contains(topic)) {
+                String[] arrays = topicAtGroup.split(TOPIC_GROUP_SEPARATOR);
+                if (arrays.length == 2 && topic.equals(arrays[0])) {
+                    it.remove();
+                    LOG.warn("Clean topic's offset, {}, {}", topicAtGroup, next.getValue());
+                }
+            }
+        }
+    }
+
     public void scanUnsubscribedTopic() {
         Iterator<Entry<String, ConcurrentMap<Integer, Long>>> it = this.offsetTable.entrySet().iterator();
         while (it.hasNext()) {
@@ -62,7 +98,7 @@ public class ConsumerOffsetManager extends ConfigManager {
                 if (null == brokerController.getConsumerManager().findSubscriptionData(group, topic)
                     && this.offsetBehindMuchThanData(topic, next.getValue())) {
                     it.remove();
-                    log.warn("remove topic offset, {}", topicAtGroup);
+                    LOG.warn("remove topic offset, {}", topicAtGroup);
                 }
             }
         }
@@ -118,6 +154,28 @@ public class ConsumerOffsetManager extends ConfigManager {
         return groups;
     }
 
+    public Map<String, Set<String>> getGroupTopicMap() {
+        Map<String, Set<String>> retMap = new HashMap<String, Set<String>>(128);
+
+        for (String key : this.offsetTable.keySet()) {
+            String[] arr = key.split(TOPIC_GROUP_SEPARATOR);
+            if (arr.length == 2) {
+                String topic = arr[0];
+                String group = arr[1];
+
+                Set<String> topics = retMap.get(group);
+                if (topics == null) {
+                    topics = new HashSet<String>(8);
+                    retMap.put(group, topics);
+                }
+
+                topics.add(topic);
+            }
+        }
+
+        return retMap;
+    }
+
     public void commitOffset(final String clientHost, final String group, final String topic, final int queueId,
         final long offset) {
         // topic@group
@@ -134,9 +192,13 @@ public class ConsumerOffsetManager extends ConfigManager {
         } else {
             Long storeOffset = map.put(queueId, offset);
             if (storeOffset != null && offset < storeOffset) {
-                log.warn("[NOTIFYME]update consumer offset less than store. clientHost={}, key={}, queueId={}, requestOffset={}, storeOffset={}", clientHost, key, queueId, offset, storeOffset);
+                LOG.warn("[NOTIFYME]update consumer offset less than store. clientHost={}, key={}, queueId={}, requestOffset={}, storeOffset={}", clientHost, key, queueId, offset, storeOffset);
             }
         }
+        if (versionChangeCounter.incrementAndGet() % brokerController.getBrokerConfig().getConsumerOffsetUpdateVersionStep() == 0) {
+            long stateMachineVersion = brokerController.getMessageStore() != null ? brokerController.getMessageStore().getStateMachineVersion() : 0;
+            dataVersion.nextVersion(stateMachineVersion);
+        }
     }
 
     public long queryOffset(final String group, final String topic, final int queueId) {
@@ -145,8 +207,9 @@ public class ConsumerOffsetManager extends ConfigManager {
         ConcurrentMap<Integer, Long> map = this.offsetTable.get(key);
         if (null != map) {
             Long offset = map.get(queueId);
-            if (offset != null)
+            if (offset != null) {
                 return offset;
+            }
         }
 
         return -1;
@@ -167,10 +230,12 @@ public class ConsumerOffsetManager extends ConfigManager {
             ConsumerOffsetManager obj = RemotingSerializable.fromJson(jsonString, ConsumerOffsetManager.class);
             if (obj != null) {
                 this.offsetTable = obj.offsetTable;
+                this.dataVersion = obj.dataVersion;
             }
         }
     }
 
+    @Override
     public String encode(final boolean prettyFormat) {
         return RemotingSerializable.toJson(this, prettyFormat);
     }
@@ -232,6 +297,14 @@ public class ConsumerOffsetManager extends ConfigManager {
         }
     }
 
+    public DataVersion getDataVersion() {
+        return dataVersion;
+    }
+
+    public void setDataVersion(DataVersion dataVersion) {
+        this.dataVersion = dataVersion;
+    }
+
     public void removeOffset(final String group) {
         Iterator<Entry<String, ConcurrentMap<Integer, Long>>> it = this.offsetTable.entrySet().iterator();
         while (it.hasNext()) {
@@ -241,7 +314,7 @@ public class ConsumerOffsetManager extends ConfigManager {
                 String[] arrays = topicAtGroup.split(TOPIC_GROUP_SEPARATOR);
                 if (arrays.length == 2 && group.equals(arrays[1])) {
                     it.remove();
-                    log.warn("clean group offset {}", topicAtGroup);
+                    LOG.warn("clean group offset {}", topicAtGroup);
                 }
             }
         }
diff --git a/broker/src/main/java/org/apache/rocketmq/broker/out/BrokerOuterAPI.java b/broker/src/main/java/org/apache/rocketmq/broker/out/BrokerOuterAPI.java
index 8a2093a..b637951 100644
--- a/broker/src/main/java/org/apache/rocketmq/broker/out/BrokerOuterAPI.java
+++ b/broker/src/main/java/org/apache/rocketmq/broker/out/BrokerOuterAPI.java
@@ -16,35 +16,67 @@
  */
 package org.apache.rocketmq.broker.out;
 
-import org.apache.rocketmq.broker.BrokerController;
 import org.apache.rocketmq.broker.latency.BrokerFixedThreadPoolExecutor;
 import org.apache.rocketmq.client.exception.MQBrokerException;
+import org.apache.rocketmq.client.producer.SendResult;
+import org.apache.rocketmq.client.producer.SendStatus;
+import org.apache.rocketmq.common.AbstractBrokerRunnable;
+import org.apache.rocketmq.common.BrokerIdentity;
+import org.apache.rocketmq.common.BrokerSyncInfo;
 import org.apache.rocketmq.common.DataVersion;
+import org.apache.rocketmq.common.LockCallback;
 import org.apache.rocketmq.common.MixAll;
 import org.apache.rocketmq.common.ThreadFactoryImpl;
+import org.apache.rocketmq.common.UnlockCallback;
 import org.apache.rocketmq.common.UtilAll;
 import org.apache.rocketmq.common.constant.LoggerName;
+import org.apache.rocketmq.common.message.Message;
+import org.apache.rocketmq.common.message.MessageBatch;
+import org.apache.rocketmq.common.message.MessageClientIDSetter;
+import org.apache.rocketmq.common.message.MessageConst;
+import org.apache.rocketmq.common.message.MessageDecoder;
+import org.apache.rocketmq.common.message.MessageExt;
+import org.apache.rocketmq.common.message.MessageQueue;
+import org.apache.rocketmq.common.namesrv.DefaultTopAddressing;
 import org.apache.rocketmq.common.namesrv.RegisterBrokerResult;
 import org.apache.rocketmq.common.namesrv.TopAddressing;
 import org.apache.rocketmq.common.protocol.RequestCode;
 import org.apache.rocketmq.common.protocol.ResponseCode;
+import org.apache.rocketmq.common.protocol.body.BrokerMemberGroup;
 import org.apache.rocketmq.common.protocol.body.ClusterInfo;
 import org.apache.rocketmq.common.protocol.body.ConsumerOffsetSerializeWrapper;
+import org.apache.rocketmq.common.protocol.body.GetBrokerMemberGroupResponseBody;
 import org.apache.rocketmq.common.protocol.body.KVTable;
+import org.apache.rocketmq.common.protocol.body.LockBatchRequestBody;
+import org.apache.rocketmq.common.protocol.body.LockBatchResponseBody;
 import org.apache.rocketmq.common.protocol.body.RegisterBrokerBody;
 import org.apache.rocketmq.common.protocol.body.SubscriptionGroupWrapper;
 import org.apache.rocketmq.common.protocol.body.TopicConfigAndMappingSerializeWrapper;
 import org.apache.rocketmq.common.protocol.body.TopicConfigSerializeWrapper;
+import org.apache.rocketmq.common.protocol.body.UnlockBatchRequestBody;
+import org.apache.rocketmq.common.protocol.header.ExchangeHAInfoRequestHeader;
+import org.apache.rocketmq.common.protocol.header.ExchangeHAInfoResponseHeader;
+import org.apache.rocketmq.common.protocol.header.GetBrokerMemberGroupRequestHeader;
+import org.apache.rocketmq.common.protocol.header.GetMaxOffsetRequestHeader;
+import org.apache.rocketmq.common.protocol.header.GetMaxOffsetResponseHeader;
+import org.apache.rocketmq.common.protocol.header.GetMinOffsetRequestHeader;
+import org.apache.rocketmq.common.protocol.header.GetMinOffsetResponseHeader;
+import org.apache.rocketmq.common.protocol.header.SendMessageRequestHeader;
+import org.apache.rocketmq.common.protocol.header.SendMessageRequestHeaderV2;
+import org.apache.rocketmq.common.protocol.header.SendMessageResponseHeader;
+import org.apache.rocketmq.common.protocol.header.namesrv.BrokerHeartbeatRequestHeader;
 import org.apache.rocketmq.common.protocol.header.namesrv.GetRouteInfoRequestHeader;
 import org.apache.rocketmq.common.protocol.header.namesrv.QueryDataVersionRequestHeader;
 import org.apache.rocketmq.common.protocol.header.namesrv.QueryDataVersionResponseHeader;
 import org.apache.rocketmq.common.protocol.header.namesrv.RegisterBrokerRequestHeader;
 import org.apache.rocketmq.common.protocol.header.namesrv.RegisterBrokerResponseHeader;
 import org.apache.rocketmq.common.protocol.header.namesrv.UnRegisterBrokerRequestHeader;
+import org.apache.rocketmq.common.protocol.route.BrokerData;
 import org.apache.rocketmq.common.protocol.route.TopicRouteData;
 import org.apache.rocketmq.common.rpc.ClientMetadata;
 import org.apache.rocketmq.common.rpc.RpcClient;
 import org.apache.rocketmq.common.rpc.RpcClientImpl;
+import org.apache.rocketmq.common.topic.TopicValidator;
 import org.apache.rocketmq.logging.InternalLogger;
 import org.apache.rocketmq.logging.InternalLoggerFactory;
 import org.apache.rocketmq.remoting.InvokeCallback;
@@ -61,37 +93,34 @@ import org.apache.rocketmq.remoting.netty.NettyRemotingClient;
 import org.apache.rocketmq.remoting.protocol.RemotingCommand;
 
 import java.io.UnsupportedEncodingException;
+import java.util.Arrays;
 import java.util.ArrayList;
 import java.util.List;
+import java.util.Set;
 import java.util.concurrent.ArrayBlockingQueue;
 import java.util.concurrent.CopyOnWriteArrayList;
 import java.util.concurrent.CountDownLatch;
 import java.util.concurrent.TimeUnit;
 
 public class BrokerOuterAPI {
-    private static final InternalLogger log = InternalLoggerFactory.getLogger(LoggerName.BROKER_LOGGER_NAME);
+    private static final InternalLogger LOGGER = InternalLoggerFactory.getLogger(LoggerName.BROKER_LOGGER_NAME);
     private final RemotingClient remotingClient;
-    private final BrokerController brokerController;
-    private final TopAddressing topAddressing = new TopAddressing(MixAll.getWSAddr());
+    private final TopAddressing topAddressing = new DefaultTopAddressing(MixAll.getWSAddr());
     private String nameSrvAddr = null;
-    private final String currBrokerName;
     private BrokerFixedThreadPoolExecutor brokerOuterExecutor = new BrokerFixedThreadPoolExecutor(4, 10, 1, TimeUnit.MINUTES,
         new ArrayBlockingQueue<Runnable>(32), new ThreadFactoryImpl("brokerOutApi_thread_", true));
 
     private ClientMetadata clientMetadata;
     private RpcClient rpcClient;
 
-    public BrokerOuterAPI(final NettyClientConfig nettyClientConfig, final BrokerController brokerController) {
-
-        this(nettyClientConfig, null, brokerController, new ClientMetadata());
+    public BrokerOuterAPI(final NettyClientConfig nettyClientConfig) {
+        this(nettyClientConfig, null, new ClientMetadata());
     }
 
-    private BrokerOuterAPI(final NettyClientConfig nettyClientConfig, RPCHook rpcHook, final BrokerController brokerController, ClientMetadata clientMetadata) {
+    private BrokerOuterAPI(final NettyClientConfig nettyClientConfig, RPCHook rpcHook, ClientMetadata clientMetadata) {
         this.remotingClient = new NettyRemotingClient(nettyClientConfig);
         this.clientMetadata = clientMetadata;
         this.remotingClient.registerRPCHook(rpcHook);
-        this.brokerController = brokerController;
-        this.currBrokerName =  brokerController.getBrokerConfig().getBrokerName();
         this.rpcClient = new RpcClientImpl(this.clientMetadata, this.remotingClient);
     }
 
@@ -104,31 +133,224 @@ public class BrokerOuterAPI {
         this.brokerOuterExecutor.shutdown();
     }
 
+    public List<String> getNameServerAddressList() {
+        return this.remotingClient.getNameServerAddressList();
+    }
+
     public String fetchNameServerAddr() {
         try {
             String addrs = this.topAddressing.fetchNSAddr();
             if (addrs != null) {
                 if (!addrs.equals(this.nameSrvAddr)) {
-                    log.info("name server address changed, old: {} new: {}", this.nameSrvAddr, addrs);
+                    LOGGER.info("name server address changed, old: {} new: {}", this.nameSrvAddr, addrs);
                     this.updateNameServerAddressList(addrs);
                     this.nameSrvAddr = addrs;
                     return nameSrvAddr;
                 }
             }
         } catch (Exception e) {
-            log.error("fetchNameServerAddr Exception", e);
+            LOGGER.error("fetchNameServerAddr Exception", e);
         }
         return nameSrvAddr;
     }
 
     public void updateNameServerAddressList(final String addrs) {
-        List<String> lst = new ArrayList<String>();
         String[] addrArray = addrs.split(";");
-        for (String addr : addrArray) {
-            lst.add(addr);
+        List<String> lst = new ArrayList<String>(Arrays.asList(addrArray));
+        this.remotingClient.updateNameServerAddressList(lst);
+    }
+
+    public BrokerMemberGroup syncBrokerMemberGroup(String clusterName, String brokerName)
+        throws InterruptedException, RemotingTimeoutException, RemotingSendRequestException, RemotingConnectException {
+        return syncBrokerMemberGroup(clusterName, brokerName, false);
+    }
+
+    public BrokerMemberGroup syncBrokerMemberGroup(String clusterName, String brokerName,
+        boolean isCompatibleWithOldNameSrv)
+        throws InterruptedException, RemotingTimeoutException, RemotingSendRequestException, RemotingConnectException {
+        if (isCompatibleWithOldNameSrv) {
+            return getBrokerMemberGroupCompatible(clusterName, brokerName);
+        } else {
+            return getBrokerMemberGroup(clusterName, brokerName);
         }
+    }
 
-        this.remotingClient.updateNameServerAddressList(lst);
+    public BrokerMemberGroup getBrokerMemberGroup(String clusterName, String brokerName)
+        throws InterruptedException, RemotingTimeoutException, RemotingSendRequestException, RemotingConnectException {
+        BrokerMemberGroup brokerMemberGroup = new BrokerMemberGroup(clusterName, brokerName);
+
+        GetBrokerMemberGroupRequestHeader requestHeader = new GetBrokerMemberGroupRequestHeader();
+        requestHeader.setClusterName(clusterName);
+        requestHeader.setBrokerName(brokerName);
+
+        RemotingCommand request = RemotingCommand.createRequestCommand(RequestCode.GET_BROKER_MEMBER_GROUP, requestHeader);
+
+        RemotingCommand response = null;
+        response = this.remotingClient.invokeSync(null, request, 3000);
+        assert response != null;
+
+        switch (response.getCode()) {
+            case ResponseCode.SUCCESS: {
+                byte[] body = response.getBody();
+                if (body != null) {
+                    GetBrokerMemberGroupResponseBody brokerMemberGroupResponseBody =
+                        GetBrokerMemberGroupResponseBody.decode(body, GetBrokerMemberGroupResponseBody.class);
+
+                    return brokerMemberGroupResponseBody.getBrokerMemberGroup();
+                }
+            }
+            default:
+                break;
+        }
+
+        return brokerMemberGroup;
+    }
+
+    public BrokerMemberGroup getBrokerMemberGroupCompatible(String clusterName, String brokerName)
+        throws InterruptedException, RemotingTimeoutException, RemotingSendRequestException, RemotingConnectException {
+        BrokerMemberGroup brokerMemberGroup = new BrokerMemberGroup(clusterName, brokerName);
+
+        GetRouteInfoRequestHeader requestHeader = new GetRouteInfoRequestHeader();
+        requestHeader.setTopic(TopicValidator.SYNC_BROKER_MEMBER_GROUP_PREFIX + brokerName);
+
+        RemotingCommand request = RemotingCommand.createRequestCommand(RequestCode.GET_ROUTEINFO_BY_TOPIC, requestHeader);
+
+        RemotingCommand response;
+        response = this.remotingClient.invokeSync(null, request, 3000);
+        assert response != null;
+
+        switch (response.getCode()) {
+            case ResponseCode.SUCCESS: {
+                byte[] body = response.getBody();
+                if (body != null) {
+                    TopicRouteData topicRouteData = TopicRouteData.decode(body, TopicRouteData.class);
+                    for (BrokerData brokerData : topicRouteData.getBrokerDatas()) {
+                        if (brokerData != null
+                            && brokerData.getBrokerName().equals(brokerName)
+                            && brokerData.getCluster().equals(clusterName)) {
+                            brokerMemberGroup.getBrokerAddrs().putAll(brokerData.getBrokerAddrs());
+                            break;
+                        }
+                    }
+                    return brokerMemberGroup;
+                }
+            }
+            default:
+                break;
+        }
+
+        return brokerMemberGroup;
+    }
+
+    public void sendHeartbeatViaDataVersion(
+        final String clusterName,
+        final String brokerAddr,
+        final String brokerName,
+        final Long brokerId,
+        final int timeoutMillis,
+        final DataVersion dataVersion,
+        final boolean isInBrokerContainer) {
+        List<String> nameServerAddressList = this.remotingClient.getAvailableNameSrvList();
+        if (nameServerAddressList != null && nameServerAddressList.size() > 0) {
+            final QueryDataVersionRequestHeader requestHeader = new QueryDataVersionRequestHeader();
+            requestHeader.setBrokerAddr(brokerAddr);
+            requestHeader.setBrokerName(brokerName);
+            requestHeader.setBrokerId(brokerId);
+            requestHeader.setClusterName(clusterName);
+
+            for (final String namesrvAddr : nameServerAddressList) {
+                brokerOuterExecutor.execute(new AbstractBrokerRunnable(new BrokerIdentity(clusterName, brokerName, brokerId, isInBrokerContainer)) {
+
+                    @Override
+                    public void run2() {
+                        RemotingCommand request = RemotingCommand.createRequestCommand(RequestCode.QUERY_DATA_VERSION, requestHeader);
+                        request.setBody(dataVersion.encode());
+
+                        try {
+                            BrokerOuterAPI.this.remotingClient.invokeOneway(namesrvAddr, request, timeoutMillis);
+                        } catch (Exception e) {
+                            LOGGER.error("sendHeartbeat Exception " + namesrvAddr, e);
+                        }
+                    }
+                });
+            }
+        }
+    }
+
+    public void sendHeartbeat(final String clusterName,
+        final String brokerAddr,
+        final String brokerName,
+        final Long brokerId,
+        final int timeoutMills,
+        final boolean isInBrokerContainer) {
+        List<String> nameServerAddressList = this.remotingClient.getAvailableNameSrvList();
+
+        final BrokerHeartbeatRequestHeader requestHeader = new BrokerHeartbeatRequestHeader();
+        requestHeader.setClusterName(clusterName);
+        requestHeader.setBrokerAddr(brokerAddr);
+        requestHeader.setBrokerName(brokerName);
+
+        if (nameServerAddressList != null && nameServerAddressList.size() > 0) {
+            for (final String namesrvAddr : nameServerAddressList) {
+                brokerOuterExecutor.execute(new AbstractBrokerRunnable(new BrokerIdentity(clusterName, brokerName, brokerId, isInBrokerContainer)) {
+                    @Override
+                    public void run2() {
+                        RemotingCommand request = RemotingCommand.createRequestCommand(RequestCode.BROKER_HEARTBEAT, requestHeader);
+
+                        try {
+                            BrokerOuterAPI.this.remotingClient.invokeOneway(namesrvAddr, request, timeoutMills);
+                        } catch (Exception e) {
+                            LOGGER.error("sendHeartbeat Exception " + namesrvAddr, e);
+                        }
+                    }
+                });
+            }
+        }
+    }
+
+    public BrokerSyncInfo retrieveBrokerHaInfo(String masterBrokerAddr)
+        throws InterruptedException, RemotingTimeoutException, RemotingSendRequestException, RemotingConnectException,
+        MQBrokerException, RemotingCommandException {
+        ExchangeHAInfoRequestHeader requestHeader = new ExchangeHAInfoRequestHeader();
+        requestHeader.setMasterHaAddress(null);
+
+        RemotingCommand request = RemotingCommand.createRequestCommand(RequestCode.EXCHANGE_BROKER_HA_INFO, requestHeader);
+
+        RemotingCommand response = this.remotingClient.invokeSync(masterBrokerAddr, request, 3000);
+        assert response != null;
+        switch (response.getCode()) {
+            case ResponseCode.SUCCESS: {
+                ExchangeHAInfoResponseHeader responseHeader = response.decodeCommandCustomHeader(ExchangeHAInfoResponseHeader.class);
+                return new BrokerSyncInfo(responseHeader.getMasterHaAddress(), responseHeader.getMasterFlushOffset(), responseHeader.getMasterAddress());
+            }
+            default:
+                break;
+        }
+
+        throw new MQBrokerException(response.getCode(), response.getRemark());
+    }
+
+    public void sendBrokerHaInfo(String brokerAddr, String masterHaAddr, long brokerInitMaxOffset, String masterAddr)
+        throws InterruptedException, RemotingTimeoutException, RemotingSendRequestException, RemotingConnectException, MQBrokerException {
+        ExchangeHAInfoRequestHeader requestHeader = new ExchangeHAInfoRequestHeader();
+        requestHeader.setMasterHaAddress(masterHaAddr);
+        requestHeader.setMasterFlushOffset(brokerInitMaxOffset);
+        requestHeader.setMasterAddress(masterAddr);
+
+        RemotingCommand request = RemotingCommand.createRequestCommand(RequestCode.EXCHANGE_BROKER_HA_INFO, requestHeader);
+
+        RemotingCommand response = this.remotingClient.invokeSync(brokerAddr, request, 3000);
+
+        assert response != null;
+        switch (response.getCode()) {
+            case ResponseCode.SUCCESS: {
+                return;
+            }
+            default:
+                break;
+        }
+
+        throw new MQBrokerException(response.getCode(), response.getRemark());
     }
 
     public List<RegisterBrokerResult> registerBrokerAll(
@@ -141,10 +363,56 @@ public class BrokerOuterAPI {
         final List<String> filterServerList,
         final boolean oneway,
         final int timeoutMills,
-        final boolean compressed) {
+        final boolean enableActingMaster,
+        final boolean compressed,
+        final boolean isInBrokerContainer) {
+        return registerBrokerAll(clusterName,
+            brokerAddr,
+            brokerName,
+            brokerId,
+            haServerAddr,
+            topicConfigWrapper,
+            filterServerList,
+            oneway, timeoutMills,
+            enableActingMaster,
+            compressed,
+            null,
+            isInBrokerContainer);
+    }
+
+    /**
+     * Considering compression brings much CPU overhead to name server, stream API will not support compression and
+     * compression feature is deprecated.
+     *
+     * @param clusterName
+     * @param brokerAddr
+     * @param brokerName
+     * @param brokerId
+     * @param haServerAddr
+     * @param topicConfigWrapper
+     * @param filterServerList
+     * @param oneway
+     * @param timeoutMills
+     * @param compressed default false
+     * @return
+     */
+    public List<RegisterBrokerResult> registerBrokerAll(
+        final String clusterName,
+        final String brokerAddr,
+        final String brokerName,
+        final long brokerId,
+        final String haServerAddr,
+        final TopicConfigSerializeWrapper topicConfigWrapper,
+        final List<String> filterServerList,
+        final boolean oneway,
+        final int timeoutMills,
+        final boolean enableActingMaster,
+        final boolean compressed,
+        final Long heartbeatTimeoutMillis,
+        final boolean isInBrokerContainer) {
 
         final List<RegisterBrokerResult> registerBrokerResultList = new CopyOnWriteArrayList<>();
-        List<String> nameServerAddressList = this.remotingClient.getNameServerAddressList();
+        List<String> nameServerAddressList = this.remotingClient.getAvailableNameSrvList();
         if (nameServerAddressList != null && nameServerAddressList.size() > 0) {
 
             final RegisterBrokerRequestHeader requestHeader = new RegisterBrokerRequestHeader();
@@ -153,7 +421,11 @@ public class BrokerOuterAPI {
             requestHeader.setBrokerName(brokerName);
             requestHeader.setClusterName(clusterName);
             requestHeader.setHaServerAddr(haServerAddr);
-            requestHeader.setCompressed(compressed);
+            requestHeader.setEnableActingMaster(enableActingMaster);
+            requestHeader.setCompressed(false);
+            if (heartbeatTimeoutMillis != null) {
+                requestHeader.setHeartbeatTimeoutMillis(heartbeatTimeoutMillis);
+            }
 
             RegisterBrokerBody requestBody = new RegisterBrokerBody();
             requestBody.setTopicConfigSerializeWrapper(TopicConfigAndMappingSerializeWrapper.from(topicConfigWrapper));
@@ -163,18 +435,17 @@ public class BrokerOuterAPI {
             requestHeader.setBodyCrc32(bodyCrc32);
             final CountDownLatch countDownLatch = new CountDownLatch(nameServerAddressList.size());
             for (final String namesrvAddr : nameServerAddressList) {
-                brokerOuterExecutor.execute(new Runnable() {
-                    @Override
-                    public void run() {
+                brokerOuterExecutor.execute(new AbstractBrokerRunnable(new BrokerIdentity(clusterName, brokerName, brokerId, isInBrokerContainer)) {
+                    @Override public void run2() {
                         try {
                             RegisterBrokerResult result = registerBroker(namesrvAddr, oneway, timeoutMills, requestHeader, body);
                             if (result != null) {
                                 registerBrokerResultList.add(result);
                             }
 
-                            log.info("register broker[{}]to name server {} OK", brokerId, namesrvAddr);
+                            LOGGER.info("Registering current broker to name server completed. TargetHost={}", namesrvAddr);
                         } catch (Exception e) {
-                            log.warn("registerBroker Exception, {}", namesrvAddr, e);
+                            LOGGER.error("Failed to register current broker to name server. TargetHost={}", namesrvAddr, e);
                         } finally {
                             countDownLatch.countDown();
                         }
@@ -183,8 +454,10 @@ public class BrokerOuterAPI {
             }
 
             try {
-                countDownLatch.await(timeoutMills, TimeUnit.MILLISECONDS);
-            } catch (InterruptedException e) {
+                if (!countDownLatch.await(timeoutMills, TimeUnit.MILLISECONDS)) {
+                    LOGGER.warn("Registration to one or more name servers does NOT complete within deadline. Timeout threshold: {}ms", timeoutMills);
+                }
+            } catch (InterruptedException ignore) {
             }
         }
 
@@ -243,9 +516,9 @@ public class BrokerOuterAPI {
             for (String namesrvAddr : nameServerAddressList) {
                 try {
                     this.unregisterBroker(namesrvAddr, clusterName, brokerAddr, brokerName, brokerId);
-                    log.info("unregisterBroker OK, NamesrvAddr: {}", namesrvAddr);
+                    LOGGER.info("unregisterBroker OK, NamesrvAddr: {}", namesrvAddr);
                 } catch (Exception e) {
-                    log.warn("unregisterBroker Exception, {}", namesrvAddr, e);
+                    LOGGER.warn("unregisterBroker Exception, {}", namesrvAddr, e);
                 }
             }
         }
@@ -284,15 +557,15 @@ public class BrokerOuterAPI {
         final String brokerName,
         final long brokerId,
         final TopicConfigSerializeWrapper topicConfigWrapper,
-        final int timeoutMills) {
+        final int timeoutMills,
+        final boolean isInBrokerContainer) {
         final List<Boolean> changedList = new CopyOnWriteArrayList<>();
         List<String> nameServerAddressList = this.remotingClient.getNameServerAddressList();
         if (nameServerAddressList != null && nameServerAddressList.size() > 0) {
             final CountDownLatch countDownLatch = new CountDownLatch(nameServerAddressList.size());
             for (final String namesrvAddr : nameServerAddressList) {
-                brokerOuterExecutor.execute(new Runnable() {
-                    @Override
-                    public void run() {
+                brokerOuterExecutor.execute(new AbstractBrokerRunnable(new BrokerIdentity(clusterName, brokerName, brokerId, isInBrokerContainer)) {
+                    @Override public void run2() {
                         try {
                             QueryDataVersionRequestHeader requestHeader = new QueryDataVersionRequestHeader();
                             requestHeader.setBrokerAddr(brokerAddr);
@@ -323,10 +596,10 @@ public class BrokerOuterAPI {
                                 default:
                                     break;
                             }
-                            log.warn("Query data version from name server {} OK,changed {}, broker {},name server {}", namesrvAddr, changed, topicConfigWrapper.getDataVersion(), nameServerDataVersion == null ? "" : nameServerDataVersion);
+                            LOGGER.warn("Query data version from name server {} OK, changed {}, broker {},name server {}", namesrvAddr, changed, topicConfigWrapper.getDataVersion(), nameServerDataVersion == null ? "" : nameServerDataVersion);
                         } catch (Exception e) {
                             changedList.add(Boolean.TRUE);
-                            log.error("Query data version from name server {}  Exception, {}", namesrvAddr, e);
+                            LOGGER.error("Query data version from name server {}  Exception, {}", namesrvAddr, e);
                         } finally {
                             countDownLatch.countDown();
                         }
@@ -337,7 +610,7 @@ public class BrokerOuterAPI {
             try {
                 countDownLatch.await(timeoutMills, TimeUnit.MILLISECONDS);
             } catch (InterruptedException e) {
-                log.error("query dataversion from nameserver countDownLatch await Exception", e);
+                LOGGER.error("query dataversion from nameserver countDownLatch await Exception", e);
             }
         }
         return changedList;
@@ -416,6 +689,220 @@ public class BrokerOuterAPI {
         remotingClient.registerRPCHook(rpcHook);
     }
 
+    public void clearRPCHook() {
+        remotingClient.clearRPCHook();
+    }
+
+    public long getMaxOffset(final String addr, final String topic, final int queueId, final boolean committed,
+        final boolean isOnlyThisBroker)
+        throws RemotingException, MQBrokerException, InterruptedException {
+        GetMaxOffsetRequestHeader requestHeader = new GetMaxOffsetRequestHeader();
+        requestHeader.setTopic(topic);
+        requestHeader.setQueueId(queueId);
+        requestHeader.setCommitted(committed);
+        RemotingCommand request = RemotingCommand.createRequestCommand(RequestCode.GET_MAX_OFFSET, requestHeader);
+
+        RemotingCommand response = this.remotingClient.invokeSync(addr, request, 3000);
+        assert response != null;
+        switch (response.getCode()) {
+            case ResponseCode.SUCCESS: {
+                GetMaxOffsetResponseHeader responseHeader = response.decodeCommandCustomHeader(GetMaxOffsetResponseHeader.class);
+
+                return responseHeader.getOffset();
+            }
+            default:
+                break;
+        }
+
+        throw new MQBrokerException(response.getCode(), response.getRemark());
+    }
+
+    public long getMinOffset(final String addr, final String topic, final int queueId, final boolean isOnlyThisBroker)
+        throws RemotingException, MQBrokerException, InterruptedException {
+        GetMinOffsetRequestHeader requestHeader = new GetMinOffsetRequestHeader();
+        requestHeader.setTopic(topic);
+        requestHeader.setQueueId(queueId);
+        RemotingCommand request = RemotingCommand.createRequestCommand(RequestCode.GET_MIN_OFFSET, requestHeader);
+
+        RemotingCommand response = this.remotingClient.invokeSync(addr, request, 3000);
+        assert response != null;
+        switch (response.getCode()) {
+            case ResponseCode.SUCCESS: {
+                GetMinOffsetResponseHeader responseHeader = response.decodeCommandCustomHeader(GetMinOffsetResponseHeader.class);
+
+                return responseHeader.getOffset();
+            }
+            default:
+                break;
+        }
+
+        throw new MQBrokerException(response.getCode(), response.getRemark());
+    }
+
+    public void lockBatchMQAsync(
+        final String addr,
+        final LockBatchRequestBody requestBody,
+        final long timeoutMillis,
+        final LockCallback callback) throws RemotingException, InterruptedException {
+        RemotingCommand request = RemotingCommand.createRequestCommand(RequestCode.LOCK_BATCH_MQ, null);
+
+        request.setBody(requestBody.encode());
+        this.remotingClient.invokeAsync(addr, request, timeoutMillis, responseFuture -> {
+            if (callback == null) {
+                return;
+            }
+
+            try {
+                RemotingCommand response = responseFuture.getResponseCommand();
+                if (response != null) {
+                    if (response.getCode() == ResponseCode.SUCCESS) {
+                        LockBatchResponseBody responseBody = LockBatchResponseBody.decode(response.getBody(),
+                            LockBatchResponseBody.class);
+                        Set<MessageQueue> messageQueues = responseBody.getLockOKMQSet();
+                        callback.onSuccess(messageQueues);
+                    } else {
+                        callback.onException(new MQBrokerException(response.getCode(), response.getRemark()));
+                    }
+                }
+            } catch (Throwable ignored) {
+
+            }
+        });
+    }
+
+    public void unlockBatchMQAsync(
+        final String addr,
+        final UnlockBatchRequestBody requestBody,
+        final long timeoutMillis,
+        final UnlockCallback callback) throws RemotingException, InterruptedException {
+        RemotingCommand request = RemotingCommand.createRequestCommand(RequestCode.UNLOCK_BATCH_MQ, null);
+
+        request.setBody(requestBody.encode());
+
+        this.remotingClient.invokeAsync(addr, request, timeoutMillis, responseFuture -> {
+            if (callback == null) {
+                return;
+            }
+
+            try {
+                RemotingCommand response = responseFuture.getResponseCommand();
+                if (response != null) {
+                    if (response.getCode() == ResponseCode.SUCCESS) {
+                        callback.onSuccess();
+                    } else {
+                        callback.onException(new MQBrokerException(response.getCode(), response.getRemark()));
+                    }
+                }
+            } catch (Throwable ignored) {
+
+            }
+        });
+    }
+
+    public RemotingClient getRemotingClient() {
+        return this.remotingClient;
+    }
+
+    public SendResult sendMessageToSpecificBroker(String brokerAddr, final String brokerName,
+        final MessageExt msg, String group,
+        long timeoutMillis) throws RemotingException, MQBrokerException, InterruptedException {
+
+        SendMessageRequestHeader requestHeader = new SendMessageRequestHeader();
+        requestHeader.setProducerGroup(group);
+        requestHeader.setTopic(msg.getTopic());
+        requestHeader.setDefaultTopic(TopicValidator.AUTO_CREATE_TOPIC_KEY_TOPIC);
+        requestHeader.setDefaultTopicQueueNums(8);
+        requestHeader.setQueueId(msg.getQueueId());
+        requestHeader.setSysFlag(msg.getSysFlag());
+        requestHeader.setBornTimestamp(msg.getBornTimestamp());
+        requestHeader.setFlag(msg.getFlag());
+        requestHeader.setProperties(MessageDecoder.messageProperties2String(msg.getProperties()));
+        requestHeader.setReconsumeTimes(msg.getReconsumeTimes());
+        requestHeader.setBatch(false);
+
+        SendMessageRequestHeaderV2 requestHeaderV2 = SendMessageRequestHeaderV2.createSendMessageRequestHeaderV2(requestHeader);
+        RemotingCommand request = RemotingCommand.createRequestCommand(RequestCode.SEND_MESSAGE_V2, requestHeaderV2);
+
+        request.setBody(msg.getBody());
+
+        RemotingCommand response = this.remotingClient.invokeSync(brokerAddr, request, timeoutMillis);
+
+        return this.processSendResponse(brokerName, msg, response);
+    }
+
+    private SendResult processSendResponse(
+        final String brokerName,
+        final Message msg,
+        final RemotingCommand response
+    ) throws MQBrokerException, RemotingCommandException {
+        switch (response.getCode()) {
+            case ResponseCode.FLUSH_DISK_TIMEOUT:
+            case ResponseCode.FLUSH_SLAVE_TIMEOUT:
+            case ResponseCode.SLAVE_NOT_AVAILABLE:
+            case ResponseCode.SUCCESS: {
+                SendStatus sendStatus = SendStatus.SEND_OK;
+                switch (response.getCode()) {
+                    case ResponseCode.FLUSH_DISK_TIMEOUT:
+                        sendStatus = SendStatus.FLUSH_DISK_TIMEOUT;
+                        break;
+                    case ResponseCode.FLUSH_SLAVE_TIMEOUT:
+                        sendStatus = SendStatus.FLUSH_SLAVE_TIMEOUT;
+                        break;
+                    case ResponseCode.SLAVE_NOT_AVAILABLE:
+                        sendStatus = SendStatus.SLAVE_NOT_AVAILABLE;
+                        break;
+                    case ResponseCode.SUCCESS:
+                        sendStatus = SendStatus.SEND_OK;
+                        break;
+                    default:
+                        assert false;
+                        break;
+                }
+
+                SendMessageResponseHeader responseHeader =
+                    (SendMessageResponseHeader) response.decodeCommandCustomHeader(SendMessageResponseHeader.class);
+
+                //If namespace not null , reset Topic without namespace.
+                String topic = msg.getTopic();
+
+                MessageQueue messageQueue = new MessageQueue(topic, brokerName, responseHeader.getQueueId());
+
+                String uniqMsgId = MessageClientIDSetter.getUniqID(msg);
+                if (msg instanceof MessageBatch) {
+                    StringBuilder sb = new StringBuilder();
+                    for (Message message : (MessageBatch) msg) {
+                        sb.append(sb.length() == 0 ? "" : ",").append(MessageClientIDSetter.getUniqID(message));
+                    }
+                    uniqMsgId = sb.toString();
+                }
+                SendResult sendResult = new SendResult(sendStatus,
+                    uniqMsgId,
+                    responseHeader.getMsgId(), messageQueue, responseHeader.getQueueOffset());
+                sendResult.setTransactionId(responseHeader.getTransactionId());
+                String regionId = response.getExtFields().get(MessageConst.PROPERTY_MSG_REGION);
+                String traceOn = response.getExtFields().get(MessageConst.PROPERTY_TRACE_SWITCH);
+                if (regionId == null || regionId.isEmpty()) {
+                    regionId = MixAll.DEFAULT_TRACE_REGION_ID;
+                }
+                if (traceOn != null && traceOn.equals("false")) {
+                    sendResult.setTraceOn(false);
+                } else {
+                    sendResult.setTraceOn(true);
+                }
+                sendResult.setRegionId(regionId);
+                return sendResult;
+            }
+            default:
+                break;
+        }
+
+        throw new MQBrokerException(response.getCode(), response.getRemark());
+    }
+
+    public BrokerFixedThreadPoolExecutor getBrokerOuterExecutor() {
+        return brokerOuterExecutor;
+    }
+
     public TopicRouteData getTopicRouteInfoFromNameServer(final String topic, final long timeoutMillis)
         throws RemotingException, MQBrokerException, InterruptedException {
         return getTopicRouteInfoFromNameServer(topic, timeoutMillis, true);
@@ -433,7 +920,7 @@ public class BrokerOuterAPI {
         switch (response.getCode()) {
             case ResponseCode.TOPIC_NOT_EXIST: {
                 if (allowTopicNotExist) {
-                    log.warn("get Topic [{}] RouteInfoFromNameServer is not exist value", topic);
+                    LOGGER.warn("get Topic [{}] RouteInfoFromNameServer is not exist value", topic);
                 }
 
                 break;
@@ -466,17 +953,16 @@ public class BrokerOuterAPI {
         throw new MQBrokerException(response.getCode(), response.getRemark());
     }
 
-    public void forwardRequest(String brokerAddr, RemotingCommand request, long timeoutMillis, InvokeCallback invokeCallback) throws InterruptedException, RemotingSendRequestException, RemotingTimeoutException, RemotingTooMuchRequestException, RemotingConnectException {
+    public void forwardRequest(String brokerAddr, RemotingCommand request, long timeoutMillis,
+        InvokeCallback invokeCallback) throws InterruptedException, RemotingSendRequestException, RemotingTimeoutException, RemotingTooMuchRequestException, RemotingConnectException {
         this.remotingClient.invokeAsync(brokerAddr, request, timeoutMillis, invokeCallback);
     }
 
-
     public void refreshMetadata() throws Exception {
         ClusterInfo brokerClusterInfo = getBrokerClusterInfo();
         clientMetadata.refreshClusterInfo(brokerClusterInfo);
     }
 
-
     public ClientMetadata getClientMetadata() {
         return clientMetadata;
     }
diff --git a/broker/src/main/java/org/apache/rocketmq/broker/pagecache/OneMessageTransfer.java b/broker/src/main/java/org/apache/rocketmq/broker/pagecache/OneMessageTransfer.java
index 558f091..952cf4f 100644
--- a/broker/src/main/java/org/apache/rocketmq/broker/pagecache/OneMessageTransfer.java
+++ b/broker/src/main/java/org/apache/rocketmq/broker/pagecache/OneMessageTransfer.java
@@ -70,6 +70,7 @@ public class OneMessageTransfer extends AbstractReferenceCounted implements File
         return 0;
     }
 
+
     @Override
     public FileRegion retain() {
         super.retain();
diff --git a/broker/src/main/java/org/apache/rocketmq/broker/plugin/AbstractPluginMessageStore.java b/broker/src/main/java/org/apache/rocketmq/broker/plugin/AbstractPluginMessageStore.java
index 9aa1d9b..4254221 100644
--- a/broker/src/main/java/org/apache/rocketmq/broker/plugin/AbstractPluginMessageStore.java
+++ b/broker/src/main/java/org/apache/rocketmq/broker/plugin/AbstractPluginMessageStore.java
@@ -17,22 +17,43 @@
 
 package org.apache.rocketmq.broker.plugin;
 
+import java.nio.ByteBuffer;
 import java.util.HashMap;
 import java.util.LinkedList;
+import java.util.Optional;
 import java.util.Set;
+import java.util.List;
 import java.util.concurrent.CompletableFuture;
+import org.apache.rocketmq.common.SystemClock;
+import org.apache.rocketmq.common.TopicConfig;
 import org.apache.rocketmq.common.message.MessageExt;
+import org.apache.rocketmq.common.message.MessageExtBatch;
+import org.apache.rocketmq.common.protocol.body.HARuntimeInfo;
+import org.apache.rocketmq.store.AllocateMappedFileService;
+import org.apache.rocketmq.store.AppendMessageResult;
+import org.apache.rocketmq.store.CommitLog;
 import org.apache.rocketmq.store.CommitLogDispatcher;
+import org.apache.rocketmq.store.DispatchRequest;
 import org.apache.rocketmq.store.GetMessageResult;
-import org.apache.rocketmq.store.MessageExtBatch;
-import org.apache.rocketmq.store.MessageExtBrokerInner;
+import org.apache.rocketmq.common.message.MessageExtBrokerInner;
 import org.apache.rocketmq.store.MessageFilter;
 import org.apache.rocketmq.store.MessageStore;
 import org.apache.rocketmq.store.PutMessageResult;
 import org.apache.rocketmq.store.QueryMessageResult;
+import org.apache.rocketmq.store.RunningFlags;
 import org.apache.rocketmq.store.SelectMappedBufferResult;
+import org.apache.rocketmq.store.StoreCheckpoint;
+import org.apache.rocketmq.store.StoreStatsService;
+import org.apache.rocketmq.store.TransientStorePool;
+import org.apache.rocketmq.store.config.MessageStoreConfig;
+import org.apache.rocketmq.store.ha.HAService;
+import org.apache.rocketmq.store.hook.PutMessageHook;
+import org.apache.rocketmq.store.hook.SendMessageBackHook;
+import org.apache.rocketmq.store.logfile.MappedFile;
 import org.apache.rocketmq.store.queue.ConsumeQueueInterface;
+import org.apache.rocketmq.store.queue.ConsumeQueueStore;
 import org.apache.rocketmq.store.stats.BrokerStatsManager;
+import org.apache.rocketmq.store.util.PerfCounter;
 
 public abstract class AbstractPluginMessageStore implements MessageStore {
     protected MessageStore next = null;
@@ -201,16 +222,6 @@ public abstract class AbstractPluginMessageStore implements MessageStore {
     }
 
     @Override
-    public void updateHaMasterAddress(String newAddr) {
-        next.updateHaMasterAddress(newAddr);
-    }
-
-    @Override
-    public long slaveFallBehindMuch() {
-        return next.slaveFallBehindMuch();
-    }
-
-    @Override
     public long now() {
         return next.now();
     }
@@ -269,4 +280,255 @@ public abstract class AbstractPluginMessageStore implements MessageStore {
     public BrokerStatsManager getBrokerStatsManager() {
         return next.getBrokerStatsManager();
     }
+
+    @Override
+    public int remainTransientStoreBufferNumbs() {
+        return next.remainTransientStoreBufferNumbs();
+    }
+
+    @Override
+    public long remainHowManyDataToCommit() {
+        return next.remainHowManyDataToCommit();
+    }
+
+    @Override
+    public long remainHowManyDataToFlush() {
+        return next.remainHowManyDataToFlush();
+    }
+
+    @Override
+    public DispatchRequest checkMessageAndReturnSize(final ByteBuffer byteBuffer, final boolean checkCRC,
+        final boolean checkDupInfo, final boolean readBody) {
+        return next.checkMessageAndReturnSize(byteBuffer, checkCRC, checkDupInfo, readBody);
+    }
+
+    @Override
+    public long getStateMachineVersion() {
+        return next.getStateMachineVersion();
+    }
+
+    @Override public PutMessageResult putMessages(MessageExtBatch messageExtBatch) {
+        return next.putMessages(messageExtBatch);
+    }
+
+    @Override public HARuntimeInfo getHARuntimeInfo() {
+        return next.getHARuntimeInfo();
+    }
+
+    @Override public boolean getLastMappedFile(long startOffset) {
+        return next.getLastMappedFile(startOffset);
+    }
+
+    @Override public void updateHaMasterAddress(String newAddr) {
+        next.updateHaMasterAddress(newAddr);
+    }
+
+    @Override public void updateMasterAddress(String newAddr) {
+        next.updateMasterAddress(newAddr);
+    }
+
+    @Override public long slaveFallBehindMuch() {
+        return next.slaveFallBehindMuch();
+    }
+
+    @Override public long getFlushedWhere() {
+        return next.getFlushedWhere();
+    }
+
+    @Override public MessageStore getMasterStoreInProcess() {
+        return next.getMasterStoreInProcess();
+    }
+
+    @Override public void setMasterStoreInProcess(MessageStore masterStoreInProcess) {
+        next.setMasterStoreInProcess(masterStoreInProcess);
+    }
+
+    @Override
+    public boolean getData(long offset, int size, ByteBuffer byteBuffer) {
+        return next.getData(offset, size, byteBuffer);
+    }
+
+    @Override public void setAliveReplicaNumInGroup(int aliveReplicaNums) {
+        next.setAliveReplicaNumInGroup(aliveReplicaNums);
+    }
+
+    @Override public int getAliveReplicaNumInGroup() {
+        return next.getAliveReplicaNumInGroup();
+    }
+
+    @Override public void wakeupHAClient() {
+        next.wakeupHAClient();
+    }
+
+    @Override public long getMasterFlushedOffset() {
+        return next.getMasterFlushedOffset();
+    }
+
+    @Override public long getBrokerInitMaxOffset() {
+        return next.getBrokerInitMaxOffset();
+    }
+
+    @Override public void setMasterFlushedOffset(long masterFlushedOffset) {
+        next.setMasterFlushedOffset(masterFlushedOffset);
+    }
+
+    @Override public void setBrokerInitMaxOffset(long brokerInitMaxOffset) {
+        next.setBrokerInitMaxOffset(brokerInitMaxOffset);
+    }
+
+    @Override public byte[] calcDeltaChecksum(long from, long to) {
+        return next.calcDeltaChecksum(from, to);
+    }
+
+    @Override public HAService getHaService() {
+        return next.getHaService();
+    }
+
+    @Override public boolean truncateFiles(long offsetToTruncate) {
+        return next.truncateFiles(offsetToTruncate);
+    }
+
+    @Override public boolean isOffsetAligned(long offset) {
+        return next.isOffsetAligned(offset);
+    }
+
+    @Override public RunningFlags getRunningFlags() {
+        return next.getRunningFlags();
+    }
+
+    @Override public void setSendMessageBackHook(SendMessageBackHook sendMessageBackHook) {
+        next.setSendMessageBackHook(sendMessageBackHook);
+    }
+
+    @Override public SendMessageBackHook getSendMessageBackHook() {
+        return next.getSendMessageBackHook();
+    }
+
+    @Override
+    public GetMessageResult getMessage(String group, String topic, int queueId, long offset,
+        int maxMsgNums, int maxTotalMsgSize, MessageFilter messageFilter) {
+        return next.getMessage(group, topic, queueId, offset, maxMsgNums, maxTotalMsgSize, messageFilter);
+    }
+
+    @Override public MessageExt lookMessageByOffset(long commitLogOffset, int size) {
+        return next.lookMessageByOffset(commitLogOffset, size);
+    }
+
+    @Override public List<SelectMappedBufferResult> getBulkCommitLogData(long offset, int size) {
+        return next.getBulkCommitLogData(offset, size);
+    }
+
+    @Override
+    public void onCommitLogAppend(MessageExtBrokerInner msg, AppendMessageResult result, MappedFile commitLogFile) {
+        next.onCommitLogAppend(msg, result, commitLogFile);
+    }
+
+    @Override
+    public void onCommitLogDispatch(DispatchRequest dispatchRequest, boolean doDispatch, MappedFile commitLogFile,
+        boolean isRecover, boolean isFileEnd) {
+        next.onCommitLogDispatch(dispatchRequest, doDispatch, commitLogFile, isRecover, isFileEnd);
+    }
+
+    @Override
+    public MessageStoreConfig getMessageStoreConfig() {
+        return next.getMessageStoreConfig();
+    }
+
+    @Override
+    public StoreStatsService getStoreStatsService() {
+        return next.getStoreStatsService();
+    }
+
+    @Override
+    public StoreCheckpoint getStoreCheckpoint() {
+        return next.getStoreCheckpoint();
+    }
+
+    @Override
+    public SystemClock getSystemClock() {
+        return next.getSystemClock();
+    }
+
+    @Override
+    public CommitLog getCommitLog() {
+        return next.getCommitLog();
+    }
+
+    @Override
+    public TransientStorePool getTransientStorePool() {
+        return next.getTransientStorePool();
+    }
+
+    @Override
+    public AllocateMappedFileService getAllocateMappedFileService() {
+        return next.getAllocateMappedFileService();
+    }
+
+    @Override
+    public void truncateDirtyLogicFiles(long phyOffset) {
+        next.truncateDirtyLogicFiles(phyOffset);
+    }
+
+    @Override
+    public void destroyLogics() {
+        next.destroyLogics();
+    }
+
+    @Override
+    public void unlockMappedFile(MappedFile unlockMappedFile) {
+        next.unlockMappedFile(unlockMappedFile);
+    }
+
+    @Override
+    public PerfCounter.Ticks getPerfCounter() {
+        return next.getPerfCounter();
+    }
+
+    @Override
+    public ConsumeQueueStore getQueueStore() {
+        return next.getQueueStore();
+    }
+
+    @Override
+    public boolean isSyncDiskFlush() {
+        return next.isSyncDiskFlush();
+    }
+
+    @Override
+    public boolean isSyncMaster() {
+        return next.isSyncMaster();
+    }
+
+    @Override
+    public void assignOffset(MessageExtBrokerInner msg, short messageNum) {
+        next.assignOffset(msg, messageNum);
+    }
+
+    @Override
+    public Optional<TopicConfig> getTopicConfig(String topic) {
+        return next.getTopicConfig(topic);
+    }
+
+    @Override
+    public List<PutMessageHook> getPutMessageHookList() {
+        return next.getPutMessageHookList();
+    }
+
+    @Override
+    public long getLastFileFromOffset() {
+        return next.getLastFileFromOffset();
+    }
+
+    @Override
+    public void setPhysicalOffset(long phyOffset) {
+        next.setPhysicalOffset(phyOffset);
+    }
+
+    @Override public boolean isMappedFilesEmpty() {
+        return next.isMappedFilesEmpty();
+    }
+
+    @Override public boolean isShutdown() {
+        return next.isShutdown();
+    }
 }
diff --git a/broker/src/main/java/org/apache/rocketmq/broker/plugin/BrokerAttachedPlugin.java b/broker/src/main/java/org/apache/rocketmq/broker/plugin/BrokerAttachedPlugin.java
new file mode 100644
index 0000000..0cd2a27
--- /dev/null
+++ b/broker/src/main/java/org/apache/rocketmq/broker/plugin/BrokerAttachedPlugin.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.broker.plugin;
+
+import java.util.Map;
+
+public interface BrokerAttachedPlugin {
+
+    /**
+     * Get plugin name
+     *
+     * @return plugin name
+     */
+    String pluginName();
+
+    /**
+     * Load broker attached plugin.
+     *
+     * @return load success or failed
+     */
+    boolean load();
+
+    /**
+     * Start broker attached plugin.
+     */
+    void start();
+
+    /**
+     * Shutdown broker attached plugin.
+     */
+    void shutdown();
+
+    /**
+     * Sync metadata from master.
+     */
+    void syncMetadata();
+
+    /**
+     * Sync metadata reverse from slave
+     *
+     * @param brokerAddr
+     */
+    void syncMetadataReverse(String brokerAddr) throws Exception;
+
+    /**
+     * Some plugin need build runningInfo when prepare runtime info.
+     *
+     * @param runtimeInfo
+     */
+    void buildRuntimeInfo(Map<String, String> runtimeInfo);
+
+    /**
+     * Some plugin need do something when status changed. For example, brokerRole change to master or slave.
+     *
+     * @param shouldStart
+     */
+    void statusChanged(boolean shouldStart);
+
+}
diff --git a/broker/src/main/java/org/apache/rocketmq/broker/plugin/MessageStoreFactory.java b/broker/src/main/java/org/apache/rocketmq/broker/plugin/MessageStoreFactory.java
index 8db538b..b64ab5a 100644
--- a/broker/src/main/java/org/apache/rocketmq/broker/plugin/MessageStoreFactory.java
+++ b/broker/src/main/java/org/apache/rocketmq/broker/plugin/MessageStoreFactory.java
@@ -22,8 +22,8 @@ import java.lang.reflect.Constructor;
 import org.apache.rocketmq.store.MessageStore;
 
 public final class MessageStoreFactory {
-    public final static MessageStore build(MessageStorePluginContext context, MessageStore messageStore)
-        throws IOException {
+    public final static MessageStore build(MessageStorePluginContext context,
+        MessageStore messageStore) throws IOException {
         String plugin = context.getBrokerConfig().getMessageStorePlugIn();
         if (plugin != null && plugin.trim().length() != 0) {
             String[] pluginClasses = plugin.split(",");
@@ -31,12 +31,13 @@ public final class MessageStoreFactory {
                 String pluginClass = pluginClasses[i];
                 try {
                     @SuppressWarnings("unchecked")
-                    Class<AbstractPluginMessageStore> clazz = (Class<AbstractPluginMessageStore>) Class.forName(pluginClass);
+                    Class<AbstractPluginMessageStore> clazz = (Class<AbstractPluginMessageStore>)Class.forName(pluginClass);
                     Constructor<AbstractPluginMessageStore> construct = clazz.getConstructor(MessageStorePluginContext.class, MessageStore.class);
-                    messageStore = construct.newInstance(context, messageStore);
-                } catch (Throwable e) {
-                    throw new RuntimeException(String.format(
-                        "Initialize plugin's class %s not found!", pluginClass), e);
+                    AbstractPluginMessageStore pluginMessageStore = construct.newInstance(context, messageStore);
+                    messageStore = pluginMessageStore;
+                }
+                catch (Throwable e) {
+                    throw new RuntimeException("Initialize plugin's class: " + pluginClass + " not found!", e);
                 }
             }
         }
diff --git a/broker/src/main/java/org/apache/rocketmq/broker/plugin/MessageStorePluginContext.java b/broker/src/main/java/org/apache/rocketmq/broker/plugin/MessageStorePluginContext.java
index b822a2f..c132cf9 100644
--- a/broker/src/main/java/org/apache/rocketmq/broker/plugin/MessageStorePluginContext.java
+++ b/broker/src/main/java/org/apache/rocketmq/broker/plugin/MessageStorePluginContext.java
@@ -17,25 +17,25 @@
 
 package org.apache.rocketmq.broker.plugin;
 
+import org.apache.rocketmq.broker.BrokerController;
 import org.apache.rocketmq.common.BrokerConfig;
 import org.apache.rocketmq.store.MessageArrivingListener;
 import org.apache.rocketmq.store.config.MessageStoreConfig;
 import org.apache.rocketmq.store.stats.BrokerStatsManager;
 
 public class MessageStorePluginContext {
+    private BrokerController controller;
     private MessageStoreConfig messageStoreConfig;
     private BrokerStatsManager brokerStatsManager;
     private MessageArrivingListener messageArrivingListener;
-    private BrokerConfig brokerConfig;
 
-    public MessageStorePluginContext(MessageStoreConfig messageStoreConfig,
-        BrokerStatsManager brokerStatsManager, MessageArrivingListener messageArrivingListener,
-        BrokerConfig brokerConfig) {
+    public MessageStorePluginContext(BrokerController controller, MessageStoreConfig messageStoreConfig,
+        BrokerStatsManager brokerStatsManager, MessageArrivingListener messageArrivingListener) {
         super();
         this.messageStoreConfig = messageStoreConfig;
         this.brokerStatsManager = brokerStatsManager;
         this.messageArrivingListener = messageArrivingListener;
-        this.brokerConfig = brokerConfig;
+        this.controller = controller;
     }
 
     public MessageStoreConfig getMessageStoreConfig() {
@@ -51,7 +51,11 @@ public class MessageStorePluginContext {
     }
 
     public BrokerConfig getBrokerConfig() {
-        return brokerConfig;
+        return controller.getBrokerConfig();
+    }
+
+    public BrokerController getController() {
+        return controller;
     }
 
 }
diff --git a/broker/src/main/java/org/apache/rocketmq/broker/plugin/PullMessageResultHandler.java b/broker/src/main/java/org/apache/rocketmq/broker/plugin/PullMessageResultHandler.java
new file mode 100644
index 0000000..3383a64
--- /dev/null
+++ b/broker/src/main/java/org/apache/rocketmq/broker/plugin/PullMessageResultHandler.java
@@ -0,0 +1,53 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.rocketmq.broker.plugin;
+
+import io.netty.channel.Channel;
+import org.apache.rocketmq.common.protocol.header.PullMessageRequestHeader;
+import org.apache.rocketmq.common.protocol.heartbeat.SubscriptionData;
+import org.apache.rocketmq.common.subscription.SubscriptionGroupConfig;
+import org.apache.rocketmq.remoting.protocol.RemotingCommand;
+import org.apache.rocketmq.store.GetMessageResult;
+import org.apache.rocketmq.store.MessageFilter;
+
+public interface PullMessageResultHandler {
+
+    /**
+     * Handle result of get message from store.
+     *
+     * @param getMessageResult store result
+     * @param request request
+     * @param requestHeader request header
+     * @param channel channel
+     * @param subscriptionData sub data
+     * @param subscriptionGroupConfig sub config
+     * @param brokerAllowSuspend brokerAllowSuspend
+     * @param messageFilter store message filter
+     * @param response response
+     * @return response or null
+     */
+    RemotingCommand handle(final GetMessageResult getMessageResult,
+                           final RemotingCommand request,
+                           final PullMessageRequestHeader requestHeader,
+                           final Channel channel,
+                           final SubscriptionData subscriptionData,
+                           final SubscriptionGroupConfig subscriptionGroupConfig,
+                           final boolean brokerAllowSuspend,
+                           final MessageFilter messageFilter,
+                           final RemotingCommand response);
+}
diff --git a/broker/src/main/java/org/apache/rocketmq/broker/processor/AbstractSendMessageProcessor.java b/broker/src/main/java/org/apache/rocketmq/broker/processor/AbstractSendMessageProcessor.java
index 3303d70..1821239 100644
--- a/broker/src/main/java/org/apache/rocketmq/broker/processor/AbstractSendMessageProcessor.java
+++ b/broker/src/main/java/org/apache/rocketmq/broker/processor/AbstractSendMessageProcessor.java
@@ -17,16 +17,26 @@
 package org.apache.rocketmq.broker.processor;
 
 import io.netty.channel.ChannelHandlerContext;
-import java.net.InetSocketAddress;
 import java.net.SocketAddress;
 import java.util.HashMap;
 import java.util.List;
 import java.util.Map;
+import java.util.Random;
 import java.util.concurrent.ThreadLocalRandom;
 
 import org.apache.rocketmq.broker.BrokerController;
+import org.apache.rocketmq.broker.mqtrace.AbortProcessException;
+import org.apache.rocketmq.broker.mqtrace.ConsumeMessageContext;
+import org.apache.rocketmq.broker.mqtrace.ConsumeMessageHook;
 import org.apache.rocketmq.broker.mqtrace.SendMessageContext;
 import org.apache.rocketmq.broker.mqtrace.SendMessageHook;
+import org.apache.rocketmq.common.BrokerConfig;
+import org.apache.rocketmq.common.MQVersion;
+import org.apache.rocketmq.common.UtilAll;
+import org.apache.rocketmq.common.message.MessageExt;
+import org.apache.rocketmq.common.message.MessageType;
+import org.apache.rocketmq.common.protocol.header.ConsumerSendMsgBackRequestHeader;
+import org.apache.rocketmq.common.subscription.SubscriptionGroupConfig;
 import org.apache.rocketmq.common.topic.TopicValidator;
 import org.apache.rocketmq.common.MixAll;
 import org.apache.rocketmq.common.TopicConfig;
@@ -46,46 +56,315 @@ import org.apache.rocketmq.common.protocol.header.SendMessageRequestHeaderV2;
 import org.apache.rocketmq.common.protocol.header.SendMessageResponseHeader;
 import org.apache.rocketmq.common.sysflag.MessageSysFlag;
 import org.apache.rocketmq.common.sysflag.TopicSysFlag;
-import org.apache.rocketmq.common.utils.ChannelUtil;
 import org.apache.rocketmq.logging.InternalLogger;
 import org.apache.rocketmq.logging.InternalLoggerFactory;
 import org.apache.rocketmq.remoting.common.RemotingHelper;
 import org.apache.rocketmq.remoting.exception.RemotingCommandException;
-import org.apache.rocketmq.remoting.netty.AsyncNettyRequestProcessor;
 import org.apache.rocketmq.remoting.netty.NettyRequestProcessor;
 import org.apache.rocketmq.remoting.protocol.RemotingCommand;
-import org.apache.rocketmq.store.MessageExtBrokerInner;
+import org.apache.rocketmq.common.message.MessageExtBrokerInner;
+import org.apache.rocketmq.store.PutMessageResult;
+import org.apache.rocketmq.store.stats.BrokerStatsManager;
 
-public abstract class AbstractSendMessageProcessor extends AsyncNettyRequestProcessor implements NettyRequestProcessor {
-    protected static final InternalLogger log = InternalLoggerFactory.getLogger(LoggerName.BROKER_LOGGER_NAME);
+public abstract class AbstractSendMessageProcessor implements NettyRequestProcessor {
+    protected static final InternalLogger LOGGER = InternalLoggerFactory.getLogger(LoggerName.BROKER_LOGGER_NAME);
+    protected static final InternalLogger DLQ_LOG = InternalLoggerFactory.getLogger(LoggerName.DLQ_LOGGER_NAME);
+
+    protected List<ConsumeMessageHook> consumeMessageHookList;
 
     protected final static int DLQ_NUMS_PER_GROUP = 1;
-    protected final BrokerController brokerController;
+    protected final Random random = new Random(System.currentTimeMillis());
     protected final SocketAddress storeHost;
     private List<SendMessageHook> sendMessageHookList;
+    protected final BrokerController brokerController;
+
 
     public AbstractSendMessageProcessor(final BrokerController brokerController) {
         this.brokerController = brokerController;
-        this.storeHost =
-            new InetSocketAddress(brokerController.getBrokerConfig().getBrokerIP1(), brokerController
-                .getNettyServerConfig().getListenPort());
+        this.storeHost = brokerController.getStoreHost();
+    }
+
+    public void registerConsumeMessageHook(List<ConsumeMessageHook> consumeMessageHookList) {
+        this.consumeMessageHookList = consumeMessageHookList;
+    }
+
+    protected RemotingCommand consumerSendMsgBack(final ChannelHandlerContext ctx, final RemotingCommand request)
+        throws RemotingCommandException {
+        final RemotingCommand response = RemotingCommand.createResponseCommand(null);
+        final ConsumerSendMsgBackRequestHeader requestHeader =
+            (ConsumerSendMsgBackRequestHeader) request.decodeCommandCustomHeader(ConsumerSendMsgBackRequestHeader.class);
+
+        // The send back requests sent to SlaveBroker will be forwarded to the master broker beside
+        final BrokerController masterBroker = this.brokerController.peekMasterBroker();
+        if (null == masterBroker) {
+            response.setCode(ResponseCode.SYSTEM_ERROR);
+            response.setRemark("no master available along with " + brokerController.getBrokerConfig().getBrokerIP1());
+            return response;
+        }
+
+        // The broker that received the request.
+        // It may be a master broker or a slave broker
+        final BrokerController currentBroker = this.brokerController;
+
+        SubscriptionGroupConfig subscriptionGroupConfig =
+            masterBroker.getSubscriptionGroupManager().findSubscriptionGroupConfig(requestHeader.getGroup());
+        if (null == subscriptionGroupConfig) {
+            response.setCode(ResponseCode.SUBSCRIPTION_GROUP_NOT_EXIST);
+            response.setRemark("subscription group not exist, " + requestHeader.getGroup() + " "
+                + FAQUrl.suggestTodo(FAQUrl.SUBSCRIPTION_GROUP_NOT_EXIST));
+            return response;
+        }
+
+        BrokerConfig masterBrokerConfig = masterBroker.getBrokerConfig();
+        if (!PermName.isWriteable(masterBrokerConfig.getBrokerPermission())) {
+            response.setCode(ResponseCode.NO_PERMISSION);
+            response.setRemark("the broker[" + masterBrokerConfig.getBrokerIP1() + "] sending message is forbidden");
+            return response;
+        }
+
+        if (subscriptionGroupConfig.getRetryQueueNums() <= 0) {
+            response.setCode(ResponseCode.SUCCESS);
+            response.setRemark(null);
+            return response;
+        }
+
+        String newTopic = MixAll.getRetryTopic(requestHeader.getGroup());
+        int queueIdInt = Math.abs(this.random.nextInt() % 99999999) % subscriptionGroupConfig.getRetryQueueNums();
+
+        int topicSysFlag = 0;
+        if (requestHeader.isUnitMode()) {
+            topicSysFlag = TopicSysFlag.buildSysFlag(false, true);
+        }
+
+        // Create retry topic to master broker
+        TopicConfig topicConfig = masterBroker.getTopicConfigManager().createTopicInSendMessageBackMethod(
+            newTopic,
+            subscriptionGroupConfig.getRetryQueueNums(),
+            PermName.PERM_WRITE | PermName.PERM_READ, topicSysFlag);
+        if (null == topicConfig) {
+            response.setCode(ResponseCode.SYSTEM_ERROR);
+            response.setRemark("topic[" + newTopic + "] not exist");
+            return response;
+        }
+
+        if (!PermName.isWriteable(topicConfig.getPerm())) {
+            response.setCode(ResponseCode.NO_PERMISSION);
+            response.setRemark(String.format("the topic[%s] sending message is forbidden", newTopic));
+            return response;
+        }
+
+        // Look message from the origin message store
+        MessageExt msgExt = currentBroker.getMessageStore().lookMessageByOffset(requestHeader.getOffset());
+        if (null == msgExt) {
+            response.setCode(ResponseCode.SYSTEM_ERROR);
+            response.setRemark("look message by offset failed, " + requestHeader.getOffset());
+            return response;
+        }
+
+        //for logic queue
+        if (requestHeader.getOriginTopic() != null
+            && !msgExt.getTopic().equals(requestHeader.getOriginTopic())) {
+            //here just do some fence in case of some unexpected offset is income
+            response.setCode(ResponseCode.SYSTEM_ERROR);
+            response.setRemark("look message by offset failed to check the topic name" + requestHeader.getOffset());
+            return response;
+        }
+
+        final String retryTopic = msgExt.getProperty(MessageConst.PROPERTY_RETRY_TOPIC);
+        if (null == retryTopic) {
+            MessageAccessor.putProperty(msgExt, MessageConst.PROPERTY_RETRY_TOPIC, msgExt.getTopic());
+        }
+        msgExt.setWaitStoreMsgOK(false);
+
+        int delayLevel = requestHeader.getDelayLevel();
+
+        int maxReconsumeTimes = subscriptionGroupConfig.getRetryMaxTimes();
+        if (request.getVersion() >= MQVersion.Version.V3_4_9.ordinal()) {
+            Integer times = requestHeader.getMaxReconsumeTimes();
+            if (times != null) {
+                maxReconsumeTimes = times;
+            }
+        }
+
+        boolean isDLQ = false;
+        if (msgExt.getReconsumeTimes() >= maxReconsumeTimes
+            || delayLevel < 0) {
+
+            isDLQ = true;
+            newTopic = MixAll.getDLQTopic(requestHeader.getGroup());
+            queueIdInt = randomQueueId(DLQ_NUMS_PER_GROUP);
+
+            // Create DLQ topic to master broker
+            topicConfig = masterBroker.getTopicConfigManager().createTopicInSendMessageBackMethod(newTopic,
+                DLQ_NUMS_PER_GROUP,
+                PermName.PERM_WRITE | PermName.PERM_READ, 0);
+
+            if (null == topicConfig) {
+                response.setCode(ResponseCode.SYSTEM_ERROR);
+                response.setRemark("topic[" + newTopic + "] not exist");
+                return response;
+            }
+            msgExt.setDelayTimeLevel(0);
+        } else {
+            if (0 == delayLevel) {
+                delayLevel = 3 + msgExt.getReconsumeTimes();
+            }
+
+            msgExt.setDelayTimeLevel(delayLevel);
+        }
+
+        MessageExtBrokerInner msgInner = new MessageExtBrokerInner();
+        msgInner.setTopic(newTopic);
+        msgInner.setBody(msgExt.getBody());
+        msgInner.setFlag(msgExt.getFlag());
+        MessageAccessor.setProperties(msgInner, msgExt.getProperties());
+        msgInner.setPropertiesString(MessageDecoder.messageProperties2String(msgExt.getProperties()));
+        msgInner.setTagsCode(MessageExtBrokerInner.tagsString2tagsCode(null, msgExt.getTags()));
+
+        msgInner.setQueueId(queueIdInt);
+        msgInner.setSysFlag(msgExt.getSysFlag());
+        msgInner.setBornTimestamp(msgExt.getBornTimestamp());
+        msgInner.setBornHost(msgExt.getBornHost());
+        msgInner.setStoreHost(this.getStoreHost());
+        msgInner.setReconsumeTimes(msgExt.getReconsumeTimes() + 1);
+
+        String originMsgId = MessageAccessor.getOriginMessageId(msgExt);
+        MessageAccessor.setOriginMessageId(msgInner, UtilAll.isBlank(originMsgId) ? msgExt.getMsgId() : originMsgId);
+        msgInner.setPropertiesString(MessageDecoder.messageProperties2String(msgExt.getProperties()));
+
+        boolean succeeded = false;
+
+        // Put retry topic to master message store
+        PutMessageResult putMessageResult = masterBroker.getMessageStore().putMessage(msgInner);
+        if (putMessageResult != null) {
+            String commercialOwner = request.getExtFields().get(BrokerStatsManager.COMMERCIAL_OWNER);
+
+            switch (putMessageResult.getPutMessageStatus()) {
+                case PUT_OK:
+                    String backTopic = msgExt.getTopic();
+                    String correctTopic = msgExt.getProperty(MessageConst.PROPERTY_RETRY_TOPIC);
+                    if (correctTopic != null) {
+                        backTopic = correctTopic;
+                    }
+                    if (TopicValidator.RMQ_SYS_SCHEDULE_TOPIC.equals(msgInner.getTopic())) {
+                        masterBroker.getBrokerStatsManager().incTopicPutNums(msgInner.getTopic());
+                        masterBroker.getBrokerStatsManager().incTopicPutSize(msgInner.getTopic(), putMessageResult.getAppendMessageResult().getWroteBytes());
+                        masterBroker.getBrokerStatsManager().incQueuePutNums(msgInner.getTopic(), msgInner.getQueueId());
+                        masterBroker.getBrokerStatsManager().incQueuePutSize(msgInner.getTopic(), msgInner.getQueueId(), putMessageResult.getAppendMessageResult().getWroteBytes());
+                    }
+                    masterBroker.getBrokerStatsManager().incSendBackNums(requestHeader.getGroup(), backTopic);
+
+                    if (isDLQ) {
+                        masterBroker.getBrokerStatsManager().incDLQStatValue(
+                            BrokerStatsManager.SNDBCK2DLQ_TIMES,
+                            commercialOwner,
+                            requestHeader.getGroup(),
+                            requestHeader.getOriginTopic(),
+                            BrokerStatsManager.StatsType.SEND_BACK_TO_DLQ.name(),
+                            1);
+
+                        String uniqKey = msgInner.getProperties().get(MessageConst.PROPERTY_UNIQ_CLIENT_MESSAGE_ID_KEYIDX);
+                        DLQ_LOG.info("send msg to DLQ {}, owner={}, originalTopic={}, consumerId={}, msgUniqKey={}, storeTimestamp={}",
+                            newTopic,
+                            commercialOwner,
+                            requestHeader.getOriginTopic(),
+                            requestHeader.getGroup(),
+                            uniqKey,
+                            putMessageResult.getAppendMessageResult().getStoreTimestamp());
+                    }
+
+                    response.setCode(ResponseCode.SUCCESS);
+                    response.setRemark(null);
+
+                    succeeded = true;
+                    break;
+                default:
+                    break;
+            }
+
+            if (!succeeded) {
+                response.setCode(ResponseCode.SYSTEM_ERROR);
+                response.setRemark(putMessageResult.getPutMessageStatus().name());
+            }
+        } else {
+            if (isDLQ) {
+                String owner = request.getExtFields().get(BrokerStatsManager.COMMERCIAL_OWNER);
+                String uniqKey = msgInner.getProperties().get(MessageConst.PROPERTY_UNIQ_CLIENT_MESSAGE_ID_KEYIDX);
+                DLQ_LOG.info("failed to send msg to DLQ {}, owner={}, originalTopic={}, consumerId={}, msgUniqKey={}, result={}",
+                    newTopic,
+                    owner,
+                    requestHeader.getOriginTopic(),
+                    requestHeader.getGroup(),
+                    uniqKey,
+                    putMessageResult == null ? "null" : putMessageResult.getPutMessageStatus().toString());
+            }
+
+            response.setCode(ResponseCode.SYSTEM_ERROR);
+            response.setRemark("putMessageResult is null");
+        }
+
+        if (this.hasConsumeMessageHook() && !UtilAll.isBlank(requestHeader.getOriginMsgId())) {
+            String namespace = NamespaceUtil.getNamespaceFromResource(requestHeader.getGroup());
+            ConsumeMessageContext context = new ConsumeMessageContext();
+            context.setNamespace(namespace);
+            context.setTopic(requestHeader.getOriginTopic());
+            context.setConsumerGroup(requestHeader.getGroup());
+            context.setCommercialRcvStats(BrokerStatsManager.StatsType.SEND_BACK);
+            context.setCommercialRcvTimes(1);
+            context.setCommercialOwner(request.getExtFields().get(BrokerStatsManager.COMMERCIAL_OWNER));
+
+            context.setAccountAuthType(request.getExtFields().get(BrokerStatsManager.ACCOUNT_AUTH_TYPE));
+            context.setAccountOwnerParent(request.getExtFields().get(BrokerStatsManager.ACCOUNT_OWNER_PARENT));
+            context.setAccountOwnerSelf(request.getExtFields().get(BrokerStatsManager.ACCOUNT_OWNER_SELF));
+            context.setRcvStat(isDLQ ? BrokerStatsManager.StatsType.SEND_BACK_TO_DLQ : BrokerStatsManager.StatsType.SEND_BACK);
+            context.setSuccess(succeeded);
+            context.setRcvMsgNum(1);
+            //Set msg body size 0 when sent back by consumer.
+            context.setRcvMsgSize(0);
+            context.setCommercialRcvMsgNum(succeeded ? 1 : 0);
+
+            try {
+                this.executeConsumeMessageHookAfter(context);
+            } catch (AbortProcessException e) {
+                response.setCode(e.getResponseCode());
+                response.setRemark(e.getErrorMessage());
+            }
+        }
+
+        return response;
+    }
+
+    public boolean hasConsumeMessageHook() {
+        return consumeMessageHookList != null && !this.consumeMessageHookList.isEmpty();
+    }
+
+    public void executeConsumeMessageHookAfter(final ConsumeMessageContext context) {
+        if (hasConsumeMessageHook()) {
+            for (ConsumeMessageHook hook : this.consumeMessageHookList) {
+                try {
+                    hook.consumeMessageAfter(context);
+                } catch (Throwable e) {
+                    // Ignore
+                }
+            }
+        }
     }
 
     protected SendMessageContext buildMsgContext(ChannelHandlerContext ctx,
         SendMessageRequestHeader requestHeader) {
-        if (!this.hasSendMessageHook()) {
-            return null;
-        }
         String namespace = NamespaceUtil.getNamespaceFromResource(requestHeader.getTopic());
-        SendMessageContext mqtraceContext = new SendMessageContext();
-        mqtraceContext.setProducerGroup(requestHeader.getProducerGroup());
-        mqtraceContext.setNamespace(namespace);
-        mqtraceContext.setTopic(requestHeader.getTopic());
-        mqtraceContext.setMsgProps(requestHeader.getProperties());
-        mqtraceContext.setBornHost(RemotingHelper.parseChannelRemoteAddr(ctx.channel()));
-        mqtraceContext.setBrokerAddr(this.brokerController.getBrokerAddr());
-        mqtraceContext.setBrokerRegionId(this.brokerController.getBrokerConfig().getRegionId());
-        mqtraceContext.setBornTimeStamp(requestHeader.getBornTimestamp());
+
+        SendMessageContext traceContext;
+        traceContext = new SendMessageContext();
+        traceContext.setNamespace(namespace);
+        traceContext.setProducerGroup(requestHeader.getProducerGroup());
+        traceContext.setTopic(requestHeader.getTopic());
+        traceContext.setMsgProps(requestHeader.getProperties());
+        traceContext.setBornHost(RemotingHelper.parseChannelRemoteAddr(ctx.channel()));
+        traceContext.setBrokerAddr(this.brokerController.getBrokerAddr());
+        traceContext.setBrokerRegionId(this.brokerController.getBrokerConfig().getRegionId());
+        traceContext.setBornTimeStamp(requestHeader.getBornTimestamp());
+        traceContext.setRequestTimeStamp(System.currentTimeMillis());
 
         Map<String, String> properties = MessageDecoder.string2messageProperties(requestHeader.getProperties());
         String uniqueKey = properties.get(MessageConst.PROPERTY_UNIQ_CLIENT_MESSAGE_ID_KEYIDX);
@@ -96,8 +375,14 @@ public abstract class AbstractSendMessageProcessor extends AsyncNettyRequestProc
         if (uniqueKey == null) {
             uniqueKey = "";
         }
-        mqtraceContext.setMsgUniqueKey(uniqueKey);
-        return mqtraceContext;
+        traceContext.setMsgUniqueKey(uniqueKey);
+
+        if (properties.containsKey(MessageConst.PROPERTY_SHARDING_KEY)) {
+            traceContext.setMsgType(MessageType.Order_Msg);
+        } else {
+            traceContext.setMsgType(MessageType.Normal_Msg);
+        }
+        return traceContext;
     }
 
     public boolean hasSendMessageHook() {
@@ -108,7 +393,7 @@ public abstract class AbstractSendMessageProcessor extends AsyncNettyRequestProc
         final SendMessageRequestHeader requestHeader, final byte[] body, TopicConfig topicConfig) {
         int queueIdInt = requestHeader.getQueueId();
         if (queueIdInt < 0) {
-            queueIdInt = ThreadLocalRandom.current().nextInt(99999999) % topicConfig.getWriteQueueNums();
+            queueIdInt = randomQueueId(topicConfig.getWriteQueueNums());
         }
         int sysFlag = requestHeader.getSysFlag();
 
@@ -143,19 +428,24 @@ public abstract class AbstractSendMessageProcessor extends AsyncNettyRequestProc
     protected RemotingCommand msgContentCheck(final ChannelHandlerContext ctx,
         final SendMessageRequestHeader requestHeader, RemotingCommand request,
         final RemotingCommand response) {
-        if (requestHeader.getTopic().length() > Byte.MAX_VALUE) {
-            log.warn("putMessage message topic length too long {}", requestHeader.getTopic().length());
+        String topic = requestHeader.getTopic();
+        if (topic.length() > Byte.MAX_VALUE) {
+            LOGGER.warn("msgContentCheck: message topic length is too long, topic={}, topic length={}, threshold={}",
+                topic, topic.length(), Byte.MAX_VALUE);
             response.setCode(ResponseCode.MESSAGE_ILLEGAL);
             return response;
         }
         if (requestHeader.getProperties() != null && requestHeader.getProperties().length() > Short.MAX_VALUE) {
-            log.warn("putMessage message properties length too long {}", requestHeader.getProperties().length());
+            LOGGER.warn(
+                "msgContentCheck: message properties length is too long, topic={}, properties length={}, threshold={}",
+                topic, requestHeader.getProperties().length(), Short.MAX_VALUE);
             response.setCode(ResponseCode.MESSAGE_ILLEGAL);
             return response;
         }
         if (request.getBody().length > DBMsgConstants.MAX_BODY_SIZE) {
-            log.warn(" topic {}  msg body size {}  from {}", requestHeader.getTopic(),
-                request.getBody().length, ChannelUtil.getRemoteIp(ctx.channel()));
+            LOGGER.warn(
+                "msgContentCheck: message body size exceeds the threshold, topic={}, body size={}, threshold={}bytes",
+                topic, request.getBody().length, DBMsgConstants.MAX_BODY_SIZE);
             response.setRemark("msg body must be less 64KB");
             response.setCode(ResponseCode.MESSAGE_ILLEGAL);
             return response;
@@ -164,7 +454,8 @@ public abstract class AbstractSendMessageProcessor extends AsyncNettyRequestProc
     }
 
     protected RemotingCommand msgCheck(final ChannelHandlerContext ctx,
-        final SendMessageRequestHeader requestHeader, final RemotingCommand response) {
+        final SendMessageRequestHeader requestHeader, final RemotingCommand request,
+        final RemotingCommand response) {
         if (!PermName.isWriteable(this.brokerController.getBrokerConfig().getBrokerPermission())
             && this.brokerController.getTopicConfigManager().isOrderTopic(requestHeader.getTopic())) {
             response.setCode(ResponseCode.NO_PERMISSION);
@@ -192,7 +483,7 @@ public abstract class AbstractSendMessageProcessor extends AsyncNettyRequestProc
                 }
             }
 
-            log.warn("the topic {} not exist, producer: {}", requestHeader.getTopic(), ctx.channel().remoteAddress());
+            LOGGER.warn("the topic {} not exist, producer: {}", requestHeader.getTopic(), ctx.channel().remoteAddress());
             topicConfig = this.brokerController.getTopicConfigManager().createTopicInSendMessageMethod(
                 requestHeader.getTopic(),
                 requestHeader.getDefaultTopic(),
@@ -224,7 +515,7 @@ public abstract class AbstractSendMessageProcessor extends AsyncNettyRequestProc
                 topicConfig.toString(),
                 RemotingHelper.parseChannelRemoteAddr(ctx.channel()));
 
-            log.warn(errorInfo);
+            LOGGER.warn(errorInfo);
             response.setCode(ResponseCode.SYSTEM_ERROR);
             response.setRemark(errorInfo);
 
@@ -243,9 +534,10 @@ public abstract class AbstractSendMessageProcessor extends AsyncNettyRequestProc
             try {
                 ctx.writeAndFlush(response);
             } catch (Throwable e) {
-                log.error("SendMessageProcessor process request over, but response failed", e);
-                log.error(request.toString());
-                log.error(response.toString());
+                LOGGER.error(
+                    "SendMessageProcessor finished processing the request, but failed to send response, client "
+                        + "address={}, request={}, response={}", RemotingHelper.parseChannelRemoteAddr(ctx.channel()),
+                    request.toString(), response.toString(), e);
             }
         }
     }
@@ -257,8 +549,8 @@ public abstract class AbstractSendMessageProcessor extends AsyncNettyRequestProc
                 try {
                     final SendMessageRequestHeader requestHeader = parseRequestHeader(request);
 
+                    String namespace = NamespaceUtil.getNamespaceFromResource(requestHeader.getTopic());
                     if (null != requestHeader) {
-                        String namespace = NamespaceUtil.getNamespaceFromResource(requestHeader.getTopic());
                         context.setNamespace(namespace);
                         context.setProducerGroup(requestHeader.getProducerGroup());
                         context.setTopic(requestHeader.getTopic());
@@ -273,8 +565,10 @@ public abstract class AbstractSendMessageProcessor extends AsyncNettyRequestProc
                     if (requestHeader != null) {
                         requestHeader.setProperties(context.getMsgProps());
                     }
+                } catch (AbortProcessException e) {
+                    throw e;
                 } catch (Throwable e) {
-                    // Ignore
+                    //ignore
                 }
             }
         }
@@ -304,7 +598,7 @@ public abstract class AbstractSendMessageProcessor extends AsyncNettyRequestProc
     }
 
     static SendMessageRequestHeaderV2 decodeSendMessageHeaderV2(RemotingCommand request)
-            throws RemotingCommandException {
+        throws RemotingCommandException {
         SendMessageRequestHeaderV2 r = new SendMessageRequestHeaderV2();
         HashMap<String, String> fields = request.getExtFields();
         if (fields == null) {
@@ -376,6 +670,10 @@ public abstract class AbstractSendMessageProcessor extends AsyncNettyRequestProc
         }
     }
 
+    protected int randomQueueId(int writeQueueNums) {
+        return ThreadLocalRandom.current().nextInt(99999999) % writeQueueNums;
+    }
+
     public void executeSendMessageHookAfter(final RemotingCommand response, final SendMessageContext context) {
         if (hasSendMessageHook()) {
             for (SendMessageHook hook : this.sendMessageHookList) {
@@ -391,7 +689,7 @@ public abstract class AbstractSendMessageProcessor extends AsyncNettyRequestProc
                     }
                     hook.sendMessageAfter(context);
                 } catch (Throwable e) {
-                    // Ignore
+                    //ignore
                 }
             }
         }
diff --git a/broker/src/main/java/org/apache/rocketmq/broker/processor/AckMessageProcessor.java b/broker/src/main/java/org/apache/rocketmq/broker/processor/AckMessageProcessor.java
index 29f7507..9c7dbc4 100644
--- a/broker/src/main/java/org/apache/rocketmq/broker/processor/AckMessageProcessor.java
+++ b/broker/src/main/java/org/apache/rocketmq/broker/processor/AckMessageProcessor.java
@@ -38,7 +38,7 @@ import org.apache.rocketmq.remoting.common.RemotingHelper;
 import org.apache.rocketmq.remoting.exception.RemotingCommandException;
 import org.apache.rocketmq.remoting.netty.NettyRequestProcessor;
 import org.apache.rocketmq.remoting.protocol.RemotingCommand;
-import org.apache.rocketmq.store.MessageExtBrokerInner;
+import org.apache.rocketmq.common.message.MessageExtBrokerInner;
 import org.apache.rocketmq.store.PutMessageResult;
 import org.apache.rocketmq.store.PutMessageStatus;
 import org.apache.rocketmq.store.pop.AckMsg;
@@ -51,36 +51,54 @@ public class AckMessageProcessor implements NettyRequestProcessor {
 
     public AckMessageProcessor(final BrokerController brokerController) {
         this.brokerController = brokerController;
-        this.reviveTopic = PopAckConstants.REVIVE_TOPIC + this.brokerController.getBrokerConfig().getBrokerClusterName();
+        this.reviveTopic = PopAckConstants.buildClusterReviveTopic(this.brokerController.getBrokerConfig().getBrokerClusterName());
         this.popReviveServices = new PopReviveService[this.brokerController.getBrokerConfig().getReviveQueueNum()];
         for (int i = 0; i < this.brokerController.getBrokerConfig().getReviveQueueNum(); i++) {
-            this.popReviveServices[i] = new PopReviveService(i, brokerController, reviveTopic);
+            this.popReviveServices[i] = new PopReviveService(brokerController, reviveTopic, i);
+            this.popReviveServices[i].setShouldRunPopRevive(brokerController.getBrokerConfig().getBrokerId() == 0);
         }
     }
 
-    @Override
-    public RemotingCommand processRequest(final ChannelHandlerContext ctx, RemotingCommand request) throws RemotingCommandException {
-        return this.processRequest(ctx.channel(), request, true);
+    public void startPopReviveService() {
+        for (PopReviveService popReviveService : popReviveServices) {
+            popReviveService.start();
+        }
     }
 
-    @Override
-    public boolean rejectRequest() {
-        return false;
+    public void shutdownPopReviveService() {
+        for (PopReviveService popReviveService : popReviveServices) {
+            popReviveService.stop();
+        }
     }
 
-    public void startPopReviveService() {
+    public void setPopReviveServiceStatus(boolean shouldStart) {
         for (PopReviveService popReviveService : popReviveServices) {
-            popReviveService.start();
+            popReviveService.setShouldRunPopRevive(shouldStart);
         }
     }
 
-    public void shutdownPopReviveService() {
+    public boolean isPopReviveServiceRunning() {
         for (PopReviveService popReviveService : popReviveServices) {
-            popReviveService.shutdown();
+            if (popReviveService.isShouldRunPopRevive()) {
+                return true;
+            }
         }
+
+        return false;
+    }
+
+    @Override
+    public RemotingCommand processRequest(final ChannelHandlerContext ctx,
+        RemotingCommand request) throws RemotingCommandException {
+        return this.processRequest(ctx.channel(), request, true);
     }
 
-    private RemotingCommand processRequest(final Channel channel, RemotingCommand request, boolean brokerAllowSuspend) throws RemotingCommandException {
+    @Override public boolean rejectRequest() {
+        return false;
+    }
+
+    private RemotingCommand processRequest(final Channel channel, RemotingCommand request,
+        boolean brokerAllowSuspend) throws RemotingCommandException {
         final AckMessageRequestHeader requestHeader = (AckMessageRequestHeader) request.decodeCommandCustomHeader(AckMessageRequestHeader.class);
         MessageExtBrokerInner msgInner = new MessageExtBrokerInner();
         AckMsg ackMsg = new AckMsg();
@@ -105,7 +123,11 @@ public class AckMessageProcessor implements NettyRequestProcessor {
         long minOffset = this.brokerController.getMessageStore().getMinOffsetInQueue(requestHeader.getTopic(), requestHeader.getQueueId());
         long maxOffset = this.brokerController.getMessageStore().getMaxOffsetInQueue(requestHeader.getTopic(), requestHeader.getQueueId());
         if (requestHeader.getOffset() < minOffset || requestHeader.getOffset() > maxOffset) {
+            String errorInfo = String.format("offset is illegal, key:%s@%d, commit:%d, store:%d~%d",
+                requestHeader.getTopic(), requestHeader.getQueueId(), requestHeader.getOffset(), minOffset, maxOffset);
+            POP_LOGGER.warn(errorInfo);
             response.setCode(ResponseCode.NO_MESSAGE);
+            response.setRemark(errorInfo);
             return response;
         }
         String[] extraInfo = ExtraInfoUtil.split(requestHeader.getExtraInfo());
@@ -116,15 +138,19 @@ public class AckMessageProcessor implements NettyRequestProcessor {
         ackMsg.setTopic(requestHeader.getTopic());
         ackMsg.setQueueId(requestHeader.getQueueId());
         ackMsg.setPopTime(ExtraInfoUtil.getPopTime(extraInfo));
+        ackMsg.setBrokerName(ExtraInfoUtil.getBrokerName(extraInfo));
 
         int rqId = ExtraInfoUtil.getReviveQid(extraInfo);
 
+        this.brokerController.getBrokerStatsManager().incBrokerAckNums(1);
+        this.brokerController.getBrokerStatsManager().incGroupAckNums(requestHeader.getConsumerGroup(), requestHeader.getTopic(), 1);
+
         if (rqId == KeyBuilder.POP_ORDER_REVIVE_QUEUE) {
             // order
             String lockKey = requestHeader.getTopic() + PopAckConstants.SPLIT
-                    + requestHeader.getConsumerGroup() + PopAckConstants.SPLIT + requestHeader.getQueueId();
+                + requestHeader.getConsumerGroup() + PopAckConstants.SPLIT + requestHeader.getQueueId();
             long oldOffset = this.brokerController.getConsumerOffsetManager().queryOffset(requestHeader.getConsumerGroup(),
-                    requestHeader.getTopic(), requestHeader.getQueueId());
+                requestHeader.getTopic(), requestHeader.getQueueId());
             if (requestHeader.getOffset() < oldOffset) {
                 return response;
             }
@@ -137,18 +163,18 @@ public class AckMessageProcessor implements NettyRequestProcessor {
                     return response;
                 }
                 long nextOffset = brokerController.getConsumerOrderInfoManager().commitAndNext(
-                        requestHeader.getTopic(), requestHeader.getConsumerGroup(),
-                        requestHeader.getQueueId(), requestHeader.getOffset());
+                    requestHeader.getTopic(), requestHeader.getConsumerGroup(),
+                    requestHeader.getQueueId(), requestHeader.getOffset());
                 if (nextOffset > -1) {
                     this.brokerController.getConsumerOffsetManager().commitOffset(channel.remoteAddress().toString(),
-                            requestHeader.getConsumerGroup(), requestHeader.getTopic(),
-                            requestHeader.getQueueId(),
-                            nextOffset);
+                        requestHeader.getConsumerGroup(), requestHeader.getTopic(),
+                        requestHeader.getQueueId(),
+                        nextOffset);
                     this.brokerController.getPopMessageProcessor().notifyMessageArriving(requestHeader.getTopic(), requestHeader.getConsumerGroup(),
-                            requestHeader.getQueueId());
+                        requestHeader.getQueueId());
                 } else if (nextOffset == -1) {
                     String errorInfo = String.format("offset is illegal, key:%s, old:%d, commit:%d, next:%d, %s",
-                            lockKey, oldOffset, requestHeader.getOffset(), nextOffset, channel.remoteAddress());
+                        lockKey, oldOffset, requestHeader.getOffset(), nextOffset, channel.remoteAddress());
                     POP_LOGGER.warn(errorInfo);
                     response.setCode(ResponseCode.MESSAGE_ILLEGAL);
                     response.setRemark(errorInfo);
@@ -175,7 +201,7 @@ public class AckMessageProcessor implements NettyRequestProcessor {
         MsgUtil.setMessageDeliverTime(this.brokerController, msgInner, ExtraInfoUtil.getPopTime(extraInfo) + ExtraInfoUtil.getInvisibleTime(extraInfo));
         msgInner.getProperties().put(MessageConst.PROPERTY_UNIQ_CLIENT_MESSAGE_ID_KEYIDX, PopMessageProcessor.genAckUniqueId(ackMsg));
         msgInner.setPropertiesString(MessageDecoder.messageProperties2String(msgInner.getProperties()));
-        PutMessageResult putMessageResult = this.brokerController.getMessageStore().putMessage(msgInner);
+        PutMessageResult putMessageResult = this.brokerController.getEscapeBridge().putMessageToSpecificQueue(msgInner);
         if (putMessageResult.getPutMessageStatus() != PutMessageStatus.PUT_OK
             && putMessageResult.getPutMessageStatus() != PutMessageStatus.FLUSH_DISK_TIMEOUT
             && putMessageResult.getPutMessageStatus() != PutMessageStatus.FLUSH_SLAVE_TIMEOUT
diff --git a/broker/src/main/java/org/apache/rocketmq/broker/processor/AdminBrokerProcessor.java b/broker/src/main/java/org/apache/rocketmq/broker/processor/AdminBrokerProcessor.java
index 4606480..a283228 100644
--- a/broker/src/main/java/org/apache/rocketmq/broker/processor/AdminBrokerProcessor.java
+++ b/broker/src/main/java/org/apache/rocketmq/broker/processor/AdminBrokerProcessor.java
@@ -20,6 +20,10 @@ import com.alibaba.fastjson.JSON;
 import com.alibaba.fastjson.JSONObject;
 import io.netty.channel.Channel;
 import io.netty.channel.ChannelHandlerContext;
+import java.util.concurrent.ConcurrentHashMap;
+import java.util.concurrent.CountDownLatch;
+import java.util.concurrent.TimeUnit;
+import org.apache.commons.lang3.StringUtils;
 import org.apache.rocketmq.acl.AccessValidator;
 import org.apache.rocketmq.acl.plain.PlainAccessValidator;
 import org.apache.rocketmq.broker.BrokerController;
@@ -27,20 +31,25 @@ import org.apache.rocketmq.broker.client.ClientChannelInfo;
 import org.apache.rocketmq.broker.client.ConsumerGroupInfo;
 import org.apache.rocketmq.broker.filter.ConsumerFilterData;
 import org.apache.rocketmq.broker.filter.ExpressionMessageFilter;
+import org.apache.rocketmq.broker.plugin.BrokerAttachedPlugin;
+import org.apache.rocketmq.broker.subscription.SubscriptionGroupManager;
 import org.apache.rocketmq.broker.transaction.queue.TransactionalMessageUtil;
 import org.apache.rocketmq.common.AclConfig;
-import org.apache.rocketmq.common.BrokerConfig;
+import org.apache.rocketmq.common.LockCallback;
 import org.apache.rocketmq.common.MQVersion;
 import org.apache.rocketmq.common.MixAll;
 import org.apache.rocketmq.common.PlainAccessConfig;
 import org.apache.rocketmq.common.TopicConfig;
+import org.apache.rocketmq.common.UnlockCallback;
 import org.apache.rocketmq.common.UtilAll;
 import org.apache.rocketmq.common.admin.ConsumeStats;
 import org.apache.rocketmq.common.admin.OffsetWrapper;
 import org.apache.rocketmq.common.admin.TopicOffset;
 import org.apache.rocketmq.common.admin.TopicStatsTable;
 import org.apache.rocketmq.common.attribute.AttributeParser;
+import org.apache.rocketmq.common.constant.ConsumeInitMode;
 import org.apache.rocketmq.common.constant.LoggerName;
+import org.apache.rocketmq.common.constant.PermName;
 import org.apache.rocketmq.common.message.MessageAccessor;
 import org.apache.rocketmq.common.message.MessageConst;
 import org.apache.rocketmq.common.message.MessageDecoder;
@@ -49,6 +58,7 @@ import org.apache.rocketmq.common.message.MessageId;
 import org.apache.rocketmq.common.message.MessageQueue;
 import org.apache.rocketmq.common.protocol.RequestCode;
 import org.apache.rocketmq.common.protocol.ResponseCode;
+import org.apache.rocketmq.common.protocol.body.BrokerMemberGroup;
 import org.apache.rocketmq.common.protocol.body.BrokerStatsData;
 import org.apache.rocketmq.common.protocol.body.BrokerStatsItem;
 import org.apache.rocketmq.common.protocol.body.Connection;
@@ -56,6 +66,7 @@ import org.apache.rocketmq.common.protocol.body.ConsumeQueueData;
 import org.apache.rocketmq.common.protocol.body.ConsumeStatsList;
 import org.apache.rocketmq.common.protocol.body.ConsumerConnection;
 import org.apache.rocketmq.common.protocol.body.GroupList;
+import org.apache.rocketmq.common.protocol.body.HARuntimeInfo;
 import org.apache.rocketmq.common.protocol.body.KVTable;
 import org.apache.rocketmq.common.protocol.body.LockBatchRequestBody;
 import org.apache.rocketmq.common.protocol.body.LockBatchResponseBody;
@@ -63,6 +74,7 @@ import org.apache.rocketmq.common.protocol.body.ProducerConnection;
 import org.apache.rocketmq.common.protocol.body.QueryConsumeQueueResponseBody;
 import org.apache.rocketmq.common.protocol.body.QueryConsumeTimeSpanBody;
 import org.apache.rocketmq.common.protocol.body.QueryCorrectionOffsetBody;
+import org.apache.rocketmq.common.protocol.body.QuerySubscriptionResponseBody;
 import org.apache.rocketmq.common.protocol.body.QueueTimeSpan;
 import org.apache.rocketmq.common.protocol.body.TopicConfigAndMappingSerializeWrapper;
 import org.apache.rocketmq.common.protocol.body.TopicList;
@@ -74,6 +86,8 @@ import org.apache.rocketmq.common.protocol.header.CreateTopicRequestHeader;
 import org.apache.rocketmq.common.protocol.header.DeleteAccessConfigRequestHeader;
 import org.apache.rocketmq.common.protocol.header.DeleteSubscriptionGroupRequestHeader;
 import org.apache.rocketmq.common.protocol.header.DeleteTopicRequestHeader;
+import org.apache.rocketmq.common.protocol.header.ExchangeHAInfoRequestHeader;
+import org.apache.rocketmq.common.protocol.header.ExchangeHAInfoResponseHeader;
 import org.apache.rocketmq.common.protocol.header.GetAllTopicConfigResponseHeader;
 import org.apache.rocketmq.common.protocol.header.GetBrokerAclConfigResponseHeader;
 import org.apache.rocketmq.common.protocol.header.GetBrokerClusterAclConfigResponseBody;
@@ -91,22 +105,27 @@ import org.apache.rocketmq.common.protocol.header.GetMaxOffsetResponseHeader;
 import org.apache.rocketmq.common.protocol.header.GetMinOffsetRequestHeader;
 import org.apache.rocketmq.common.protocol.header.GetMinOffsetResponseHeader;
 import org.apache.rocketmq.common.protocol.header.GetProducerConnectionListRequestHeader;
+import org.apache.rocketmq.common.protocol.header.GetSubscriptionGroupConfigRequestHeader;
 import org.apache.rocketmq.common.protocol.header.GetTopicConfigRequestHeader;
 import org.apache.rocketmq.common.protocol.header.GetTopicStatsInfoRequestHeader;
+import org.apache.rocketmq.common.protocol.header.NotifyMinBrokerIdChangeRequestHeader;
 import org.apache.rocketmq.common.protocol.header.QueryConsumeQueueRequestHeader;
 import org.apache.rocketmq.common.protocol.header.QueryConsumeTimeSpanRequestHeader;
 import org.apache.rocketmq.common.protocol.header.QueryCorrectionOffsetHeader;
+import org.apache.rocketmq.common.protocol.header.QuerySubscriptionByConsumerRequestHeader;
 import org.apache.rocketmq.common.protocol.header.QueryTopicConsumeByWhoRequestHeader;
+import org.apache.rocketmq.common.protocol.header.QueryTopicsByConsumerRequestHeader;
+import org.apache.rocketmq.common.protocol.header.ResetMasterFlushOffsetHeader;
 import org.apache.rocketmq.common.protocol.header.ResetOffsetRequestHeader;
 import org.apache.rocketmq.common.protocol.header.ResumeCheckHalfMessageRequestHeader;
 import org.apache.rocketmq.common.protocol.header.SearchOffsetRequestHeader;
 import org.apache.rocketmq.common.protocol.header.SearchOffsetResponseHeader;
 import org.apache.rocketmq.common.protocol.header.UpdateGlobalWhiteAddrsConfigRequestHeader;
+import org.apache.rocketmq.common.protocol.header.UpdateGroupForbiddenRequestHeader;
 import org.apache.rocketmq.common.protocol.header.ViewBrokerStatsDataRequestHeader;
 import org.apache.rocketmq.common.protocol.header.filtersrv.RegisterFilterServerRequestHeader;
 import org.apache.rocketmq.common.protocol.header.filtersrv.RegisterFilterServerResponseHeader;
 import org.apache.rocketmq.common.protocol.heartbeat.SubscriptionData;
-import org.apache.rocketmq.common.rpc.RpcClient;
 import org.apache.rocketmq.common.rpc.RpcClientUtils;
 import org.apache.rocketmq.common.rpc.RpcException;
 import org.apache.rocketmq.common.rpc.RpcRequest;
@@ -118,6 +137,7 @@ import org.apache.rocketmq.common.statictopic.TopicQueueMappingDetail;
 import org.apache.rocketmq.common.statictopic.TopicQueueMappingUtils;
 import org.apache.rocketmq.common.stats.StatsItem;
 import org.apache.rocketmq.common.stats.StatsSnapshot;
+import org.apache.rocketmq.common.subscription.GroupForbidden;
 import org.apache.rocketmq.common.subscription.SubscriptionGroupConfig;
 import org.apache.rocketmq.common.topic.TopicValidator;
 import org.apache.rocketmq.filter.util.BitsArray;
@@ -126,15 +146,13 @@ import org.apache.rocketmq.logging.InternalLoggerFactory;
 import org.apache.rocketmq.remoting.common.RemotingHelper;
 import org.apache.rocketmq.remoting.exception.RemotingCommandException;
 import org.apache.rocketmq.remoting.exception.RemotingTimeoutException;
-import org.apache.rocketmq.remoting.netty.AsyncNettyRequestProcessor;
 import org.apache.rocketmq.remoting.netty.NettyRequestProcessor;
 import org.apache.rocketmq.remoting.protocol.LanguageCode;
 import org.apache.rocketmq.remoting.protocol.RemotingCommand;
 import org.apache.rocketmq.remoting.protocol.RemotingSerializable;
 import org.apache.rocketmq.remoting.protocol.RemotingSysResponseCode;
 import org.apache.rocketmq.store.ConsumeQueueExt;
-import org.apache.rocketmq.store.DefaultMessageStore;
-import org.apache.rocketmq.store.MessageExtBrokerInner;
+import org.apache.rocketmq.common.message.MessageExtBrokerInner;
 import org.apache.rocketmq.store.MessageFilter;
 import org.apache.rocketmq.store.MessageStore;
 import org.apache.rocketmq.store.PutMessageResult;
@@ -159,16 +177,12 @@ import java.util.concurrent.ConcurrentMap;
 
 import static org.apache.rocketmq.remoting.protocol.RemotingCommand.buildErrorResponse;
 
-public class AdminBrokerProcessor extends AsyncNettyRequestProcessor implements NettyRequestProcessor {
-    private static final InternalLogger log = InternalLoggerFactory.getLogger(LoggerName.BROKER_LOGGER_NAME);
-    private final BrokerController brokerController;
-    private final RpcClient rpcClient;
-    private final BrokerConfig brokerConfig;
+public class AdminBrokerProcessor implements NettyRequestProcessor {
+    private static final InternalLogger LOGGER = InternalLoggerFactory.getLogger(LoggerName.BROKER_LOGGER_NAME);
+    protected final BrokerController brokerController;
 
     public AdminBrokerProcessor(final BrokerController brokerController) {
         this.brokerController = brokerController;
-        this.brokerConfig = brokerController.getBrokerConfig();
-        this.rpcClient = brokerController.getBrokerOuterAPI().getRpcClient();
     }
 
     @Override
@@ -223,6 +237,10 @@ public class AdminBrokerProcessor extends AsyncNettyRequestProcessor implements
                 return this.getConsumerStatus(ctx, request);
             case RequestCode.QUERY_TOPIC_CONSUME_BY_WHO:
                 return this.queryTopicConsumeByWho(ctx, request);
+            case RequestCode.QUERY_TOPICS_BY_CONSUMER:
+                return this.queryTopicsByConsumer(ctx, request);
+            case RequestCode.QUERY_SUBSCRIPTION_BY_CONSUMER:
+                return this.querySubscriptionByConsumer(ctx, request);
             case RequestCode.REGISTER_FILTER_SERVER:
                 return this.registerFilterServer(ctx, request);
             case RequestCode.QUERY_CONSUME_TIME_SPAN:
@@ -247,6 +265,10 @@ public class AdminBrokerProcessor extends AsyncNettyRequestProcessor implements
                 return fetchAllConsumeStatsInBroker(ctx, request);
             case RequestCode.QUERY_CONSUME_QUEUE:
                 return queryConsumeQueue(ctx, request);
+            case RequestCode.UPDATE_AND_GET_GROUP_FORBIDDEN:
+                return this.updateAndGetGroupForbidden(ctx, request);
+            case RequestCode.GET_SUBSCRIPTIONGROUP_CONFIG:
+                return this.getSubscriptionGroup(ctx, request);
             case RequestCode.UPDATE_AND_CREATE_ACL_CONFIG:
                 return updateAndCreateAccessConfig(ctx, request);
             case RequestCode.DELETE_ACL_CONFIG:
@@ -263,11 +285,83 @@ public class AdminBrokerProcessor extends AsyncNettyRequestProcessor implements
                 return getTopicConfig(ctx, request);
             case RequestCode.UPDATE_AND_CREATE_STATIC_TOPIC:
                 return this.updateAndCreateStaticTopic(ctx, request);
+            case RequestCode.NOTIFY_MIN_BROKER_ID_CHANGE:
+                return this.notifyMinBrokerIdChange(ctx, request);
+            case RequestCode.EXCHANGE_BROKER_HA_INFO:
+                return this.updateBrokerHaInfo(ctx, request);
+            case RequestCode.GET_BROKER_HA_STATUS:
+                return this.getBrokerHaStatus(ctx, request);
+            case RequestCode.RESET_MASTER_FLUSH_OFFSET:
+                return this.resetMasterFlushOffset(ctx, request);
             default:
                 return getUnknownCmdResponse(ctx, request);
         }
     }
 
+    /**
+     * @param ctx
+     * @param request
+     * @return
+     * @throws RemotingCommandException
+     */
+    private RemotingCommand getSubscriptionGroup(ChannelHandlerContext ctx,
+        RemotingCommand request) throws RemotingCommandException {
+        GetSubscriptionGroupConfigRequestHeader requestHeader = (GetSubscriptionGroupConfigRequestHeader) request.decodeCommandCustomHeader(GetSubscriptionGroupConfigRequestHeader.class);
+        final RemotingCommand response = RemotingCommand.createResponseCommand(null);
+
+        SubscriptionGroupConfig groupConfig = this.brokerController.getSubscriptionGroupManager().getSubscriptionGroupTable().get(requestHeader.getGroup());
+        if (groupConfig == null) {
+            LOGGER.error("No group in this broker, client: {} group: {}", ctx.channel().remoteAddress(), requestHeader.getGroup());
+            response.setCode(ResponseCode.SYSTEM_ERROR);
+            response.setRemark("No group in this broker");
+            return response;
+        }
+        String content = JSONObject.toJSONString(groupConfig);
+        try {
+            response.setBody(content.getBytes(MixAll.DEFAULT_CHARSET));
+        } catch (UnsupportedEncodingException e) {
+            LOGGER.error("UnsupportedEncodingException getSubscriptionGroup: group=" + groupConfig.getGroupName(), e);
+
+            response.setCode(ResponseCode.SYSTEM_ERROR);
+            response.setRemark("UnsupportedEncodingException " + e.getMessage());
+            return response;
+        }
+        response.setCode(ResponseCode.SUCCESS);
+        response.setRemark(null);
+
+        return response;
+    }
+
+    /**
+     * @param ctx
+     * @param request
+     * @return
+     */
+    private RemotingCommand updateAndGetGroupForbidden(ChannelHandlerContext ctx, RemotingCommand request)
+        throws RemotingCommandException {
+        final RemotingCommand response = RemotingCommand.createResponseCommand(null);
+        UpdateGroupForbiddenRequestHeader requestHeader = (UpdateGroupForbiddenRequestHeader) //
+            request.decodeCommandCustomHeader(UpdateGroupForbiddenRequestHeader.class);
+        String group = requestHeader.getGroup();
+        String topic = requestHeader.getTopic();
+        LOGGER.info("updateAndGetGroupForbidden called by {} for object {}@{} readable={}",//
+            RemotingHelper.parseChannelRemoteAddr(ctx.channel()), group, //
+            topic, requestHeader.getReadable());
+        SubscriptionGroupManager groupManager = this.brokerController.getSubscriptionGroupManager();
+        if (requestHeader.getReadable() != null) {
+            groupManager.updateForbidden(group, topic, PermName.INDEX_PERM_READ, !requestHeader.getReadable());
+        }
+
+        response.setCode(ResponseCode.SUCCESS);
+        response.setRemark("");
+        GroupForbidden groupForbidden = new GroupForbidden();
+        groupForbidden.setGroup(group);
+        groupForbidden.setTopic(topic);
+        groupForbidden.setReadable(!groupManager.getForbidden(group, topic, PermName.INDEX_PERM_READ));
+        response.setBody(groupForbidden.toJson().getBytes());
+        return response;
+    }
+
     @Override
     public boolean rejectRequest() {
         return false;
@@ -278,7 +372,9 @@ public class AdminBrokerProcessor extends AsyncNettyRequestProcessor implements
         final RemotingCommand response = RemotingCommand.createResponseCommand(null);
         final CreateTopicRequestHeader requestHeader =
             (CreateTopicRequestHeader) request.decodeCommandCustomHeader(CreateTopicRequestHeader.class);
-        log.info("updateAndCreateTopic called by {}", RemotingHelper.parseChannelRemoteAddr(ctx.channel()));
+
+        LOGGER.info("Broker receive request to update or create topic={}, caller address={}",
+            requestHeader.getTopic(), RemotingHelper.parseChannelRemoteAddr(ctx.channel()));
 
         String topic = requestHeader.getTopic();
 
@@ -302,8 +398,8 @@ public class AdminBrokerProcessor extends AsyncNettyRequestProcessor implements
             this.brokerController.getTopicConfigManager().updateTopicConfig(topicConfig);
             this.brokerController.registerIncrementBrokerData(topicConfig, this.brokerController.getTopicConfigManager().getDataVersion());
             response.setCode(ResponseCode.SUCCESS);
-        }  catch (Exception e) {
-            log.error("Update / create topic failed for [{}]", request, e);
+        } catch (Exception e) {
+            LOGGER.error("Update / create topic failed for [{}]", request, e);
             response.setCode(ResponseCode.SYSTEM_ERROR);
             response.setRemark(e.getMessage());
         }
@@ -311,11 +407,11 @@ public class AdminBrokerProcessor extends AsyncNettyRequestProcessor implements
     }
 
     private synchronized RemotingCommand updateAndCreateStaticTopic(ChannelHandlerContext ctx,
-                                                              RemotingCommand request) throws RemotingCommandException {
+        RemotingCommand request) throws RemotingCommandException {
         final RemotingCommand response = RemotingCommand.createResponseCommand(null);
         final CreateTopicRequestHeader requestHeader =
-                (CreateTopicRequestHeader) request.decodeCommandCustomHeader(CreateTopicRequestHeader.class);
-        log.info("updateAndCreateTopic called by {}", RemotingHelper.parseChannelRemoteAddr(ctx.channel()));
+            (CreateTopicRequestHeader) request.decodeCommandCustomHeader(CreateTopicRequestHeader.class);
+        LOGGER.info("Broker receive request to update or create static topic={}, caller address={}", requestHeader.getTopic(), RemotingHelper.parseChannelRemoteAddr(ctx.channel()));
 
         final TopicQueueMappingDetail topicQueueMappingDetail = RemotingSerializable.decode(request.getBody(), TopicQueueMappingDetail.class);
 
@@ -347,21 +443,21 @@ public class AdminBrokerProcessor extends AsyncNettyRequestProcessor implements
             this.brokerController.registerIncrementBrokerData(topicConfig, this.brokerController.getTopicConfigManager().getDataVersion());
             response.setCode(ResponseCode.SUCCESS);
         } catch (Exception e) {
-            log.error("Update static topic failed for [{}]", request, e);
+            LOGGER.error("Update static topic failed for [{}]", request, e);
             response.setCode(ResponseCode.SYSTEM_ERROR);
             response.setRemark(e.getMessage());
         }
         return response;
     }
 
-
     private synchronized RemotingCommand deleteTopic(ChannelHandlerContext ctx,
         RemotingCommand request) throws RemotingCommandException {
         final RemotingCommand response = RemotingCommand.createResponseCommand(null);
         DeleteTopicRequestHeader requestHeader =
             (DeleteTopicRequestHeader) request.decodeCommandCustomHeader(DeleteTopicRequestHeader.class);
 
-        log.info("deleteTopic called by {}", RemotingHelper.parseChannelRemoteAddr(ctx.channel()));
+        LOGGER.info("AdminBrokerProcessor#deleteTopic: broker receive request to delete topic={}, caller={}",
+            requestHeader.getTopic(), RemotingHelper.parseChannelRemoteAddr(ctx.channel()));
 
         String topic = requestHeader.getTopic();
         if (!TopicValidator.validateTopic(topic, response)) {
@@ -371,9 +467,9 @@ public class AdminBrokerProcessor extends AsyncNettyRequestProcessor implements
             return response;
         }
 
-        this.brokerController.getTopicConfigManager().deleteTopicConfig(topic);
-        this.brokerController.getTopicQueueMappingManager().delete(topic);
-
+        this.brokerController.getTopicConfigManager().deleteTopicConfig(requestHeader.getTopic());
+        this.brokerController.getTopicQueueMappingManager().delete(requestHeader.getTopic());
+        this.brokerController.getConsumerOffsetManager().cleanOffsetByTopic(requestHeader.getTopic());
         this.brokerController.getMessageStore()
             .cleanUnusedTopic(this.brokerController.getTopicConfigManager().getTopicConfigTable().keySet());
         if (this.brokerController.getBrokerConfig().isAutoDeleteUnusedStats()) {
@@ -412,13 +508,13 @@ public class AdminBrokerProcessor extends AsyncNettyRequestProcessor implements
                 ctx.writeAndFlush(response);
             } else {
                 String errorMsg = "The accesskey[" + requestHeader.getAccessKey() + "] corresponding to accessConfig has been updated failed.";
-                log.warn(errorMsg);
+                LOGGER.warn(errorMsg);
                 response.setCode(ResponseCode.UPDATE_AND_CREATE_ACL_CONFIG_FAILED);
                 response.setRemark(errorMsg);
                 return response;
             }
         } catch (Exception e) {
-            log.error("Failed to generate a proper update accessvalidator response", e);
+            LOGGER.error("Failed to generate a proper update accessvalidator response", e);
             response.setCode(ResponseCode.UPDATE_AND_CREATE_ACL_CONFIG_FAILED);
             response.setRemark(e.getMessage());
             return response;
@@ -433,7 +529,7 @@ public class AdminBrokerProcessor extends AsyncNettyRequestProcessor implements
 
         final DeleteAccessConfigRequestHeader requestHeader =
             (DeleteAccessConfigRequestHeader) request.decodeCommandCustomHeader(DeleteAccessConfigRequestHeader.class);
-        log.info("DeleteAccessConfig called by {}", RemotingHelper.parseChannelRemoteAddr(ctx.channel()));
+        LOGGER.info("DeleteAccessConfig called by {}", RemotingHelper.parseChannelRemoteAddr(ctx.channel()));
 
         try {
             String accessKey = requestHeader.getAccessKey();
@@ -446,14 +542,14 @@ public class AdminBrokerProcessor extends AsyncNettyRequestProcessor implements
                 ctx.writeAndFlush(response);
             } else {
                 String errorMsg = "The accesskey[" + requestHeader.getAccessKey() + "] corresponding to accessConfig has been deleted failed.";
-                log.warn(errorMsg);
+                LOGGER.warn(errorMsg);
                 response.setCode(ResponseCode.DELETE_ACL_CONFIG_FAILED);
                 response.setRemark(errorMsg);
                 return response;
             }
 
         } catch (Exception e) {
-            log.error("Failed to generate a proper delete accessvalidator response", e);
+            LOGGER.error("Failed to generate a proper delete accessvalidator response", e);
             response.setCode(ResponseCode.DELETE_ACL_CONFIG_FAILED);
             response.setRemark(e.getMessage());
             return response;
@@ -480,13 +576,13 @@ public class AdminBrokerProcessor extends AsyncNettyRequestProcessor implements
                 ctx.writeAndFlush(response);
             } else {
                 String errorMsg = "The globalWhiteAddresses[" + requestHeader.getGlobalWhiteAddrs() + "] has been updated failed.";
-                log.warn(errorMsg);
+                LOGGER.warn(errorMsg);
                 response.setCode(ResponseCode.UPDATE_GLOBAL_WHITE_ADDRS_CONFIG_FAILED);
                 response.setRemark(errorMsg);
                 return response;
             }
         } catch (Exception e) {
-            log.error("Failed to generate a proper update globalWhiteAddresses response", e);
+            LOGGER.error("Failed to generate a proper update globalWhiteAddresses response", e);
             response.setCode(ResponseCode.UPDATE_GLOBAL_WHITE_ADDRS_CONFIG_FAILED);
             response.setRemark(e.getMessage());
             return response;
@@ -499,12 +595,11 @@ public class AdminBrokerProcessor extends AsyncNettyRequestProcessor implements
 
         final RemotingCommand response = RemotingCommand.createResponseCommand(GetBrokerAclConfigResponseHeader.class);
 
-        final GetBrokerAclConfigResponseHeader responseHeader = (GetBrokerAclConfigResponseHeader)response.readCustomHeader();
+        final GetBrokerAclConfigResponseHeader responseHeader = (GetBrokerAclConfigResponseHeader) response.readCustomHeader();
 
         try {
             AccessValidator accessValidator = this.brokerController.getAccessValidatorMap().get(PlainAccessValidator.class);
 
-            responseHeader.setAllAclFileVersion(JSON.toJSONString(accessValidator.getAllAclConfigVersion()));
             responseHeader.setVersion(accessValidator.getAclConfigVersion());
             responseHeader.setBrokerAddr(this.brokerController.getBrokerAddr());
             responseHeader.setBrokerName(this.brokerController.getBrokerConfig().getBrokerName());
@@ -514,7 +609,7 @@ public class AdminBrokerProcessor extends AsyncNettyRequestProcessor implements
             response.setRemark(null);
             return response;
         } catch (Exception e) {
-            log.error("Failed to generate a proper getBrokerAclConfigVersion response", e);
+            LOGGER.error("Failed to generate a proper getBrokerAclConfigVersion response", e);
         }
 
         return null;
@@ -535,7 +630,7 @@ public class AdminBrokerProcessor extends AsyncNettyRequestProcessor implements
             response.setRemark(null);
             return response;
         } catch (Exception e) {
-            log.error("Failed to generate a proper getBrokerClusterAclConfig response", e);
+            LOGGER.error("Failed to generate a proper getBrokerClusterAclConfig response", e);
         }
 
         return null;
@@ -544,7 +639,7 @@ public class AdminBrokerProcessor extends AsyncNettyRequestProcessor implements
     private RemotingCommand getUnknownCmdResponse(ChannelHandlerContext ctx, RemotingCommand request) {
         String error = " request type " + request.getCode() + " not supported";
         final RemotingCommand response =
-                RemotingCommand.createResponseCommand(RemotingSysResponseCode.REQUEST_CODE_NOT_SUPPORTED, error);
+            RemotingCommand.createResponseCommand(RemotingSysResponseCode.REQUEST_CODE_NOT_SUPPORTED, error);
         return response;
     }
 
@@ -561,20 +656,19 @@ public class AdminBrokerProcessor extends AsyncNettyRequestProcessor implements
         topicConfigAndMappingSerializeWrapper.setMappingDataVersion(this.brokerController.getTopicQueueMappingManager().getDataVersion());
         topicConfigAndMappingSerializeWrapper.setTopicQueueMappingDetailMap(this.brokerController.getTopicQueueMappingManager().getTopicQueueMappingTable());
 
-
         String content = topicConfigAndMappingSerializeWrapper.toJson();
         if (content != null && content.length() > 0) {
             try {
                 response.setBody(content.getBytes(MixAll.DEFAULT_CHARSET));
             } catch (UnsupportedEncodingException e) {
-                log.error("", e);
+                LOGGER.error("", e);
 
                 response.setCode(ResponseCode.SYSTEM_ERROR);
-                response.setRemark("UnsupportedEncodingException " + e);
+                response.setRemark("UnsupportedEncodingException " + e.getMessage());
                 return response;
             }
         } else {
-            log.error("No topic in this broker, client: {}", ctx.channel().remoteAddress());
+            LOGGER.error("No topic in this broker, client: {}", ctx.channel().remoteAddress());
             response.setCode(ResponseCode.SYSTEM_ERROR);
             response.setRemark("No topic in this broker");
             return response;
@@ -589,7 +683,8 @@ public class AdminBrokerProcessor extends AsyncNettyRequestProcessor implements
     private synchronized RemotingCommand updateBrokerConfig(ChannelHandlerContext ctx, RemotingCommand request) {
         final RemotingCommand response = RemotingCommand.createResponseCommand(null);
 
-        log.info("updateBrokerConfig called by {}", RemotingHelper.parseChannelRemoteAddr(ctx.channel()));
+        final String callerAddress = RemotingHelper.parseChannelRemoteAddr(ctx.channel());
+        LOGGER.info("Broker receive request to update config, caller address={}", callerAddress);
 
         byte[] body = request.getBody();
         if (body != null) {
@@ -597,20 +692,21 @@ public class AdminBrokerProcessor extends AsyncNettyRequestProcessor implements
                 String bodyStr = new String(body, MixAll.DEFAULT_CHARSET);
                 Properties properties = MixAll.string2Properties(bodyStr);
                 if (properties != null) {
-                    log.info("updateBrokerConfig, new config: [{}] client: {} ", properties, ctx.channel().remoteAddress());
+                    LOGGER.info("updateBrokerConfig, new config: [{}] client: {} ", properties, ctx.channel().remoteAddress());
                     this.brokerController.getConfiguration().update(properties);
                     if (properties.containsKey("brokerPermission")) {
                         this.brokerController.getTopicConfigManager().getDataVersion().nextVersion();
                         this.brokerController.registerBrokerAll(false, false, true);
                     }
                 } else {
-                    log.error("string2Properties error");
+                    LOGGER.error("string2Properties error");
                     response.setCode(ResponseCode.SYSTEM_ERROR);
                     response.setRemark("string2Properties error");
                     return response;
                 }
             } catch (UnsupportedEncodingException e) {
-                log.error("", e);
+                LOGGER.error("AdminBrokerProcessor#updateBrokerConfig: unexpected error, caller={}",
+                    callerAddress, e);
                 response.setCode(ResponseCode.SYSTEM_ERROR);
                 response.setRemark("UnsupportedEncodingException " + e);
                 return response;
@@ -632,7 +728,8 @@ public class AdminBrokerProcessor extends AsyncNettyRequestProcessor implements
             try {
                 response.setBody(content.getBytes(MixAll.DEFAULT_CHARSET));
             } catch (UnsupportedEncodingException e) {
-                log.error("", e);
+                LOGGER.error("AdminBrokerProcessor#getBrokerConfig: unexpected error, caller={}",
+                    RemotingHelper.parseChannelRemoteAddr(ctx.channel()), e);
 
                 response.setCode(ResponseCode.SYSTEM_ERROR);
                 response.setRemark("UnsupportedEncodingException " + e);
@@ -647,7 +744,8 @@ public class AdminBrokerProcessor extends AsyncNettyRequestProcessor implements
         return response;
     }
 
-    private RemotingCommand rewriteRequestForStaticTopic(SearchOffsetRequestHeader requestHeader, TopicQueueMappingContext mappingContext) {
+    private RemotingCommand rewriteRequestForStaticTopic(SearchOffsetRequestHeader requestHeader,
+        TopicQueueMappingContext mappingContext) {
         try {
             if (mappingContext.getMappingDetail() == null) {
                 return null;
@@ -684,7 +782,7 @@ public class AdminBrokerProcessor extends AsyncNettyRequestProcessor implements
                     }
                     SearchOffsetResponseHeader offsetResponseHeader = (SearchOffsetResponseHeader) rpcResponse.getHeader();
                     if (offsetResponseHeader.getOffset() < 0
-                            || (item.checkIfEndOffsetDecided() && offsetResponseHeader.getOffset() >= item.getEndOffset())) {
+                        || (item.checkIfEndOffsetDecided() && offsetResponseHeader.getOffset() >= item.getEndOffset())) {
                         continue;
                     } else {
                         offset = item.computeStaticQueueOffsetStrictly(offsetResponseHeader.getOffset());
@@ -727,7 +825,8 @@ public class AdminBrokerProcessor extends AsyncNettyRequestProcessor implements
         return response;
     }
 
-    private RemotingCommand rewriteRequestForStaticTopic(GetMaxOffsetRequestHeader requestHeader, TopicQueueMappingContext mappingContext) {
+    private RemotingCommand rewriteRequestForStaticTopic(GetMaxOffsetRequestHeader requestHeader,
+        TopicQueueMappingContext mappingContext) {
         if (mappingContext.getMappingDetail() == null) {
             return null;
         }
@@ -793,7 +892,8 @@ public class AdminBrokerProcessor extends AsyncNettyRequestProcessor implements
         return response;
     }
 
-    private CompletableFuture<RpcResponse> handleGetMinOffsetForStaticTopic(RpcRequest request, TopicQueueMappingContext mappingContext)  {
+    private CompletableFuture<RpcResponse> handleGetMinOffsetForStaticTopic(RpcRequest request,
+        TopicQueueMappingContext mappingContext) {
         if (mappingContext.getMappingDetail() == null) {
             return null;
         }
@@ -801,11 +901,11 @@ public class AdminBrokerProcessor extends AsyncNettyRequestProcessor implements
         if (!mappingContext.isLeader()) {
             //this may not
             return CompletableFuture.completedFuture(new RpcResponse(new RpcException(ResponseCode.NOT_LEADER_FOR_QUEUE,
-                    String.format("%s-%d is not leader in broker %s, request code %d", mappingContext.getTopic(), mappingContext.getGlobalId(), mappingDetail.getBname(), request.getCode()))));
+                String.format("%s-%d is not leader in broker %s, request code %d", mappingContext.getTopic(), mappingContext.getGlobalId(), mappingDetail.getBname(), request.getCode()))));
         }
         GetMinOffsetRequestHeader requestHeader = (GetMinOffsetRequestHeader) request.getHeader();
         LogicQueueMappingItem mappingItem = TopicQueueMappingUtils.findLogicQueueMappingItem(mappingContext.getMappingItemList(), 0L, true);
-        assert  mappingItem != null;
+        assert mappingItem != null;
         try {
             requestHeader.setBname(mappingItem.getBname());
             requestHeader.setLo(false);
@@ -829,12 +929,12 @@ public class AdminBrokerProcessor extends AsyncNettyRequestProcessor implements
             responseHeader.setOffset(offset);
             return CompletableFuture.completedFuture(new RpcResponse(ResponseCode.SUCCESS, responseHeader, null));
         } catch (Throwable t) {
-            log.error("rewriteRequestForStaticTopic failed", t);
+            LOGGER.error("rewriteRequestForStaticTopic failed", t);
             return CompletableFuture.completedFuture(new RpcResponse(new RpcException(ResponseCode.SYSTEM_ERROR, t.getMessage(), t)));
         }
     }
 
-    private CompletableFuture<RpcResponse>  handleGetMinOffset(RpcRequest request) {
+    private CompletableFuture<RpcResponse> handleGetMinOffset(RpcRequest request) {
         assert request.getCode() == RequestCode.GET_MIN_OFFSET;
         GetMinOffsetRequestHeader requestHeader = (GetMinOffsetRequestHeader) request.getHeader();
         TopicQueueMappingContext mappingContext = this.brokerController.getTopicQueueMappingManager().buildTopicQueueMappingContext(requestHeader, false);
@@ -854,14 +954,15 @@ public class AdminBrokerProcessor extends AsyncNettyRequestProcessor implements
             (GetMinOffsetRequestHeader) request.decodeCommandCustomHeader(GetMinOffsetRequestHeader.class);
         try {
             CompletableFuture<RpcResponse> responseFuture = handleGetMinOffset(new RpcRequest(RequestCode.GET_MIN_OFFSET, requestHeader, null));
-            RpcResponse  rpcResponse = responseFuture.get();
+            RpcResponse rpcResponse = responseFuture.get();
             return RpcClientUtils.createCommandForRpcResponse(rpcResponse);
         } catch (Throwable t) {
             return buildErrorResponse(ResponseCode.SYSTEM_ERROR, t.getMessage());
         }
     }
 
-    private RemotingCommand rewriteRequestForStaticTopic(GetEarliestMsgStoretimeRequestHeader requestHeader, TopicQueueMappingContext mappingContext)  {
+    private RemotingCommand rewriteRequestForStaticTopic(GetEarliestMsgStoretimeRequestHeader requestHeader,
+        TopicQueueMappingContext mappingContext) {
         if (mappingContext.getMappingDetail() == null) {
             return null;
         }
@@ -935,10 +1036,78 @@ public class AdminBrokerProcessor extends AsyncNettyRequestProcessor implements
         final RemotingCommand response = RemotingCommand.createResponseCommand(null);
         LockBatchRequestBody requestBody = LockBatchRequestBody.decode(request.getBody(), LockBatchRequestBody.class);
 
-        Set<MessageQueue> lockOKMQSet = this.brokerController.getRebalanceLockManager().tryLockBatch(
+        Set<MessageQueue> lockOKMQSet = new HashSet<>();
+        Set<MessageQueue> selfLockOKMQSet = this.brokerController.getRebalanceLockManager().tryLockBatch(
             requestBody.getConsumerGroup(),
             requestBody.getMqSet(),
             requestBody.getClientId());
+        if (requestBody.isOnlyThisBroker() || !brokerController.getBrokerConfig().isLockInStrictMode()) {
+            lockOKMQSet = selfLockOKMQSet;
+        } else {
+            requestBody.setOnlyThisBroker(true);
+            int replicaSize = this.brokerController.getMessageStoreConfig().getTotalReplicas();
+
+            int quorum = replicaSize / 2 + 1;
+
+            if (quorum <= 1) {
+                lockOKMQSet = selfLockOKMQSet;
+            } else {
+                final ConcurrentMap<MessageQueue, Integer> mqLockMap = new ConcurrentHashMap<>();
+                for (MessageQueue mq : selfLockOKMQSet) {
+                    if (!mqLockMap.containsKey(mq)) {
+                        mqLockMap.put(mq, 0);
+                    }
+                    mqLockMap.put(mq, mqLockMap.get(mq) + 1);
+                }
+
+                BrokerMemberGroup memberGroup = this.brokerController.getBrokerMemberGroup();
+
+                if (memberGroup != null) {
+                    Map<Long, String> addrMap = new HashMap<>(memberGroup.getBrokerAddrs());
+                    addrMap.remove(this.brokerController.getBrokerConfig().getBrokerId());
+                    final CountDownLatch countDownLatch = new CountDownLatch(addrMap.size());
+                    requestBody.setMqSet(selfLockOKMQSet);
+                    requestBody.setOnlyThisBroker(true);
+                    for (Long brokerId : addrMap.keySet()) {
+                        try {
+                            this.brokerController.getBrokerOuterAPI().lockBatchMQAsync(addrMap.get(brokerId),
+                                requestBody, 1000, new LockCallback() {
+                                    @Override
+                                    public void onSuccess(Set<MessageQueue> lockOKMQSet) {
+                                        for (MessageQueue mq : lockOKMQSet) {
+                                            if (!mqLockMap.containsKey(mq)) {
+                                                mqLockMap.put(mq, 0);
+                                            }
+                                            mqLockMap.put(mq, mqLockMap.get(mq) + 1);
+                                        }
+                                        countDownLatch.countDown();
+                                    }
+
+                                    @Override
+                                    public void onException(Throwable e) {
+                                        LOGGER.warn("lockBatchMQAsync on {} failed, {}", addrMap.get(brokerId), e);
+                                        countDownLatch.countDown();
+                                    }
+                                });
+                        } catch (Exception e) {
+                            LOGGER.warn("lockBatchMQAsync on {} failed, {}", addrMap.get(brokerId), e);
+                            countDownLatch.countDown();
+                        }
+                    }
+                    try {
+                        countDownLatch.await(2000, TimeUnit.MILLISECONDS);
+                    } catch (InterruptedException e) {
+                        LOGGER.warn("lockBatchMQ exception on {}, {}", this.brokerController.getBrokerConfig().getBrokerName(), e);
+                    }
+                }
+
+                for (MessageQueue mq : mqLockMap.keySet()) {
+                    if (mqLockMap.get(mq) >= quorum) {
+                        lockOKMQSet.add(mq);
+                    }
+                }
+            }
+        }
 
         LockBatchResponseBody responseBody = new LockBatchResponseBody();
         responseBody.setLockOKMQSet(lockOKMQSet);
@@ -954,10 +1123,36 @@ public class AdminBrokerProcessor extends AsyncNettyRequestProcessor implements
         final RemotingCommand response = RemotingCommand.createResponseCommand(null);
         UnlockBatchRequestBody requestBody = UnlockBatchRequestBody.decode(request.getBody(), UnlockBatchRequestBody.class);
 
-        this.brokerController.getRebalanceLockManager().unlockBatch(
-            requestBody.getConsumerGroup(),
-            requestBody.getMqSet(),
-            requestBody.getClientId());
+        if (requestBody.isOnlyThisBroker() || !this.brokerController.getBrokerConfig().isLockInStrictMode()) {
+            this.brokerController.getRebalanceLockManager().unlockBatch(
+                requestBody.getConsumerGroup(),
+                requestBody.getMqSet(),
+                requestBody.getClientId());
+        } else {
+            requestBody.setOnlyThisBroker(true);
+            BrokerMemberGroup memberGroup = this.brokerController.getBrokerMemberGroup();
+
+            if (memberGroup != null) {
+                Map<Long, String> addrMap = memberGroup.getBrokerAddrs();
+                for (Long brokerId : addrMap.keySet()) {
+                    try {
+                        this.brokerController.getBrokerOuterAPI().unlockBatchMQAsync(addrMap.get(brokerId), requestBody, 1000, new UnlockCallback() {
+                            @Override
+                            public void onSuccess() {
+
+                            }
+
+                            @Override
+                            public void onException(Throwable e) {
+                                LOGGER.warn("unlockBatchMQ exception on {}, {}", addrMap.get(brokerId), e);
+                            }
+                        });
+                    } catch (Exception e) {
+                        LOGGER.warn("unlockBatchMQ exception on {}, {}", addrMap.get(brokerId), e);
+                    }
+                }
+            }
+        }
 
         response.setCode(ResponseCode.SUCCESS);
         response.setRemark(null);
@@ -968,7 +1163,8 @@ public class AdminBrokerProcessor extends AsyncNettyRequestProcessor implements
         throws RemotingCommandException {
         final RemotingCommand response = RemotingCommand.createResponseCommand(null);
 
-        log.info("updateAndCreateSubscriptionGroup called by {}", RemotingHelper.parseChannelRemoteAddr(ctx.channel()));
+        LOGGER.info("AdminBrokerProcessor#updateAndCreateSubscriptionGroup called by {}",
+            RemotingHelper.parseChannelRemoteAddr(ctx.channel()));
 
         SubscriptionGroupConfig config = RemotingSerializable.decode(request.getBody(), SubscriptionGroupConfig.class);
         if (config != null) {
@@ -980,6 +1176,26 @@ public class AdminBrokerProcessor extends AsyncNettyRequestProcessor implements
         return response;
     }
 
+    private void initConsumerOffset(String clientHost, String groupName, int mode, TopicConfig topicConfig) {
+        String topic = topicConfig.getTopicName();
+        for (int queueId = 0; queueId < topicConfig.getReadQueueNums(); queueId++) {
+            if (this.brokerController.getConsumerOffsetManager().queryOffset(groupName, topic, queueId) > -1) {
+                continue;
+            }
+            long offset = 0;
+            if (this.brokerController.getMessageStore().getConsumeQueue(topic, queueId) != null) {
+                if (ConsumeInitMode.MAX == mode) {
+                    offset = this.brokerController.getMessageStore().getMaxOffsetInQueue(topic, queueId);
+                } else if (ConsumeInitMode.MIN == mode) {
+                    offset = this.brokerController.getMessageStore().getMinOffsetInQueue(topic, queueId);
+                }
+            }
+            this.brokerController.getConsumerOffsetManager().commitOffset(clientHost, groupName, topic, queueId, offset);
+            LOGGER.info("AdminBrokerProcessor#initConsumerOffset: consumerGroup={}, topic={}, queueId={}, offset={}",
+                groupName, topic, queueId, offset);
+        }
+    }
+
     private RemotingCommand getAllSubscriptionGroup(ChannelHandlerContext ctx,
         RemotingCommand request) throws RemotingCommandException {
         final RemotingCommand response = RemotingCommand.createResponseCommand(null);
@@ -988,14 +1204,14 @@ public class AdminBrokerProcessor extends AsyncNettyRequestProcessor implements
             try {
                 response.setBody(content.getBytes(MixAll.DEFAULT_CHARSET));
             } catch (UnsupportedEncodingException e) {
-                log.error("", e);
+                LOGGER.error("UnsupportedEncodingException getAllSubscriptionGroup", e);
 
                 response.setCode(ResponseCode.SYSTEM_ERROR);
-                response.setRemark("UnsupportedEncodingException " + e);
+                response.setRemark("UnsupportedEncodingException " + e.getMessage());
                 return response;
             }
         } else {
-            log.error("No subscription group in this broker, client:{} ", ctx.channel().remoteAddress());
+            LOGGER.error("No subscription group in this broker, client:{} ", ctx.channel().remoteAddress());
             response.setCode(ResponseCode.SYSTEM_ERROR);
             response.setRemark("No subscription group in this broker");
             return response;
@@ -1013,11 +1229,12 @@ public class AdminBrokerProcessor extends AsyncNettyRequestProcessor implements
         DeleteSubscriptionGroupRequestHeader requestHeader =
             (DeleteSubscriptionGroupRequestHeader) request.decodeCommandCustomHeader(DeleteSubscriptionGroupRequestHeader.class);
 
-        log.info("deleteSubscriptionGroup called by {}", RemotingHelper.parseChannelRemoteAddr(ctx.channel()));
+        LOGGER.info("AdminBrokerProcessor#deleteSubscriptionGroup, caller={}",
+            RemotingHelper.parseChannelRemoteAddr(ctx.channel()));
 
         this.brokerController.getSubscriptionGroupManager().deleteSubscriptionGroupConfig(requestHeader.getGroupName());
 
-        if (requestHeader.isRemoveOffset()) {
+        if (requestHeader.isCleanOffset()) {
             this.brokerController.getConsumerOffsetManager().removeOffset(requestHeader.getGroupName());
         }
 
@@ -1029,8 +1246,6 @@ public class AdminBrokerProcessor extends AsyncNettyRequestProcessor implements
         return response;
     }
 
-
-
     private RemotingCommand getTopicStatsInfo(ChannelHandlerContext ctx,
         RemotingCommand request) throws RemotingCommandException {
         final RemotingCommand response = RemotingCommand.createResponseCommand(null);
@@ -1044,8 +1259,8 @@ public class AdminBrokerProcessor extends AsyncNettyRequestProcessor implements
             response.setRemark("topic[" + topic + "] not exist");
             return response;
         }
-        TopicStatsTable topicStatsTable = new TopicStatsTable();
 
+        TopicStatsTable topicStatsTable = new TopicStatsTable();
         for (int i = 0; i < topicConfig.getWriteQueueNums(); i++) {
             MessageQueue mq = new MessageQueue();
             mq.setTopic(topic);
@@ -1054,12 +1269,14 @@ public class AdminBrokerProcessor extends AsyncNettyRequestProcessor implements
 
             TopicOffset topicOffset = new TopicOffset();
             long min = this.brokerController.getMessageStore().getMinOffsetInQueue(topic, i);
-            if (min < 0)
+            if (min < 0) {
                 min = 0;
+            }
 
             long max = this.brokerController.getMessageStore().getMaxOffsetInQueue(topic, i);
-            if (max < 0)
+            if (max < 0) {
                 max = 0;
+            }
 
             long timestamp = 0;
             if (max > 0) {
@@ -1172,7 +1389,7 @@ public class AdminBrokerProcessor extends AsyncNettyRequestProcessor implements
         for (String topic : topics) {
             TopicConfig topicConfig = this.brokerController.getTopicConfigManager().selectTopicConfig(topic);
             if (null == topicConfig) {
-                log.warn("consumeStats, topic config not exist, {}", topic);
+                LOGGER.warn("AdminBrokerProcessor#getConsumeStats: topic config does not exist, topic={}", topic);
                 continue;
             }
 
@@ -1184,7 +1401,9 @@ public class AdminBrokerProcessor extends AsyncNettyRequestProcessor implements
 
                 if (null == findSubscriptionData
                     && this.brokerController.getConsumerManager().findSubscriptionDataCount(requestHeader.getConsumerGroup()) > 0) {
-                    log.warn("consumeStats, the consumer group[{}], topic[{}] not exist", requestHeader.getConsumerGroup(), topic);
+                    LOGGER.warn(
+                        "AdminBrokerProcessor#getConsumeStats: topic does not exist in consumer group's subscription, "
+                            + "topic={}, consumer group={}", topic, requestHeader.getConsumerGroup());
                     continue;
                 }
             }
@@ -1198,19 +1417,20 @@ public class AdminBrokerProcessor extends AsyncNettyRequestProcessor implements
                 OffsetWrapper offsetWrapper = new OffsetWrapper();
 
                 long brokerOffset = this.brokerController.getMessageStore().getMaxOffsetInQueue(topic, i);
-                if (brokerOffset < 0)
+                if (brokerOffset < 0) {
                     brokerOffset = 0;
+                }
 
                 long consumerOffset = this.brokerController.getConsumerOffsetManager().queryOffset(
-                    requestHeader.getConsumerGroup(),
-                    topic,
-                    i);
+                    requestHeader.getConsumerGroup(), topic, i);
+
                 // the consumerOffset cannot be zero for static topic because of the "double read check" strategy
                 // just remain the logic for dynamic topic
                 // maybe we should remove it in the future
                 if (mappingDetail == null) {
-                    if (consumerOffset < 0)
+                    if (consumerOffset < 0) {
                         consumerOffset = 0;
+                    }
                 }
 
                 offsetWrapper.setBrokerOffset(brokerOffset);
@@ -1223,6 +1443,7 @@ public class AdminBrokerProcessor extends AsyncNettyRequestProcessor implements
                         offsetWrapper.setLastTimestamp(lastTimestamp);
                     }
                 }
+
                 consumeStats.getOffsetTable().put(mq, offsetWrapper);
             }
 
@@ -1247,14 +1468,14 @@ public class AdminBrokerProcessor extends AsyncNettyRequestProcessor implements
             try {
                 response.setBody(content.getBytes(MixAll.DEFAULT_CHARSET));
             } catch (UnsupportedEncodingException e) {
-                log.error("get all consumer offset from master error.", e);
+                LOGGER.error("get all consumer offset from master error.", e);
 
                 response.setCode(ResponseCode.SYSTEM_ERROR);
-                response.setRemark("UnsupportedEncodingException " + e);
+                response.setRemark("UnsupportedEncodingException " + e.getMessage());
                 return response;
             }
         } else {
-            log.error("No consumer offset in this broker, client: {} ", ctx.channel().remoteAddress());
+            LOGGER.error("No consumer offset in this broker, client: {} ", ctx.channel().remoteAddress());
             response.setCode(ResponseCode.SYSTEM_ERROR);
             response.setRemark("No consumer offset in this broker");
             return response;
@@ -1269,26 +1490,21 @@ public class AdminBrokerProcessor extends AsyncNettyRequestProcessor implements
     private RemotingCommand getAllDelayOffset(ChannelHandlerContext ctx, RemotingCommand request) {
         final RemotingCommand response = RemotingCommand.createResponseCommand(null);
 
-        if (!(this.brokerController.getMessageStore() instanceof DefaultMessageStore)) {
-            log.error("Delay offset not supported in this messagetore, client: {} ", ctx.channel().remoteAddress());
-            response.setCode(ResponseCode.SYSTEM_ERROR);
-            response.setRemark("Delay offset not supported in this messagetore");
-            return response;
-        }
-
-        String content = ((DefaultMessageStore) this.brokerController.getMessageStore()).getScheduleMessageService().encode();
+        String content = this.brokerController.getScheduleMessageService().encode();
         if (content != null && content.length() > 0) {
             try {
                 response.setBody(content.getBytes(MixAll.DEFAULT_CHARSET));
             } catch (UnsupportedEncodingException e) {
-                log.error("Get all delay offset from master error.", e);
+                LOGGER.error("AdminBrokerProcessor#getAllDelayOffset: unexpected error, caller={}.",
+                    RemotingHelper.parseChannelRemoteAddr(ctx.channel()), e);
 
                 response.setCode(ResponseCode.SYSTEM_ERROR);
                 response.setRemark("UnsupportedEncodingException " + e);
                 return response;
             }
         } else {
-            log.error("No delay offset in this broker, client: {} ", ctx.channel().remoteAddress());
+            LOGGER.error("AdminBrokerProcessor#getAllDelayOffset: no delay offset in this broker, caller={}",
+                RemotingHelper.parseChannelRemoteAddr(ctx.channel()));
             response.setCode(ResponseCode.SYSTEM_ERROR);
             response.setRemark("No delay offset in this broker");
             return response;
@@ -1304,7 +1520,7 @@ public class AdminBrokerProcessor extends AsyncNettyRequestProcessor implements
         RemotingCommand request) throws RemotingCommandException {
         final ResetOffsetRequestHeader requestHeader =
             (ResetOffsetRequestHeader) request.decodeCommandCustomHeader(ResetOffsetRequestHeader.class);
-        log.info("[reset-offset] reset offset started by {}. topic={}, group={}, timestamp={}, isForce={}",
+        LOGGER.info("[reset-offset] reset offset started by {}. topic={}, group={}, timestamp={}, isForce={}",
             RemotingHelper.parseChannelRemoteAddr(ctx.channel()), requestHeader.getTopic(), requestHeader.getGroup(),
             requestHeader.getTimestamp(), requestHeader.isForce());
         boolean isC = false;
@@ -1323,7 +1539,7 @@ public class AdminBrokerProcessor extends AsyncNettyRequestProcessor implements
         final GetConsumerStatusRequestHeader requestHeader =
             (GetConsumerStatusRequestHeader) request.decodeCommandCustomHeader(GetConsumerStatusRequestHeader.class);
 
-        log.info("[get-consumer-status] get consumer status by {}. topic={}, group={}",
+        LOGGER.info("[get-consumer-status] get consumer status by {}. topic={}, group={}",
             RemotingHelper.parseChannelRemoteAddr(ctx.channel()), requestHeader.getTopic(), requestHeader.getGroup());
 
         return this.brokerController.getBroker2Client().getConsumeStatus(requestHeader.getTopic(), requestHeader.getGroup(),
@@ -1353,6 +1569,47 @@ public class AdminBrokerProcessor extends AsyncNettyRequestProcessor implements
         return response;
     }
 
+    private RemotingCommand queryTopicsByConsumer(ChannelHandlerContext ctx,
+        RemotingCommand request) throws RemotingCommandException {
+        final RemotingCommand response = RemotingCommand.createResponseCommand(null);
+        QueryTopicsByConsumerRequestHeader requestHeader =
+            (QueryTopicsByConsumerRequestHeader) request.decodeCommandCustomHeader(QueryTopicsByConsumerRequestHeader.class);
+
+        Set<String> topics = this.brokerController.getConsumerOffsetManager().whichTopicByConsumer(requestHeader.getGroup());
+
+        TopicList topicList = new TopicList();
+        topicList.setTopicList(topics);
+        topicList.setBrokerAddr(brokerController.getBrokerAddr());
+        byte[] body = topicList.encode();
+
+        response.setBody(body);
+        response.setCode(ResponseCode.SUCCESS);
+        response.setRemark(null);
+        return response;
+    }
+
+    private RemotingCommand querySubscriptionByConsumer(ChannelHandlerContext ctx,
+        RemotingCommand request) throws RemotingCommandException {
+        final RemotingCommand response = RemotingCommand.createResponseCommand(null);
+        QuerySubscriptionByConsumerRequestHeader requestHeader =
+            (QuerySubscriptionByConsumerRequestHeader) request.decodeCommandCustomHeader(QuerySubscriptionByConsumerRequestHeader.class);
+
+        SubscriptionData subscriptionData =
+            this.brokerController.getConsumerManager().findSubscriptionData(requestHeader.getGroup(), requestHeader.getTopic());
+
+        QuerySubscriptionResponseBody responseBody = new QuerySubscriptionResponseBody();
+        responseBody.setGroup(requestHeader.getGroup());
+        responseBody.setTopic(requestHeader.getTopic());
+        responseBody.setSubscriptionData(subscriptionData);
+        byte[] body = responseBody.encode();
+
+        response.setBody(body);
+        response.setCode(ResponseCode.SUCCESS);
+        response.setRemark(null);
+        return response;
+
+    }
+
     private RemotingCommand registerFilterServer(ChannelHandlerContext ctx,
         RemotingCommand request) throws RemotingCommandException {
         final RemotingCommand response = RemotingCommand.createResponseCommand(RegisterFilterServerResponseHeader.class);
@@ -1440,20 +1697,20 @@ public class AdminBrokerProcessor extends AsyncNettyRequestProcessor implements
     }
 
     public RemotingCommand cleanExpiredConsumeQueue() {
-        log.warn("invoke cleanExpiredConsumeQueue start.");
+        LOGGER.info("AdminBrokerProcessor#cleanExpiredConsumeQueue: start.");
         final RemotingCommand response = RemotingCommand.createResponseCommand(null);
         brokerController.getMessageStore().cleanExpiredConsumerQueue();
-        log.warn("invoke cleanExpiredConsumeQueue end.");
+        LOGGER.info("AdminBrokerProcessor#cleanExpiredConsumeQueue: end.");
         response.setCode(ResponseCode.SUCCESS);
         response.setRemark(null);
         return response;
     }
 
     public RemotingCommand cleanUnusedTopic() {
-        log.warn("invoke cleanUnusedTopic start.");
+        LOGGER.warn("invoke cleanUnusedTopic start.");
         final RemotingCommand response = RemotingCommand.createResponseCommand(null);
         brokerController.getMessageStore().cleanUnusedTopic(brokerController.getTopicConfigManager().getTopicConfigTable().keySet());
-        log.warn("invoke cleanUnusedTopic end.");
+        LOGGER.warn("invoke cleanUnusedTopic end.");
         response.setCode(ResponseCode.SUCCESS);
         response.setRemark(null);
         return response;
@@ -1501,7 +1758,22 @@ public class AdminBrokerProcessor extends AsyncNettyRequestProcessor implements
         final ConsumeMessageDirectlyResultRequestHeader requestHeader = (ConsumeMessageDirectlyResultRequestHeader) request
             .decodeCommandCustomHeader(ConsumeMessageDirectlyResultRequestHeader.class);
 
+        // brokerName
         request.getExtFields().put("brokerName", this.brokerController.getBrokerConfig().getBrokerName());
+        // topicSysFlag
+        if (StringUtils.isNotEmpty(requestHeader.getTopic())) {
+            TopicConfig topicConfig = this.brokerController.getTopicConfigManager().getTopicConfigTable().get(requestHeader.getTopic());
+            if (topicConfig != null) {
+                request.addExtField("topicSysFlag", String.valueOf(topicConfig.getTopicSysFlag()));
+            }
+        }
+        // groupSysFlag
+        if (StringUtils.isNotEmpty(requestHeader.getConsumerGroup())) {
+            SubscriptionGroupConfig groupConfig = brokerController.getSubscriptionGroupManager().getSubscriptionGroupTable().get(requestHeader.getConsumerGroup());
+            if (groupConfig != null) {
+                request.addExtField("groupSysFlag", String.valueOf(groupConfig.getGroupSysFlag()));
+            }
+        }
         SelectMappedBufferResult selectMappedBufferResult = null;
         try {
             MessageId messageId = MessageDecoder.decodeMessageId(requestHeader.getMsgId());
@@ -1538,7 +1810,7 @@ public class AdminBrokerProcessor extends AsyncNettyRequestProcessor implements
         for (String topic : topics) {
             TopicConfig topicConfig = this.brokerController.getTopicConfigManager().selectTopicConfig(topic);
             if (null == topicConfig) {
-                log.warn("[cloneGroupOffset], topic config not exist, {}", topic);
+                LOGGER.warn("[cloneGroupOffset], topic config not exist, {}", topic);
                 continue;
             }
 
@@ -1548,7 +1820,9 @@ public class AdminBrokerProcessor extends AsyncNettyRequestProcessor implements
                     this.brokerController.getConsumerManager().findSubscriptionData(requestHeader.getSrcGroup(), topic);
                 if (this.brokerController.getConsumerManager().findSubscriptionDataCount(requestHeader.getSrcGroup()) > 0
                     && findSubscriptionData == null) {
-                    log.warn("[cloneGroupOffset], the consumer group[{}], topic[{}] not exist", requestHeader.getSrcGroup(), topic);
+                    LOGGER.warn(
+                        "AdminBrokerProcessor#cloneGroupOffset: topic does not exist in consumer group's "
+                            + "subscription, topic={}, consumer group={}", topic, requestHeader.getSrcGroup());
                     continue;
                 }
             }
@@ -1633,7 +1907,9 @@ public class AdminBrokerProcessor extends AsyncNettyRequestProcessor implements
                 ConsumeStats consumeStats = new ConsumeStats();
                 TopicConfig topicConfig = this.brokerController.getTopicConfigManager().selectTopicConfig(topic);
                 if (null == topicConfig) {
-                    log.warn("consumeStats, topic config not exist, {}", topic);
+                    LOGGER.warn(
+                        "AdminBrokerProcessor#fetchAllConsumeStatsInBroker: topic config does not exist, topic={}",
+                        topic);
                     continue;
                 }
 
@@ -1646,7 +1922,9 @@ public class AdminBrokerProcessor extends AsyncNettyRequestProcessor implements
 
                     if (null == findSubscriptionData
                         && this.brokerController.getConsumerManager().findSubscriptionDataCount(group) > 0) {
-                        log.warn("consumeStats, the consumer group[{}], topic[{}] not exist", group, topic);
+                        LOGGER.warn(
+                            "AdminBrokerProcessor#fetchAllConsumeStatsInBroker: topic does not exist in consumer "
+                                + "group's subscription, topic={}, consumer group={}", topic, group);
                         continue;
                     }
                 }
@@ -1658,8 +1936,9 @@ public class AdminBrokerProcessor extends AsyncNettyRequestProcessor implements
                     mq.setQueueId(i);
                     OffsetWrapper offsetWrapper = new OffsetWrapper();
                     long brokerOffset = this.brokerController.getMessageStore().getMaxOffsetInQueue(topic, i);
-                    if (brokerOffset < 0)
+                    if (brokerOffset < 0) {
                         brokerOffset = 0;
+                    }
                     long consumerOffset = this.brokerController.getConsumerOffsetManager().queryOffset(
                         group,
                         topic,
@@ -1700,6 +1979,15 @@ public class AdminBrokerProcessor extends AsyncNettyRequestProcessor implements
 
     private HashMap<String, String> prepareRuntimeInfo() {
         HashMap<String, String> runtimeInfo = this.brokerController.getMessageStore().getRuntimeInfo();
+
+        for (BrokerAttachedPlugin brokerAttachedPlugin : brokerController.getBrokerAttachedPlugins()) {
+            if (brokerAttachedPlugin != null) {
+                brokerAttachedPlugin.buildRuntimeInfo(runtimeInfo);
+            }
+        }
+
+        this.brokerController.getScheduleMessageService().buildRunningStats(runtimeInfo);
+        runtimeInfo.put("brokerActive", String.valueOf(this.brokerController.isSpecialServiceRunning()));
         runtimeInfo.put("brokerVersionDesc", MQVersion.getVersionDesc(MQVersion.CURRENT_VERSION));
         runtimeInfo.put("brokerVersion", String.valueOf(MQVersion.CURRENT_VERSION));
 
@@ -1713,8 +2001,25 @@ public class AdminBrokerProcessor extends AsyncNettyRequestProcessor implements
         runtimeInfo.put("msgGetTotalTodayMorning", String.valueOf(this.brokerController.getBrokerStats().getMsgGetTotalTodayMorning()));
         runtimeInfo.put("msgGetTotalTodayNow", String.valueOf(this.brokerController.getBrokerStats().getMsgGetTotalTodayNow()));
 
-        runtimeInfo.put("sendThreadPoolQueueSize", String.valueOf(this.brokerController.getSendThreadPoolQueue().size()));
+        runtimeInfo.put("dispatchBehindBytes", String.valueOf(this.brokerController.getMessageStore().dispatchBehindBytes()));
+        runtimeInfo.put("pageCacheLockTimeMills", String.valueOf(this.brokerController.getMessageStore().lockTimeMills()));
+
+        runtimeInfo.put("earliestMessageTimeStamp", String.valueOf(this.brokerController.getMessageStore().getEarliestMessageTime()));
+        runtimeInfo.put("startAcceptSendRequestTimeStamp", String.valueOf(this.brokerController.getBrokerConfig().getStartAcceptSendRequestTimeStamp()));
 
+        MessageStore messageStore = this.brokerController.getMessageStore();
+        runtimeInfo.put("remainTransientStoreBufferNumbs", String.valueOf(messageStore.remainTransientStoreBufferNumbs()));
+        if (this.brokerController.getMessageStoreConfig().isTransientStorePoolEnable()) {
+            runtimeInfo.put("remainHowManyDataToCommit", MixAll.humanReadableByteCount(messageStore.remainHowManyDataToCommit(), false));
+        }
+        runtimeInfo.put("remainHowManyDataToFlush", MixAll.humanReadableByteCount(messageStore.remainHowManyDataToFlush(), false));
+
+        java.io.File commitLogDir = new java.io.File(this.brokerController.getMessageStoreConfig().getStorePathRootDir());
+        if (commitLogDir.exists()) {
+            runtimeInfo.put("commitLogDirCapacity", String.format("Total : %s, Free : %s.", MixAll.humanReadableByteCount(commitLogDir.getTotalSpace(), false), MixAll.humanReadableByteCount(commitLogDir.getFreeSpace(), false)));
+        }
+
+        runtimeInfo.put("sendThreadPoolQueueSize", String.valueOf(this.brokerController.getSendThreadPoolQueue().size()));
         runtimeInfo.put("sendThreadPoolQueueCapacity",
             String.valueOf(this.brokerController.getBrokerConfig().getSendThreadPoolQueueCapacity()));
 
@@ -1722,36 +2027,22 @@ public class AdminBrokerProcessor extends AsyncNettyRequestProcessor implements
         runtimeInfo.put("pullThreadPoolQueueCapacity",
             String.valueOf(this.brokerController.getBrokerConfig().getPullThreadPoolQueueCapacity()));
 
+        runtimeInfo.put("litePullThreadPoolQueueSize", String.valueOf(brokerController.getLitePullThreadPoolQueue().size()));
+        runtimeInfo.put("litePullThreadPoolQueueCapacity",
+            String.valueOf(this.brokerController.getBrokerConfig().getLitePullThreadPoolQueueCapacity()));
+
         runtimeInfo.put("queryThreadPoolQueueSize", String.valueOf(this.brokerController.getQueryThreadPoolQueue().size()));
         runtimeInfo.put("queryThreadPoolQueueCapacity",
             String.valueOf(this.brokerController.getBrokerConfig().getQueryThreadPoolQueueCapacity()));
 
-        runtimeInfo.put("EndTransactionQueueSize", String.valueOf(this.brokerController.getEndTransactionThreadPoolQueue().size()));
-        runtimeInfo.put("EndTransactionThreadPoolQueueCapacity",
-            String.valueOf(this.brokerController.getBrokerConfig().getEndTransactionPoolQueueCapacity()));
-
-        runtimeInfo.put("dispatchBehindBytes", String.valueOf(this.brokerController.getMessageStore().dispatchBehindBytes()));
-        runtimeInfo.put("pageCacheLockTimeMills", String.valueOf(this.brokerController.getMessageStore().lockTimeMills()));
-
         runtimeInfo.put("sendThreadPoolQueueHeadWaitTimeMills", String.valueOf(this.brokerController.headSlowTimeMills4SendThreadPoolQueue()));
-        runtimeInfo.put("pullThreadPoolQueueHeadWaitTimeMills", String.valueOf(this.brokerController.headSlowTimeMills4PullThreadPoolQueue()));
+        runtimeInfo.put("pullThreadPoolQueueHeadWaitTimeMills", String.valueOf(brokerController.headSlowTimeMills4PullThreadPoolQueue()));
         runtimeInfo.put("queryThreadPoolQueueHeadWaitTimeMills", String.valueOf(this.brokerController.headSlowTimeMills4QueryThreadPoolQueue()));
+        runtimeInfo.put("litePullThreadPoolQueueHeadWaitTimeMills", String.valueOf(brokerController.headSlowTimeMills4LitePullThreadPoolQueue()));
 
-        runtimeInfo.put("earliestMessageTimeStamp", String.valueOf(this.brokerController.getMessageStore().getEarliestMessageTime()));
-        runtimeInfo.put("startAcceptSendRequestTimeStamp", String.valueOf(this.brokerController.getBrokerConfig().getStartAcceptSendRequestTimeStamp()));
-        if (this.brokerController.getMessageStore() instanceof DefaultMessageStore) {
-            DefaultMessageStore defaultMessageStore = (DefaultMessageStore) this.brokerController.getMessageStore();
-            runtimeInfo.put("remainTransientStoreBufferNumbs", String.valueOf(defaultMessageStore.remainTransientStoreBufferNumbs()));
-            if (defaultMessageStore.getMessageStoreConfig().isTransientStorePoolEnable()) {
-                runtimeInfo.put("remainHowManyDataToCommit", MixAll.humanReadableByteCount(defaultMessageStore.getCommitLog().remainHowManyDataToCommit(), false));
-            }
-            runtimeInfo.put("remainHowManyDataToFlush", MixAll.humanReadableByteCount(defaultMessageStore.getCommitLog().remainHowManyDataToFlush(), false));
-        }
-
-        java.io.File commitLogDir = new java.io.File(this.brokerController.getMessageStoreConfig().getStorePathRootDir());
-        if (commitLogDir.exists()) {
-            runtimeInfo.put("commitLogDirCapacity", String.format("Total : %s, Free : %s.", MixAll.humanReadableByteCount(commitLogDir.getTotalSpace(), false), MixAll.humanReadableByteCount(commitLogDir.getUsableSpace(), false)));
-        }
+        runtimeInfo.put("EndTransactionQueueSize", String.valueOf(this.brokerController.getEndTransactionThreadPoolQueue().size()));
+        runtimeInfo.put("EndTransactionThreadPoolQueueCapacity",
+            String.valueOf(this.brokerController.getBrokerConfig().getEndTransactionPoolQueueCapacity()));
 
         return runtimeInfo;
     }
@@ -1844,7 +2135,7 @@ public class AdminBrokerProcessor extends AsyncNettyRequestProcessor implements
             List<ConsumeQueueData> queues = new ArrayList<>();
             while (result.hasNext()) {
                 CqUnit cqUnit = result.next();
-                if (cqUnit.getQueueOffset() - requestHeader.getIndex() >=  requestHeader.getCount()) {
+                if (cqUnit.getQueueOffset() - requestHeader.getIndex() >= requestHeader.getCount()) {
                     break;
                 }
 
@@ -1898,18 +2189,18 @@ public class AdminBrokerProcessor extends AsyncNettyRequestProcessor implements
                 .putMessage(toMessageExtBrokerInner(msg));
             if (putMessageResult != null
                 && putMessageResult.getPutMessageStatus() == PutMessageStatus.PUT_OK) {
-                log.info(
+                LOGGER.info(
                     "Put message back to RMQ_SYS_TRANS_HALF_TOPIC. real topic={}",
                     msg.getUserProperty(MessageConst.PROPERTY_REAL_TOPIC));
                 response.setCode(ResponseCode.SUCCESS);
                 response.setRemark(null);
             } else {
-                log.error("Put message back to RMQ_SYS_TRANS_HALF_TOPIC failed.");
+                LOGGER.error("Put message back to RMQ_SYS_TRANS_HALF_TOPIC failed.");
                 response.setCode(ResponseCode.SYSTEM_ERROR);
                 response.setRemark("Put message back to RMQ_SYS_TRANS_HALF_TOPIC failed.");
             }
         } catch (Exception e) {
-            log.error("Exception was thrown when putting message back to RMQ_SYS_TRANS_HALF_TOPIC.");
+            LOGGER.error("Exception was thrown when putting message back to RMQ_SYS_TRANS_HALF_TOPIC.");
             response.setCode(ResponseCode.SYSTEM_ERROR);
             response.setRemark("Exception was thrown when putting message back to RMQ_SYS_TRANS_HALF_TOPIC.");
         } finally {
@@ -1946,7 +2237,7 @@ public class AdminBrokerProcessor extends AsyncNettyRequestProcessor implements
 
         TopicConfig topicConfig = this.brokerController.getTopicConfigManager().getTopicConfigTable().get(requestHeader.getTopic());
         if (topicConfig == null) {
-            log.error("No topic in this broker, client: {} topic: {}", ctx.channel().remoteAddress(), requestHeader.getTopic());
+            LOGGER.error("No topic in this broker, client: {} topic: {}", ctx.channel().remoteAddress(), requestHeader.getTopic());
             //be care of the response code, should set "not-exist" explictly
             response.setCode(ResponseCode.TOPIC_NOT_EXIST);
             response.setRemark("No topic in this broker. topic: " + requestHeader.getTopic());
@@ -1960,7 +2251,7 @@ public class AdminBrokerProcessor extends AsyncNettyRequestProcessor implements
         try {
             response.setBody(content.getBytes(MixAll.DEFAULT_CHARSET));
         } catch (UnsupportedEncodingException e) {
-            log.error("UnsupportedEncodingException getTopicConfig: topic=" + topicConfig.getTopicName(), e);
+            LOGGER.error("UnsupportedEncodingException getTopicConfig: topic=" + topicConfig.getTopicName(), e);
 
             response.setCode(ResponseCode.SYSTEM_ERROR);
             response.setRemark("UnsupportedEncodingException " + e.getMessage());
@@ -1971,4 +2262,84 @@ public class AdminBrokerProcessor extends AsyncNettyRequestProcessor implements
 
         return response;
     }
+
+    private RemotingCommand notifyMinBrokerIdChange(ChannelHandlerContext ctx,
+        RemotingCommand request) throws RemotingCommandException {
+        NotifyMinBrokerIdChangeRequestHeader requestHeader = request.decodeCommandCustomHeader(NotifyMinBrokerIdChangeRequestHeader.class);
+
+        RemotingCommand response = RemotingCommand.createResponseCommand(null);
+
+        LOGGER.warn("min broker id changed, prev {}, new {}", this.brokerController.getMinBrokerIdInGroup(), requestHeader.getMinBrokerId());
+
+        this.brokerController.updateMinBroker(requestHeader.getMinBrokerId(), requestHeader.getMinBrokerAddr(),
+            requestHeader.getOfflineBrokerAddr(),
+            requestHeader.getHaBrokerAddr());
+
+        response.setCode(ResponseCode.SUCCESS);
+        response.setRemark(null);
+
+        return response;
+    }
+
+    private RemotingCommand updateBrokerHaInfo(ChannelHandlerContext ctx,
+        RemotingCommand request) throws RemotingCommandException {
+        RemotingCommand response = RemotingCommand.createResponseCommand(ExchangeHAInfoResponseHeader.class);
+
+        ExchangeHAInfoRequestHeader requestHeader = request.decodeCommandCustomHeader(ExchangeHAInfoRequestHeader.class);
+        if (requestHeader.getMasterHaAddress() != null) {
+            this.brokerController.getMessageStore().updateHaMasterAddress(requestHeader.getMasterHaAddress());
+            this.brokerController.getMessageStore().updateMasterAddress(requestHeader.getMasterAddress());
+            if (this.brokerController.getMessageStore().getMasterFlushedOffset() == 0
+                && this.brokerController.getMessageStoreConfig().isSyncMasterFlushOffsetWhenStartup()) {
+                LOGGER.info("Set master flush offset in slave to {}", requestHeader.getMasterFlushOffset());
+                this.brokerController.getMessageStore().setMasterFlushedOffset(requestHeader.getMasterFlushOffset());
+            }
+        } else if (this.brokerController.getBrokerConfig().getBrokerId() == MixAll.MASTER_ID) {
+            final ExchangeHAInfoResponseHeader responseHeader = (ExchangeHAInfoResponseHeader) response.readCustomHeader();
+            responseHeader.setMasterHaAddress(this.brokerController.getHAServerAddr());
+            responseHeader.setMasterFlushOffset(this.brokerController.getMessageStore().getBrokerInitMaxOffset());
+            responseHeader.setMasterAddress(this.brokerController.getBrokerAddr());
+        }
+
+        response.setCode(ResponseCode.SUCCESS);
+        response.setRemark(null);
+
+        return response;
+    }
+
+    private RemotingCommand getBrokerHaStatus(ChannelHandlerContext ctx, RemotingCommand request) {
+        final RemotingCommand response = RemotingCommand.createResponseCommand(null);
+
+        HARuntimeInfo runtimeInfo = this.brokerController.getMessageStore().getHARuntimeInfo();
+
+        if (runtimeInfo != null) {
+            byte[] body = runtimeInfo.encode();
+            response.setBody(body);
+            response.setCode(ResponseCode.SUCCESS);
+            response.setRemark(null);
+        } else {
+            response.setCode(ResponseCode.SYSTEM_ERROR);
+            response.setRemark("Can not get HARuntimeInfo, may be duplicationEnable is true");
+        }
+
+        return response;
+    }
+
+    private RemotingCommand resetMasterFlushOffset(ChannelHandlerContext ctx,
+        RemotingCommand request) throws RemotingCommandException {
+        final RemotingCommand response = RemotingCommand.createResponseCommand(null);
+
+        if (this.brokerController.getBrokerConfig().getBrokerId() != MixAll.MASTER_ID) {
+
+            ResetMasterFlushOffsetHeader requestHeader = request.decodeCommandCustomHeader(ResetMasterFlushOffsetHeader.class);
+
+            if (requestHeader.getMasterFlushOffset() != null) {
+                this.brokerController.getMessageStore().setMasterFlushedOffset(requestHeader.getMasterFlushOffset());
+            }
+        }
+
+        response.setCode(ResponseCode.SUCCESS);
+        response.setRemark(null);
+        return response;
+    }
 }
diff --git a/broker/src/main/java/org/apache/rocketmq/broker/processor/ChangeInvisibleTimeProcessor.java b/broker/src/main/java/org/apache/rocketmq/broker/processor/ChangeInvisibleTimeProcessor.java
index e40a5e8..991a705 100644
--- a/broker/src/main/java/org/apache/rocketmq/broker/processor/ChangeInvisibleTimeProcessor.java
+++ b/broker/src/main/java/org/apache/rocketmq/broker/processor/ChangeInvisibleTimeProcessor.java
@@ -38,7 +38,7 @@ import org.apache.rocketmq.remoting.common.RemotingHelper;
 import org.apache.rocketmq.remoting.exception.RemotingCommandException;
 import org.apache.rocketmq.remoting.netty.NettyRequestProcessor;
 import org.apache.rocketmq.remoting.protocol.RemotingCommand;
-import org.apache.rocketmq.store.MessageExtBrokerInner;
+import org.apache.rocketmq.common.message.MessageExtBrokerInner;
 import org.apache.rocketmq.store.PutMessageResult;
 import org.apache.rocketmq.store.PutMessageStatus;
 import org.apache.rocketmq.store.pop.AckMsg;
@@ -47,16 +47,16 @@ import org.apache.rocketmq.store.pop.PopCheckPoint;
 public class ChangeInvisibleTimeProcessor implements NettyRequestProcessor {
     private static final InternalLogger POP_LOGGER = InternalLoggerFactory.getLogger(LoggerName.ROCKETMQ_POP_LOGGER_NAME);
     private final BrokerController brokerController;
-    private String reviveTopic;
+    private final String reviveTopic;
 
     public ChangeInvisibleTimeProcessor(final BrokerController brokerController) {
         this.brokerController = brokerController;
-        this.reviveTopic = PopAckConstants.REVIVE_TOPIC + this.brokerController.getBrokerConfig().getBrokerClusterName();
-
+        this.reviveTopic = PopAckConstants.buildClusterReviveTopic(this.brokerController.getBrokerConfig().getBrokerClusterName());
     }
 
     @Override
-    public RemotingCommand processRequest(final ChannelHandlerContext ctx, RemotingCommand request) throws RemotingCommandException {
+    public RemotingCommand processRequest(final ChannelHandlerContext ctx,
+        RemotingCommand request) throws RemotingCommandException {
         return this.processRequest(ctx.channel(), request, true);
     }
 
@@ -65,7 +65,8 @@ public class ChangeInvisibleTimeProcessor implements NettyRequestProcessor {
         return false;
     }
 
-    private RemotingCommand processRequest(final Channel channel, RemotingCommand request, boolean brokerAllowSuspend) throws RemotingCommandException {
+    private RemotingCommand processRequest(final Channel channel, RemotingCommand request,
+        boolean brokerAllowSuspend) throws RemotingCommandException {
         final ChangeInvisibleTimeRequestHeader requestHeader = (ChangeInvisibleTimeRequestHeader) request.decodeCommandCustomHeader(ChangeInvisibleTimeRequestHeader.class);
         RemotingCommand response = RemotingCommand.createResponseCommand(ChangeInvisibleTimeResponseHeader.class);
         response.setCode(ResponseCode.SUCCESS);
@@ -98,7 +99,7 @@ public class ChangeInvisibleTimeProcessor implements NettyRequestProcessor {
 
         // add new ck
         long now = System.currentTimeMillis();
-        PutMessageResult ckResult = appendCheckPoint(requestHeader, ExtraInfoUtil.getReviveQid(extraInfo), requestHeader.getQueueId(), requestHeader.getOffset(), now);
+        PutMessageResult ckResult = appendCheckPoint(requestHeader, ExtraInfoUtil.getReviveQid(extraInfo), requestHeader.getQueueId(), requestHeader.getOffset(), now, ExtraInfoUtil.getBrokerName(extraInfo));
 
         if (ckResult.getPutMessageStatus() != PutMessageStatus.PUT_OK
             && ckResult.getPutMessageStatus() != PutMessageStatus.FLUSH_DISK_TIMEOUT
@@ -133,9 +134,13 @@ public class ChangeInvisibleTimeProcessor implements NettyRequestProcessor {
         ackMsg.setTopic(requestHeader.getTopic());
         ackMsg.setQueueId(requestHeader.getQueueId());
         ackMsg.setPopTime(ExtraInfoUtil.getPopTime(extraInfo));
+        ackMsg.setBrokerName(ExtraInfoUtil.getBrokerName(extraInfo));
 
         int rqId = ExtraInfoUtil.getReviveQid(extraInfo);
 
+        this.brokerController.getBrokerStatsManager().incBrokerAckNums(1);
+        this.brokerController.getBrokerStatsManager().incGroupAckNums(requestHeader.getConsumerGroup(), requestHeader.getTopic(), 1);
+
         if (brokerController.getPopMessageProcessor().getPopBufferMergeService().addAk(rqId, ackMsg)) {
             return;
         }
@@ -150,7 +155,7 @@ public class ChangeInvisibleTimeProcessor implements NettyRequestProcessor {
         MsgUtil.setMessageDeliverTime(this.brokerController, msgInner, ExtraInfoUtil.getPopTime(extraInfo) + ExtraInfoUtil.getInvisibleTime(extraInfo));
         msgInner.getProperties().put(MessageConst.PROPERTY_UNIQ_CLIENT_MESSAGE_ID_KEYIDX, PopMessageProcessor.genAckUniqueId(ackMsg));
         msgInner.setPropertiesString(MessageDecoder.messageProperties2String(msgInner.getProperties()));
-        PutMessageResult putMessageResult = this.brokerController.getMessageStore().putMessage(msgInner);
+        PutMessageResult putMessageResult = this.brokerController.getEscapeBridge().putMessageToSpecificQueue(msgInner);
         if (putMessageResult.getPutMessageStatus() != PutMessageStatus.PUT_OK
             && putMessageResult.getPutMessageStatus() != PutMessageStatus.FLUSH_DISK_TIMEOUT
             && putMessageResult.getPutMessageStatus() != PutMessageStatus.FLUSH_SLAVE_TIMEOUT
@@ -159,7 +164,8 @@ public class ChangeInvisibleTimeProcessor implements NettyRequestProcessor {
         }
     }
 
-    private PutMessageResult appendCheckPoint(final ChangeInvisibleTimeRequestHeader requestHeader, int reviveQid, int queueId, long offset, long popTime) {
+    private PutMessageResult appendCheckPoint(final ChangeInvisibleTimeRequestHeader requestHeader, int reviveQid,
+        int queueId, long offset, long popTime, String brokerName) {
         // add check point msg to revive log
         MessageExtBrokerInner msgInner = new MessageExtBrokerInner();
         msgInner.setTopic(reviveTopic);
@@ -187,7 +193,12 @@ public class ChangeInvisibleTimeProcessor implements NettyRequestProcessor {
 
         if (brokerController.getBrokerConfig().isEnablePopLog()) {
             POP_LOGGER.info("change Invisible , appendCheckPoint, topic {}, queueId {},reviveId {}, cid {}, startOffset {}, rt {}, result {}", requestHeader.getTopic(), queueId, reviveQid, requestHeader.getConsumerGroup(), offset,
-                    ck.getReviveTime(), putMessageResult);
+                ck.getReviveTime(), putMessageResult);
+        }
+
+        if (putMessageResult != null && putMessageResult.isOk()) {
+            this.brokerController.getBrokerStatsManager().incBrokerCkNums(1);
+            this.brokerController.getBrokerStatsManager().incGroupCkNums(requestHeader.getConsumerGroup(), requestHeader.getTopic(), 1);
         }
 
         return putMessageResult;
diff --git a/broker/src/main/java/org/apache/rocketmq/broker/processor/ClientManageProcessor.java b/broker/src/main/java/org/apache/rocketmq/broker/processor/ClientManageProcessor.java
index aa7d0a3..3a453b6 100644
--- a/broker/src/main/java/org/apache/rocketmq/broker/processor/ClientManageProcessor.java
+++ b/broker/src/main/java/org/apache/rocketmq/broker/processor/ClientManageProcessor.java
@@ -28,6 +28,7 @@ import org.apache.rocketmq.common.protocol.ResponseCode;
 import org.apache.rocketmq.common.protocol.body.CheckClientRequestBody;
 import org.apache.rocketmq.common.protocol.header.UnregisterClientRequestHeader;
 import org.apache.rocketmq.common.protocol.header.UnregisterClientResponseHeader;
+import org.apache.rocketmq.common.protocol.heartbeat.ConsumeType;
 import org.apache.rocketmq.common.protocol.heartbeat.ConsumerData;
 import org.apache.rocketmq.common.protocol.heartbeat.HeartbeatData;
 import org.apache.rocketmq.common.protocol.heartbeat.ProducerData;
@@ -40,11 +41,10 @@ import org.apache.rocketmq.logging.InternalLoggerFactory;
 import org.apache.rocketmq.remoting.common.RemotingHelper;
 import org.apache.rocketmq.remoting.exception.RemotingCommandException;
 import org.apache.rocketmq.remoting.netty.NettyRequestProcessor;
-import org.apache.rocketmq.remoting.netty.AsyncNettyRequestProcessor;
 import org.apache.rocketmq.remoting.protocol.RemotingCommand;
 
-public class ClientManageProcessor extends AsyncNettyRequestProcessor implements NettyRequestProcessor {
-    private static final InternalLogger log = InternalLoggerFactory.getLogger(LoggerName.BROKER_LOGGER_NAME);
+public class ClientManageProcessor implements NettyRequestProcessor {
+    private static final InternalLogger LOGGER = InternalLoggerFactory.getLogger(LoggerName.BROKER_LOGGER_NAME);
     private final BrokerController brokerController;
 
     public ClientManageProcessor(final BrokerController brokerController) {
@@ -82,39 +82,54 @@ public class ClientManageProcessor extends AsyncNettyRequestProcessor implements
             request.getVersion()
         );
 
-        for (ConsumerData data : heartbeatData.getConsumerDataSet()) {
+        for (ConsumerData consumerData : heartbeatData.getConsumerDataSet()) {
+            //Reject the PullConsumer
+            if (brokerController.getBrokerConfig().isRejectPullConsumerEnable()) {
+                if (ConsumeType.CONSUME_ACTIVELY == consumerData.getConsumeType()) {
+                    continue;
+                }
+            }
+
+            boolean hasOrderTopicSub = false;
+
+            for (final SubscriptionData subscriptionData : consumerData.getSubscriptionDataSet()) {
+                if (this.brokerController.getTopicConfigManager().isOrderTopic(subscriptionData.getTopic())) {
+                    hasOrderTopicSub = true;
+                    break;
+                }
+            }
+
             SubscriptionGroupConfig subscriptionGroupConfig =
                 this.brokerController.getSubscriptionGroupManager().findSubscriptionGroupConfig(
-                    data.getGroupName());
+                    consumerData.getGroupName());
             boolean isNotifyConsumerIdsChangedEnable = true;
             if (null != subscriptionGroupConfig) {
                 isNotifyConsumerIdsChangedEnable = subscriptionGroupConfig.isNotifyConsumerIdsChangedEnable();
                 int topicSysFlag = 0;
-                if (data.isUnitMode()) {
+                if (consumerData.isUnitMode()) {
                     topicSysFlag = TopicSysFlag.buildSysFlag(false, true);
                 }
-                String newTopic = MixAll.getRetryTopic(data.getGroupName());
+                String newTopic = MixAll.getRetryTopic(consumerData.getGroupName());
                 this.brokerController.getTopicConfigManager().createTopicInSendMessageBackMethod(
                     newTopic,
                     subscriptionGroupConfig.getRetryQueueNums(),
-                    PermName.PERM_WRITE | PermName.PERM_READ, topicSysFlag);
+                    PermName.PERM_WRITE | PermName.PERM_READ, hasOrderTopicSub, topicSysFlag);
             }
 
             boolean changed = this.brokerController.getConsumerManager().registerConsumer(
-                data.getGroupName(),
+                consumerData.getGroupName(),
                 clientChannelInfo,
-                data.getConsumeType(),
-                data.getMessageModel(),
-                data.getConsumeFromWhere(),
-                data.getSubscriptionDataSet(),
+                consumerData.getConsumeType(),
+                consumerData.getMessageModel(),
+                consumerData.getConsumeFromWhere(),
+                consumerData.getSubscriptionDataSet(),
                 isNotifyConsumerIdsChangedEnable
             );
 
             if (changed) {
-                log.info("registerConsumer info changed {} {}",
-                    data.toString(),
-                    RemotingHelper.parseChannelRemoteAddr(ctx.channel())
-                );
+                LOGGER.info(
+                    "ClientManageProcessor: registerConsumer info changed, SDK address={}, consumerData={}",
+                    RemotingHelper.parseChannelRemoteAddr(ctx.channel()), consumerData.toString());
             }
         }
 
@@ -190,7 +205,7 @@ public class ClientManageProcessor extends AsyncNettyRequestProcessor implements
             try {
                 FilterFactory.INSTANCE.get(subscriptionData.getExpressionType()).compile(subscriptionData.getSubString());
             } catch (Exception e) {
-                log.warn("Client {}@{} filter message, but failed to compile expression! sub={}, error={}",
+                LOGGER.warn("Client {}@{} filter message, but failed to compile expression! sub={}, error={}",
                     requestBody.getClientId(), requestBody.getGroup(), requestBody.getSubscriptionData(), e.getMessage());
                 response.setCode(ResponseCode.SUBSCRIPTION_PARSE_FAILED);
                 response.setRemark(e.getMessage());
diff --git a/broker/src/main/java/org/apache/rocketmq/broker/processor/ConsumerManageProcessor.java b/broker/src/main/java/org/apache/rocketmq/broker/processor/ConsumerManageProcessor.java
index 31a7993..afa1aa4 100644
--- a/broker/src/main/java/org/apache/rocketmq/broker/processor/ConsumerManageProcessor.java
+++ b/broker/src/main/java/org/apache/rocketmq/broker/processor/ConsumerManageProcessor.java
@@ -17,6 +17,7 @@
 package org.apache.rocketmq.broker.processor;
 
 import io.netty.channel.ChannelHandlerContext;
+import java.util.Set;
 import org.apache.rocketmq.broker.BrokerController;
 import org.apache.rocketmq.broker.client.ConsumerGroupInfo;
 import org.apache.rocketmq.common.constant.LoggerName;
@@ -40,7 +41,6 @@ import org.apache.rocketmq.logging.InternalLogger;
 import org.apache.rocketmq.logging.InternalLoggerFactory;
 import org.apache.rocketmq.remoting.common.RemotingHelper;
 import org.apache.rocketmq.remoting.exception.RemotingCommandException;
-import org.apache.rocketmq.remoting.netty.AsyncNettyRequestProcessor;
 import org.apache.rocketmq.remoting.netty.NettyRequestProcessor;
 import org.apache.rocketmq.remoting.protocol.RemotingCommand;
 
@@ -48,9 +48,8 @@ import java.util.List;
 
 import static org.apache.rocketmq.remoting.protocol.RemotingCommand.buildErrorResponse;
 
-public class ConsumerManageProcessor extends AsyncNettyRequestProcessor implements NettyRequestProcessor {
-    private static final InternalLogger log = InternalLoggerFactory.getLogger(LoggerName.BROKER_LOGGER_NAME);
-
+public class ConsumerManageProcessor implements NettyRequestProcessor {
+    private static final InternalLogger LOGGER = InternalLoggerFactory.getLogger(LoggerName.BROKER_LOGGER_NAME);
     private final BrokerController brokerController;
 
     public ConsumerManageProcessor(final BrokerController brokerController) {
@@ -99,11 +98,11 @@ public class ConsumerManageProcessor extends AsyncNettyRequestProcessor implemen
                 response.setRemark(null);
                 return response;
             } else {
-                log.warn("getAllClientId failed, {} {}", requestHeader.getConsumerGroup(),
+                LOGGER.warn("getAllClientId failed, {} {}", requestHeader.getConsumerGroup(),
                     RemotingHelper.parseChannelRemoteAddr(ctx.channel()));
             }
         } else {
-            log.warn("getConsumerGroupInfo failed, {} {}", requestHeader.getConsumerGroup(),
+            LOGGER.warn("getConsumerGroupInfo failed, {} {}", requestHeader.getConsumerGroup(),
                 RemotingHelper.parseChannelRemoteAddr(ctx.channel()));
         }
 
@@ -156,10 +155,17 @@ public class ConsumerManageProcessor extends AsyncNettyRequestProcessor implemen
         if (rewriteResult != null) {
             return rewriteResult;
         }
-        this.brokerController.getConsumerOffsetManager().commitOffset(RemotingHelper.parseChannelRemoteAddr(ctx.channel()), requestHeader.getConsumerGroup(),
-            requestHeader.getTopic(), requestHeader.getQueueId(), requestHeader.getCommitOffset());
-        response.setCode(ResponseCode.SUCCESS);
-        response.setRemark(null);
+        Set<String> topicSets = this.brokerController.getTopicConfigManager().getTopicConfigTable().keySet();
+        if (topicSets.contains(requestHeader.getTopic())) {
+            this.brokerController.getConsumerOffsetManager().commitOffset(RemotingHelper.parseChannelRemoteAddr(ctx.channel()), requestHeader.getConsumerGroup(),
+                requestHeader.getTopic(), requestHeader.getQueueId(), requestHeader.getCommitOffset());
+            response.setCode(ResponseCode.SUCCESS);
+            response.setRemark(null);
+        } else {
+            response.setCode(ResponseCode.TOPIC_NOT_EXIST);
+            response.setRemark("Topic " + requestHeader.getTopic() + " not exist!");
+        }
+
         return response;
     }
 
diff --git a/broker/src/main/java/org/apache/rocketmq/broker/processor/DefaultPullMessageResultHandler.java b/broker/src/main/java/org/apache/rocketmq/broker/processor/DefaultPullMessageResultHandler.java
new file mode 100644
index 0000000..0e6a112
--- /dev/null
+++ b/broker/src/main/java/org/apache/rocketmq/broker/processor/DefaultPullMessageResultHandler.java
@@ -0,0 +1,244 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.rocketmq.broker.processor;
+
+import io.netty.channel.Channel;
+import io.netty.channel.ChannelFuture;
+import io.netty.channel.ChannelFutureListener;
+import io.netty.channel.FileRegion;
+import org.apache.rocketmq.broker.BrokerController;
+import org.apache.rocketmq.broker.longpolling.PullRequest;
+import org.apache.rocketmq.broker.pagecache.ManyMessageTransfer;
+import org.apache.rocketmq.broker.plugin.PullMessageResultHandler;
+import org.apache.rocketmq.common.TopicFilterType;
+import org.apache.rocketmq.common.constant.LoggerName;
+import org.apache.rocketmq.common.message.MessageDecoder;
+import org.apache.rocketmq.common.message.MessageQueue;
+import org.apache.rocketmq.common.protocol.ResponseCode;
+import org.apache.rocketmq.common.protocol.header.PullMessageRequestHeader;
+import org.apache.rocketmq.common.protocol.header.PullMessageResponseHeader;
+import org.apache.rocketmq.common.protocol.heartbeat.SubscriptionData;
+import org.apache.rocketmq.common.protocol.topic.OffsetMovedEvent;
+import org.apache.rocketmq.common.subscription.SubscriptionGroupConfig;
+import org.apache.rocketmq.common.sysflag.MessageSysFlag;
+import org.apache.rocketmq.common.sysflag.PullSysFlag;
+import org.apache.rocketmq.common.topic.TopicValidator;
+import org.apache.rocketmq.logging.InternalLogger;
+import org.apache.rocketmq.logging.InternalLoggerFactory;
+import org.apache.rocketmq.remoting.common.RemotingUtil;
+import org.apache.rocketmq.remoting.protocol.RemotingCommand;
+import org.apache.rocketmq.store.GetMessageResult;
+import org.apache.rocketmq.common.message.MessageExtBrokerInner;
+import org.apache.rocketmq.store.MessageFilter;
+import org.apache.rocketmq.store.PutMessageResult;
+import org.apache.rocketmq.store.config.BrokerRole;
+
+import java.nio.ByteBuffer;
+import java.util.List;
+
+public class DefaultPullMessageResultHandler implements PullMessageResultHandler {
+
+    protected static final InternalLogger log = InternalLoggerFactory.getLogger(LoggerName.BROKER_LOGGER_NAME);
+    protected final BrokerController brokerController;
+
+    public DefaultPullMessageResultHandler(final BrokerController brokerController) {
+        this.brokerController = brokerController;
+    }
+
+    @Override
+    public RemotingCommand handle(final GetMessageResult getMessageResult,
+                                  final RemotingCommand request,
+                                  final PullMessageRequestHeader requestHeader,
+                                  final Channel channel,
+                                  final SubscriptionData subscriptionData,
+                                  final SubscriptionGroupConfig subscriptionGroupConfig,
+                                  final boolean brokerAllowSuspend,
+                                  final MessageFilter messageFilter,
+                                  RemotingCommand response) {
+
+        final PullMessageResponseHeader responseHeader = (PullMessageResponseHeader) response.readCustomHeader();
+
+        switch (response.getCode()) {
+            case ResponseCode.SUCCESS:
+                this.brokerController.getBrokerStatsManager().incGroupGetNums(requestHeader.getConsumerGroup(), requestHeader.getTopic(),
+                        getMessageResult.getMessageCount());
+
+                this.brokerController.getBrokerStatsManager().incGroupGetSize(requestHeader.getConsumerGroup(), requestHeader.getTopic(),
+                        getMessageResult.getBufferTotalSize());
+
+                this.brokerController.getBrokerStatsManager().incBrokerGetNums(getMessageResult.getMessageCount());
+
+                if (!channelIsWritable(channel, requestHeader)) {
+                    getMessageResult.release();
+                    //ignore pull request
+                    return null;
+                }
+
+                if (this.brokerController.getBrokerConfig().isTransferMsgByHeap()) {
+
+                    final long beginTimeMills = this.brokerController.getMessageStore().now();
+                    final byte[] r = this.readGetMessageResult(getMessageResult, requestHeader.getConsumerGroup(), requestHeader.getTopic(), requestHeader.getQueueId());
+                    this.brokerController.getBrokerStatsManager().incGroupGetLatency(requestHeader.getConsumerGroup(),
+                            requestHeader.getTopic(), requestHeader.getQueueId(),
+                            (int) (this.brokerController.getMessageStore().now() - beginTimeMills));
+                    response.setBody(r);
+                    return response;
+                } else {
+                    try {
+                        FileRegion fileRegion =
+                                new ManyMessageTransfer(response.encodeHeader(getMessageResult.getBufferTotalSize()), getMessageResult);
+                        channel.writeAndFlush(fileRegion).addListener(new ChannelFutureListener() {
+                            @Override
+                            public void operationComplete(ChannelFuture future) throws Exception {
+                                getMessageResult.release();
+                                if (!future.isSuccess()) {
+                                    log.error("Fail to transfer messages from page cache to {}", channel.remoteAddress(), future.cause());
+                                }
+                            }
+                        });
+                    } catch (Throwable e) {
+                        log.error("Error occurred when transferring messages from page cache", e);
+                        getMessageResult.release();
+                    }
+                    return null;
+                }
+            case ResponseCode.PULL_NOT_FOUND:
+                final boolean hasSuspendFlag = PullSysFlag.hasSuspendFlag(requestHeader.getSysFlag());
+                final long suspendTimeoutMillisLong = hasSuspendFlag ? requestHeader.getSuspendTimeoutMillis() : 0;
+
+                if (brokerAllowSuspend && hasSuspendFlag) {
+                    long pollingTimeMills = suspendTimeoutMillisLong;
+                    if (!this.brokerController.getBrokerConfig().isLongPollingEnable()) {
+                        pollingTimeMills = this.brokerController.getBrokerConfig().getShortPollingTimeMills();
+                    }
+
+                    String topic = requestHeader.getTopic();
+                    long offset = requestHeader.getQueueOffset();
+                    int queueId = requestHeader.getQueueId();
+                    PullRequest pullRequest = new PullRequest(request, channel, pollingTimeMills,
+                            this.brokerController.getMessageStore().now(), offset, subscriptionData, messageFilter);
+                    this.brokerController.getPullRequestHoldService().suspendPullRequest(topic, queueId, pullRequest);
+                    return null;
+                }
+            case ResponseCode.PULL_RETRY_IMMEDIATELY:
+                break;
+            case ResponseCode.PULL_OFFSET_MOVED:
+                if (this.brokerController.getMessageStoreConfig().getBrokerRole() != BrokerRole.SLAVE
+                        || this.brokerController.getMessageStoreConfig().isOffsetCheckInSlave()) {
+                    MessageQueue mq = new MessageQueue();
+                    mq.setTopic(requestHeader.getTopic());
+                    mq.setQueueId(requestHeader.getQueueId());
+                    mq.setBrokerName(this.brokerController.getBrokerConfig().getBrokerName());
+
+                    OffsetMovedEvent event = new OffsetMovedEvent();
+                    event.setConsumerGroup(requestHeader.getConsumerGroup());
+                    event.setMessageQueue(mq);
+                    event.setOffsetRequest(requestHeader.getQueueOffset());
+                    event.setOffsetNew(getMessageResult.getNextBeginOffset());
+                    log.warn(
+                            "PULL_OFFSET_MOVED:correction offset. topic={}, groupId={}, requestOffset={}, newOffset={}, suggestBrokerId={}",
+                            requestHeader.getTopic(), requestHeader.getConsumerGroup(), event.getOffsetRequest(), event.getOffsetNew(),
+                            responseHeader.getSuggestWhichBrokerId());
+                } else {
+                    responseHeader.setSuggestWhichBrokerId(subscriptionGroupConfig.getBrokerId());
+                    response.setCode(ResponseCode.PULL_RETRY_IMMEDIATELY);
+                    log.warn("PULL_OFFSET_MOVED:none correction. topic={}, groupId={}, requestOffset={}, suggestBrokerId={}",
+                            requestHeader.getTopic(), requestHeader.getConsumerGroup(), requestHeader.getQueueOffset(),
+                            responseHeader.getSuggestWhichBrokerId());
+                }
+
+                break;
+            default:
+                log.warn("[BUG] impossible result code of get message: {}", response.getCode());
+                assert false;
+        }
+
+        return response;
+    }
+
+    private boolean channelIsWritable(Channel channel, PullMessageRequestHeader requestHeader) {
+        if (this.brokerController.getBrokerConfig().isNetWorkFlowController()) {
+            if (!channel.isWritable()) {
+                log.warn("channel {} not writable ,cid {}", channel.remoteAddress(), requestHeader.getConsumerGroup());
+                return false;
+            }
+
+        }
+        return true;
+    }
+
+    protected byte[] readGetMessageResult(final GetMessageResult getMessageResult, final String group, final String topic,
+        final int queueId) {
+        final ByteBuffer byteBuffer = ByteBuffer.allocate(getMessageResult.getBufferTotalSize());
+
+        long storeTimestamp = 0;
+        try {
+            List<ByteBuffer> messageBufferList = getMessageResult.getMessageBufferList();
+            for (ByteBuffer bb : messageBufferList) {
+
+                byteBuffer.put(bb);
+                int sysFlag = bb.getInt(MessageDecoder.SYSFLAG_POSITION);
+//                bornhost has the IPv4 ip if the MessageSysFlag.BORNHOST_V6_FLAG bit of sysFlag is 0
+//                IPv4 host = ip(4 byte) + port(4 byte); IPv6 host = ip(16 byte) + port(4 byte)
+                int bornhostLength = (sysFlag & MessageSysFlag.BORNHOST_V6_FLAG) == 0 ? 8 : 20;
+                int msgStoreTimePos = 4 // 1 TOTALSIZE
+                    + 4 // 2 MAGICCODE
+                    + 4 // 3 BODYCRC
+                    + 4 // 4 QUEUEID
+                    + 4 // 5 FLAG
+                    + 8 // 6 QUEUEOFFSET
+                    + 8 // 7 PHYSICALOFFSET
+                    + 4 // 8 SYSFLAG
+                    + 8 // 9 BORNTIMESTAMP
+                    + bornhostLength; // 10 BORNHOST
+                storeTimestamp = bb.getLong(msgStoreTimePos);
+            }
+        } finally {
+            getMessageResult.release();
+        }
+
+        this.brokerController.getBrokerStatsManager().recordDiskFallBehindTime(group, topic, queueId, this.brokerController.getMessageStore().now() - storeTimestamp);
+        return byteBuffer.array();
+    }
+
+    protected void generateOffsetMovedEvent(final OffsetMovedEvent event) {
+        try {
+            MessageExtBrokerInner msgInner = new MessageExtBrokerInner();
+            msgInner.setTopic(TopicValidator.RMQ_SYS_OFFSET_MOVED_EVENT);
+            msgInner.setTags(event.getConsumerGroup());
+            msgInner.setDelayTimeLevel(0);
+            msgInner.setKeys(event.getConsumerGroup());
+            msgInner.setBody(event.encode());
+            msgInner.setFlag(0);
+            msgInner.setPropertiesString(MessageDecoder.messageProperties2String(msgInner.getProperties()));
+            msgInner.setTagsCode(MessageExtBrokerInner.tagsString2tagsCode(TopicFilterType.SINGLE_TAG, msgInner.getTags()));
+
+            msgInner.setQueueId(0);
+            msgInner.setSysFlag(0);
+            msgInner.setBornTimestamp(System.currentTimeMillis());
+            msgInner.setBornHost(RemotingUtil.string2SocketAddress(this.brokerController.getBrokerAddr()));
+            msgInner.setStoreHost(msgInner.getBornHost());
+
+            msgInner.setReconsumeTimes(0);
+
+            PutMessageResult putMessageResult = this.brokerController.getMessageStore().putMessage(msgInner);
+        } catch (Exception e) {
+            log.warn(String.format("generateOffsetMovedEvent Exception, %s", event.toString()), e);
+        }
+    }
+}
diff --git a/broker/src/main/java/org/apache/rocketmq/broker/processor/EndTransactionProcessor.java b/broker/src/main/java/org/apache/rocketmq/broker/processor/EndTransactionProcessor.java
index 41e7df3..8ac52a4 100644
--- a/broker/src/main/java/org/apache/rocketmq/broker/processor/EndTransactionProcessor.java
+++ b/broker/src/main/java/org/apache/rocketmq/broker/processor/EndTransactionProcessor.java
@@ -32,17 +32,16 @@ import org.apache.rocketmq.logging.InternalLogger;
 import org.apache.rocketmq.logging.InternalLoggerFactory;
 import org.apache.rocketmq.remoting.common.RemotingHelper;
 import org.apache.rocketmq.remoting.exception.RemotingCommandException;
-import org.apache.rocketmq.remoting.netty.AsyncNettyRequestProcessor;
 import org.apache.rocketmq.remoting.netty.NettyRequestProcessor;
 import org.apache.rocketmq.remoting.protocol.RemotingCommand;
-import org.apache.rocketmq.store.MessageExtBrokerInner;
+import org.apache.rocketmq.common.message.MessageExtBrokerInner;
 import org.apache.rocketmq.store.PutMessageResult;
 import org.apache.rocketmq.store.config.BrokerRole;
 
 /**
  * EndTransaction processor: process commit and rollback message
  */
-public class EndTransactionProcessor extends AsyncNettyRequestProcessor implements NettyRequestProcessor {
+public class EndTransactionProcessor implements NettyRequestProcessor {
     private static final InternalLogger LOGGER = InternalLoggerFactory.getLogger(LoggerName.TRANSACTION_LOGGER_NAME);
     private final BrokerController brokerController;
 
@@ -231,7 +230,7 @@ public class EndTransactionProcessor extends AsyncNettyRequestProcessor implemen
                     response.setRemark(null);
                     break;
                 // Failed
-                case CREATE_MAPEDFILE_FAILED:
+                case CREATE_MAPPED_FILE_FAILED:
                     response.setCode(ResponseCode.SYSTEM_ERROR);
                     response.setRemark("Create mapped file failed.");
                     break;
@@ -244,7 +243,7 @@ public class EndTransactionProcessor extends AsyncNettyRequestProcessor implemen
                     response.setCode(ResponseCode.SERVICE_NOT_AVAILABLE);
                     response.setRemark("Service not available now.");
                     break;
-                case OS_PAGECACHE_BUSY:
+                case OS_PAGE_CACHE_BUSY:
                     response.setCode(ResponseCode.SYSTEM_ERROR);
                     response.setRemark("OS page cache busy, please try another machine");
                     break;
diff --git a/broker/src/main/java/org/apache/rocketmq/broker/processor/ForwardRequestProcessor.java b/broker/src/main/java/org/apache/rocketmq/broker/processor/ForwardRequestProcessor.java
index cd93598..b0f0a05 100644
--- a/broker/src/main/java/org/apache/rocketmq/broker/processor/ForwardRequestProcessor.java
+++ b/broker/src/main/java/org/apache/rocketmq/broker/processor/ForwardRequestProcessor.java
@@ -21,11 +21,10 @@ import org.apache.rocketmq.broker.BrokerController;
 import org.apache.rocketmq.common.constant.LoggerName;
 import org.apache.rocketmq.logging.InternalLogger;
 import org.apache.rocketmq.logging.InternalLoggerFactory;
-import org.apache.rocketmq.remoting.netty.AsyncNettyRequestProcessor;
 import org.apache.rocketmq.remoting.netty.NettyRequestProcessor;
 import org.apache.rocketmq.remoting.protocol.RemotingCommand;
 
-public class ForwardRequestProcessor extends AsyncNettyRequestProcessor implements NettyRequestProcessor {
+public class ForwardRequestProcessor implements NettyRequestProcessor {
     private static final InternalLogger log = InternalLoggerFactory.getLogger(LoggerName.BROKER_LOGGER_NAME);
 
     private final BrokerController brokerController;
diff --git a/broker/src/main/java/org/apache/rocketmq/broker/processor/NotificationProcessor.java b/broker/src/main/java/org/apache/rocketmq/broker/processor/NotificationProcessor.java
new file mode 100644
index 0000000..e007367
--- /dev/null
+++ b/broker/src/main/java/org/apache/rocketmq/broker/processor/NotificationProcessor.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.broker.processor;
+
+import com.googlecode.concurrentlinkedhashmap.ConcurrentLinkedHashMap;
+import io.netty.channel.Channel;
+import io.netty.channel.ChannelFuture;
+import io.netty.channel.ChannelFutureListener;
+import io.netty.channel.ChannelHandlerContext;
+
+import java.util.ArrayList;
+import java.util.Collection;
+import java.util.List;
+import java.util.Random;
+import java.util.concurrent.ArrayBlockingQueue;
+
+import org.apache.rocketmq.broker.BrokerController;
+import org.apache.rocketmq.broker.longpolling.NotificationRequest;
+import org.apache.rocketmq.common.AbstractBrokerRunnable;
+import org.apache.rocketmq.common.KeyBuilder;
+import org.apache.rocketmq.common.TopicConfig;
+import org.apache.rocketmq.common.constant.LoggerName;
+import org.apache.rocketmq.common.constant.PermName;
+import org.apache.rocketmq.common.help.FAQUrl;
+import org.apache.rocketmq.common.protocol.ResponseCode;
+import org.apache.rocketmq.common.protocol.header.NotificationRequestHeader;
+import org.apache.rocketmq.common.protocol.header.NotificationResponseHeader;
+import org.apache.rocketmq.common.subscription.SubscriptionGroupConfig;
+import org.apache.rocketmq.logging.InternalLogger;
+import org.apache.rocketmq.logging.InternalLoggerFactory;
+import org.apache.rocketmq.remoting.common.RemotingHelper;
+import org.apache.rocketmq.remoting.exception.RemotingCommandException;
+import org.apache.rocketmq.remoting.netty.NettyRequestProcessor;
+import org.apache.rocketmq.remoting.netty.RequestTask;
+import org.apache.rocketmq.remoting.protocol.RemotingCommand;
+
+public class NotificationProcessor implements NettyRequestProcessor {
+    private static final InternalLogger POP_LOGGER = InternalLoggerFactory.getLogger(LoggerName.ROCKETMQ_POP_LOGGER_NAME);
+    private final BrokerController brokerController;
+    private Random random = new Random(System.currentTimeMillis());
+    private static final String BORN_TIME = "bornTime";
+    private ConcurrentLinkedHashMap<String, ArrayBlockingQueue<NotificationRequest>> pollingMap = new ConcurrentLinkedHashMap.Builder<String, ArrayBlockingQueue<NotificationRequest>>().maximumWeightedCapacity(100000).build();
+    private Thread checkNotificationPollingThread;
+
+    public NotificationProcessor(final BrokerController brokerController) {
+        this.brokerController = brokerController;
+        this.checkNotificationPollingThread = new Thread(new AbstractBrokerRunnable(brokerController.getBrokerConfig()) {
+            @Override public void run2() {
+                while (true) {
+                    if (Thread.currentThread().isInterrupted()) {
+                        break;
+                    }
+                    try {
+                        Thread.sleep(2000L);
+                        Collection<ArrayBlockingQueue<NotificationRequest>> pops = pollingMap.values();
+                        for (ArrayBlockingQueue<NotificationRequest> popQ : pops) {
+                            NotificationRequest tmPopRequest = popQ.peek();
+                            while (tmPopRequest != null) {
+                                if (tmPopRequest.isTimeout()) {
+                                    tmPopRequest = popQ.poll();
+                                    if (tmPopRequest == null) {
+                                        break;
+                                    }
+                                    if (!tmPopRequest.isTimeout()) {
+                                        POP_LOGGER.info("not timeout , but wakeUp Notification in advance: {}", tmPopRequest);
+                                        wakeUp(tmPopRequest, false);
+                                        break;
+                                    } else {
+                                        POP_LOGGER.info("timeout , wakeUp Notification : {}", tmPopRequest);
+                                        wakeUp(tmPopRequest, false);
+                                        tmPopRequest = popQ.peek();
+                                    }
+                                } else {
+                                    break;
+                                }
+                            }
+                        }
+                    } catch (InterruptedException e) {
+                        break;
+                    } catch (Exception e) {
+                        POP_LOGGER.error("checkNotificationPolling error", e);
+                    }
+                }
+            }
+        });
+        this.checkNotificationPollingThread.setDaemon(true);
+        this.checkNotificationPollingThread.setName("checkNotificationPolling");
+        this.checkNotificationPollingThread.start();
+    }
+
+    public void shutdown() {
+        this.checkNotificationPollingThread.interrupt();
+    }
+
+    @Override
+    public RemotingCommand processRequest(final ChannelHandlerContext ctx,
+        RemotingCommand request) throws RemotingCommandException {
+        request.addExtField(BORN_TIME, String.valueOf(System.currentTimeMillis()));
+        return this.processRequest(ctx.channel(), request);
+    }
+
+    @Override
+    public boolean rejectRequest() {
+        return false;
+    }
+
+    public void notifyMessageArriving(final String topic, final int queueId) {
+        ArrayBlockingQueue<NotificationRequest> remotingCommands = pollingMap.get(KeyBuilder.buildPollingNotificationKey(topic, -1));
+        if (remotingCommands != null) {
+            List<NotificationRequest> c = new ArrayList<>();
+            remotingCommands.drainTo(c);
+            for (NotificationRequest notificationRequest : c) {
+                POP_LOGGER.info("new msg arrive , wakeUp : {}", notificationRequest);
+                wakeUp(notificationRequest, true);
+
+            }
+        }
+        remotingCommands = pollingMap.get(KeyBuilder.buildPollingNotificationKey(topic, queueId));
+        if (remotingCommands != null) {
+            List<NotificationRequest> c = new ArrayList<>();
+            remotingCommands.drainTo(c);
+            for (NotificationRequest notificationRequest : c) {
+                POP_LOGGER.info("new msg arrive , wakeUp : {}", notificationRequest);
+                wakeUp(notificationRequest, true);
+            }
+        }
+    }
+
+    private void wakeUp(final NotificationRequest request, final boolean hasMsg) {
+        if (request == null || !request.complete()) {
+            return;
+        }
+        if (!request.getChannel().isActive()) {
+            return;
+        }
+        Runnable run = new Runnable() {
+            @Override
+            public void run() {
+                final RemotingCommand response = NotificationProcessor.this.responseNotification(request.getChannel(), hasMsg);
+
+                if (response != null) {
+                    response.setOpaque(request.getRemotingCommand().getOpaque());
+                    response.markResponseType();
+                    try {
+                        request.getChannel().writeAndFlush(response).addListener(new ChannelFutureListener() {
+                            @Override
+                            public void operationComplete(ChannelFuture future) throws Exception {
+                                if (!future.isSuccess()) {
+                                    POP_LOGGER.error("ProcessRequestWrapper response to {} failed", future.channel().remoteAddress(), future.cause());
+                                    POP_LOGGER.error(request.toString());
+                                    POP_LOGGER.error(response.toString());
+                                }
+                            }
+                        });
+                    } catch (Throwable e) {
+                        POP_LOGGER.error("ProcessRequestWrapper process request over, but response failed", e);
+                        POP_LOGGER.error(request.toString());
+                        POP_LOGGER.error(response.toString());
+                    }
+                }
+            }
+        };
+        this.brokerController.getPullMessageExecutor().submit(new RequestTask(run, request.getChannel(), request.getRemotingCommand()));
+    }
+
+    public RemotingCommand responseNotification(final Channel channel, boolean hasMsg) {
+        RemotingCommand response = RemotingCommand.createResponseCommand(NotificationResponseHeader.class);
+        final NotificationResponseHeader responseHeader = (NotificationResponseHeader) response.readCustomHeader();
+        responseHeader.setHasMsg(hasMsg);
+        response.setCode(ResponseCode.SUCCESS);
+        return response;
+    }
+
+    private RemotingCommand processRequest(final Channel channel, RemotingCommand request)
+        throws RemotingCommandException {
+        RemotingCommand response = RemotingCommand.createResponseCommand(NotificationResponseHeader.class);
+        final NotificationResponseHeader responseHeader = (NotificationResponseHeader) response.readCustomHeader();
+        final NotificationRequestHeader requestHeader =
+            (NotificationRequestHeader) request.decodeCommandCustomHeader(NotificationRequestHeader.class);
+
+        response.setOpaque(request.getOpaque());
+
+        if (!PermName.isReadable(this.brokerController.getBrokerConfig().getBrokerPermission())) {
+            response.setCode(ResponseCode.NO_PERMISSION);
+            response.setRemark(String.format("the broker[%s] peeking message is forbidden", this.brokerController.getBrokerConfig().getBrokerIP1()));
+            return response;
+        }
+
+        TopicConfig topicConfig = this.brokerController.getTopicConfigManager().selectTopicConfig(requestHeader.getTopic());
+        if (null == topicConfig) {
+            POP_LOGGER.error("The topic {} not exist, consumer: {} ", requestHeader.getTopic(), RemotingHelper.parseChannelRemoteAddr(channel));
+            response.setCode(ResponseCode.TOPIC_NOT_EXIST);
+            response.setRemark(String.format("topic[%s] not exist, apply first please! %s", requestHeader.getTopic(), FAQUrl.suggestTodo(FAQUrl.APPLY_TOPIC_URL)));
+            return response;
+        }
+
+        if (!PermName.isReadable(topicConfig.getPerm())) {
+            response.setCode(ResponseCode.NO_PERMISSION);
+            response.setRemark("the topic[" + requestHeader.getTopic() + "] peeking message is forbidden");
+            return response;
+        }
+
+        if (requestHeader.getQueueId() >= topicConfig.getReadQueueNums()) {
+            String errorInfo = String.format("queueId[%d] is illegal, topic:[%s] topicConfig.readQueueNums:[%d] consumer:[%s]",
+                requestHeader.getQueueId(), requestHeader.getTopic(), topicConfig.getReadQueueNums(), channel.remoteAddress());
+            POP_LOGGER.warn(errorInfo);
+            response.setCode(ResponseCode.SYSTEM_ERROR);
+            response.setRemark(errorInfo);
+            return response;
+        }
+        SubscriptionGroupConfig subscriptionGroupConfig = this.brokerController.getSubscriptionGroupManager().findSubscriptionGroupConfig(requestHeader.getConsumerGroup());
+        if (null == subscriptionGroupConfig) {
+            response.setCode(ResponseCode.SUBSCRIPTION_GROUP_NOT_EXIST);
+            response.setRemark(String.format("subscription group [%s] does not exist, %s", requestHeader.getConsumerGroup(), FAQUrl.suggestTodo(FAQUrl.SUBSCRIPTION_GROUP_NOT_EXIST)));
+            return response;
+        }
+
+        if (!subscriptionGroupConfig.isConsumeEnable()) {
+            response.setCode(ResponseCode.NO_PERMISSION);
+            response.setRemark("subscription group no permission, " + requestHeader.getConsumerGroup());
+            return response;
+        }
+        int randomQ = random.nextInt(100);
+        boolean hasMsg = false;
+        boolean needRetry = randomQ % 5 == 0;
+        if (needRetry) {
+            TopicConfig retryTopicConfig = this.brokerController.getTopicConfigManager().selectTopicConfig(KeyBuilder.buildPopRetryTopic(requestHeader.getTopic(), requestHeader.getConsumerGroup()));
+            if (retryTopicConfig != null) {
+                for (int i = 0; i < retryTopicConfig.getReadQueueNums(); i++) {
+                    int queueId = (randomQ + i) % retryTopicConfig.getReadQueueNums();
+                    hasMsg = hasMsgFromQueue(true, requestHeader, queueId);
+                }
+            }
+        }
+        if (!hasMsg && requestHeader.getQueueId() < 0) {
+            // read all queue
+            for (int i = 0; i < topicConfig.getReadQueueNums(); i++) {
+                int queueId = (randomQ + i) % topicConfig.getReadQueueNums();
+                hasMsg = hasMsgFromQueue(false, requestHeader, queueId);
+                if (hasMsg) {
+                    break;
+                }
+            }
+        } else {
+            int queueId = requestHeader.getQueueId();
+            hasMsg = hasMsgFromQueue(false, requestHeader, queueId);
+        }
+
+        if (!hasMsg) {
+            if (polling(channel, request, requestHeader)) {
+                return null;
+            }
+        }
+        response.setCode(ResponseCode.SUCCESS);
+        responseHeader.setHasMsg(hasMsg);
+        return response;
+    }
+
+    private boolean hasMsgFromQueue(boolean isRetry, NotificationRequestHeader requestHeader, int queueId) {
+        String topic = isRetry ? KeyBuilder.buildPopRetryTopic(requestHeader.getTopic(), requestHeader.getConsumerGroup()) : requestHeader.getTopic();
+        long offset = getPopOffset(topic, requestHeader.getConsumerGroup(), queueId);
+        long restNum = this.brokerController.getMessageStore().getMaxOffsetInQueue(topic, queueId) - offset;
+        return restNum > 0;
+    }
+
+    private long getPopOffset(String topic, String cid, int queueId) {
+        long offset = this.brokerController.getConsumerOffsetManager().queryOffset(cid, topic, queueId);
+        if (offset < 0) {
+            offset = this.brokerController.getMessageStore().getMinOffsetInQueue(topic, queueId);
+        }
+        long bufferOffset = this.brokerController.getPopMessageProcessor().getPopBufferMergeService()
+            .getLatestOffset(topic, cid, queueId);
+        if (bufferOffset < 0) {
+            return offset;
+        } else {
+            return bufferOffset > offset ? bufferOffset : offset;
+        }
+    }
+
+    private boolean polling(final Channel channel, RemotingCommand remotingCommand,
+        final NotificationRequestHeader requestHeader) {
+        if (requestHeader.getPollTime() <= 0) {
+            return false;
+        }
+
+        long expired = requestHeader.getBornTime() + requestHeader.getPollTime();
+        final NotificationRequest request = new NotificationRequest(remotingCommand, channel, expired);
+        boolean result = false;
+        if (!request.isTimeout()) {
+            String key = KeyBuilder.buildPollingNotificationKey(requestHeader.getTopic(), requestHeader.getQueueId());
+            ArrayBlockingQueue<NotificationRequest> queue = pollingMap.get(key);
+            if (queue == null) {
+                queue = new ArrayBlockingQueue<>(this.brokerController.getBrokerConfig().getPopPollingSize());
+                pollingMap.put(key, queue);
+                result = queue.offer(request);
+            } else {
+                result = queue.offer(request);
+            }
+        }
+        POP_LOGGER.info("polling {}, result {}", remotingCommand, result);
+        return result;
+
+    }
+}
diff --git a/broker/src/main/java/org/apache/rocketmq/broker/processor/PeekMessageProcessor.java b/broker/src/main/java/org/apache/rocketmq/broker/processor/PeekMessageProcessor.java
new file mode 100644
index 0000000..7cc15f2
--- /dev/null
+++ b/broker/src/main/java/org/apache/rocketmq/broker/processor/PeekMessageProcessor.java
@@ -0,0 +1,271 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.rocketmq.broker.processor;
+
+import io.netty.channel.Channel;
+import io.netty.channel.ChannelFuture;
+import io.netty.channel.ChannelFutureListener;
+import io.netty.channel.ChannelHandlerContext;
+import io.netty.channel.FileRegion;
+
+import java.nio.ByteBuffer;
+import java.util.List;
+import java.util.Random;
+
+import org.apache.rocketmq.broker.BrokerController;
+import org.apache.rocketmq.broker.pagecache.ManyMessageTransfer;
+import org.apache.rocketmq.common.KeyBuilder;
+import org.apache.rocketmq.common.TopicConfig;
+import org.apache.rocketmq.common.constant.LoggerName;
+import org.apache.rocketmq.common.constant.PermName;
+import org.apache.rocketmq.common.help.FAQUrl;
+import org.apache.rocketmq.common.message.MessageDecoder;
+import org.apache.rocketmq.common.protocol.ResponseCode;
+import org.apache.rocketmq.common.protocol.header.PeekMessageRequestHeader;
+import org.apache.rocketmq.common.protocol.header.PopMessageResponseHeader;
+import org.apache.rocketmq.common.subscription.SubscriptionGroupConfig;
+import org.apache.rocketmq.logging.InternalLogger;
+import org.apache.rocketmq.logging.InternalLoggerFactory;
+import org.apache.rocketmq.remoting.common.RemotingHelper;
+import org.apache.rocketmq.remoting.exception.RemotingCommandException;
+import org.apache.rocketmq.remoting.netty.NettyRequestProcessor;
+import org.apache.rocketmq.remoting.protocol.RemotingCommand;
+import org.apache.rocketmq.store.GetMessageResult;
+import org.apache.rocketmq.store.GetMessageStatus;
+import org.apache.rocketmq.store.SelectMappedBufferResult;
+
+public class PeekMessageProcessor implements NettyRequestProcessor {
+    private static final InternalLogger LOG = InternalLoggerFactory.getLogger(LoggerName.BROKER_LOGGER_NAME);
+    private final BrokerController brokerController;
+    private Random random = new Random(System.currentTimeMillis());
+
+    public PeekMessageProcessor(final BrokerController brokerController) {
+        this.brokerController = brokerController;
+    }
+
+    @Override
+    public RemotingCommand processRequest(final ChannelHandlerContext ctx,
+        RemotingCommand request) throws RemotingCommandException {
+        return this.processRequest(ctx.channel(), request, true);
+    }
+
+    @Override
+    public boolean rejectRequest() {
+        return false;
+    }
+
+    private RemotingCommand processRequest(final Channel channel, RemotingCommand request, boolean brokerAllowSuspend)
+        throws RemotingCommandException {
+        RemotingCommand response = RemotingCommand.createResponseCommand(PopMessageResponseHeader.class);
+        final PopMessageResponseHeader responseHeader = (PopMessageResponseHeader) response.readCustomHeader();
+        final PeekMessageRequestHeader requestHeader =
+            (PeekMessageRequestHeader) request.decodeCommandCustomHeader(PeekMessageRequestHeader.class);
+
+        response.setOpaque(request.getOpaque());
+
+        if (!PermName.isReadable(this.brokerController.getBrokerConfig().getBrokerPermission())) {
+            response.setCode(ResponseCode.NO_PERMISSION);
+            response.setRemark(String.format("the broker[%s] peeking message is forbidden", this.brokerController.getBrokerConfig().getBrokerIP1()));
+            return response;
+        }
+
+        TopicConfig topicConfig = this.brokerController.getTopicConfigManager().selectTopicConfig(requestHeader.getTopic());
+        if (null == topicConfig) {
+            LOG.error("The topic {} not exist, consumer: {} ", requestHeader.getTopic(), RemotingHelper.parseChannelRemoteAddr(channel));
+            response.setCode(ResponseCode.TOPIC_NOT_EXIST);
+            response.setRemark(String.format("topic[%s] not exist, apply first please! %s", requestHeader.getTopic(), FAQUrl.suggestTodo(FAQUrl.APPLY_TOPIC_URL)));
+            return response;
+        }
+
+        if (!PermName.isReadable(topicConfig.getPerm())) {
+            response.setCode(ResponseCode.NO_PERMISSION);
+            response.setRemark("the topic[" + requestHeader.getTopic() + "] peeking message is forbidden");
+            return response;
+        }
+
+        if (requestHeader.getQueueId() >= topicConfig.getReadQueueNums()) {
+            String errorInfo = String.format("queueId[%d] is illegal, topic:[%s] topicConfig.readQueueNums:[%d] consumer:[%s]",
+                requestHeader.getQueueId(), requestHeader.getTopic(), topicConfig.getReadQueueNums(), channel.remoteAddress());
+            LOG.warn(errorInfo);
+            response.setCode(ResponseCode.SYSTEM_ERROR);
+            response.setRemark(errorInfo);
+            return response;
+        }
+        SubscriptionGroupConfig subscriptionGroupConfig = this.brokerController.getSubscriptionGroupManager().findSubscriptionGroupConfig(requestHeader.getConsumerGroup());
+        if (null == subscriptionGroupConfig) {
+            response.setCode(ResponseCode.SUBSCRIPTION_GROUP_NOT_EXIST);
+            response.setRemark(String.format("subscription group [%s] does not exist, %s", requestHeader.getConsumerGroup(), FAQUrl.suggestTodo(FAQUrl.SUBSCRIPTION_GROUP_NOT_EXIST)));
+            return response;
+        }
+
+        if (!subscriptionGroupConfig.isConsumeEnable()) {
+            response.setCode(ResponseCode.NO_PERMISSION);
+            response.setRemark("subscription group no permission, " + requestHeader.getConsumerGroup());
+            return response;
+        }
+        int randomQ = random.nextInt(100);
+        int reviveQid = randomQ % this.brokerController.getBrokerConfig().getReviveQueueNum();
+        int commercialSizePerMsg = this.brokerController.getBrokerConfig().getCommercialSizePerMsg();
+        GetMessageResult getMessageResult = new GetMessageResult(commercialSizePerMsg);
+        boolean needRetry = randomQ % 5 == 0;
+        long popTime = System.currentTimeMillis();
+        long restNum = 0;
+        if (needRetry) {
+            TopicConfig retryTopicConfig = this.brokerController.getTopicConfigManager().selectTopicConfig(KeyBuilder.buildPopRetryTopic(requestHeader.getTopic(), requestHeader.getConsumerGroup()));
+            if (retryTopicConfig != null) {
+                for (int i = 0; i < retryTopicConfig.getReadQueueNums(); i++) {
+                    int queueId = (randomQ + i) % retryTopicConfig.getReadQueueNums();
+                    restNum = peekMsgFromQueue(true, getMessageResult, requestHeader, queueId, restNum, reviveQid, channel, popTime);
+                }
+            }
+        }
+        if (requestHeader.getQueueId() < 0) {
+            // read all queue
+            for (int i = 0; i < topicConfig.getReadQueueNums(); i++) {
+                int queueId = (randomQ + i) % topicConfig.getReadQueueNums();
+                restNum = peekMsgFromQueue(false, getMessageResult, requestHeader, queueId, restNum, reviveQid, channel, popTime);
+            }
+        } else {
+            int queueId = requestHeader.getQueueId();
+            restNum = peekMsgFromQueue(false, getMessageResult, requestHeader, queueId, restNum, reviveQid, channel, popTime);
+        }
+        // if not full , fetch retry again
+        if (!needRetry && getMessageResult.getMessageMapedList().size() < requestHeader.getMaxMsgNums()) {
+            TopicConfig retryTopicConfig = this.brokerController.getTopicConfigManager().selectTopicConfig(KeyBuilder.buildPopRetryTopic(requestHeader.getTopic(), requestHeader.getConsumerGroup()));
+            if (retryTopicConfig != null) {
+                for (int i = 0; i < retryTopicConfig.getReadQueueNums(); i++) {
+                    int queueId = (randomQ + i) % retryTopicConfig.getReadQueueNums();
+                    restNum = peekMsgFromQueue(true, getMessageResult, requestHeader, queueId, restNum, reviveQid, channel, popTime);
+                }
+            }
+        }
+        if (!getMessageResult.getMessageBufferList().isEmpty()) {
+            response.setCode(ResponseCode.SUCCESS);
+            getMessageResult.setStatus(GetMessageStatus.FOUND);
+        } else {
+            response.setCode(ResponseCode.PULL_NOT_FOUND);
+            getMessageResult.setStatus(GetMessageStatus.NO_MESSAGE_IN_QUEUE);
+
+        }
+        responseHeader.setRestNum(restNum);
+        response.setRemark(getMessageResult.getStatus().name());
+        switch (response.getCode()) {
+            case ResponseCode.SUCCESS:
+
+                this.brokerController.getBrokerStatsManager().incGroupGetNums(requestHeader.getConsumerGroup(), requestHeader.getTopic(),
+                    getMessageResult.getMessageCount());
+
+                this.brokerController.getBrokerStatsManager().incGroupGetSize(requestHeader.getConsumerGroup(), requestHeader.getTopic(),
+                    getMessageResult.getBufferTotalSize());
+
+                this.brokerController.getBrokerStatsManager().incBrokerGetNums(getMessageResult.getMessageCount());
+                if (this.brokerController.getBrokerConfig().isTransferMsgByHeap()) {
+                    final long beginTimeMills = this.brokerController.getMessageStore().now();
+                    final byte[] r = this.readGetMessageResult(getMessageResult, requestHeader.getConsumerGroup(), requestHeader.getTopic(), requestHeader.getQueueId());
+                    this.brokerController.getBrokerStatsManager().incGroupGetLatency(requestHeader.getConsumerGroup(),
+                        requestHeader.getTopic(), requestHeader.getQueueId(),
+                        (int) (this.brokerController.getMessageStore().now() - beginTimeMills));
+                    response.setBody(r);
+                } else {
+                    final GetMessageResult tmpGetMessageResult = getMessageResult;
+                    try {
+                        FileRegion fileRegion =
+                            new ManyMessageTransfer(response.encodeHeader(getMessageResult.getBufferTotalSize()), getMessageResult);
+                        channel.writeAndFlush(fileRegion).addListener(new ChannelFutureListener() {
+                            @Override
+                            public void operationComplete(ChannelFuture future) throws Exception {
+                                tmpGetMessageResult.release();
+                                if (!future.isSuccess()) {
+                                    LOG.error("Fail to transfer messages from page cache to {}", channel.remoteAddress(), future.cause());
+                                }
+                            }
+                        });
+                    } catch (Throwable e) {
+                        LOG.error("Error occurred when transferring messages from page cache", e);
+                        getMessageResult.release();
+                    }
+
+                    response = null;
+                }
+                break;
+            default:
+                assert false;
+        }
+        return response;
+    }
+
+    private long peekMsgFromQueue(boolean isRetry, GetMessageResult getMessageResult,
+        PeekMessageRequestHeader requestHeader, int queueId, long restNum, int reviveQid, Channel channel,
+        long popTime) {
+        String topic = isRetry ? KeyBuilder.buildPopRetryTopic(requestHeader.getTopic(), requestHeader.getConsumerGroup()) : requestHeader.getTopic();
+        GetMessageResult getMessageTmpResult;
+        long offset = getPopOffset(topic, requestHeader.getConsumerGroup(), queueId);
+        restNum = this.brokerController.getMessageStore().getMaxOffsetInQueue(topic, queueId) - offset + restNum;
+        if (getMessageResult.getMessageMapedList().size() >= requestHeader.getMaxMsgNums()) {
+            return restNum;
+        }
+        getMessageTmpResult = this.brokerController.getMessageStore().getMessage(requestHeader.getConsumerGroup(), topic, queueId, offset,
+            requestHeader.getMaxMsgNums() - getMessageResult.getMessageMapedList().size(), null);
+        // maybe store offset is not correct.
+        if (GetMessageStatus.OFFSET_TOO_SMALL.equals(getMessageTmpResult.getStatus()) || GetMessageStatus.OFFSET_OVERFLOW_BADLY.equals(getMessageTmpResult.getStatus())) {
+            offset = getMessageTmpResult.getNextBeginOffset();
+            getMessageTmpResult = this.brokerController.getMessageStore().getMessage(requestHeader.getConsumerGroup(), topic, queueId, offset,
+                requestHeader.getMaxMsgNums() - getMessageResult.getMessageMapedList().size(), null);
+        }
+        if (getMessageTmpResult != null) {
+            for (SelectMappedBufferResult mapedBuffer : getMessageTmpResult.getMessageMapedList()) {
+                getMessageResult.addMessage(mapedBuffer);
+            }
+        }
+        return restNum;
+    }
+
+    private long getPopOffset(String topic, String cid, int queueId) {
+        long offset = this.brokerController.getConsumerOffsetManager().queryOffset(cid, topic, queueId);
+        if (offset < 0) {
+            offset = this.brokerController.getMessageStore().getMinOffsetInQueue(topic, queueId);
+        }
+        long bufferOffset = this.brokerController.getPopMessageProcessor().getPopBufferMergeService()
+            .getLatestOffset(topic, cid, queueId);
+        if (bufferOffset < 0) {
+            return offset;
+        } else {
+            return bufferOffset > offset ? bufferOffset : offset;
+        }
+    }
+
+    private byte[] readGetMessageResult(final GetMessageResult getMessageResult, final String group, final String topic,
+        final int queueId) {
+        final ByteBuffer byteBuffer = ByteBuffer.allocate(getMessageResult.getBufferTotalSize());
+
+        long storeTimestamp = 0;
+        try {
+            List<ByteBuffer> messageBufferList = getMessageResult.getMessageBufferList();
+            for (ByteBuffer bb : messageBufferList) {
+
+                byteBuffer.put(bb);
+                storeTimestamp = bb.getLong(MessageDecoder.MESSAGE_STORE_TIMESTAMP_POSITION);
+            }
+        } finally {
+            getMessageResult.release();
+        }
+
+        this.brokerController.getBrokerStatsManager().recordDiskFallBehindTime(group, topic, queueId, this.brokerController.getMessageStore().now() - storeTimestamp);
+        return byteBuffer.array();
+    }
+
+}
diff --git a/broker/src/main/java/org/apache/rocketmq/broker/processor/PollingInfoProcessor.java b/broker/src/main/java/org/apache/rocketmq/broker/processor/PollingInfoProcessor.java
new file mode 100644
index 0000000..827a8d6
--- /dev/null
+++ b/broker/src/main/java/org/apache/rocketmq/broker/processor/PollingInfoProcessor.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.broker.processor;
+
+import io.netty.channel.Channel;
+import io.netty.channel.ChannelHandlerContext;
+import java.util.concurrent.ConcurrentSkipListSet;
+import org.apache.rocketmq.broker.BrokerController;
+import org.apache.rocketmq.broker.longpolling.PopRequest;
+import org.apache.rocketmq.common.KeyBuilder;
+import org.apache.rocketmq.common.TopicConfig;
+import org.apache.rocketmq.common.constant.LoggerName;
+import org.apache.rocketmq.common.constant.PermName;
+import org.apache.rocketmq.common.help.FAQUrl;
+import org.apache.rocketmq.common.protocol.ResponseCode;
+import org.apache.rocketmq.common.protocol.header.PollingInfoRequestHeader;
+import org.apache.rocketmq.common.protocol.header.PollingInfoResponseHeader;
+import org.apache.rocketmq.common.subscription.SubscriptionGroupConfig;
+import org.apache.rocketmq.logging.InternalLogger;
+import org.apache.rocketmq.logging.InternalLoggerFactory;
+import org.apache.rocketmq.remoting.common.RemotingHelper;
+import org.apache.rocketmq.remoting.exception.RemotingCommandException;
+import org.apache.rocketmq.remoting.netty.NettyRequestProcessor;
+import org.apache.rocketmq.remoting.protocol.RemotingCommand;
+
+public class PollingInfoProcessor implements NettyRequestProcessor {
+    private static final InternalLogger POP_LOGGER = InternalLoggerFactory.getLogger(LoggerName.ROCKETMQ_POP_LOGGER_NAME);
+    private final BrokerController brokerController;
+
+    public PollingInfoProcessor(final BrokerController brokerController) {
+        this.brokerController = brokerController;
+    }
+
+    @Override
+    public RemotingCommand processRequest(final ChannelHandlerContext ctx,
+        RemotingCommand request) throws RemotingCommandException {
+        return this.processRequest(ctx.channel(), request);
+    }
+
+    @Override
+    public boolean rejectRequest() {
+        return false;
+    }
+
+    private RemotingCommand processRequest(final Channel channel, RemotingCommand request)
+        throws RemotingCommandException {
+        RemotingCommand response = RemotingCommand.createResponseCommand(PollingInfoResponseHeader.class);
+        final PollingInfoResponseHeader responseHeader = (PollingInfoResponseHeader) response.readCustomHeader();
+        final PollingInfoRequestHeader requestHeader =
+            (PollingInfoRequestHeader) request.decodeCommandCustomHeader(PollingInfoRequestHeader.class);
+
+        response.setOpaque(request.getOpaque());
+
+        if (!PermName.isReadable(this.brokerController.getBrokerConfig().getBrokerPermission())) {
+            response.setCode(ResponseCode.NO_PERMISSION);
+            response.setRemark(String.format("the broker[%s] peeking message is forbidden", this.brokerController.getBrokerConfig().getBrokerIP1()));
+            return response;
+        }
+
+        TopicConfig topicConfig = this.brokerController.getTopicConfigManager().selectTopicConfig(requestHeader.getTopic());
+        if (null == topicConfig) {
+            POP_LOGGER.error("The topic {} not exist, consumer: {} ", requestHeader.getTopic(), RemotingHelper.parseChannelRemoteAddr(channel));
+            response.setCode(ResponseCode.TOPIC_NOT_EXIST);
+            response.setRemark(String.format("topic[%s] not exist, apply first please! %s", requestHeader.getTopic(), FAQUrl.suggestTodo(FAQUrl.APPLY_TOPIC_URL)));
+            return response;
+        }
+
+        if (!PermName.isReadable(topicConfig.getPerm())) {
+            response.setCode(ResponseCode.NO_PERMISSION);
+            response.setRemark("the topic[" + requestHeader.getTopic() + "] peeking message is forbidden");
+            return response;
+        }
+
+        if (requestHeader.getQueueId() >= topicConfig.getReadQueueNums()) {
+            String errorInfo = String.format("queueId[%d] is illegal, topic:[%s] topicConfig.readQueueNums:[%d] consumer:[%s]",
+                requestHeader.getQueueId(), requestHeader.getTopic(), topicConfig.getReadQueueNums(), channel.remoteAddress());
+            POP_LOGGER.warn(errorInfo);
+            response.setCode(ResponseCode.SYSTEM_ERROR);
+            response.setRemark(errorInfo);
+            return response;
+        }
+        SubscriptionGroupConfig subscriptionGroupConfig = this.brokerController.getSubscriptionGroupManager().findSubscriptionGroupConfig(requestHeader.getConsumerGroup());
+        if (null == subscriptionGroupConfig) {
+            response.setCode(ResponseCode.SUBSCRIPTION_GROUP_NOT_EXIST);
+            response.setRemark(String.format("subscription group [%s] does not exist, %s", requestHeader.getConsumerGroup(), FAQUrl.suggestTodo(FAQUrl.SUBSCRIPTION_GROUP_NOT_EXIST)));
+            return response;
+        }
+
+        if (!subscriptionGroupConfig.isConsumeEnable()) {
+            response.setCode(ResponseCode.NO_PERMISSION);
+            response.setRemark("subscription group no permission, " + requestHeader.getConsumerGroup());
+            return response;
+        }
+        String key = KeyBuilder.buildPollingKey(requestHeader.getTopic(), requestHeader.getConsumerGroup(), requestHeader.getQueueId());
+        ConcurrentSkipListSet<PopRequest> queue = this.brokerController.getPopMessageProcessor().getPollingMap().get(key);
+        if (queue != null) {
+            responseHeader.setPollingNum(queue.size());
+        } else {
+            responseHeader.setPollingNum(0);
+        }
+        response.setCode(ResponseCode.SUCCESS);
+        return response;
+    }
+
+}
diff --git a/broker/src/main/java/org/apache/rocketmq/broker/processor/PopBufferMergeService.java b/broker/src/main/java/org/apache/rocketmq/broker/processor/PopBufferMergeService.java
index 615a70e..cee84e6 100644
--- a/broker/src/main/java/org/apache/rocketmq/broker/processor/PopBufferMergeService.java
+++ b/broker/src/main/java/org/apache/rocketmq/broker/processor/PopBufferMergeService.java
@@ -33,7 +33,7 @@ import org.apache.rocketmq.common.message.MessageDecoder;
 import org.apache.rocketmq.common.utils.DataConverter;
 import org.apache.rocketmq.logging.InternalLogger;
 import org.apache.rocketmq.logging.InternalLoggerFactory;
-import org.apache.rocketmq.store.MessageExtBrokerInner;
+import org.apache.rocketmq.common.message.MessageExtBrokerInner;
 import org.apache.rocketmq.store.PutMessageResult;
 import org.apache.rocketmq.store.PutMessageStatus;
 import org.apache.rocketmq.store.config.BrokerRole;
@@ -43,9 +43,9 @@ import org.apache.rocketmq.store.pop.PopCheckPoint;
 public class PopBufferMergeService extends ServiceThread {
     private static final InternalLogger POP_LOGGER = InternalLoggerFactory.getLogger(LoggerName.ROCKETMQ_POP_LOGGER_NAME);
     ConcurrentHashMap<String/*mergeKey*/, PopCheckPointWrapper>
-            buffer = new ConcurrentHashMap<>(1024 * 16);
+        buffer = new ConcurrentHashMap<>(1024 * 16);
     ConcurrentHashMap<String/*topic@cid@queueId*/, QueueWithTime<PopCheckPointWrapper>> commitOffsets =
-            new ConcurrentHashMap<>();
+        new ConcurrentHashMap<>();
     private volatile boolean serving = true;
     private AtomicInteger counter = new AtomicInteger(0);
     private int scanTimes = 0;
@@ -61,20 +61,25 @@ public class PopBufferMergeService extends ServiceThread {
     private volatile boolean master = false;
 
     public PopBufferMergeService(BrokerController brokerController, PopMessageProcessor popMessageProcessor) {
-        super();
         this.brokerController = brokerController;
         this.popMessageProcessor = popMessageProcessor;
         this.queueLockManager = popMessageProcessor.getQueueLockManager();
     }
 
-    private boolean checkAndSetMaster() {
+    private boolean isShouldRunning() {
+        if (this.brokerController.getBrokerConfig().isEnableSlaveActingMaster()) {
+            return true;
+        }
         this.master = brokerController.getMessageStoreConfig().getBrokerRole() != BrokerRole.SLAVE;
         return this.master;
     }
 
     @Override
     public String getServiceName() {
-        return "PopBufferMergeService";
+        if (this.brokerController != null && this.brokerController.getBrokerConfig().isInBrokerContainer()) {
+            return brokerController.getBrokerConfig().getLoggerIdentifier() + PopBufferMergeService.class.getSimpleName();
+        }
+        return PopBufferMergeService.class.getSimpleName();
     }
 
     @Override
@@ -82,11 +87,11 @@ public class PopBufferMergeService extends ServiceThread {
         // scan
         while (!this.isStopped()) {
             try {
-                if (!checkAndSetMaster()) {
+                if (!isShouldRunning()) {
                     // slave
                     this.waitForRunning(interval * 200 * 5);
                     POP_LOGGER.info("Broker is {}, {}, clear all data",
-                            brokerController.getMessageStoreConfig().getBrokerRole(), this.master);
+                        brokerController.getMessageStoreConfig().getBrokerRole(), this.master);
                     this.buffer.clear();
                     this.commitOffsets.clear();
                     continue;
@@ -113,7 +118,7 @@ public class PopBufferMergeService extends ServiceThread {
             Thread.sleep(2000);
         } catch (InterruptedException e) {
         }
-        if (!checkAndSetMaster()) {
+        if (!isShouldRunning()) {
             return;
         }
         while (this.buffer.size() > 0 || totalSize() > 0) {
@@ -133,7 +138,7 @@ public class PopBufferMergeService extends ServiceThread {
                 // 2. ck is buffer(acked)
                 // 3. ck is buffer(not all acked), all ak are stored and ck is stored
                 if ((pointWrapper.isJustOffset() && pointWrapper.isCkStored()) || isCkDone(pointWrapper)
-                        || (isCkDoneForFinish(pointWrapper) && pointWrapper.isCkStored())) {
+                    || (isCkDoneForFinish(pointWrapper) && pointWrapper.isCkStored())) {
                     if (commitOffset(pointWrapper)) {
                         queue.poll();
                     } else {
@@ -141,7 +146,7 @@ public class PopBufferMergeService extends ServiceThread {
                     }
                 } else {
                     if (System.currentTimeMillis() - pointWrapper.getCk().getPopTime()
-                            > brokerController.getBrokerConfig().getPopCkStayBufferTime() * 2) {
+                        > brokerController.getBrokerConfig().getPopCkStayBufferTime() * 2) {
                         POP_LOGGER.warn("[PopBuffer] ck offset long time not commit, {}", pointWrapper);
                     }
                     break;
@@ -151,7 +156,7 @@ public class PopBufferMergeService extends ServiceThread {
             count += qs;
             if (qs > 5000 && scanTimes % countOfSecond1 == 0) {
                 POP_LOGGER.info("[PopBuffer] offset queue size too long, {}, {}",
-                        entry.getKey(), qs);
+                    entry.getKey(), qs);
             }
         }
         return count;
@@ -214,7 +219,7 @@ public class PopBufferMergeService extends ServiceThread {
 
             // just process offset(already stored at pull thread), or buffer ck(not stored and ack finish)
             if ((pointWrapper.isJustOffset() && pointWrapper.isCkStored()) || isCkDone(pointWrapper)
-                    || (isCkDoneForFinish(pointWrapper) && pointWrapper.isCkStored())) {
+                || (isCkDoneForFinish(pointWrapper) && pointWrapper.isCkStored())) {
                 if (brokerController.getBrokerConfig().isEnablePopLog()) {
                     POP_LOGGER.info("[PopBuffer]ck done, {}", pointWrapper);
                 }
@@ -226,7 +231,7 @@ public class PopBufferMergeService extends ServiceThread {
             PopCheckPoint point = pointWrapper.getCk();
             long now = System.currentTimeMillis();
 
-            boolean removeCk = !this.serving;
+            boolean removeCk = this.serving ? false : true;
             // ck will be timeout
             if (point.getReviveTime() - now < brokerController.getBrokerConfig().getPopCkStayBufferTimeOut()) {
                 removeCk = true;
@@ -265,7 +270,7 @@ public class PopBufferMergeService extends ServiceThread {
                 for (byte i = 0; i < point.getNum(); i++) {
                     // reput buffer ak to store
                     if (DataConverter.getBit(pointWrapper.getBits().get(), i)
-                            && !DataConverter.getBit(pointWrapper.getToStoreBits().get(), i)) {
+                        && !DataConverter.getBit(pointWrapper.getToStoreBits().get(), i)) {
                         if (putAckToStore(pointWrapper, i)) {
                             count++;
                             markBitCAS(pointWrapper.getToStoreBits(), i);
@@ -289,14 +294,14 @@ public class PopBufferMergeService extends ServiceThread {
         long eclipse = System.currentTimeMillis() - startTime;
         if (eclipse > brokerController.getBrokerConfig().getPopCkStayBufferTimeOut() - 1000) {
             POP_LOGGER.warn("[PopBuffer]scan stop, because eclipse too long, PopBufferEclipse={}, " +
-                            "PopBufferToStoreAck={}, PopBufferToStoreCk={}, PopBufferSize={}, PopBufferOffsetSize={}",
-                    eclipse, count, countCk, counter.get(), offsetBufferSize);
+                    "PopBufferToStoreAck={}, PopBufferToStoreCk={}, PopBufferSize={}, PopBufferOffsetSize={}",
+                eclipse, count, countCk, counter.get(), offsetBufferSize);
             this.serving = false;
         } else {
             if (scanTimes % countOfSecond1 == 0) {
                 POP_LOGGER.info("[PopBuffer]scan, PopBufferEclipse={}, " +
-                                "PopBufferToStoreAck={}, PopBufferToStoreCk={}, PopBufferSize={}, PopBufferOffsetSize={}",
-                        eclipse, count, countCk, counter.get(), offsetBufferSize);
+                        "PopBufferToStoreAck={}, PopBufferToStoreCk={}, PopBufferSize={}, PopBufferOffsetSize={}",
+                    eclipse, count, countCk, counter.get(), offsetBufferSize);
             }
         }
         scanTimes++;
@@ -354,7 +359,7 @@ public class PopBufferMergeService extends ServiceThread {
                 POP_LOGGER.warn("Commit offset, consumer offset less than store, {}, {}", wrapper, offset);
             }
             brokerController.getConsumerOffsetManager().commitOffset(getServiceName(),
-                    popCheckPoint.getCId(), popCheckPoint.getTopic(), popCheckPoint.getQueueId(), wrapper.getNextBeginOffset());
+                popCheckPoint.getCId(), popCheckPoint.getTopic(), popCheckPoint.getQueueId(), wrapper.getNextBeginOffset());
         } finally {
             queueLockManager.unLock(lockKey);
         }
@@ -384,6 +389,7 @@ public class PopBufferMergeService extends ServiceThread {
 
     /**
      * put to store && add to buffer.
+     *
      * @param point
      * @param reviveQueueId
      * @param reviveQueueOffset
@@ -404,7 +410,7 @@ public class PopBufferMergeService extends ServiceThread {
     }
 
     public void addCkMock(String group, String topic, int queueId, long startOffset, long invisibleTime,
-                          long popTime, int reviveQueueId, long nextBeginOffset) {
+        long popTime, int reviveQueueId, long nextBeginOffset, String brokerName) {
         final PopCheckPoint ck = new PopCheckPoint();
         ck.setBitMap(0);
         ck.setNum((byte) 0);
@@ -414,6 +420,7 @@ public class PopBufferMergeService extends ServiceThread {
         ck.setCId(group);
         ck.setTopic(topic);
         ck.setQueueId((byte) queueId);
+        ck.setBrokerName(brokerName);
 
         PopCheckPointWrapper pointWrapper = new PopCheckPointWrapper(reviveQueueId, Long.MAX_VALUE, ck, nextBeginOffset, true);
         pointWrapper.setCkStored(true);
@@ -469,7 +476,7 @@ public class PopBufferMergeService extends ServiceThread {
             return false;
         }
         try {
-            PopCheckPointWrapper pointWrapper = this.buffer.get(ackMsg.getTopic() + ackMsg.getConsumerGroup() + ackMsg.getQueueId() + ackMsg.getStartOffset() + ackMsg.getPopTime());
+            PopCheckPointWrapper pointWrapper = this.buffer.get(ackMsg.getTopic() + ackMsg.getConsumerGroup() + ackMsg.getQueueId() + ackMsg.getStartOffset() + ackMsg.getPopTime() + ackMsg.getBrokerName());
             if (pointWrapper == null) {
                 if (brokerController.getBrokerConfig().isEnablePopLog()) {
                     POP_LOGGER.warn("[PopBuffer]add ack fail, rqId={}, no ck, {}", reviveQid, ackMsg);
@@ -523,6 +530,10 @@ public class PopBufferMergeService extends ServiceThread {
         return false;
     }
 
+    public void clearOffsetQueue(String lockKey) {
+        this.commitOffsets.remove(lockKey);
+    }
+
     private void putCkToStore(final PopCheckPointWrapper pointWrapper, final boolean runInCurrent) {
         if (pointWrapper.getReviveQueueOffset() >= 0) {
             return;
@@ -530,9 +541,9 @@ public class PopBufferMergeService extends ServiceThread {
         MessageExtBrokerInner msgInner = popMessageProcessor.buildCkMsg(pointWrapper.getCk(), pointWrapper.getReviveQueueId());
         PutMessageResult putMessageResult = brokerController.getMessageStore().putMessage(msgInner);
         if (putMessageResult.getPutMessageStatus() != PutMessageStatus.PUT_OK
-                && putMessageResult.getPutMessageStatus() != PutMessageStatus.FLUSH_DISK_TIMEOUT
-                && putMessageResult.getPutMessageStatus() != PutMessageStatus.FLUSH_SLAVE_TIMEOUT
-                && putMessageResult.getPutMessageStatus() != PutMessageStatus.SLAVE_NOT_AVAILABLE) {
+            && putMessageResult.getPutMessageStatus() != PutMessageStatus.FLUSH_DISK_TIMEOUT
+            && putMessageResult.getPutMessageStatus() != PutMessageStatus.FLUSH_SLAVE_TIMEOUT
+            && putMessageResult.getPutMessageStatus() != PutMessageStatus.SLAVE_NOT_AVAILABLE) {
             POP_LOGGER.error("[PopBuffer]put ck to store fail: {}, {}", pointWrapper, putMessageResult);
             return;
         }
@@ -568,9 +579,9 @@ public class PopBufferMergeService extends ServiceThread {
         msgInner.setPropertiesString(MessageDecoder.messageProperties2String(msgInner.getProperties()));
         PutMessageResult putMessageResult = brokerController.getMessageStore().putMessage(msgInner);
         if (putMessageResult.getPutMessageStatus() != PutMessageStatus.PUT_OK
-                && putMessageResult.getPutMessageStatus() != PutMessageStatus.FLUSH_DISK_TIMEOUT
-                && putMessageResult.getPutMessageStatus() != PutMessageStatus.FLUSH_SLAVE_TIMEOUT
-                && putMessageResult.getPutMessageStatus() != PutMessageStatus.SLAVE_NOT_AVAILABLE) {
+            && putMessageResult.getPutMessageStatus() != PutMessageStatus.FLUSH_DISK_TIMEOUT
+            && putMessageResult.getPutMessageStatus() != PutMessageStatus.FLUSH_SLAVE_TIMEOUT
+            && putMessageResult.getPutMessageStatus() != PutMessageStatus.SLAVE_NOT_AVAILABLE) {
             POP_LOGGER.error("[PopBuffer]put ack to store fail: {}, {}, {}", pointWrapper, ackMsg, putMessageResult);
             return false;
         }
@@ -581,6 +592,38 @@ public class PopBufferMergeService extends ServiceThread {
         return true;
     }
 
+    private boolean cancelCkTimer(final PopCheckPointWrapper pointWrapper) {
+        // not stored, no need cancel
+        if (pointWrapper.getReviveQueueOffset() < 0) {
+            return true;
+        }
+        PopCheckPoint point = pointWrapper.getCk();
+        MessageExtBrokerInner msgInner = new MessageExtBrokerInner();
+        msgInner.setTopic(popMessageProcessor.reviveTopic);
+        msgInner.setBody((pointWrapper.getReviveQueueId() + "-" + pointWrapper.getReviveQueueOffset()).getBytes());
+        msgInner.setQueueId(pointWrapper.getReviveQueueId());
+        msgInner.setTags(PopAckConstants.CK_TAG);
+        msgInner.setBornTimestamp(System.currentTimeMillis());
+        msgInner.setBornHost(brokerController.getStoreHost());
+        msgInner.setStoreHost(brokerController.getStoreHost());
+
+        MsgUtil.setMessageDeliverTime(brokerController, msgInner, point.getReviveTime() - PopAckConstants.ackTimeInterval);
+
+        msgInner.setPropertiesString(MessageDecoder.messageProperties2String(msgInner.getProperties()));
+        PutMessageResult putMessageResult = brokerController.getEscapeBridge().putMessageToSpecificQueue(msgInner);
+        if (putMessageResult.getPutMessageStatus() != PutMessageStatus.PUT_OK
+            && putMessageResult.getPutMessageStatus() != PutMessageStatus.FLUSH_DISK_TIMEOUT
+            && putMessageResult.getPutMessageStatus() != PutMessageStatus.FLUSH_SLAVE_TIMEOUT
+            && putMessageResult.getPutMessageStatus() != PutMessageStatus.SLAVE_NOT_AVAILABLE) {
+            POP_LOGGER.error("[PopBuffer]PutMessageCallback cancelCheckPoint fail, {}, {}", pointWrapper, putMessageResult);
+            return false;
+        }
+        if (brokerController.getBrokerConfig().isEnablePopLog()) {
+            POP_LOGGER.info("[PopBuffer]cancelCheckPoint, {}", pointWrapper);
+        }
+        return true;
+    }
+
     private boolean isCkDone(PopCheckPointWrapper pointWrapper) {
         byte num = pointWrapper.getCk().getNum();
         for (byte i = 0; i < num; i++) {
@@ -639,7 +682,8 @@ public class PopBufferMergeService extends ServiceThread {
         private final boolean justOffset;
         private volatile boolean ckStored = false;
 
-        public PopCheckPointWrapper(int reviveQueueId, long reviveQueueOffset, PopCheckPoint point, long nextBeginOffset) {
+        public PopCheckPointWrapper(int reviveQueueId, long reviveQueueOffset, PopCheckPoint point,
+            long nextBeginOffset) {
             this.reviveQueueId = reviveQueueId;
             this.reviveQueueOffset = reviveQueueOffset;
             this.ck = point;
@@ -647,12 +691,13 @@ public class PopBufferMergeService extends ServiceThread {
             this.toStoreBits = new AtomicInteger(0);
             this.nextBeginOffset = nextBeginOffset;
             this.lockKey = ck.getTopic() + PopAckConstants.SPLIT + ck.getCId() + PopAckConstants.SPLIT + ck.getQueueId();
-            this.mergeKey = point.getTopic() + point.getCId() + point.getQueueId() + point.getStartOffset() + point.getPopTime();
+            this.mergeKey = point.getTopic() + point.getCId() + point.getQueueId() + point.getStartOffset() + point.getPopTime() + point.getBrokerName();
             this.justOffset = false;
         }
 
-        public PopCheckPointWrapper(int reviveQueueId, long reviveQueueOffset, PopCheckPoint point, long nextBeginOffset,
-                                    boolean justOffset) {
+        public PopCheckPointWrapper(int reviveQueueId, long reviveQueueOffset, PopCheckPoint point,
+            long nextBeginOffset,
+            boolean justOffset) {
             this.reviveQueueId = reviveQueueId;
             this.reviveQueueOffset = reviveQueueOffset;
             this.ck = point;
@@ -660,7 +705,7 @@ public class PopBufferMergeService extends ServiceThread {
             this.toStoreBits = new AtomicInteger(0);
             this.nextBeginOffset = nextBeginOffset;
             this.lockKey = ck.getTopic() + PopAckConstants.SPLIT + ck.getCId() + PopAckConstants.SPLIT + ck.getQueueId();
-            this.mergeKey = point.getTopic() + point.getCId() + point.getQueueId() + point.getStartOffset() + point.getPopTime();
+            this.mergeKey = point.getTopic() + point.getCId() + point.getQueueId() + point.getStartOffset() + point.getPopTime() + point.getBrokerName();
             this.justOffset = justOffset;
         }
 
diff --git a/broker/src/main/java/org/apache/rocketmq/broker/processor/PopMessageProcessor.java b/broker/src/main/java/org/apache/rocketmq/broker/processor/PopMessageProcessor.java
index fcc972d..c9f4485 100644
--- a/broker/src/main/java/org/apache/rocketmq/broker/processor/PopMessageProcessor.java
+++ b/broker/src/main/java/org/apache/rocketmq/broker/processor/PopMessageProcessor.java
@@ -68,14 +68,14 @@ import org.apache.rocketmq.remoting.netty.RequestTask;
 import org.apache.rocketmq.remoting.protocol.RemotingCommand;
 import org.apache.rocketmq.store.GetMessageResult;
 import org.apache.rocketmq.store.GetMessageStatus;
-import org.apache.rocketmq.store.MessageExtBrokerInner;
+import org.apache.rocketmq.common.message.MessageExtBrokerInner;
 import org.apache.rocketmq.store.SelectMappedBufferResult;
 import org.apache.rocketmq.store.pop.AckMsg;
 import org.apache.rocketmq.store.pop.PopCheckPoint;
 
 public class PopMessageProcessor implements NettyRequestProcessor {
     private static final InternalLogger POP_LOGGER =
-            InternalLoggerFactory.getLogger(LoggerName.ROCKETMQ_POP_LOGGER_NAME);
+        InternalLoggerFactory.getLogger(LoggerName.ROCKETMQ_POP_LOGGER_NAME);
     private final BrokerController brokerController;
     private Random random = new Random(System.currentTimeMillis());
     String reviveTopic;
@@ -96,12 +96,11 @@ public class PopMessageProcessor implements NettyRequestProcessor {
 
     public PopMessageProcessor(final BrokerController brokerController) {
         this.brokerController = brokerController;
-        this.reviveTopic =
-                PopAckConstants.REVIVE_TOPIC + this.brokerController.getBrokerConfig().getBrokerClusterName();
-        // 100000 topic default,  100000 lru topic + cid + qid 
+        this.reviveTopic = PopAckConstants.buildClusterReviveTopic(this.brokerController.getBrokerConfig().getBrokerClusterName());
+        // 100000 topic default,  100000 lru topic + cid + qid
         this.topicCidMap = new ConcurrentHashMap<>(this.brokerController.getBrokerConfig().getPopPollingMapSize());
         this.pollingMap = new ConcurrentLinkedHashMap.Builder<String, ConcurrentSkipListSet<PopRequest>>()
-                .maximumWeightedCapacity(this.brokerController.getBrokerConfig().getPopPollingMapSize()).build();
+            .maximumWeightedCapacity(this.brokerController.getBrokerConfig().getPopPollingMapSize()).build();
         this.popLongPollingService = new PopLongPollingService();
         this.queueLockManager = new QueueLockManager();
         this.popBufferMergeService = new PopBufferMergeService(this.brokerController, this);
@@ -122,24 +121,27 @@ public class PopMessageProcessor implements NettyRequestProcessor {
 
     public static String genAckUniqueId(AckMsg ackMsg) {
         return ackMsg.getTopic()
-                + PopAckConstants.SPLIT + ackMsg.getQueueId()
-                + PopAckConstants.SPLIT + ackMsg.getAckOffset()
-                + PopAckConstants.SPLIT + ackMsg.getConsumerGroup()
-                + PopAckConstants.SPLIT + ackMsg.getPopTime()
-                + PopAckConstants.SPLIT + PopAckConstants.ACK_TAG;
+            + PopAckConstants.SPLIT + ackMsg.getQueueId()
+            + PopAckConstants.SPLIT + ackMsg.getAckOffset()
+            + PopAckConstants.SPLIT + ackMsg.getConsumerGroup()
+            + PopAckConstants.SPLIT + ackMsg.getPopTime()
+            + PopAckConstants.SPLIT + ackMsg.getBrokerName()
+            + PopAckConstants.SPLIT + PopAckConstants.ACK_TAG;
     }
 
     public static String genCkUniqueId(PopCheckPoint ck) {
         return ck.getTopic()
-                + PopAckConstants.SPLIT + ck.getQueueId()
-                + PopAckConstants.SPLIT + ck.getStartOffset()
-                + PopAckConstants.SPLIT + ck.getCId()
-                + PopAckConstants.SPLIT + ck.getPopTime()
-                + PopAckConstants.SPLIT + PopAckConstants.CK_TAG;
+            + PopAckConstants.SPLIT + ck.getQueueId()
+            + PopAckConstants.SPLIT + ck.getStartOffset()
+            + PopAckConstants.SPLIT + ck.getCId()
+            + PopAckConstants.SPLIT + ck.getPopTime()
+            + PopAckConstants.SPLIT + ck.getBrokerName()
+            + PopAckConstants.SPLIT + PopAckConstants.CK_TAG;
     }
 
     @Override
-    public RemotingCommand processRequest(final ChannelHandlerContext ctx, RemotingCommand request) throws RemotingCommandException {
+    public RemotingCommand processRequest(final ChannelHandlerContext ctx,
+        RemotingCommand request) throws RemotingCommandException {
         request.addExtField(BORN_TIME, String.valueOf(System.currentTimeMillis()));
         return this.processRequest(ctx.channel(), request);
     }
@@ -167,8 +169,7 @@ public class PopMessageProcessor implements NettyRequestProcessor {
     }
 
     public void notifyMessageArriving(final String topic, final String cid, final int queueId) {
-        ConcurrentSkipListSet<PopRequest> remotingCommands = pollingMap.get(KeyBuilder.buildPollingKey(topic, cid,
-                queueId));
+        ConcurrentSkipListSet<PopRequest> remotingCommands = pollingMap.get(KeyBuilder.buildPollingKey(topic, cid, queueId));
         if (remotingCommands == null || remotingCommands.isEmpty()) {
             return;
         }
@@ -192,12 +193,14 @@ public class PopMessageProcessor implements NettyRequestProcessor {
         if (request == null || !request.complete()) {
             return;
         }
+        if (!request.getChannel().isActive()) {
+            return;
+        }
         Runnable run = new Runnable() {
             @Override
             public void run() {
                 try {
-                    final RemotingCommand response = PopMessageProcessor.this.processRequest(request.getChannel(),
-                            request.getRemotingCommand());
+                    final RemotingCommand response = PopMessageProcessor.this.processRequest(request.getChannel(), request.getRemotingCommand());
 
                     if (response != null) {
                         response.setOpaque(request.getRemotingCommand().getOpaque());
@@ -207,8 +210,7 @@ public class PopMessageProcessor implements NettyRequestProcessor {
                                 @Override
                                 public void operationComplete(ChannelFuture future) throws Exception {
                                     if (!future.isSuccess()) {
-                                        POP_LOGGER.error("ProcessRequestWrapper response to {} failed",
-                                                future.channel().remoteAddress(), future.cause());
+                                        POP_LOGGER.error("ProcessRequestWrapper response to {} failed", future.channel().remoteAddress(), future.cause());
                                         POP_LOGGER.error(request.toString());
                                         POP_LOGGER.error(response.toString());
                                     }
@@ -225,16 +227,15 @@ public class PopMessageProcessor implements NettyRequestProcessor {
                 }
             }
         };
-        this.brokerController.getPullMessageExecutor().submit(new RequestTask(run, request.getChannel(),
-                request.getRemotingCommand()));
+        this.brokerController.getPullMessageExecutor().submit(new RequestTask(run, request.getChannel(), request.getRemotingCommand()));
     }
 
     private RemotingCommand processRequest(final Channel channel, RemotingCommand request)
-            throws RemotingCommandException {
+        throws RemotingCommandException {
         RemotingCommand response = RemotingCommand.createResponseCommand(PopMessageResponseHeader.class);
         final PopMessageResponseHeader responseHeader = (PopMessageResponseHeader) response.readCustomHeader();
         final PopMessageRequestHeader requestHeader =
-                (PopMessageRequestHeader) request.decodeCommandCustomHeader(PopMessageRequestHeader.class);
+            (PopMessageRequestHeader) request.decodeCommandCustomHeader(PopMessageRequestHeader.class);
         StringBuilder startOffsetInfo = new StringBuilder(64);
         StringBuilder msgOffsetInfo = new StringBuilder(64);
         StringBuilder orderCountInfo = null;
@@ -251,30 +252,30 @@ public class PopMessageProcessor implements NettyRequestProcessor {
         if (requestHeader.isTimeoutTooMuch()) {
             response.setCode(POLLING_TIMEOUT);
             response.setRemark(String.format("the broker[%s] poping message is timeout too much",
-                    this.brokerController.getBrokerConfig().getBrokerIP1()));
+                this.brokerController.getBrokerConfig().getBrokerIP1()));
             return response;
         }
         if (!PermName.isReadable(this.brokerController.getBrokerConfig().getBrokerPermission())) {
             response.setCode(ResponseCode.NO_PERMISSION);
             response.setRemark(String.format("the broker[%s] poping message is forbidden",
-                    this.brokerController.getBrokerConfig().getBrokerIP1()));
+                this.brokerController.getBrokerConfig().getBrokerIP1()));
             return response;
         }
         if (requestHeader.getMaxMsgNums() > 32) {
             response.setCode(ResponseCode.SYSTEM_ERROR);
             response.setRemark(String.format("the broker[%s] poping message's num is greater than 32",
-                    this.brokerController.getBrokerConfig().getBrokerIP1()));
+                this.brokerController.getBrokerConfig().getBrokerIP1()));
             return response;
         }
 
         TopicConfig topicConfig =
-                this.brokerController.getTopicConfigManager().selectTopicConfig(requestHeader.getTopic());
+            this.brokerController.getTopicConfigManager().selectTopicConfig(requestHeader.getTopic());
         if (null == topicConfig) {
             POP_LOGGER.error("The topic {} not exist, consumer: {} ", requestHeader.getTopic(),
-                    RemotingHelper.parseChannelRemoteAddr(channel));
+                RemotingHelper.parseChannelRemoteAddr(channel));
             response.setCode(ResponseCode.TOPIC_NOT_EXIST);
             response.setRemark(String.format("topic[%s] not exist, apply first please! %s", requestHeader.getTopic(),
-                    FAQUrl.suggestTodo(FAQUrl.APPLY_TOPIC_URL)));
+                FAQUrl.suggestTodo(FAQUrl.APPLY_TOPIC_URL)));
             return response;
         }
 
@@ -286,20 +287,20 @@ public class PopMessageProcessor implements NettyRequestProcessor {
 
         if (requestHeader.getQueueId() >= topicConfig.getReadQueueNums()) {
             String errorInfo = String.format("queueId[%d] is illegal, topic:[%s] topicConfig.readQueueNums:[%d] " +
-                            "consumer:[%s]",
-                    requestHeader.getQueueId(), requestHeader.getTopic(), topicConfig.getReadQueueNums(),
-                    channel.remoteAddress());
+                    "consumer:[%s]",
+                requestHeader.getQueueId(), requestHeader.getTopic(), topicConfig.getReadQueueNums(),
+                channel.remoteAddress());
             POP_LOGGER.warn(errorInfo);
             response.setCode(ResponseCode.SYSTEM_ERROR);
             response.setRemark(errorInfo);
             return response;
         }
         SubscriptionGroupConfig subscriptionGroupConfig =
-                this.brokerController.getSubscriptionGroupManager().findSubscriptionGroupConfig(requestHeader.getConsumerGroup());
+            this.brokerController.getSubscriptionGroupManager().findSubscriptionGroupConfig(requestHeader.getConsumerGroup());
         if (null == subscriptionGroupConfig) {
             response.setCode(ResponseCode.SUBSCRIPTION_GROUP_NOT_EXIST);
             response.setRemark(String.format("subscription group [%s] does not exist, %s",
-                    requestHeader.getConsumerGroup(), FAQUrl.suggestTodo(FAQUrl.SUBSCRIPTION_GROUP_NOT_EXIST)));
+                requestHeader.getConsumerGroup(), FAQUrl.suggestTodo(FAQUrl.SUBSCRIPTION_GROUP_NOT_EXIST)));
             return response;
         }
         ConsumerGroupInfo consumerGroupInfo =
@@ -311,7 +312,6 @@ public class PopMessageProcessor implements NettyRequestProcessor {
             return response;
         }
 
-
         if (!subscriptionGroupConfig.isConsumeEnable()) {
             response.setCode(ResponseCode.NO_PERMISSION);
             response.setRemark("subscription group no permission, " + requestHeader.getConsumerGroup());
@@ -325,22 +325,22 @@ public class PopMessageProcessor implements NettyRequestProcessor {
                 ConsumerFilterData consumerFilterData = null;
                 if (!ExpressionType.isTagType(subscriptionData.getExpressionType())) {
                     consumerFilterData = ConsumerFilterManager.build(
-                            requestHeader.getTopic(), requestHeader.getConsumerGroup(), requestHeader.getExp(),
-                            requestHeader.getExpType(), System.currentTimeMillis()
+                        requestHeader.getTopic(), requestHeader.getConsumerGroup(), requestHeader.getExp(),
+                        requestHeader.getExpType(), System.currentTimeMillis()
                     );
                     if (consumerFilterData == null) {
                         POP_LOGGER.warn("Parse the consumer's subscription[{}] failed, group: {}",
-                                requestHeader.getExp(), requestHeader.getConsumerGroup());
+                            requestHeader.getExp(), requestHeader.getConsumerGroup());
                         response.setCode(ResponseCode.SUBSCRIPTION_PARSE_FAILED);
                         response.setRemark("parse the consumer's subscription failed");
                         return response;
                     }
                 }
                 messageFilter = new ExpressionMessageFilter(subscriptionData, consumerFilterData,
-                        brokerController.getConsumerFilterManager());
+                    brokerController.getConsumerFilterManager());
             } catch (Exception e) {
                 POP_LOGGER.warn("Parse the consumer's subscription[{}] error, group: {}", requestHeader.getExp(),
-                        requestHeader.getConsumerGroup());
+                    requestHeader.getConsumerGroup());
                 response.setCode(ResponseCode.SUBSCRIPTION_PARSE_FAILED);
                 response.setRemark("parse the consumer's subscription failed");
                 return response;
@@ -355,20 +355,21 @@ public class PopMessageProcessor implements NettyRequestProcessor {
             reviveQid = (int) Math.abs(ckMessageNumber.getAndIncrement() % this.brokerController.getBrokerConfig().getReviveQueueNum());
         }
 
-        GetMessageResult getMessageResult = new GetMessageResult();
+        int commercialSizePerMsg = this.brokerController.getBrokerConfig().getCommercialSizePerMsg();
+        GetMessageResult getMessageResult = new GetMessageResult(commercialSizePerMsg);
 
         long restNum = 0;
         boolean needRetry = randomQ % 5 == 0;
         long popTime = System.currentTimeMillis();
         if (needRetry && !requestHeader.isOrder()) {
             TopicConfig retryTopicConfig =
-                    this.brokerController.getTopicConfigManager().selectTopicConfig(KeyBuilder.buildPopRetryTopic(requestHeader.getTopic(), requestHeader.getConsumerGroup()));
+                this.brokerController.getTopicConfigManager().selectTopicConfig(KeyBuilder.buildPopRetryTopic(requestHeader.getTopic(), requestHeader.getConsumerGroup()));
             if (retryTopicConfig != null) {
                 for (int i = 0; i < retryTopicConfig.getReadQueueNums(); i++) {
                     int queueId = (randomQ + i) % retryTopicConfig.getReadQueueNums();
                     restNum = popMsgFromQueue(true, getMessageResult, requestHeader, queueId, restNum, reviveQid,
-                            channel, popTime, messageFilter,
-                            startOffsetInfo, msgOffsetInfo, orderCountInfo);
+                        channel, popTime, messageFilter,
+                        startOffsetInfo, msgOffsetInfo, orderCountInfo);
                 }
             }
         }
@@ -382,19 +383,19 @@ public class PopMessageProcessor implements NettyRequestProcessor {
         } else {
             int queueId = requestHeader.getQueueId();
             restNum = popMsgFromQueue(false, getMessageResult, requestHeader, queueId, restNum, reviveQid, channel,
-                    popTime, messageFilter,
+                popTime, messageFilter,
                 startOffsetInfo, msgOffsetInfo, orderCountInfo);
         }
         // if not full , fetch retry again
         if (!needRetry && getMessageResult.getMessageMapedList().size() < requestHeader.getMaxMsgNums() && !requestHeader.isOrder()) {
             TopicConfig retryTopicConfig =
-                    this.brokerController.getTopicConfigManager().selectTopicConfig(KeyBuilder.buildPopRetryTopic(requestHeader.getTopic(), requestHeader.getConsumerGroup()));
+                this.brokerController.getTopicConfigManager().selectTopicConfig(KeyBuilder.buildPopRetryTopic(requestHeader.getTopic(), requestHeader.getConsumerGroup()));
             if (retryTopicConfig != null) {
                 for (int i = 0; i < retryTopicConfig.getReadQueueNums(); i++) {
                     int queueId = (randomQ + i) % retryTopicConfig.getReadQueueNums();
                     restNum = popMsgFromQueue(true, getMessageResult, requestHeader, queueId, restNum, reviveQid,
-                            channel, popTime, messageFilter,
-                            startOffsetInfo, msgOffsetInfo, orderCountInfo);
+                        channel, popTime, messageFilter,
+                        startOffsetInfo, msgOffsetInfo, orderCountInfo);
                 }
             }
         }
@@ -404,7 +405,7 @@ public class PopMessageProcessor implements NettyRequestProcessor {
             if (restNum > 0) {
                 // all queue pop can not notify specified queue pop, and vice versa
                 notifyMessageArriving(requestHeader.getTopic(), requestHeader.getConsumerGroup(),
-                        requestHeader.getQueueId());
+                    requestHeader.getQueueId());
             }
         } else {
             int pollingResult = polling(channel, request, requestHeader);
@@ -432,24 +433,24 @@ public class PopMessageProcessor implements NettyRequestProcessor {
                 if (this.brokerController.getBrokerConfig().isTransferMsgByHeap()) {
                     final long beginTimeMills = this.brokerController.getMessageStore().now();
                     final byte[] r = this.readGetMessageResult(getMessageResult, requestHeader.getConsumerGroup(),
-                            requestHeader.getTopic(), requestHeader.getQueueId());
+                        requestHeader.getTopic(), requestHeader.getQueueId());
                     this.brokerController.getBrokerStatsManager().incGroupGetLatency(requestHeader.getConsumerGroup(),
-                            requestHeader.getTopic(), requestHeader.getQueueId(),
-                            (int) (this.brokerController.getMessageStore().now() - beginTimeMills));
+                        requestHeader.getTopic(), requestHeader.getQueueId(),
+                        (int) (this.brokerController.getMessageStore().now() - beginTimeMills));
                     response.setBody(r);
                 } else {
                     final GetMessageResult tmpGetMessageResult = getMessageResult;
                     try {
                         FileRegion fileRegion =
-                                new ManyMessageTransfer(response.encodeHeader(getMessageResult.getBufferTotalSize()),
-                                        getMessageResult);
+                            new ManyMessageTransfer(response.encodeHeader(getMessageResult.getBufferTotalSize()),
+                                getMessageResult);
                         channel.writeAndFlush(fileRegion).addListener(new ChannelFutureListener() {
                             @Override
                             public void operationComplete(ChannelFuture future) throws Exception {
                                 tmpGetMessageResult.release();
                                 if (!future.isSuccess()) {
                                     POP_LOGGER.error("Fail to transfer messages from page cache to {}",
-                                            channel.remoteAddress(), future.cause());
+                                        channel.remoteAddress(), future.cause());
                                 }
                             }
                         });
@@ -468,14 +469,14 @@ public class PopMessageProcessor implements NettyRequestProcessor {
     }
 
     private long popMsgFromQueue(boolean isRetry, GetMessageResult getMessageResult,
-                                 PopMessageRequestHeader requestHeader, int queueId, long restNum, int reviveQid,
-                                 Channel channel, long popTime,
-                                 ExpressionMessageFilter messageFilter, StringBuilder startOffsetInfo,
-                                 StringBuilder msgOffsetInfo, StringBuilder orderCountInfo) {
+        PopMessageRequestHeader requestHeader, int queueId, long restNum, int reviveQid,
+        Channel channel, long popTime,
+        ExpressionMessageFilter messageFilter, StringBuilder startOffsetInfo,
+        StringBuilder msgOffsetInfo, StringBuilder orderCountInfo) {
         String topic = isRetry ? KeyBuilder.buildPopRetryTopic(requestHeader.getTopic(),
-                requestHeader.getConsumerGroup()) : requestHeader.getTopic();
+            requestHeader.getConsumerGroup()) : requestHeader.getTopic();
         String lockKey =
-                topic + PopAckConstants.SPLIT + requestHeader.getConsumerGroup() + PopAckConstants.SPLIT + queueId;
+            topic + PopAckConstants.SPLIT + requestHeader.getConsumerGroup() + PopAckConstants.SPLIT + queueId;
         boolean isOrder = requestHeader.isOrder();
         long offset = getPopOffset(topic, requestHeader, queueId, false, lockKey);
         if (!queueLockManager.tryLock(lockKey)) {
@@ -486,34 +487,34 @@ public class PopMessageProcessor implements NettyRequestProcessor {
         GetMessageResult getMessageTmpResult;
         try {
             if (isOrder && brokerController.getConsumerOrderInfoManager().checkBlock(topic,
-                    requestHeader.getConsumerGroup(), queueId, requestHeader.getInvisibleTime())) {
+                requestHeader.getConsumerGroup(), queueId, requestHeader.getInvisibleTime())) {
                 return this.brokerController.getMessageStore().getMaxOffsetInQueue(topic, queueId) - offset + restNum;
             }
 
             if (getMessageResult.getMessageMapedList().size() >= requestHeader.getMaxMsgNums()) {
                 restNum =
-                        this.brokerController.getMessageStore().getMaxOffsetInQueue(topic, queueId) - offset + restNum;
+                    this.brokerController.getMessageStore().getMaxOffsetInQueue(topic, queueId) - offset + restNum;
                 return restNum;
             }
             getMessageTmpResult = this.brokerController.getMessageStore().getMessage(requestHeader.getConsumerGroup()
-                    , topic, queueId, offset,
-                    requestHeader.getMaxMsgNums() - getMessageResult.getMessageMapedList().size(), messageFilter);
+                , topic, queueId, offset,
+                requestHeader.getMaxMsgNums() - getMessageResult.getMessageMapedList().size(), messageFilter);
             // maybe store offset is not correct.
             if (getMessageTmpResult == null
                 || GetMessageStatus.OFFSET_TOO_SMALL.equals(getMessageTmpResult.getStatus())
-                    || GetMessageStatus.OFFSET_OVERFLOW_BADLY.equals(getMessageTmpResult.getStatus())
-                    || GetMessageStatus.OFFSET_FOUND_NULL.equals(getMessageTmpResult.getStatus())) {
+                || GetMessageStatus.OFFSET_OVERFLOW_BADLY.equals(getMessageTmpResult.getStatus())
+                || GetMessageStatus.OFFSET_FOUND_NULL.equals(getMessageTmpResult.getStatus())) {
                 // commit offset, because the offset is not correct
                 // If offset in store is greater than cq offset, it will cause duplicate messages,
                 // because offset in PopBuffer is not committed.
                 POP_LOGGER.warn("Pop initial offset, because store is no correct, {}, {}->{}",
-                        lockKey, offset, getMessageTmpResult != null ? getMessageTmpResult.getNextBeginOffset() : "null");
+                    lockKey, offset, getMessageTmpResult != null ? getMessageTmpResult.getNextBeginOffset() : "null");
                 offset = getMessageTmpResult != null ? getMessageTmpResult.getNextBeginOffset() : 0;
                 this.brokerController.getConsumerOffsetManager().commitOffset(channel.remoteAddress().toString(), requestHeader.getConsumerGroup(), topic,
-                        queueId, offset);
+                    queueId, offset);
                 getMessageTmpResult =
-                        this.brokerController.getMessageStore().getMessage(requestHeader.getConsumerGroup(), topic,
-                                queueId, offset,
+                    this.brokerController.getMessageStore().getMessage(requestHeader.getConsumerGroup(), topic,
+                        queueId, offset,
                         requestHeader.getMaxMsgNums() - getMessageResult.getMessageMapedList().size(), messageFilter);
             }
 
@@ -521,30 +522,32 @@ public class PopMessageProcessor implements NettyRequestProcessor {
             if (!getMessageTmpResult.getMessageMapedList().isEmpty()) {
                 this.brokerController.getBrokerStatsManager().incBrokerGetNums(getMessageTmpResult.getMessageCount());
                 this.brokerController.getBrokerStatsManager().incGroupGetNums(requestHeader.getConsumerGroup(), topic,
-                        getMessageTmpResult.getMessageCount());
+                    getMessageTmpResult.getMessageCount());
                 this.brokerController.getBrokerStatsManager().incGroupGetSize(requestHeader.getConsumerGroup(), topic,
-                        getMessageTmpResult.getBufferTotalSize());
+                    getMessageTmpResult.getBufferTotalSize());
 
                 if (isOrder) {
                     int count = brokerController.getConsumerOrderInfoManager().update(topic,
-                            requestHeader.getConsumerGroup(),
-                            queueId, getMessageTmpResult.getMessageQueueOffset());
+                        requestHeader.getConsumerGroup(),
+                        queueId, getMessageTmpResult.getMessageQueueOffset());
                     this.brokerController.getConsumerOffsetManager().commitOffset(channel.remoteAddress().toString(),
-                            requestHeader.getConsumerGroup(), topic, queueId, offset);
+                        requestHeader.getConsumerGroup(), topic, queueId, offset);
                     ExtraInfoUtil.buildOrderCountInfo(orderCountInfo, isRetry, queueId, count);
                 } else {
-                    appendCheckPoint(requestHeader, topic, reviveQid, queueId, offset, getMessageTmpResult, popTime);
+                    appendCheckPoint(requestHeader, topic, reviveQid, queueId, offset, getMessageTmpResult, popTime, this.brokerController.getBrokerConfig().getBrokerName());
                 }
                 ExtraInfoUtil.buildStartOffsetInfo(startOffsetInfo, isRetry, queueId, offset);
                 ExtraInfoUtil.buildMsgOffsetInfo(msgOffsetInfo, isRetry, queueId,
-                        getMessageTmpResult.getMessageQueueOffset());
+                    getMessageTmpResult.getMessageQueueOffset());
             } else if ((GetMessageStatus.NO_MATCHED_MESSAGE.equals(getMessageTmpResult.getStatus())
-                    || GetMessageStatus.OFFSET_FOUND_NULL.equals(getMessageTmpResult.getStatus())
-                    || GetMessageStatus.MESSAGE_WAS_REMOVING.equals(getMessageTmpResult.getStatus())
-                    || GetMessageStatus.NO_MATCHED_LOGIC_QUEUE.equals(getMessageTmpResult.getStatus()))
-                    && getMessageTmpResult.getNextBeginOffset() > -1) {
+                || GetMessageStatus.OFFSET_FOUND_NULL.equals(getMessageTmpResult.getStatus())
+                || GetMessageStatus.MESSAGE_WAS_REMOVING.equals(getMessageTmpResult.getStatus())
+                || GetMessageStatus.NO_MATCHED_LOGIC_QUEUE.equals(getMessageTmpResult.getStatus()))
+                && getMessageTmpResult.getNextBeginOffset() > -1) {
                 popBufferMergeService.addCkMock(requestHeader.getConsumerGroup(), topic, queueId, offset,
-                        requestHeader.getInvisibleTime(), popTime, reviveQid, getMessageTmpResult.getNextBeginOffset());
+                    requestHeader.getInvisibleTime(), popTime, reviveQid, getMessageTmpResult.getNextBeginOffset(), brokerController.getBrokerConfig().getBrokerName());
+//                this.brokerController.getConsumerOffsetManager().commitOffset(channel.remoteAddress().toString(), requestHeader.getConsumerGroup(), topic,
+//                        queueId, getMessageTmpResult.getNextBeginOffset());
             }
         } finally {
             queueLockManager.unLock(lockKey);
@@ -558,9 +561,9 @@ public class PopMessageProcessor implements NettyRequestProcessor {
     }
 
     private long getPopOffset(String topic, PopMessageRequestHeader requestHeader, int queueId, boolean init,
-                              String lockKey) {
+        String lockKey) {
         long offset = this.brokerController.getConsumerOffsetManager().queryOffset(requestHeader.getConsumerGroup(),
-                topic, queueId);
+            topic, queueId);
         if (offset < 0) {
             if (ConsumeInitMode.MIN == requestHeader.getInitMode()) {
                 offset = this.brokerController.getMessageStore().getMinOffsetInQueue(topic, queueId);
@@ -573,8 +576,8 @@ public class PopMessageProcessor implements NettyRequestProcessor {
                 }
                 if (init) {
                     this.brokerController.getConsumerOffsetManager().commitOffset("getPopOffset",
-                            requestHeader.getConsumerGroup(), topic,
-                            queueId, offset);
+                        requestHeader.getConsumerGroup(), topic,
+                        queueId, offset);
                 }
             }
         }
@@ -593,7 +596,7 @@ public class PopMessageProcessor implements NettyRequestProcessor {
      * @return
      */
     private int polling(final Channel channel, RemotingCommand remotingCommand,
-                        final PopMessageRequestHeader requestHeader) {
+        final PopMessageRequestHeader requestHeader) {
         if (requestHeader.getPollTime() <= 0 || this.popLongPollingService.isStopped()) {
             return NOT_POLLING;
         }
@@ -621,7 +624,7 @@ public class PopMessageProcessor implements NettyRequestProcessor {
             return POLLING_TIMEOUT;
         }
         String key = KeyBuilder.buildPollingKey(requestHeader.getTopic(), requestHeader.getConsumerGroup(),
-                requestHeader.getQueueId());
+            requestHeader.getQueueId());
         ConcurrentSkipListSet<PopRequest> queue = pollingMap.get(key);
         if (queue == null) {
             queue = new ConcurrentSkipListSet<>(PopRequest.COMPARATOR);
@@ -667,8 +670,8 @@ public class PopMessageProcessor implements NettyRequestProcessor {
     }
 
     private void appendCheckPoint(final PopMessageRequestHeader requestHeader,
-                                  final String topic, final int reviveQid, final int queueId, final long offset,
-                                  final GetMessageResult getMessageTmpResult, final long popTime) {
+        final String topic, final int reviveQid, final int queueId, final long offset,
+        final GetMessageResult getMessageTmpResult, final long popTime, final String brokerName) {
         // add check point msg to revive log
         final PopCheckPoint ck = new PopCheckPoint();
         ck.setBitMap(0);
@@ -679,12 +682,13 @@ public class PopMessageProcessor implements NettyRequestProcessor {
         ck.setCId(requestHeader.getConsumerGroup());
         ck.setTopic(topic);
         ck.setQueueId((byte) queueId);
+        ck.setBrokerName(brokerName);
         for (Long msgQueueOffset : getMessageTmpResult.getMessageQueueOffset()) {
             ck.addDiff((int) (msgQueueOffset - offset));
         }
 
         final boolean addBufferSuc = this.popBufferMergeService.addCk(
-                ck, reviveQid, -1, getMessageTmpResult.getNextBeginOffset()
+            ck, reviveQid, -1, getMessageTmpResult.getNextBeginOffset()
         );
 
         if (addBufferSuc) {
@@ -692,12 +696,12 @@ public class PopMessageProcessor implements NettyRequestProcessor {
         }
 
         this.popBufferMergeService.addCkJustOffset(
-                ck, reviveQid, -1, getMessageTmpResult.getNextBeginOffset()
+            ck, reviveQid, -1, getMessageTmpResult.getNextBeginOffset()
         );
     }
 
-    private byte[] readGetMessageResult(final GetMessageResult getMessageResult, final String group,
-                                        final String topic, final int queueId) {
+    private byte[] readGetMessageResult(final GetMessageResult getMessageResult, final String group, final String topic,
+        final int queueId) {
         final ByteBuffer byteBuffer = ByteBuffer.allocate(getMessageResult.getBufferTotalSize());
 
         long storeTimestamp = 0;
@@ -713,7 +717,7 @@ public class PopMessageProcessor implements NettyRequestProcessor {
         }
 
         this.brokerController.getBrokerStatsManager().recordDiskFallBehindTime(group, topic, queueId,
-                this.brokerController.getMessageStore().now() - storeTimestamp);
+            this.brokerController.getMessageStore().now() - storeTimestamp);
         return byteBuffer.array();
     }
 
@@ -723,7 +727,10 @@ public class PopMessageProcessor implements NettyRequestProcessor {
 
         @Override
         public String getServiceName() {
-            return "PopLongPollingService";
+            if (PopMessageProcessor.this.brokerController.getBrokerConfig().isInBrokerContainer()) {
+                return PopMessageProcessor.this.brokerController.getBrokerConfig().getLoggerIdentifier() + PopLongPollingService.class.getName();
+            }
+            return PopLongPollingService.class.getName();
         }
 
         private void cleanUnusedResource() {
@@ -819,7 +826,8 @@ public class PopMessageProcessor implements NettyRequestProcessor {
                             }
                             totalPollingNum.decrementAndGet();
                             wakeUp(first);
-                        } while (true);
+                        }
+                        while (true);
                         if (i >= 100) {
                             long tmpPollingNum = popQ.size();
                             tmpTotalPollingNum = tmpTotalPollingNum + tmpPollingNum;
@@ -831,8 +839,8 @@ public class PopMessageProcessor implements NettyRequestProcessor {
 
                     if (i >= 100) {
                         POP_LOGGER.info("pollingMapSize={},tmpTotalSize={},atomicTotalSize={},diffSize={}",
-                                pollingMap.size(), tmpTotalPollingNum, totalPollingNum.get(),
-                                Math.abs(totalPollingNum.get() - tmpTotalPollingNum));
+                            pollingMap.size(), tmpTotalPollingNum, totalPollingNum.get(),
+                            Math.abs(totalPollingNum.get() - tmpTotalPollingNum));
                         totalPollingNum.set(tmpTotalPollingNum);
                         i = 0;
                     }
@@ -893,7 +901,7 @@ public class PopMessageProcessor implements NettyRequestProcessor {
         }
     }
 
-    public static class QueueLockManager extends ServiceThread {
+    public class QueueLockManager extends ServiceThread {
         private ConcurrentHashMap<String, TimedLock> expiredLocalCache = new ConcurrentHashMap<>(100000);
 
         public boolean tryLock(String key) {
@@ -931,8 +939,8 @@ public class PopMessageProcessor implements NettyRequestProcessor {
                 if (System.currentTimeMillis() - entry.getValue().getLockTime() > usedExpireMillis) {
                     iterator.remove();
                     POP_LOGGER.info("Remove unused queue lock: {}, {}, {}", entry.getKey(),
-                            entry.getValue().getLockTime(),
-                            entry.getValue().isLock());
+                        entry.getValue().getLockTime(),
+                        entry.getValue().isLock());
                 }
 
                 total++;
@@ -950,7 +958,10 @@ public class PopMessageProcessor implements NettyRequestProcessor {
 
         @Override
         public String getServiceName() {
-            return "QueueLockManager";
+            if (PopMessageProcessor.this.brokerController.getBrokerConfig().isInBrokerContainer()) {
+                return PopMessageProcessor.this.brokerController.getBrokerConfig().getLoggerIdentifier() + QueueLockManager.class.getName();
+            }
+            return QueueLockManager.class.getName();
         }
 
         @Override
diff --git a/broker/src/main/java/org/apache/rocketmq/broker/processor/PopReviveService.java b/broker/src/main/java/org/apache/rocketmq/broker/processor/PopReviveService.java
index 6d94a75..858ad61 100644
--- a/broker/src/main/java/org/apache/rocketmq/broker/processor/PopReviveService.java
+++ b/broker/src/main/java/org/apache/rocketmq/broker/processor/PopReviveService.java
@@ -23,6 +23,7 @@ import java.util.Collections;
 import java.util.Comparator;
 import java.util.HashMap;
 import java.util.List;
+
 import org.apache.rocketmq.broker.BrokerController;
 import org.apache.rocketmq.broker.util.MsgUtil;
 import org.apache.rocketmq.client.consumer.PullResult;
@@ -43,9 +44,8 @@ import org.apache.rocketmq.logging.InternalLogger;
 import org.apache.rocketmq.logging.InternalLoggerFactory;
 import org.apache.rocketmq.store.AppendMessageStatus;
 import org.apache.rocketmq.store.GetMessageResult;
-import org.apache.rocketmq.store.MessageExtBrokerInner;
+import org.apache.rocketmq.common.message.MessageExtBrokerInner;
 import org.apache.rocketmq.store.PutMessageResult;
-import org.apache.rocketmq.store.config.BrokerRole;
 import org.apache.rocketmq.store.pop.AckMsg;
 import org.apache.rocketmq.store.pop.PopCheckPoint;
 
@@ -55,10 +55,9 @@ public class PopReviveService extends ServiceThread {
     private int queueId;
     private BrokerController brokerController;
     private String reviveTopic;
-    private static volatile boolean isMaster = false;
+    private volatile boolean shouldRunPopRevive = false;
 
-    public PopReviveService(int queueId, BrokerController brokerController, String reviveTopic) {
-        super();
+    public PopReviveService(BrokerController brokerController, String reviveTopic, int queueId) {
         this.queueId = queueId;
         this.brokerController = brokerController;
         this.reviveTopic = reviveTopic;
@@ -66,20 +65,22 @@ public class PopReviveService extends ServiceThread {
 
     @Override
     public String getServiceName() {
+        if (brokerController != null && brokerController.getBrokerConfig().isInBrokerContainer()) {
+            return brokerController.getBrokerConfig().getLoggerIdentifier() + "PopReviveService_" + this.queueId;
+        }
         return "PopReviveService_" + this.queueId;
     }
 
-    private boolean checkMaster() {
-        return brokerController.getMessageStoreConfig().getBrokerRole() != BrokerRole.SLAVE;
+    public void setShouldRunPopRevive(final boolean shouldRunPopRevive) {
+        this.shouldRunPopRevive = shouldRunPopRevive;
     }
 
-    private boolean checkAndSetMaster() {
-        isMaster = checkMaster();
-        return isMaster;
+    public boolean isShouldRunPopRevive() {
+        return shouldRunPopRevive;
     }
 
     private void reviveRetry(PopCheckPoint popCheckPoint, MessageExt messageExt) throws Exception {
-        if (!checkAndSetMaster()) {
+        if (!shouldRunPopRevive) {
             POP_LOGGER.info("slave skip retry , revive topic={}, reviveQueueId={}", reviveTopic, queueId);
             return;
         }
@@ -109,8 +110,8 @@ public class PopReviveService extends ServiceThread {
         PutMessageResult putMessageResult = brokerController.getMessageStore().putMessage(msgInner);
         if (brokerController.getBrokerConfig().isEnablePopLog()) {
             POP_LOGGER.info("reviveQueueId={},retry msg , ck={}, msg queueId {}, offset {}, reviveDelay={}, result is {} ",
-                    queueId, popCheckPoint, messageExt.getQueueId(), messageExt.getQueueOffset(),
-                    (System.currentTimeMillis() - popCheckPoint.getReviveTime()) / 1000, putMessageResult);
+                queueId, popCheckPoint, messageExt.getQueueId(), messageExt.getQueueOffset(),
+                (System.currentTimeMillis() - popCheckPoint.getReviveTime()) / 1000, putMessageResult);
         }
         if (putMessageResult.getAppendMessageResult() == null || putMessageResult.getAppendMessageResult().getStatus() != AppendMessageStatus.PUT_OK) {
             throw new Exception("reviveQueueId=" + queueId + ",revive error ,msg is :" + msgInner);
@@ -120,9 +121,9 @@ public class PopReviveService extends ServiceThread {
         this.brokerController.getBrokerStatsManager().incTopicPutSize(msgInner.getTopic(), putMessageResult.getAppendMessageResult().getWroteBytes());
         if (brokerController.getPopMessageProcessor() != null) {
             brokerController.getPopMessageProcessor().notifyMessageArriving(
-                    KeyBuilder.parseNormalTopic(popCheckPoint.getTopic(), popCheckPoint.getCId()),
-                    popCheckPoint.getCId(),
-                    -1
+                KeyBuilder.parseNormalTopic(popCheckPoint.getTopic(), popCheckPoint.getCId()),
+                popCheckPoint.getCId(),
+                -1
             );
         }
     }
@@ -136,19 +137,21 @@ public class PopReviveService extends ServiceThread {
     }
 
     private void addRetryTopicIfNoExit(String topic, String consumerGroup) {
-        TopicConfig topicConfig = brokerController.getTopicConfigManager().selectTopicConfig(topic);
-        if (topicConfig != null) {
-            return;
+        if (brokerController != null) {
+            TopicConfig topicConfig = brokerController.getTopicConfigManager().selectTopicConfig(topic);
+            if (topicConfig != null) {
+                return;
+            }
+            topicConfig = new TopicConfig(topic);
+            topicConfig.setReadQueueNums(PopAckConstants.retryQueueNum);
+            topicConfig.setWriteQueueNums(PopAckConstants.retryQueueNum);
+            topicConfig.setTopicFilterType(TopicFilterType.SINGLE_TAG);
+            topicConfig.setPerm(6);
+            topicConfig.setTopicSysFlag(0);
+            brokerController.getTopicConfigManager().updateTopicConfig(topicConfig);
+
+            initPopRetryOffset(topic, consumerGroup);
         }
-        topicConfig = new TopicConfig(topic);
-        topicConfig.setReadQueueNums(PopAckConstants.retryQueueNum);
-        topicConfig.setWriteQueueNums(PopAckConstants.retryQueueNum);
-        topicConfig.setTopicFilterType(TopicFilterType.SINGLE_TAG);
-        topicConfig.setPerm(6);
-        topicConfig.setTopicSysFlag(0);
-        brokerController.getTopicConfigManager().updateTopicConfig(topicConfig);
-
-        initPopRetryOffset(topic, consumerGroup);
     }
 
     private List<MessageExt> getReviveMessage(long offset, int queueId) {
@@ -160,33 +163,26 @@ public class PopReviveService extends ServiceThread {
             POP_LOGGER.info("reviveQueueId={}, reach tail,offset {}", queueId, offset);
         } else if (pullResult.getPullStatus() == PullStatus.OFFSET_ILLEGAL || pullResult.getPullStatus() == PullStatus.NO_MATCHED_MSG) {
             POP_LOGGER.error("reviveQueueId={}, OFFSET_ILLEGAL {}, result is {}", queueId, offset, pullResult);
-            if (!checkAndSetMaster()) {
+            if (!shouldRunPopRevive) {
                 POP_LOGGER.info("slave skip offset correct topic={}, reviveQueueId={}", reviveTopic, queueId);
                 return null;
             }
-            brokerController.getConsumerOffsetManager().commitOffset(PopAckConstants.LOCAL_HOST, PopAckConstants.REVIVE_GROUP, reviveTopic, queueId, pullResult.getNextBeginOffset() - 1);
+            this.brokerController.getConsumerOffsetManager().commitOffset(PopAckConstants.LOCAL_HOST, PopAckConstants.REVIVE_GROUP, reviveTopic, queueId, pullResult.getNextBeginOffset() - 1);
         }
         return pullResult.getMsgFoundList();
     }
 
     private boolean reachTail(PullResult pullResult, long offset) {
         return pullResult.getPullStatus() == PullStatus.NO_NEW_MSG
-                || (pullResult.getPullStatus() == PullStatus.OFFSET_ILLEGAL && offset == pullResult.getMaxOffset());
+            || (pullResult.getPullStatus() == PullStatus.OFFSET_ILLEGAL && offset == pullResult.getMaxOffset());
     }
 
-    private MessageExt getBizMessage(String topic, long offset, int queueId) {
-        final GetMessageResult getMessageTmpResult = brokerController.getMessageStore().getMessage(PopAckConstants.REVIVE_GROUP, topic, queueId, offset, 1, null);
-        List<MessageExt> list = decodeMsgList(getMessageTmpResult);
-        if (list == null || list.isEmpty()) {
-            POP_LOGGER.warn("can not get msg , topic {}, offset {}, queueId {}, result is {}", topic, offset, queueId, getMessageTmpResult);
-            return null;
-        } else {
-            return list.get(0);
-        }
+    private MessageExt getBizMessage(String topic, long offset, int queueId, String brokerName) {
+        return this.brokerController.getEscapeBridge().getMessage(topic, offset, queueId, brokerName);
     }
 
     public PullResult getMessage(String group, String topic, int queueId, long offset, int nums) {
-        GetMessageResult getMessageResult = brokerController.getMessageStore().getMessage(group, topic, queueId, offset, nums, null);
+        GetMessageResult getMessageResult = this.brokerController.getMessageStore().getMessage(group, topic, queueId, offset, nums, null);
 
         if (getMessageResult != null) {
             PullStatus pullStatus = PullStatus.NO_NEW_MSG;
@@ -199,17 +195,17 @@ public class PopReviveService extends ServiceThread {
... 6336 lines suppressed ...

[rocketmq] 03/07: feature(logging&tools&distribution&docs):[RIP-31][PART-A]Support RocketMQ BrokerContainer (#3977)

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

jinrongtong pushed a commit to branch 5.0.0-beta-tmp
in repository https://gitbox.apache.org/repos/asf/rocketmq.git

commit fb9e3f05bb96ca7a951baa83b5cd3891d568ee44
Author: rongtong <ji...@163.com>
AuthorDate: Mon Mar 14 14:06:24 2022 +0800

    feature(logging&tools&distribution&docs):[RIP-31][PART-A]Support RocketMQ BrokerContainer (#3977)
---
 distribution/bin/{mqshutdown => mqbrokercontainer} |   46 +-
 distribution/bin/mqshutdown                        |   14 +
 .../2container-2m-2s/broker-a-in-container1.conf   |   20 +
 .../2container-2m-2s/broker-a-in-container2.conf   |   20 +
 .../2container-2m-2s/broker-b-in-container1.conf   |   20 +
 .../2container-2m-2s/broker-b-in-container2.conf   |   20 +
 .../2container-2m-2s/broker-container1.conf        |    9 +
 .../2container-2m-2s/broker-container2.conf        |    9 +
 .../container/2container-2m-2s/nameserver.conf     |    1 +
 distribution/conf/container/broker-a.conf          |   19 +
 distribution/conf/container/broker-b.conf          |   19 +
 distribution/conf/container/broker-container.conf  |    9 +
 distribution/pom.xml                               |    4 +
 distribution/release.xml                           |    1 +
 docs/cn/BrokerContainer.md                         |  152 +++
 docs/cn/QuorumACK.md                               |   70 ++
 docs/cn/README.md                                  |   12 +-
 docs/cn/SlaveActingMasterMode.md                   |  161 +++
 .../rocketmq/logging/InternalLoggerFactory.java    |   11 +
 .../rocketmq/logging/Slf4jLoggerFactory.java       |   87 +-
 .../org/apache/rocketmq/logging/inner/Level.java   |    1 +
 .../apache/rocketmq/logging/inner/SysLogger.java   |    4 +-
 .../rocketmq/logging/Slf4jLoggerFactoryTest.java   |    2 +-
 .../rocketmq/logging/inner/LoggingBuilderTest.java |    8 +-
 srvutil/pom.xml                                    |    8 +
 .../apache/rocketmq/srvutil/FileWatchService.java  |    4 +-
 .../rocketmq/util/cache/CacheEvictHandler.java     |   23 +
 .../apache/rocketmq/util/cache/CacheObject.java    |   36 +
 .../rocketmq/util/cache/ExpiredLocalCache.java     |   84 ++
 .../org/apache/rocketmq/util/cache/LocalCache.java |   58 ++
 .../apache/rocketmq/util/cache/LockManager.java    |   54 +
 .../rocketmq/tools/admin/DefaultMQAdminExt.java    |  157 ++-
 .../tools/admin/DefaultMQAdminExtImpl.java         | 1098 ++++++++++++++------
 .../apache/rocketmq/tools/admin/MQAdminExt.java    |   71 +-
 .../tools/admin/api/BrokerOperatorResult.java      |   50 +
 .../tools/admin/common/AdminToolHandler.java       |   21 +
 .../tools/admin/common/AdminToolResult.java        |   76 ++
 .../admin/common/AdminToolsResultCodeEnum.java     |   43 +
 .../apache/rocketmq/tools/command/CommandUtil.java |   17 +-
 .../rocketmq/tools/command/MQAdminStartup.java     |   18 +-
 .../command/broker/GetBrokerConfigCommand.java     |   29 +-
 .../broker/ResetMasterFlushOffsetSubCommand.java   |   71 ++
 .../broker/UpdateBrokerConfigSubCommand.java       |   16 +-
 .../command/cluster/CLusterSendMsgRTCommand.java   |    4 +-
 .../command/cluster/ClusterListSubCommand.java     |  106 +-
 .../consumer/ConsumerProgressSubCommand.java       |   17 +-
 .../consumer/DeleteSubscriptionGroupCommand.java   |   19 +-
 .../consumer/GetConsumerConfigSubCommand.java      |    2 +-
 .../command/container/AddBrokerSubCommand.java     |   66 ++
 .../command/container/RemoveBrokerSubCommand.java  |   79 ++
 .../tools/command/ha/HAStatusSubCommand.java       |  151 +++
 .../command/offset/ResetOffsetByTimeCommand.java   |   34 +
 .../tools/admin/DefaultMQAdminExtTest.java         |    4 +-
 .../rocketmq/tools/command/CommandUtilTest.java    |    2 +-
 .../message/QueryMsgByUniqueKeySubCommandTest.java |   18 +-
 55 files changed, 2655 insertions(+), 500 deletions(-)

diff --git a/distribution/bin/mqshutdown b/distribution/bin/mqbrokercontainer
similarity index 51%
copy from distribution/bin/mqshutdown
copy to distribution/bin/mqbrokercontainer
index d2d51fc..0ce383f 100644
--- a/distribution/bin/mqshutdown
+++ b/distribution/bin/mqbrokercontainer
@@ -15,35 +15,31 @@
 # See the License for the specific language governing permissions and
 # limitations under the License.
 
-case $1 in
-    broker)
-
-    pid=`ps ax | grep -i 'org.apache.rocketmq.broker.BrokerStartup' |grep java | grep -v grep | awk '{print $1}'`
-    if [ -z "$pid" ] ; then
-            echo "No mqbroker running."
-            exit -1;
+if [ -z "$ROCKETMQ_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
 
-    echo "The mqbroker(${pid}) is running..."
-
-    kill ${pid}
+  saveddir=`pwd`
 
-    echo "Send shutdown request to mqbroker(${pid}) OK"
-    ;;
-    namesrv)
+  ROCKETMQ_HOME=`dirname "$PRG"`/..
 
-    pid=`ps ax | grep -i 'org.apache.rocketmq.namesrv.NamesrvStartup' |grep java | grep -v grep | awk '{print $1}'`
-    if [ -z "$pid" ] ; then
-            echo "No mqnamesrv running."
-            exit -1;
-    fi
+  # make it fully qualified
+  ROCKETMQ_HOME=`cd "$ROCKETMQ_HOME" && pwd`
 
-    echo "The mqnamesrv(${pid}) is running..."
+  cd "$saveddir"
+fi
 
-    kill ${pid}
+export ROCKETMQ_HOME
 
-    echo "Send shutdown request to mqnamesrv(${pid}) OK"
-    ;;
-    *)
-    echo "Useage: mqshutdown broker | namesrv"
-esac
+sh ${ROCKETMQ_HOME}/bin/runbroker.sh org.apache.rocketmq.container.BrokerContainerStartup $@
diff --git a/distribution/bin/mqshutdown b/distribution/bin/mqshutdown
index d2d51fc..d91fce9 100644
--- a/distribution/bin/mqshutdown
+++ b/distribution/bin/mqshutdown
@@ -30,6 +30,20 @@ case $1 in
 
     echo "Send shutdown request to mqbroker(${pid}) OK"
     ;;
+    brokerContainer)
+
+    pid=`ps ax | grep -i 'org.apache.rocketmq.container.BrokerContainerStartup' |grep java | grep -v grep | awk '{print $1}'`
+    if [ -z "$pid" ] ; then
+            echo "No broker container running."
+            exit -1;
+    fi
+
+    echo "The broker container(${pid}) is running..."
+
+    kill ${pid}
+
+    echo "Send shutdown request to broker container(${pid}) OK"
+    ;;
     namesrv)
 
     pid=`ps ax | grep -i 'org.apache.rocketmq.namesrv.NamesrvStartup' |grep java | grep -v grep | awk '{print $1}'`
diff --git a/distribution/conf/container/2container-2m-2s/broker-a-in-container1.conf b/distribution/conf/container/2container-2m-2s/broker-a-in-container1.conf
new file mode 100644
index 0000000..6421fc8
--- /dev/null
+++ b/distribution/conf/container/2container-2m-2s/broker-a-in-container1.conf
@@ -0,0 +1,20 @@
+#Master配置
+brokerClusterName=DefaultCluster
+brokerName=broker-a
+brokerId=0
+brokerRole=SYNC_MASTER
+flushDiskType=ASYNC_FLUSH
+storePathRootDir=/root/broker-a/store
+storePathCommitLog=/root/broker-a/store/commitlog
+listenPort=10911
+haListenPort=10912
+totalReplicas=2
+inSyncReplicas=2
+minInSyncReplicas=1
+enableAutoInSyncReplicas=true
+slaveReadEnable=true
+brokerHeartbeatInterval=1000
+brokerNotActiveTimeoutMillis=5000
+sendHeartbeatTimeoutMillis=1000
+enableSlaveActingMaster=true
+isolateLogEnable=true
\ No newline at end of file
diff --git a/distribution/conf/container/2container-2m-2s/broker-a-in-container2.conf b/distribution/conf/container/2container-2m-2s/broker-a-in-container2.conf
new file mode 100644
index 0000000..4be2249
--- /dev/null
+++ b/distribution/conf/container/2container-2m-2s/broker-a-in-container2.conf
@@ -0,0 +1,20 @@
+#Master配置
+brokerClusterName=DefaultCluster
+brokerName=broker-a
+brokerId=1
+brokerRole=SLAVE
+flushDiskType=ASYNC_FLUSH
+storePathRootDir=/root/broker-a/store
+storePathCommitLog=/root/broker-a/store/commitlog
+listenPort=10911
+haListenPort=10912
+totalReplicas=2
+inSyncReplicas=2
+minInSyncReplicas=1
+enableAutoInSyncReplicas=true
+slaveReadEnable=true
+brokerHeartbeatInterval=1000
+brokerNotActiveTimeoutMillis=5000
+sendHeartbeatTimeoutMillis=1000
+enableSlaveActingMaster=true
+isolateLogEnable=true
\ No newline at end of file
diff --git a/distribution/conf/container/2container-2m-2s/broker-b-in-container1.conf b/distribution/conf/container/2container-2m-2s/broker-b-in-container1.conf
new file mode 100644
index 0000000..35d469a
--- /dev/null
+++ b/distribution/conf/container/2container-2m-2s/broker-b-in-container1.conf
@@ -0,0 +1,20 @@
+#Slave配置
+brokerClusterName=DefaultCluster
+brokerName=broker-b
+brokerId=1
+brokerRole=SLAVE
+flushDiskType=ASYNC_FLUSH
+storePathRootDir=/root/broker-b/store
+storePathCommitLog=/root/broker-b/store/commitlog
+listenPort=20911
+haListenPort=20912
+totalReplicas=2
+inSyncReplicas=2
+minInSyncReplicas=1
+enableAutoInSyncReplicas=true
+slaveReadEnable=true
+brokerHeartbeatInterval=1000
+brokerNotActiveTimeoutMillis=5000
+sendHeartbeatTimeoutMillis=1000
+enableSlaveActingMaster=true
+isolateLogEnable=true
\ No newline at end of file
diff --git a/distribution/conf/container/2container-2m-2s/broker-b-in-container2.conf b/distribution/conf/container/2container-2m-2s/broker-b-in-container2.conf
new file mode 100644
index 0000000..1594975
--- /dev/null
+++ b/distribution/conf/container/2container-2m-2s/broker-b-in-container2.conf
@@ -0,0 +1,20 @@
+#Slave配置
+brokerClusterName=DefaultCluster
+brokerName=broker-b
+brokerId=0
+brokerRole=SYNC_MASTER
+flushDiskType=ASYNC_FLUSH
+storePathRootDir=/root/broker-b/store
+storePathCommitLog=/root/broker-b/store/commitlog
+listenPort=20911
+haListenPort=20912
+totalReplicas=2
+inSyncReplicas=2
+minInSyncReplicas=1
+enableAutoInSyncReplicas=true
+slaveReadEnable=true
+brokerHeartbeatInterval=1000
+brokerNotActiveTimeoutMillis=5000
+sendHeartbeatTimeoutMillis=1000
+enableSlaveActingMaster=true
+isolateLogEnable=true
\ No newline at end of file
diff --git a/distribution/conf/container/2container-2m-2s/broker-container1.conf b/distribution/conf/container/2container-2m-2s/broker-container1.conf
new file mode 100644
index 0000000..e5b7aac
--- /dev/null
+++ b/distribution/conf/container/2container-2m-2s/broker-container1.conf
@@ -0,0 +1,9 @@
+#配置端口,用于接收mqadmin命令
+listenPort=10811
+#指定namesrv
+namesrvAddr=172.22.144.49:9876
+#或指定自动获取namesrv
+fetchNamesrvAddrByAddressServer=false
+#指定要向BrokerContainer内添加的brokerConfig路径,多个config间用“:”分隔;
+#不指定则只启动BrokerConainer,具体broker可通过mqadmin工具添加
+brokerConfigPaths=/root/2container-2m-2s/broker-a-in-container1.conf:/root/2container-2m-2s/broker-b-in-container1.conf
\ No newline at end of file
diff --git a/distribution/conf/container/2container-2m-2s/broker-container2.conf b/distribution/conf/container/2container-2m-2s/broker-container2.conf
new file mode 100644
index 0000000..1d4f1c3
--- /dev/null
+++ b/distribution/conf/container/2container-2m-2s/broker-container2.conf
@@ -0,0 +1,9 @@
+#配置端口,用于接收mqadmin命令
+listenPort=10811
+#指定namesrv
+namesrvAddr=172.22.144.49:9876
+#或指定自动获取namesrv
+fetchNamesrvAddrByAddressServer=false
+#指定要向BrokerContainer内添加的brokerConfig路径,多个config间用“:”分隔;
+#不指定则只启动BrokerConainer,具体broker可通过mqadmin工具添加
+brokerConfigPaths=/root/2container-2m-2s/broker-a-in-container2.conf:/root/2container-2m-2s/broker-b-in-container2.conf
\ No newline at end of file
diff --git a/distribution/conf/container/2container-2m-2s/nameserver.conf b/distribution/conf/container/2container-2m-2s/nameserver.conf
new file mode 100644
index 0000000..d716e7f
--- /dev/null
+++ b/distribution/conf/container/2container-2m-2s/nameserver.conf
@@ -0,0 +1 @@
+supportActingMaster=true
\ No newline at end of file
diff --git a/distribution/conf/container/broker-a.conf b/distribution/conf/container/broker-a.conf
new file mode 100644
index 0000000..5324f26
--- /dev/null
+++ b/distribution/conf/container/broker-a.conf
@@ -0,0 +1,19 @@
+#Master配置
+brokerClusterName=DefaultCluster
+brokerName=broker-a
+brokerId=0
+brokerRole=SYNC_MASTER
+flushDiskType=ASYNC_FLUSH
+storePathRootDir=/disk1/rocketmq/broker-a/store
+storePathCommitLog=/disk1/rocketmq/broker-a/store/commitlog
+listenPort=10911
+haListenPort=10912
+totalReplicas=2
+inSyncReplicas=2
+minInSyncReplicas=1
+enableAutoInSyncReplicas=true
+slaveReadEnable=true
+brokerHeartbeatInterval=1000
+brokerNotActiveTimeoutMillis=5000
+sendHeartbeatTimeoutMillis=1000
+enableSlaveActingMaster=true
\ No newline at end of file
diff --git a/distribution/conf/container/broker-b.conf b/distribution/conf/container/broker-b.conf
new file mode 100644
index 0000000..82ca2ea
--- /dev/null
+++ b/distribution/conf/container/broker-b.conf
@@ -0,0 +1,19 @@
+#Slave配置
+brokerClusterName=DefaultCluster
+brokerName=broker-b
+brokerId=1
+brokerRole=SLAVE
+flushDiskType=ASYNC_FLUSH
+storePathRootDir=/disk2/rocketmq/broker-b/store
+storePathCommitLog=/disk2/rocketmq/broker-b/store/commitlog
+listenPort=20911
+haListenPort=20912
+totalReplicas=2
+inSyncReplicas=2
+minInSyncReplicas=1
+enableAutoInSyncReplicas=true
+slaveReadEnable=true
+brokerHeartbeatInterval=1000
+brokerNotActiveTimeoutMillis=5000
+sendHeartbeatTimeoutMillis=1000
+enableSlaveActingMaster=true
\ No newline at end of file
diff --git a/distribution/conf/container/broker-container.conf b/distribution/conf/container/broker-container.conf
new file mode 100644
index 0000000..4798eae
--- /dev/null
+++ b/distribution/conf/container/broker-container.conf
@@ -0,0 +1,9 @@
+#配置端口,用于接收mqadmin命令
+listenPort=10811
+#指定namesrv
+namesrvAddr=127.0.0.1:9876
+#或指定自动获取namesrv
+fetchNamesrvAddrByAddressServer=true
+#指定要向BrokerContainer内添加的brokerConfig路径,多个config间用“:”分隔;
+#不指定则只启动BrokerConainer,具体broker可通过mqadmin工具添加
+brokerConfigPaths=/home/admin/broker-a.conf:/home/admin/broker-b.conf
\ No newline at end of file
diff --git a/distribution/pom.xml b/distribution/pom.xml
index 8059a9a..439e477 100644
--- a/distribution/pom.xml
+++ b/distribution/pom.xml
@@ -32,6 +32,10 @@
             <dependencies>
                 <dependency>
                     <groupId>org.apache.rocketmq</groupId>
+                    <artifactId>rocketmq-container</artifactId>
+                </dependency>
+                <dependency>
+                    <groupId>org.apache.rocketmq</groupId>
                     <artifactId>rocketmq-broker</artifactId>
                 </dependency>
                 <dependency>
diff --git a/distribution/release.xml b/distribution/release.xml
index fd6e3db..b3a9e5e 100644
--- a/distribution/release.xml
+++ b/distribution/release.xml
@@ -61,6 +61,7 @@
         <moduleSet>
             <useAllReactorProjects>true</useAllReactorProjects>
             <includes>
+                <include>org.apache.rocketmq:rocketmq-container</include>
                 <include>org.apache.rocketmq:rocketmq-broker</include>
                 <include>org.apache.rocketmq:rocketmq-tools</include>
                 <include>org.apache.rocketmq:rocketmq-client</include>
diff --git a/docs/cn/BrokerContainer.md b/docs/cn/BrokerContainer.md
new file mode 100644
index 0000000..94a2355
--- /dev/null
+++ b/docs/cn/BrokerContainer.md
@@ -0,0 +1,152 @@
+# BrokerContainer
+
+## 背景
+
+在RocketMQ 4.x版本中,一个进程只有一个broker,通常会以主备或者DLedger(Raft)的形式部署,但是一个进程中只有一个broker,而slave一般只承担冷备或热备的作用,节点之间角色的不对等导致slave节点资源没有充分被利用。
+因此在RocketMQ 5.x版本中,提供一种新的模式BrokerContainer,在一个BrokerContainer进程中可以加入多个Broker(Master Broker、Slave Broker、DLedger Broker),来提高单个节点的资源利用率,并且可以通过各种形式的交叉部署来实现节点之间的对等部署。
+该特性的优点包括:
+
+1. 一个BrokerContainer进程中可以加入多个broker,通过进程内混部来提高单个节点的资源利用率
+2. 通过各种形式的交叉部署来实现节点之间的对等部署,增强单节点的高可用能力
+3. 利用BrokerContainer可以实现单进程内多CommitLog写入,也可以实现单机的多磁盘写入
+4. BrokerContainer中的CommitLog天然隔离的,不同的CommitLog(broker)可以采取不同作用,比如可以用来比如创建单独的broker做不同TTL的CommitLog。
+
+## 架构
+
+### 单进程视图
+
+![](https://s4.ax1x.com/2022/01/26/7LMZHP.png)
+
+相比于原来一个Broker一个进程,RocketMQ 5.0将增加BrokerContainer概念,一个BrokerContainer可以存放多个Broker,每个Broker拥有不同的端口,但它们共享同一个传输层(remoting层),而每一个broker在功能上是完全独立的。BrokerContainer也拥有自己端口,在运行时可以通过admin命令来增加或减少Broker。
+
+### 对等部署形态
+
+在BrokerContainer模式下,可以通过各种形式的交叉部署完成节点的对等部署
+
+- 二副本对等部署
+
+![](https://s4.ax1x.com/2022/01/26/7LQi5T.png)
+
+二副本对等部署情况下,每个节点都会有一主一备,资源利用率均等。另外假设图中Node1宕机,由于Node2的broker_2可读可写,broker_1可以备读,因此普通消息的收发不会收到影响,单节点的高可用能力得到了增强。
+
+- 三副本对等部署
+
+![](https://s4.ax1x.com/2022/01/26/7LQMa6.png)
+
+三副本对等部署情况下,每个节点都会有一主两备,资源利用率均等。此外,和二副本一样,任意一个节点的宕机也不会影响到普通消息的收发。
+
+### 传输层共享
+
+![](https://s4.ax1x.com/2022/02/07/HMNIVs.png)
+
+BrokerContainer中的所有broker共享同一个传输层,就像RocketMQ客户端中同进程的Consumer和Producer共享同一个传输层一样。
+
+这里为NettyRemotingServer提供SubRemotingServer支持,通过为一个RemotingServer绑定另一个端口即可生成SubRemotingServer,其共享NettyRemotingServer的Netty实例、计算资源、以及协议栈等,但拥有不同的端口以及ProcessorTable。另外同一个BrokerContainer中的所有的broker也会共享同一个BrokerOutAPI(RemotingClient)。
+
+## 启动方式和配置
+
+![](https://s4.ax1x.com/2022/01/26/7LQ1PO.png)
+
+像Broker启动利用BrokerStartup一样,使用BrokerContainerStartup来启动BrokerContainer。我们可以通过两种方式向BrokerContainer中增加broker,一种是通过启动时通过在配置文件中指定
+
+BrokerContainer配置文件内容主要是Netty网络层参数(由于传输层共享),BrokerContainer的监听端口、namesrv配置,以及最重要的brokerConfigPaths参数,brokerConfigPaths是指需要向BrokerContainer内添加的brokerConfig路径,多个config间用“:”分隔,不指定则只启动BrokerConainer,具体broker可通过mqadmin工具添加
+
+broker-container.conf(distribution/conf/container/broker-container.conf):
+
+```
+#配置端口,用于接收mqadmin命令
+listenPort=10811
+#指定namesrv
+namesrvAddr=127.0.0.1:9876
+#或指定自动获取namesrv
+fetchNamesrvAddrByAddressServer=true
+#指定要向BrokerContainer内添加的brokerConfig路径,多个config间用“:”分隔;
+#不指定则只启动BrokerConainer,具体broker可通过mqadmin工具添加
+brokerConfigPaths=/home/admin/broker-a.conf:/home/admin/broker-b.conf
+```
+broker的配置和以前一样,但在BrokerContainer模式下broker配置文件中下Netty网络层参数和nameserver参数不生效,均使用BrokerContainer的配置参数。
+
+完成配置文件后,可以以如下命令启动
+```
+sh mqbrokercontainer -c broker-container.conf
+```
+mqbrokercontainer脚本路径为distribution/bin/mqbrokercontainer。
+
+## 运行时增加或较少Broker
+
+当BrokerContainer进程启动后,也可以通过Admin命令来增加或减少Broker。
+
+AddBrokerCommand
+```
+usage: mqadmin addBroker -b <arg> -c <arg> [-h] [-n <arg>]
+ -b,--brokerConfigPath <arg>      Broker config path
+ -c,--brokerContainerAddr <arg>   Broker container address
+ -h,--help                        Print help
+ -n,--namesrvAddr <arg>           Name server address list, eg: 192.168.0.1:9876;192.168.0.2:9876
+```
+
+RemoveBroker Command
+```
+usage: mqadmin removeBroker -b <arg> -c <arg> [-h] [-n <arg>]
+ -b,--brokerIdentity <arg>        Information to identify a broker: clusterName:brokerName:brokerId
+ -c,--brokerContainerAddr <arg>   Broker container address
+ -h,--help                        Print help
+ -n,--namesrvAddr <arg>           Name server address list, eg: 192.168.0.1:9876;192.168.0.2:9876
+```
+
+## 存储变化
+
+storePathRootDir, storePathCommitLog路径依然为MessageStoreConfig中配置值,需要注意的是同一个brokerContainer中的broker不能使用相同的storePathRootDir, storePathCommitLog,否则不同的broker占用同一个存储目录,发生数据混乱。
+
+在文件删除策略上,仍然单个Broker的视角来进行删除,但MessageStoreConfig新增replicasPerDiskPartition参数和logicalDiskSpaceCleanForciblyThreshold。
+
+replicasPerDiskPartition表示同一磁盘分区上有多少个副本,即该broker的存储目录所在的磁盘分区被几个broker共享,默认值为1。该配置用于计算当同一节点上的多个broker共享同一磁盘分区时,各broker的磁盘配额
+
+e.g. replicasPerDiskPartition==2且broker所在磁盘空间为1T时,则该broker磁盘配额为512G,该broker的逻辑磁盘空间利用率基于512G的空间进行计算。
+
+logicalDiskSpaceCleanForciblyThreshold,该值只在quotaPercentForDiskPartition小于1时生效,表示逻辑磁盘空间强制清理阈值,默认为0.80(80%), 逻辑磁盘空间利用率为该broker在自身磁盘配额内的空间利用率,物理磁盘空间利用率为该磁盘分区总空间利用率。由于在BrokerContainer实现中,考虑计算效率的情况下,仅统计了commitLog+consumeQueue(+ BCQ)+indexFile作为broker的存储空间占用,其余文件如元数据、消费进度、磁盘脏数据等未统计在内,故在多个broker存储空间达到动态平衡时,各broker所占空间可能有相差,以一个BrokerContainer中有两个broker为例,两broker存储空间差异可表示为:
+![](https://s4.ax1x.com/2022/01/26/7L14v4.png)
+其中,R_logical为logicalDiskSpaceCleanForciblyThreshold,R_phy为diskSpaceCleanForciblyRatio,T为磁盘分区总空间,x为除上述计算的broker存储空间外的其他文件所占磁盘总空间比例,可见,当
+![](https://s4.ax1x.com/2022/01/26/7L1TbR.png)
+时,可保证BrokerContainer各Broker存储空间在达到动态平衡时相差无几。
+
+eg.假设broker获取到的配额是500g(根据replicasPerDiskPartition计算获得),logicalDiskSpaceCleanForciblyThreshold为默认值0.8,则默认commitLog+consumeQueue(+ BCQ)+indexFile总量超过400g就会强制清理文件。
+
+其他清理阈值(diskSpaceCleanForciblyRatio、diskSpaceWarningLevelRatio),文件保存时间(fileReservedTime)等逻辑与之前保持一致。
+
+注意:当以普通broker方式启动而非brokerContainer启动时,且replicasPerDiskPartition=1(默认值)时,清理逻辑与之前完全一致。replicasPerDiskPartition>1时,逻辑磁盘空间强制清理阈值logicalDiskSpaceCleanForciblyThreshold将会生效。
+
+
+## 日志变化
+
+在BrokerContainer模式下并开启日志分离后,日志的默认输出路径将发生变化,每个broker日志的具体路径变化为
+```
+{user.home}/logs/{$brokerCanonicalName}_rocketmqlogs/
+```
+
+其中brokerCanonicalName为{BrokerClusterName_BrokerName_BrokerId},{BrokerClusterName_BrokerName_BrokerId}。
+
+**开发者需要注意!**
+
+在BrokerContainer模式下,多个broker会在同一个BrokerContainer进程中,因此所有broker的日志将会输出到同一个日志文件中,BrokerContainer模式下将提供broker日志分离功能,通过BrokerConfig配置isolateLogEnable=true进行开启,开启后不同broker的日志将会输出到不同文件中。
+
+主要通过线程名(ThreadName)或者通过设置线程本地变量(ThreadLocal)来区分不同broker线程,并且hack logback的logAppender将日志重定向到不同的文件中。
+
+通过设置线程名来区分不同broker线程,线程名前缀必须是#BrokerClusterName_BrokerName_BrokerId#
+
+通过设置线程本地变量区分不同broker线程,设置的变量为BrokerClusterName_BrokerName_BrokerId
+```java
+// set threadlocal broker identity to forward logging to corresponding broker
+InnerLoggerFactory.brokerIdentity.set(brokerIdentity.getCanonicalName())
+```
+
+如果线程没有上述区分,日志将仍然会输出在原来的目录下。
+
+以普通方式启动Broker(非BrokerContainer模式)时,日志将仍然会输出在原来的目录下。
+
+具体实现方式可以参考Slf4jLoggerFactory和BrokerLogbackConfigurator两个类。
+
+通过线程名和线程本地变量区分可以参考org.apache.rocketmq.common.AbstractBrokerRunnable、org.apache.rocketmq.common.ThreadFactoryImpl以及各个ServiceThread中getServiceName的实现。
+
+
+参考文档: [原RIP](https://github.com/apache/rocketmq/wiki/RIP-31-Support-RocketMQ-BrokerContainer)
\ No newline at end of file
diff --git a/docs/cn/QuorumACK.md b/docs/cn/QuorumACK.md
new file mode 100644
index 0000000..be609c6
--- /dev/null
+++ b/docs/cn/QuorumACK.md
@@ -0,0 +1,70 @@
+# Quorum Write和自动降级
+
+## 背景
+
+![](https://s4.ax1x.com/2022/02/05/HnWo2d.png)
+
+在RocketMQ中,主备之间的复制模式主要有同步复制和异步复制,如上图所示,Slave1的复制是同步的,在向Producer报告成功写入之前,Master需要等待Slave1成功复制该消息并确认,Slave2的复制是异步的,Master不需要等待Slave2的响应。在RocketMQ中,发送一条消息,如果一切都顺利,那最后会返回给Producer客户端一个PUT_OK的状态,如果是Slave同步超时则返回FLUSH_SLAVE_TIMEOUT状态,如果是Slave不可用或者Slave与Master之间CommitLog差距超过一定的值(默认是256MB),则返回SLAVE_NOT_AVAILABLE,后面两个状态并不会导致系统异常而无法写入下一条消息。
+
+同步复制可以保证Master失效后,数据仍然能在Slave中找到,适合可靠性要求较高的场景。异步复制虽然消息可能会丢失,但是由于无需等待Slave的确认,效率上要高于同步复制,适合对效率有一定要求的场景。但是只有两种模式仍然不够灵活,比如在三副本甚至五副本且对可靠性要求高场景中,采用异步复制无法满足需求,但采用同步复制则需要每一个副本确认后才会返回,在副本数多的情况下严重影响效率。另一方面,在同步复制的模式下,如果副本组中的某一个Slave出现假死,整个发送将一直失败直到进行手动处理。
+
+因此,RocketMQ 5 提出了副本组的quorum write,在同步复制的模式下,用户可以在broker端指定发送后至少需要写入多少副本数后才能返回,并且提供自适应降级的方式,可以根据存活的副本数以及CommitLog差距自动完成降级。
+
+## 架构和参数
+
+### Quorum Write
+
+通过增加两个参数来支持quorum write。
+
+- **totalReplicas**:副本组broker总数。默认为1。
+- **inSyncReplicas**:正常情况需保持同步的副本组数量。默认为1。
+
+通过这两个参数,可以在同步复制的模式下,灵活指定需要ACK的副本数。
+
+![](https://s4.ax1x.com/2022/02/05/HnWHKI.png)
+
+如上图所示,在两副本情况下,如果inSyncReplicas为2,则该条消息需要在Master和Slave中均复制完成后才会返回给客户端;在三副本情况下,如果inSyncReplicas为2,则该条消息除了需要复制在Master上,还需要复制到任意一个slave上,才会返回给客户端。在四副本情况下,如果inSyncReplicas为3,则条消息除了需要复制在Master上,还需要复制到任意两个slave上,才会返回给客户端。通过灵活设置totalReplicas和inSyncReplicas,可以满足用户各类场景的需求。
+
+### 自动降级
+
+自动降级的标准是
+
+- 当前副本组的存活副本数
+- Master Commitlog和Slave CommitLog的高度差
+
+通过Nameserver的反向通知以及GetBrokerMemberGroup请求可以获取当前副本组的存活信息,而Master与Slave的Commitlog高度差也可以通过HA服务中的位点记录计算出来。将增加以下参数完成自动降级:
+
+- **minInSyncReplicas**:最小需保持同步的副本组数量,仅在enableAutoInSyncReplicas为true时生效,默认为1。
+- **enableAutoInSyncReplicas**:自动同步降级开关,开启后,若当前副本组处于同步状态的broker数量(包括master自身)不满足inSyncReplicas指定的数量,则按照minInSyncReplicas进行同步。同步状态判断条件为:slave commitLog落后master长度不超过haSlaveFallBehindMax。默认为false。
+- **haSlaveFallBehindMax**:slave是否与master处于in-sync状态的判断值,slave commitLog落后master长度超过该值则认为slave已处于非同步状态。当enableAutoInSyncReplicas打开时,该值越小,越容易触发master的自动降级,当enableAutoInSyncReplicas关闭,且totalReplicas==inSyncReplicas时,该值越小,越容易导致在大流量时发送请求失败,故在该情况下可适当调大haSlaveFallBehindMax。默认为256K。
+
+注意:在RocketMQ 4.x中存在haSlaveFallbehindMax参数,默认256MB,表明Slave与Master的CommitLog高度差多少后判定其为不可用,在RocketMQ 5中该参数被取消,由haSlaveFallBehindMax代替,含义如上。
+
+```java
+//计算needAckNums
+int inSyncReplicas = Math.min(this.defaultMessageStore.getAliveReplicaNumInGroup(),
+                              this.defaultMessageStore.getHaService().inSyncSlaveNums(currOffset) + 1);
+needAckNums = calcNeedAckNums(inSyncReplicas);
+if (needAckNums > inSyncReplicas) {
+    // Tell the producer, don't have enough slaves to handle the send request
+    return CompletableFuture.completedFuture(new PutMessageResult(PutMessageStatus.IN_SYNC_REPLICAS_NOT_ENOUGH, null));
+}
+
+private int calcNeedAckNums(int inSyncReplicas) {
+    int needAckNums = this.defaultMessageStore.getMessageStoreConfig().getInSyncReplicas();
+    if (this.defaultMessageStore.getMessageStoreConfig().isEnableAutoInSyncReplicas()) {
+        needAckNums = Math.min(needAckNums, inSyncReplicas);
+        needAckNums = Math.max(needAckNums, this.defaultMessageStore.getMessageStoreConfig().getMinInSyncReplicas());
+    }
+    return needAckNums;
+}
+```
+
+当enableAutoInSyncReplicas=true是开启自适应降级模式,当副本组中存活的副本数减少或Master和Slave Commitlog高度差过大时,都会进行自动降级,最小降级到minInSyncReplicas副本数。比如在两副本中,如果设置totalReplicas=2,InSyncReplicas=2,minInSyncReplicas=1,enableAutoInSyncReplicas=true,正常情况下,两个副本均会处于同步复制,当Slave下线或假死时,会进行自适应降级,producer只需要发送到master即成功。
+
+## 兼容性
+
+用户需要设置正确的参数才能完成正确的向后兼容。举个例子,假设用户原集群为两副本同步复制,在没有修改任何参数的情况下,升级到RocketMQ 5的版本,由于totalReplicas、inSyncReplicas默认都为1,将降级为异步复制,如果需要和以前行为保持一致,则需要将totalReplicas和inSyncReplicas均设置为2。
+
+
+参考文档: [原RIP](https://github.com/apache/rocketmq/wiki/RIP-34-Support-quorum-write-and-adaptive-degradation-in-master-slave-architecture)
\ No newline at end of file
diff --git a/docs/cn/README.md b/docs/cn/README.md
index 2dbd854..fa2e7a5 100644
--- a/docs/cn/README.md
+++ b/docs/cn/README.md
@@ -34,9 +34,15 @@
 ### 5. 运维管理
 - [集群部署(Operation)](operation.md):介绍单Master模式、多Master模式、多Master多slave模式等RocketMQ集群各种形式的部署方法以及运维工具mqadmin的使用方式。
 
-
-
-### 6. API Reference(待补充)
+### 6. RocketMQ 5.0 新特性
+- [POP消费](https://github.com/apache/rocketmq/wiki/%5BRIP-19%5D-Server-side-rebalance,--lightweight-consumer-client-support)
+- [StaticTopic](RocketMQ_Static_Topic_Logic_Queue_设计.md)
+- [BatchConsumeQueue](https://github.com/apache/rocketmq/wiki/RIP-26-Improve-Batch-Message-Processing-Throughput)
+- [BrokerContainer](BrokerContainer.md)
+- [SlaveActingMaster模式](SlaveActingMasterMode.md)
+- [Quorum Write和自动降级](QuorumACK.md)
+
+### 7. API Reference(待补充)
 
 - [DefaultMQProducer API Reference](client/java/API_Reference_DefaultMQProducer.md)
 
diff --git a/docs/cn/SlaveActingMasterMode.md b/docs/cn/SlaveActingMasterMode.md
new file mode 100644
index 0000000..03978bf
--- /dev/null
+++ b/docs/cn/SlaveActingMasterMode.md
@@ -0,0 +1,161 @@
+# Slave Acting Master模式
+
+## 背景
+
+![](https://s4.ax1x.com/2022/02/05/HnW3CQ.png)
+
+上图为当前RocketMQ Master-Slave冷备部署,在该部署方式下,即使一个Master掉线,发送端仍然可以向其他Master发送消息,对于消费端而言,若开启备读,Consumer会自动重连到对应的Slave机器,不会出现消费停滞的情况。但也存在以下问题:
+
+1. 一些仅限于在Master上进行的操作将无法进行,包括且不限于:
+
+- searchOffset 
+- maxOffset 
+- minOffset 
+- earliestMsgStoreTime 
+- endTransaction
+
+所有锁MQ相关操作,包括lock, unlock, lockBatch, unlockAll
+
+具体影响为:
+- 客户端无法获取位于该副本组的mq的锁,故当本地锁过期后,将无法消费该组的顺序消息 
+- 客户端无法主动结束处于半状态的事务消息,只能等待broker回查事务状态 
+- Admin tools或控制中依赖查询offset及earliestMsgStoreTime等操作在该组上无法生效
+
+2. 故障Broker组上的二级消息消费将会中断,该类消息特点依赖Master Broker上的线程扫描CommitLog上的特殊Topic,并将满足要求的消息投放回CommitLog,如果Master Broker下线,会出现二级消息的消费延迟或丢失。具体会影响到当前版本的延迟消息消费、事务消息消费、Pop消费。
+
+3. 没有元数据的反向同步。Master重新被人工拉起后,容易造成元数据的回退,如Master上线后将落后的消费位点同步给备,该组broker的消费位点回退,造成大量消费重复。
+
+![](https://s4.ax1x.com/2022/02/05/HnWwUU.png)
+
+上图为DLedger(Raft)架构,其可以通过选主一定程度上规避上述存在的问题,但可以看到DLedger模式下当前需要强制三副本及以上。
+
+提出一个新的方案,Slave代理Master模式,作为Master-Slave部署模式的升级。在原先Master-Slave部署模式下,通过备代理主、轻量级心跳、副本组信息获取、broker预上线机制、二级消息逃逸等方式,当同组Master发生故障时,Slave将承担更加重要的作用,包括:
+
+- 当Master下线后,该组中brokerId最小的Slave会承担备读 以及 一些 客户端和管控会访问 但却只能在Master节点上完成的任务。包括且不限于searchOffset、maxOffset、minOffset、earliestMsgStoreTime、endTransaction以及所有锁MQ相关操作lock, unlock, lockBatch, unlockAll。
+- 当Master下线后,故障Broker组上的二级消息消费将不会中断,由该组中该组中brokerId最小的Slave承担起该任务,定时消息、Pop消息、事务消息等仍然可以正常运行。
+- 当Master下线后,在Slave代理Master一段时间主后,然后当Master再次上线后,通过预上线机制,Master会自动完成元数据的反向同步后再上线,不会出现元数据回退,造成消息大量重复消费或二级消息大量重放。
+
+## 架构
+
+### 备代理主
+
+Master下线后Slave能正常消费,且在不修改客户端代码情况下完成只能在Master完成的操作源自于Namesrv对“代理”Master的支持。此处“代理”Master指的是,当副本组处于无主状态时,Namesrv将把brokerId最小的存活Slave视为“代理”Master,具体表现为在构建TopicRouteData时,将该Slave的brokerId替换为0,并将brokerPermission修改为4(Read-Only),从而使得该Slave在客户端视图中充当只读模式的Master的角色。
+
+此外,当Master下线后,brokerId最小的Slave会承担起二级消息的扫描和重新投递功能,这也是“代理”的一部分。
+
+```java
+//改变二级消息扫描状态
+public void changeSpecialServiceStatus(boolean shouldStart) {
+	……
+
+    //改变延迟消息服务的状态
+    changeScheduleServiceStatus(shouldStart);
+
+    //改变事务消息服务的状态
+    changeTransactionCheckServiceStatus(shouldStart);
+
+    //改变Pop消息服务状态
+    if (this.ackMessageProcessor != null) {
+        LOG.info("Set PopReviveService Status to {}", shouldStart);
+        this.ackMessageProcessor.setPopReviveServiceStatus(shouldStart);
+    }
+}
+```
+
+### 轻量级心跳
+
+如上文所述,brokerId最小的存活Slave在Master故障后开启自动代理Master模式,因此需要一种机制,这个机制需要保证:
+
+1. Nameserver能及时发现broker上下线并完成路由替换以及下线broker的路由剔除。
+
+2. Broker能及时感知到同组Broker的上下线情况。
+
+针对1,Nameserver原本就存在判活机制,定时会扫描不活跃的broker使其下线,而原本broker与nameserver的“心跳”则依赖于registerBroker操作,而这个操作涉及到topic信息上报,过于“重”,而且注册间隔过于长,因此需要一个轻量级的心跳机制,RoccketMQ 5.0在nameserver和broker间新增BrokerHeartbeat请求,broker会定时向nameserver发送心跳,若nameserver定时任务扫描发现超过心跳超时时间仍未收到该broker的心跳,将unregister该broker。registerBroker时会完成心跳超时时间的设置,并且注册时如果发现broker组内最小brokerId发生变化,将反向通知该组所有broker,并在路由获取时将最小brokerId的Slave路由替换使其充当只读模式的Master的角色
+
+针对2,通过两个机制来及时感知同组broker上下线情况,1是上文中介绍的当nameserver发现该broker组内最小brokerId发生变化,反向通知该组所有broker。2是broker自身会有定时任务,向nameserver同步本broker组存活broker的信息,RoccketMQ 5.0会新增GetBrokerMemberGroup请求来完成该工作。
+
+Slave Broker发现自己是该组中最小的brokerId,将会开启代理模式,而一旦Master Broker重新上线,Slave Broker同样会通过Nameserver反向通知或自身定时任务同步同组broker的信息感知到,并自动结束代理模式。
+
+### 二级消息逃逸
+
+代理模式开启后,brokerId最小的Slave会承担起二级消息的扫描和重新投递功能。
+
+二级消息一般分为两个阶段,发送或者消费时会发送到一个特殊topic中,后台会有线程会扫描,最终的满足要求的消息会被重新投递到Commitlog中。我们可以让brokerId最小的Slave进行扫描,但如果扫描之后的消息重新投递到本Commitlog,那将会破坏Slave不可写的语义,造成Commitlog分叉。因此RoccketMQ 5.0提出一种逃逸机制,将重放的二级消息远程或本地投放到其他Master的Commitlog中。
+
+- 远程逃逸
+
+![](https://s4.ax1x.com/2022/02/05/HnWWVK.png)
+
+如上图所示,假设Region A发生故障,Region B中的节点2将会承担二级消息的扫描任务,同时将最终的满足要求的消息通过EscapeBridge远程发送到当前Broker集群中仍然存活的Master上。
+
+- 本地逃逸
+
+![](https://s4.ax1x.com/2022/02/05/HnWfUO.png)
+
+本地逃逸需要在BrokerContainer下进行,如果BrokerContainer中存在存活的Master,会优先向同进程的Master Commitlog中逃逸,避免远程RPC。
+
+#### 各类二级消息变化
+
+**延迟消息**
+
+Slave代理Master时,ScheduleMessageService将启动,时间到期的延迟消息将通过EscapeBridge优先往本地Master逃逸,若没有则向远程的Master逃逸。该broker上存量的时间未到期的消息将会被逃逸到存活的其他Master上,数据量上如果该broker上有大量的延迟消息未到期,远程逃逸会造成集群内部会有较大数据流转,但基本可控。
+
+
+**POP消息**
+
+1. CK/ACK拼key的时候增加brokerName属性。这样每个broker能在扫描自身commitlog的revive topic时抵消其他broker的CK/ACK消息。
+
+2. Slave上的CK/ACK消息将被逃逸到其他指定的Master A上(需要同一个Master,否则CK/ACK无法抵消,造成消息重复),Master A扫描自身Commitlog revive消息并进行抵消,若超时,则将根据CK消息中的信息向Slave拉取消息(若本地有则拉取本地,否则远程拉取),然后投放到本地的retry topic中。
+
+数据量上,如果是远程投递或拉取,且有消费者大量通过Pop消费存量的Slave消息,并且长时间不ACK,则在集群内部会有较大数据流转。
+
+### 预上线机制
+
+![](https://s4.ax1x.com/2022/02/05/HnW5Pe.png)
+
+当Master Broker下线后,Slave Broker将承担备读的作用,并对二级消息进行代理,因此Slave Broker中的部分元数据包括消费位点、定时消息进度等会比下线的Master Broker更加超前。如果Master Broker重新上线,Slave Broker元数据将被Master Broker覆盖,该组Broker元数据将发生回退,可能造成大量消息重复。因此,需要一套预上线机制来完成元数据的反向同步。
+
+需要为consumerOffset和delayOffset等元数据增加版本号(DataVersion)的概念,并且为了防止版本号更新太频繁,增加更新步长的概念,比如对于消费位点来说,默认每更新位点超过500次,版本号增加到下一个版本。
+
+如上图所示,Master Broker启动前会进行预上线,再预上线之前,对外不可见(Broker会有isIsolated标记自己的状态,当其为true时,不会像nameserver注册和发送心跳),因此也不会对外提供服务,二级消息的扫描流程也不会进行启动,具体预上线机制如下:
+
+1. Master Broker向NameServer获取Slave Broker地址(GetBrokerMemberGroup请求),但不注册
+2. Master Broker向Slave Broker发送自己的状态信息和地址
+3. Slave Broker得到Master Broker地址后和状态信息后,建立HA连接,并完成握手,进入Transfer状态
+4. Master Broker再完成握手后,反向获取备的元数据,包括消费位点、定时消息进度等,根据版本号决定是否更新。
+5. Master Broker对broker组内所有Slave Broker都完成1-4步操作后,正式上线,向NameServer注册,正式对外提供服务。
+
+### 锁Quorum
+
+当Slave代理Master时,外部看到的是“只读”的Master,因此顺序消息仍然可以对队列上锁,消费不会中断。但当真的Master重新上线后,在一定的时间差内可能会造成多个consumer锁定同一个队列,比如一个consumer仍然锁着代理的备某一个队列,一个consumer锁刚上线的主的同一队列,造成顺序消息的乱序和重复。
+
+因此在lock操作时要求,需锁broker副本组的大多数成员(quorum原则)均成功才算锁成功。但两副本下达不到quorum的原则,所以提供了lockInStrictMode参数,表示消费端消费顺序消息锁队列时是否使用严格模式。严格模式即对单个队列而言,需锁副本组的大多数成员(quorum原则)均成功才算锁成功,非严格模式即锁任意一副本成功就算锁成功,该参数默认为false。当对消息顺序性高于可用性时,需将该参数设置为false。
+
+## 配置更新
+
+Nameserver
+- scanNotActiveBrokerInterval:扫描不活跃broker间隔,每次扫描将判断broker心跳是否超时,默认5s。
+- supportActingMaster:nameserver端是否支持Slave代理Master模式,开启后,副本组在无master状态下,brokerId==1的slave将在TopicRoute中被替换成master(即brokerId=0),并以只读模式对客户端提供服务,默认为false。
+Broker
+- enableSlaveActingMaster:broker端开启slave代理master模式总开关,默认为false。
+- enableRemoteEscape:是否允许远程逃逸,默认为false。
+- brokerHeartbeatInterval:broker向nameserver发送心跳间隔(不同于注册间隔),默认1s。
+- brokerNotActiveTimeoutMillis:broker不活跃超时时间,超过此时间nameserver仍未收到broker心跳,则判定broker下线,默认10s。
+- sendHeartbeatTimeoutMillis:broker发送心跳请求超时时间,默认1s。
+- lockInStrictMode:消费端消费顺序消息锁队列时是否使用严格模式,默认为false,上文已介绍。
+- skipPreOnline:broker跳过预上线流程,默认为false。
+- compatibleWithOldNameSrv:是否以兼容模式访问旧nameserver,默认为true。
+
+## 兼容性方案
+
+新版nameserver和旧版broker:新版nameserver可以完全兼容旧版broker,无兼容问题。
+
+旧版nameserver和新版Broker:新版Broker开启Slave代理Master,会向Nameserver发送 BROKER_HEARTBEAT以及GET_BROKER_MEMBER_GROUP请求,但由于旧版本nameserver无法处理这些请求。因此需要在brokerConfig中配置compatibleWithOldNameSrv=true,开启对旧版nameserver的兼容模式,在该模式下,broker的一些新增RPC将通过复用原有RequestCode实现,具体为:
+新增轻量级心跳将通过复用QUERY_DATA_VERSION实现
+新增获取BrokerMemberGroup数据将通过复用GET_ROUTEINFO_BY_TOPIC实现,具体实现方式是每个broker都会新增rmq_sys_{brokerName}的系统topic,通过获取该系统topic的路由来获取该副本组的存活信息。
+但旧版nameserver无法提供代理功能,Slave代理Master的功能将无法生效,但不影响其他功能。
+
+客户端对新旧版本的nameserver和broker均无兼容性问题。
+
+
+参考文档: [原RIP](https://github.com/apache/rocketmq/wiki/RIP-32-Slave-Acting-Master-Mode)
\ No newline at end of file
diff --git a/logging/src/main/java/org/apache/rocketmq/logging/InternalLoggerFactory.java b/logging/src/main/java/org/apache/rocketmq/logging/InternalLoggerFactory.java
index 17c56bd..e4b5732 100644
--- a/logging/src/main/java/org/apache/rocketmq/logging/InternalLoggerFactory.java
+++ b/logging/src/main/java/org/apache/rocketmq/logging/InternalLoggerFactory.java
@@ -27,8 +27,19 @@ public abstract class InternalLoggerFactory {
 
     public static final String DEFAULT_LOGGER = LOGGER_SLF4J;
 
+    public static final String BROKER_CONTAINER_NAME = "BrokerContainer";
+
+    /**
+     * Loggers with following name will be directed to default logger for LogTail parser.
+     */
+    public static final String CONSUMER_STATS_LOGGER_NAME = "RocketmqConsumerStats";
+    public static final String COMMERCIAL_LOGGER_NAME = "RocketmqCommercial";
+    public static final String ACCOUNT_LOGGER_NAME = "RocketmqAccount";
+
     private static String loggerType = null;
 
+    public static ThreadLocal<String> brokerIdentity = new ThreadLocal<String>();
+
     private static ConcurrentHashMap<String, InternalLoggerFactory> loggerFactoryCache = new ConcurrentHashMap<String, InternalLoggerFactory>();
 
     public static InternalLogger getLogger(Class clazz) {
diff --git a/logging/src/main/java/org/apache/rocketmq/logging/Slf4jLoggerFactory.java b/logging/src/main/java/org/apache/rocketmq/logging/Slf4jLoggerFactory.java
index 53dbc94..c8516c3 100644
--- a/logging/src/main/java/org/apache/rocketmq/logging/Slf4jLoggerFactory.java
+++ b/logging/src/main/java/org/apache/rocketmq/logging/Slf4jLoggerFactory.java
@@ -17,6 +17,10 @@
 
 package org.apache.rocketmq.logging;
 
+import java.util.HashMap;
+import java.util.Map;
+import java.util.regex.Matcher;
+import java.util.regex.Pattern;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 
@@ -43,116 +47,147 @@ public class Slf4jLoggerFactory extends InternalLoggerFactory {
     }
 
     public static class Slf4jLogger implements InternalLogger {
-
-        private Logger logger = null;
-
-        public Slf4jLogger(String name) {
-            logger = LoggerFactory.getLogger(name);
+        private static final Pattern PATTERN = Pattern.compile("#.*#");
+
+        private final String loggerSuffix;
+        private final Logger defaultLogger;
+
+        private final Map<String, Logger> loggerMap = new HashMap<String, Logger>();
+
+        public Slf4jLogger(String loggerSuffix) {
+            this.loggerSuffix = loggerSuffix;
+            this.defaultLogger = LoggerFactory.getLogger(loggerSuffix);
+        }
+
+        private Logger getLogger() {
+            if (loggerSuffix.equals(ACCOUNT_LOGGER_NAME)
+                || loggerSuffix.equals(CONSUMER_STATS_LOGGER_NAME)
+                || loggerSuffix.equals(COMMERCIAL_LOGGER_NAME)) {
+                return defaultLogger;
+            }
+            String brokerIdentity = InnerLoggerFactory.brokerIdentity.get();
+            if (brokerIdentity == null) {
+                Matcher m = PATTERN.matcher(Thread.currentThread().getName());
+                if (m.find()) {
+                    String match = m.group();
+                    brokerIdentity = match.substring(1, match.length() - 1);
+                }
+            }
+            if (InnerLoggerFactory.BROKER_CONTAINER_NAME.equals(brokerIdentity)) {
+                return defaultLogger;
+            }
+            if (brokerIdentity != null) {
+                if (!loggerMap.containsKey(brokerIdentity)) {
+                    loggerMap.put(brokerIdentity, LoggerFactory.getLogger("#" + brokerIdentity + "#" + loggerSuffix));
+                }
+                return loggerMap.get(brokerIdentity);
+            }
+            return defaultLogger;
         }
 
         @Override
         public String getName() {
-            return logger.getName();
+            return getLogger().getName();
         }
 
         @Override
         public void debug(String s) {
-            logger.debug(s);
+            getLogger().debug(s);
         }
 
         @Override
         public void debug(String s, Object o) {
-            logger.debug(s, o);
+            getLogger().debug(s, o);
         }
 
         @Override
         public void debug(String s, Object o, Object o1) {
-            logger.debug(s, o, o1);
+            getLogger().debug(s, o, o1);
         }
 
         @Override
         public void debug(String s, Object... objects) {
-            logger.debug(s, objects);
+            getLogger().debug(s, objects);
         }
 
         @Override
         public void debug(String s, Throwable throwable) {
-            logger.debug(s, throwable);
+            getLogger().debug(s, throwable);
         }
 
         @Override
         public void info(String s) {
-            logger.info(s);
+            getLogger().info(s);
         }
 
         @Override
         public void info(String s, Object o) {
-            logger.info(s, o);
+            getLogger().info(s, o);
         }
 
         @Override
         public void info(String s, Object o, Object o1) {
-            logger.info(s, o, o1);
+            getLogger().info(s, o, o1);
         }
 
         @Override
         public void info(String s, Object... objects) {
-            logger.info(s, objects);
+            getLogger().info(s, objects);
         }
 
         @Override
         public void info(String s, Throwable throwable) {
-            logger.info(s, throwable);
+            getLogger().info(s, throwable);
         }
 
         @Override
         public void warn(String s) {
-            logger.warn(s);
+            getLogger().warn(s);
         }
 
         @Override
         public void warn(String s, Object o) {
-            logger.warn(s, o);
+            getLogger().warn(s, o);
         }
 
         @Override
         public void warn(String s, Object... objects) {
-            logger.warn(s, objects);
+            getLogger().warn(s, objects);
         }
 
         @Override
         public void warn(String s, Object o, Object o1) {
-            logger.warn(s, o, o1);
+            getLogger().warn(s, o, o1);
         }
 
         @Override
         public void warn(String s, Throwable throwable) {
-            logger.warn(s, throwable);
+            getLogger().warn(s, throwable);
         }
 
         @Override
         public void error(String s) {
-            logger.error(s);
+            getLogger().error(s);
         }
 
         @Override
         public void error(String s, Object o) {
-            logger.error(s, o);
+            getLogger().error(s, o);
         }
 
         @Override
         public void error(String s, Object o, Object o1) {
-            logger.error(s, o, o1);
+            getLogger().error(s, o, o1);
         }
 
         @Override
         public void error(String s, Object... objects) {
-            logger.error(s, objects);
+            getLogger().error(s, objects);
         }
 
         @Override
         public void error(String s, Throwable throwable) {
-            logger.error(s, throwable);
+            getLogger().error(s, throwable);
         }
     }
 }
diff --git a/logging/src/main/java/org/apache/rocketmq/logging/inner/Level.java b/logging/src/main/java/org/apache/rocketmq/logging/inner/Level.java
index 0dc81d7..487682c 100755
--- a/logging/src/main/java/org/apache/rocketmq/logging/inner/Level.java
+++ b/logging/src/main/java/org/apache/rocketmq/logging/inner/Level.java
@@ -116,6 +116,7 @@ public class Level implements Serializable {
         if (s.equals(OFF_NAME)) {
             return Level.OFF;
         }
+
         if (s.equals(INFO_NAME)) {
             return Level.INFO;
         }
diff --git a/logging/src/main/java/org/apache/rocketmq/logging/inner/SysLogger.java b/logging/src/main/java/org/apache/rocketmq/logging/inner/SysLogger.java
index aaba4d6..b6d1049 100755
--- a/logging/src/main/java/org/apache/rocketmq/logging/inner/SysLogger.java
+++ b/logging/src/main/java/org/apache/rocketmq/logging/inner/SysLogger.java
@@ -33,13 +33,13 @@ public class SysLogger {
 
     public static void debug(String msg) {
         if (debugEnabled && !quietMode) {
-            System.err.println(PREFIX + msg);
+            System.out.printf("%s", PREFIX + msg);
         }
     }
 
     public static void debug(String msg, Throwable t) {
         if (debugEnabled && !quietMode) {
-            System.err.println(PREFIX + msg);
+            System.out.printf("%s", PREFIX + msg);
             if (t != null) {
                 t.printStackTrace(System.out);
             }
diff --git a/logging/src/test/java/org/apache/rocketmq/logging/Slf4jLoggerFactoryTest.java b/logging/src/test/java/org/apache/rocketmq/logging/Slf4jLoggerFactoryTest.java
index 4bed745..2fe2abf 100644
--- a/logging/src/test/java/org/apache/rocketmq/logging/Slf4jLoggerFactoryTest.java
+++ b/logging/src/test/java/org/apache/rocketmq/logging/Slf4jLoggerFactoryTest.java
@@ -69,7 +69,7 @@ public class Slf4jLoggerFactoryTest extends BasicLoggerTest {
         logger3.error("error {}", "hahahah");
         logger3.debug("debug {}", "hahahah");
         String content = readFile(file);
-        System.out.println(content);
+        System.out.printf(content);
 
         Assert.assertTrue(content.contains("Slf4jLoggerFactoryTest"));
         Assert.assertTrue(content.contains("info"));
diff --git a/logging/src/test/java/org/apache/rocketmq/logging/inner/LoggingBuilderTest.java b/logging/src/test/java/org/apache/rocketmq/logging/inner/LoggingBuilderTest.java
index 6c816a6..e3dbb14 100644
--- a/logging/src/test/java/org/apache/rocketmq/logging/inner/LoggingBuilderTest.java
+++ b/logging/src/test/java/org/apache/rocketmq/logging/inner/LoggingBuilderTest.java
@@ -17,15 +17,15 @@
 
 package org.apache.rocketmq.logging.inner;
 
-import org.apache.rocketmq.logging.BasicLoggerTest;
-import org.junit.Assert;
-import org.junit.Test;
-
 import java.io.ByteArrayOutputStream;
 import java.io.File;
 import java.io.FilenameFilter;
 import java.io.PrintStream;
 
+import org.apache.rocketmq.logging.BasicLoggerTest;
+import org.junit.Assert;
+import org.junit.Test;
+
 public class LoggingBuilderTest extends BasicLoggerTest {
 
     @Test
diff --git a/srvutil/pom.xml b/srvutil/pom.xml
index 820a406..ee1617e 100644
--- a/srvutil/pom.xml
+++ b/srvutil/pom.xml
@@ -41,5 +41,13 @@
             <groupId>commons-cli</groupId>
             <artifactId>commons-cli</artifactId>
         </dependency>
+        <dependency>
+            <groupId>com.google.guava</groupId>
+            <artifactId>guava</artifactId>
+        </dependency>
+        <dependency>
+            <groupId>com.googlecode.concurrentlinkedhashmap</groupId>
+            <artifactId>concurrentlinkedhashmap-lru</artifactId>
+        </dependency>
     </dependencies>
 </project>
diff --git a/srvutil/src/main/java/org/apache/rocketmq/srvutil/FileWatchService.java b/srvutil/src/main/java/org/apache/rocketmq/srvutil/FileWatchService.java
index 111cede..adf61ad 100644
--- a/srvutil/src/main/java/org/apache/rocketmq/srvutil/FileWatchService.java
+++ b/srvutil/src/main/java/org/apache/rocketmq/srvutil/FileWatchService.java
@@ -17,6 +17,7 @@
 
 package org.apache.rocketmq.srvutil;
 
+import com.google.common.base.Strings;
 import java.io.File;
 import java.io.IOException;
 import java.nio.file.Files;
@@ -26,7 +27,6 @@ import java.security.MessageDigest;
 import java.security.NoSuchAlgorithmException;
 import java.util.ArrayList;
 import java.util.List;
-import org.apache.commons.lang3.StringUtils;
 import org.apache.rocketmq.common.ServiceThread;
 import org.apache.rocketmq.common.UtilAll;
 import org.apache.rocketmq.common.constant.LoggerName;
@@ -49,7 +49,7 @@ public class FileWatchService extends ServiceThread {
         this.fileCurrentHash = new ArrayList<>();
 
         for (int i = 0; i < watchFiles.length; i++) {
-            if (StringUtils.isNotEmpty(watchFiles[i]) && new File(watchFiles[i]).exists()) {
+            if (!Strings.isNullOrEmpty(watchFiles[i]) && new File(watchFiles[i]).exists()) {
                 this.watchFiles.add(watchFiles[i]);
                 this.fileCurrentHash.add(hash(watchFiles[i]));
             }
diff --git a/srvutil/src/main/java/org/apache/rocketmq/util/cache/CacheEvictHandler.java b/srvutil/src/main/java/org/apache/rocketmq/util/cache/CacheEvictHandler.java
new file mode 100644
index 0000000..13dcd9e
--- /dev/null
+++ b/srvutil/src/main/java/org/apache/rocketmq/util/cache/CacheEvictHandler.java
@@ -0,0 +1,23 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.rocketmq.util.cache;
+
+import java.util.Map;
+
+public interface CacheEvictHandler<K, V> {
+    void onEvict(Map.Entry<K, V> eldest);
+}
diff --git a/srvutil/src/main/java/org/apache/rocketmq/util/cache/CacheObject.java b/srvutil/src/main/java/org/apache/rocketmq/util/cache/CacheObject.java
new file mode 100644
index 0000000..39c64ce
--- /dev/null
+++ b/srvutil/src/main/java/org/apache/rocketmq/util/cache/CacheObject.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.util.cache;
+
+
+public class CacheObject<T> {
+    private T target;
+    private long bornTime = System.currentTimeMillis();
+    private long exp;
+
+    public CacheObject(long exp, T target) {
+        this.exp = exp;
+        this.target = target;
+    }
+
+    public T getTarget() {
+        if (System.currentTimeMillis() - bornTime > exp) {
+            return null;
+        }
+        return target;
+    }
+}
diff --git a/srvutil/src/main/java/org/apache/rocketmq/util/cache/ExpiredLocalCache.java b/srvutil/src/main/java/org/apache/rocketmq/util/cache/ExpiredLocalCache.java
new file mode 100644
index 0000000..ab60f0f
--- /dev/null
+++ b/srvutil/src/main/java/org/apache/rocketmq/util/cache/ExpiredLocalCache.java
@@ -0,0 +1,84 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.rocketmq.util.cache;
+
+import com.googlecode.concurrentlinkedhashmap.ConcurrentLinkedHashMap;
+import com.googlecode.concurrentlinkedhashmap.EvictionListener;
+
+public class ExpiredLocalCache<K, T> {
+    private ConcurrentLinkedHashMap<K, CacheObject<T>> cache;
+    private EvictionListener<K, CacheObject<T>> listener;
+
+    public ExpiredLocalCache(int size) {
+        cache = new ConcurrentLinkedHashMap.Builder<K, CacheObject<T>>().maximumWeightedCapacity(size).build();
+    }
+
+    public ExpiredLocalCache(int size, String name) {
+        cache = new ConcurrentLinkedHashMap.Builder<K, CacheObject<T>>().maximumWeightedCapacity(size).build();
+    }
+
+    public ExpiredLocalCache(int size, boolean memoryMeter, EvictionListener<K, CacheObject<T>> listener) {
+        this.listener = listener;
+        cache = new ConcurrentLinkedHashMap.Builder<K, CacheObject<T>>().listener(listener).maximumWeightedCapacity(size).build();
+    }
+
+    public T get(K key) {
+        CacheObject<T> object = cache.get(key);
+        if (object == null) {
+            return null;
+        }
+        T ret = object.getTarget();
+        if (ret == null) {
+            this.delete(key);
+        }
+        return ret;
+    }
+
+    public T put(K key, T v, long exp) {
+        CacheObject<T> value = new CacheObject<T>(exp, v);
+        CacheObject<T> old = cache.put(key, value);
+        if (old == null) {
+            return null;
+        } else {
+            return old.getTarget();
+        }
+    }
+
+    public T putIfAbsent(K key, T v, long exp) {
+        CacheObject<T> value = new CacheObject<T>(exp, v);
+        CacheObject<T> old = cache.putIfAbsent(key, value);
+        if (old == null) {
+            return null;
+        } else {
+            return old.getTarget();
+        }
+    }
+
+    public T delete(K key) {
+        CacheObject<T> object = cache.remove(key);
+        if (object == null) {
+            return null;
+        }
+        T ret = object.getTarget();
+        return ret;
+    }
+
+    public ConcurrentLinkedHashMap<K, CacheObject<T>> getCache() {
+        return cache;
+    }
+
+}
diff --git a/srvutil/src/main/java/org/apache/rocketmq/util/cache/LocalCache.java b/srvutil/src/main/java/org/apache/rocketmq/util/cache/LocalCache.java
new file mode 100644
index 0000000..7dbb6e2
--- /dev/null
+++ b/srvutil/src/main/java/org/apache/rocketmq/util/cache/LocalCache.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.util.cache;
+
+import java.util.LinkedHashMap;
+import java.util.Map;
+
+public class LocalCache<K, V> extends LinkedHashMap<K, V> {
+
+    private static final long serialVersionUID = 1606231700062718297L;
+
+    private static final int DEFAULT_CACHE_SIZE = 1000;
+
+    private int cacheSize = DEFAULT_CACHE_SIZE;
+    private CacheEvictHandler<K, V> handler;
+
+    /**
+     * The default initial capacity - MUST be a power of two.
+     */
+    static final int DEFAULT_INITIAL_CAPACITY = 1 << 4; // aka 16
+
+
+    /**
+     * The load factor used when none specified in constructor.
+     */
+    static final float DEFAULT_LOAD_FACTOR = 0.75f;
+
+    public LocalCache(int cacheSize, boolean isLru, CacheEvictHandler<K, V> handler) {
+        super(DEFAULT_INITIAL_CAPACITY, DEFAULT_LOAD_FACTOR, isLru);
+        this.cacheSize = cacheSize;
+        this.handler = handler;
+    }
+
+
+    @Override
+    protected boolean removeEldestEntry(Map.Entry<K, V> eldest) {
+        boolean result = this.size() > cacheSize;
+        if (result && handler != null) {
+            handler.onEvict(eldest);
+        }
+        return result;
+    }
+
+}
diff --git a/srvutil/src/main/java/org/apache/rocketmq/util/cache/LockManager.java b/srvutil/src/main/java/org/apache/rocketmq/util/cache/LockManager.java
new file mode 100644
index 0000000..ae6906c
--- /dev/null
+++ b/srvutil/src/main/java/org/apache/rocketmq/util/cache/LockManager.java
@@ -0,0 +1,54 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.rocketmq.util.cache;
+
+import java.util.concurrent.atomic.AtomicBoolean;
+
+import org.apache.rocketmq.common.PopAckConstants;
+import org.apache.rocketmq.common.protocol.header.PopMessageRequestHeader;
+
+public class LockManager {
+    private static ExpiredLocalCache<String, AtomicBoolean> expiredLocalCache = new ExpiredLocalCache<String, AtomicBoolean>(100000);
+
+    public static boolean tryLock(String key, long lockTime) {
+        AtomicBoolean v = expiredLocalCache.get(key);
+        if (v == null) {
+            return expiredLocalCache.putIfAbsent(key, new AtomicBoolean(false), lockTime) == null;
+        } else {
+            return v.compareAndSet(true, false);
+        }
+    }
+
+    public static void unLock(String key) {
+        AtomicBoolean v = expiredLocalCache.get(key);
+        if (v != null) {
+            v.set(true);
+        }
+    }
+
+    public static String buildKey(PopMessageRequestHeader requestHeader, int queueId) {
+        return requestHeader.getConsumerGroup() + PopAckConstants.SPLIT + requestHeader.getTopic() + PopAckConstants.SPLIT + queueId;
+    }
+
+    public static String buildKey(String topic, String cid, int queueId) {
+        return topic + PopAckConstants.SPLIT + cid + PopAckConstants.SPLIT + queueId;
+    }
+
+    public static String buildKey(String prefix, int queueId) {
+        return prefix + PopAckConstants.SPLIT + queueId;
+    }
+}
diff --git a/tools/src/main/java/org/apache/rocketmq/tools/admin/DefaultMQAdminExt.java b/tools/src/main/java/org/apache/rocketmq/tools/admin/DefaultMQAdminExt.java
index 2aa4488..b8bb399 100644
--- a/tools/src/main/java/org/apache/rocketmq/tools/admin/DefaultMQAdminExt.java
+++ b/tools/src/main/java/org/apache/rocketmq/tools/admin/DefaultMQAdminExt.java
@@ -37,6 +37,7 @@ import org.apache.rocketmq.common.protocol.body.ConsumeStatsList;
 import org.apache.rocketmq.common.protocol.body.ConsumerConnection;
 import org.apache.rocketmq.common.protocol.body.ConsumerRunningInfo;
 import org.apache.rocketmq.common.protocol.body.GroupList;
+import org.apache.rocketmq.common.protocol.body.HARuntimeInfo;
 import org.apache.rocketmq.common.protocol.body.KVTable;
 import org.apache.rocketmq.common.protocol.body.ProducerConnection;
 import org.apache.rocketmq.common.protocol.body.QueryConsumeQueueResponseBody;
@@ -44,8 +45,10 @@ import org.apache.rocketmq.common.protocol.body.QueueTimeSpan;
 import org.apache.rocketmq.common.protocol.body.SubscriptionGroupWrapper;
 import org.apache.rocketmq.common.protocol.body.TopicConfigSerializeWrapper;
 import org.apache.rocketmq.common.protocol.body.TopicList;
+import org.apache.rocketmq.common.protocol.heartbeat.SubscriptionData;
 import org.apache.rocketmq.common.protocol.route.TopicRouteData;
 import org.apache.rocketmq.common.statictopic.TopicQueueMappingDetail;
+import org.apache.rocketmq.common.subscription.GroupForbidden;
 import org.apache.rocketmq.common.subscription.SubscriptionGroupConfig;
 import org.apache.rocketmq.common.topic.TopicValidator;
 import org.apache.rocketmq.remoting.RPCHook;
@@ -54,6 +57,7 @@ import org.apache.rocketmq.remoting.exception.RemotingConnectException;
 import org.apache.rocketmq.remoting.exception.RemotingException;
 import org.apache.rocketmq.remoting.exception.RemotingSendRequestException;
 import org.apache.rocketmq.remoting.exception.RemotingTimeoutException;
+import org.apache.rocketmq.tools.admin.api.BrokerOperatorResult;
 import org.apache.rocketmq.tools.admin.api.MessageTrack;
 
 import java.io.UnsupportedEncodingException;
@@ -61,6 +65,7 @@ import java.util.List;
 import java.util.Map;
 import java.util.Properties;
 import java.util.Set;
+import org.apache.rocketmq.tools.admin.common.AdminToolResult;
 
 public class DefaultMQAdminExt extends ClientConfig implements MQAdminExt {
     private final DefaultMQAdminExtImpl defaultMQAdminExtImpl;
@@ -95,12 +100,14 @@ public class DefaultMQAdminExt extends ClientConfig implements MQAdminExt {
     }
 
     @Override
-    public void createTopic(String key, String newTopic, int queueNum, Map<String, String> attributes) throws MQClientException {
+    public void createTopic(String key, String newTopic, int queueNum,
+        Map<String, String> attributes) throws MQClientException {
         createTopic(key, newTopic, queueNum, 0, attributes);
     }
 
     @Override
-    public void createTopic(String key, String newTopic, int queueNum, int topicSysFlag, Map<String, String> attributes) throws MQClientException {
+    public void createTopic(String key, String newTopic, int queueNum, int topicSysFlag,
+        Map<String, String> attributes) throws MQClientException {
         defaultMQAdminExtImpl.createTopic(key, newTopic, queueNum, topicSysFlag, attributes);
     }
 
@@ -133,16 +140,9 @@ public class DefaultMQAdminExt extends ClientConfig implements MQAdminExt {
     @Override
     public QueryResult queryMessage(String topic, String key, int maxNum, long begin, long end)
         throws MQClientException, InterruptedException {
-
         return defaultMQAdminExtImpl.queryMessage(topic, key, maxNum, begin, end);
     }
 
-    public QueryResult queryMessageByUniqKey(String topic, String key, int maxNum, long begin, long end)
-        throws MQClientException, InterruptedException {
-
-        return defaultMQAdminExtImpl.queryMessageByUniqKey(topic, key, maxNum, begin, end);
-    }
-
     @Override
     public void start() throws MQClientException {
         defaultMQAdminExtImpl.start();
@@ -154,6 +154,18 @@ public class DefaultMQAdminExt extends ClientConfig implements MQAdminExt {
     }
 
     @Override
+    public void addBrokerToContainer(String brokerContainerAddr, String brokerConfig) throws InterruptedException,
+        RemotingConnectException, RemotingTimeoutException, RemotingSendRequestException, MQBrokerException {
+        defaultMQAdminExtImpl.addBrokerToContainer(brokerContainerAddr, brokerConfig);
+    }
+
+    @Override public void removeBrokerFromContainer(String brokerContainerAddr, String clusterName, String brokerName,
+        long brokerId) throws InterruptedException, MQBrokerException, RemotingTimeoutException, RemotingSendRequestException,
+        RemotingConnectException {
+        defaultMQAdminExtImpl.removeBrokerFromContainer(brokerContainerAddr, clusterName, brokerName, brokerId);
+    }
+
+    @Override
     public void updateBrokerConfig(String brokerAddr,
         Properties properties) throws RemotingConnectException, RemotingSendRequestException,
         RemotingTimeoutException, UnsupportedEncodingException, InterruptedException, MQBrokerException {
@@ -225,12 +237,18 @@ public class DefaultMQAdminExt extends ClientConfig implements MQAdminExt {
     }
 
     @Override
-    public TopicStatsTable examineTopicStats(String brokerAddr, String topic) throws RemotingException, MQClientException, InterruptedException,
-            MQBrokerException {
+    public TopicStatsTable examineTopicStats(String brokerAddr,
+        String topic) throws RemotingException, MQClientException, InterruptedException,
+        MQBrokerException {
         return defaultMQAdminExtImpl.examineTopicStats(brokerAddr, topic);
     }
 
     @Override
+    public AdminToolResult<TopicStatsTable> examineTopicStatsConcurrent(String topic) {
+        return defaultMQAdminExtImpl.examineTopicStatsConcurrent(topic);
+    }
+
+    @Override
     public TopicList fetchAllTopicList() throws RemotingException, MQClientException, InterruptedException {
         return this.defaultMQAdminExtImpl.fetchAllTopicList();
     }
@@ -263,6 +281,11 @@ public class DefaultMQAdminExt extends ClientConfig implements MQAdminExt {
     }
 
     @Override
+    public AdminToolResult<ConsumeStats> examineConsumeStatsConcurrent(String consumerGroup, String topic) {
+        return defaultMQAdminExtImpl.examineConsumeStatsConcurrent(consumerGroup, topic);
+    }
+
+    @Override
     public ClusterInfo examineBrokerClusterInfo() throws InterruptedException, RemotingConnectException, RemotingTimeoutException,
         RemotingSendRequestException, MQBrokerException {
         return defaultMQAdminExtImpl.examineBrokerClusterInfo();
@@ -295,6 +318,11 @@ public class DefaultMQAdminExt extends ClientConfig implements MQAdminExt {
         return defaultMQAdminExtImpl.examineProducerConnectionInfo(producerGroup, topic);
     }
 
+    @Override public void deleteTopicInNameServer(Set<String> addrs, String clusterName,
+        String topic) throws RemotingException, MQBrokerException, InterruptedException, MQClientException {
+        defaultMQAdminExtImpl.deleteTopicInNameServer(addrs, clusterName, topic);
+    }
+
     @Override
     public List<String> getNameServerAddressList() {
         return this.defaultMQAdminExtImpl.getNameServerAddressList();
@@ -330,6 +358,13 @@ public class DefaultMQAdminExt extends ClientConfig implements MQAdminExt {
     }
 
     @Override
+    public void deleteTopic(String topicName,
+        String clusterName) throws RemotingException, MQBrokerException, InterruptedException,
+        MQClientException {
+        defaultMQAdminExtImpl.deleteTopic(topicName, clusterName);
+    }
+
+    @Override
     public void deleteTopicInBroker(Set<String> addrs,
         String topic) throws RemotingException, MQBrokerException, InterruptedException,
         MQClientException {
@@ -337,6 +372,11 @@ public class DefaultMQAdminExt extends ClientConfig implements MQAdminExt {
     }
 
     @Override
+    public AdminToolResult<BrokerOperatorResult> deleteTopicInBrokerConcurrent(Set<String> addrs, String topic) {
+        return defaultMQAdminExtImpl.deleteTopicInBrokerConcurrent(addrs, topic);
+    }
+
+    @Override
     public void deleteTopicInNameServer(Set<String> addrs,
         String topic) throws RemotingException, MQBrokerException, InterruptedException,
         MQClientException {
@@ -398,6 +438,12 @@ public class DefaultMQAdminExt extends ClientConfig implements MQAdminExt {
     }
 
     @Override
+    public AdminToolResult<BrokerOperatorResult> resetOffsetNewConcurrent(final String group, final String topic,
+        final long timestamp) {
+        return this.defaultMQAdminExtImpl.resetOffsetNewConcurrent(group, topic, timestamp);
+    }
+
+    @Override
     public Map<String, Map<MessageQueue, Long>> getConsumeStatus(String topic, String group,
         String clientAddr) throws RemotingException,
         MQBrokerException, InterruptedException, MQClientException {
@@ -419,6 +465,23 @@ public class DefaultMQAdminExt extends ClientConfig implements MQAdminExt {
     }
 
     @Override
+    public TopicList queryTopicsByConsumer(String group)
+        throws InterruptedException, MQBrokerException, RemotingException, MQClientException {
+        return this.defaultMQAdminExtImpl.queryTopicsByConsumer(group);
+    }
+
+    @Override
+    public AdminToolResult<TopicList> queryTopicsByConsumerConcurrent(String group) {
+        return defaultMQAdminExtImpl.queryTopicsByConsumerConcurrent(group);
+    }
+
+    @Override
+    public SubscriptionData querySubscription(String group, String topic)
+        throws InterruptedException, MQBrokerException, RemotingException, MQClientException {
+        return this.defaultMQAdminExtImpl.querySubscription(group, topic);
+    }
+
+    @Override
     public List<QueueTimeSpan> queryConsumeTimeSpan(final String topic,
         final String group) throws InterruptedException, MQBrokerException,
         RemotingException, MQClientException {
@@ -426,6 +489,11 @@ public class DefaultMQAdminExt extends ClientConfig implements MQAdminExt {
     }
 
     @Override
+    public AdminToolResult<List<QueueTimeSpan>> queryConsumeTimeSpanConcurrent(String topic, String group) {
+        return defaultMQAdminExtImpl.queryConsumeTimeSpanConcurrent(topic, group);
+    }
+
+    @Override
     public boolean cleanExpiredConsumerQueue(
         String cluster) throws RemotingConnectException, RemotingSendRequestException,
         RemotingTimeoutException, MQClientException, InterruptedException {
@@ -459,6 +527,13 @@ public class DefaultMQAdminExt extends ClientConfig implements MQAdminExt {
     }
 
     @Override
+    public ConsumerRunningInfo getConsumerRunningInfo(String consumerGroup, String clientId,
+        boolean jstack, boolean metrics) throws RemotingException,
+        MQClientException, InterruptedException {
+        return defaultMQAdminExtImpl.getConsumerRunningInfo(consumerGroup, clientId, jstack, metrics);
+    }
+
+    @Override
     public ConsumeMessageDirectlyResult consumeMessageDirectly(String consumerGroup, String clientId, String msgId)
         throws RemotingException, MQClientException, InterruptedException, MQBrokerException {
         return defaultMQAdminExtImpl.consumeMessageDirectly(consumerGroup, clientId, msgId);
@@ -479,6 +554,13 @@ public class DefaultMQAdminExt extends ClientConfig implements MQAdminExt {
     }
 
     @Override
+    public List<MessageTrack> messageTrackDetailConcurrent(
+        MessageExt msg) throws RemotingException, MQClientException, InterruptedException,
+        MQBrokerException {
+        return this.defaultMQAdminExtImpl.messageTrackDetailConcurrent(msg);
+    }
+
+    @Override
     public void cloneGroupOffset(String srcGroup, String destGroup, String topic,
         boolean isOffline) throws RemotingException,
         MQClientException, InterruptedException, MQBrokerException {
@@ -548,6 +630,12 @@ public class DefaultMQAdminExt extends ClientConfig implements MQAdminExt {
         return this.defaultMQAdminExtImpl.viewMessage(topic, msgId);
     }
 
+    @Override
+    public MessageExt queryMessage(String clusterName, String topic, String msgId)
+        throws RemotingException, MQBrokerException, InterruptedException, MQClientException {
+        return this.defaultMQAdminExtImpl.queryMessage(clusterName, topic, msgId);
+    }
+
     public String getAdminExtGroup() {
         return adminExtGroup;
     }
@@ -617,9 +705,54 @@ public class DefaultMQAdminExt extends ClientConfig implements MQAdminExt {
     }
 
     @Override
-    public void createStaticTopic(String addr, String defaultTopic, TopicConfig topicConfig, TopicQueueMappingDetail mappingDetail, boolean force) throws RemotingException,  InterruptedException, MQBrokerException {
+    public void createStaticTopic(String addr, String defaultTopic, TopicConfig topicConfig,
+        TopicQueueMappingDetail mappingDetail,
+        boolean force) throws RemotingException, InterruptedException, MQBrokerException {
         this.defaultMQAdminExtImpl.createStaticTopic(addr, defaultTopic, topicConfig, mappingDetail, force);
     }
 
+    @Override
+    public long searchOffset(final String brokerAddr, final String topicName,
+        final int queueId, final long timestamp, final long timeoutMillis)
+        throws RemotingException, MQBrokerException, InterruptedException {
+        return this.defaultMQAdminExtImpl.searchOffset(brokerAddr, topicName, queueId, timestamp, timeoutMillis);
+    }
 
+    @Override
+    public void resetOffsetByQueueId(final String brokerAddr, final String consumerGroup,
+        final String topicName, final int queueId, final long resetOffset)
+        throws RemotingException, InterruptedException, MQBrokerException {
+        this.defaultMQAdminExtImpl.resetOffsetByQueueId(brokerAddr, consumerGroup, topicName, queueId, resetOffset);
+    }
+
+    @Override
+    public HARuntimeInfo getBrokerHAStatus(String brokerAddr) throws RemotingConnectException,
+        RemotingSendRequestException, RemotingTimeoutException, InterruptedException, MQBrokerException {
+        return this.defaultMQAdminExtImpl.getBrokerHAStatus(brokerAddr);
+    }
+
+    @Override
+    public void resetMasterFlushOffset(String brokerAddr, long masterFlushOffset)
+        throws InterruptedException, RemotingConnectException, RemotingTimeoutException, RemotingSendRequestException, MQBrokerException {
+        this.defaultMQAdminExtImpl.resetMasterFlushOffset(brokerAddr, masterFlushOffset);
+    }
+
+    public QueryResult queryMessageByUniqKey(String topic, String key, int maxNum, long begin, long end)
+        throws MQClientException, InterruptedException {
+
+        return defaultMQAdminExtImpl.queryMessageByUniqKey(topic, key, maxNum, begin, end);
+    }
+
+    public DefaultMQAdminExtImpl getDefaultMQAdminExtImpl() {
+        return defaultMQAdminExtImpl;
+    }
+
+    public GroupForbidden updateAndGetGroupReadForbidden(String brokerAddr, //
+        String groupName, //
+        String topicName, //
+        Boolean readable) //
+        throws RemotingException,
+        InterruptedException, MQBrokerException {
+        return this.defaultMQAdminExtImpl.updateAndGetGroupReadForbidden(brokerAddr, groupName, topicName, readable);
+    }
 }
diff --git a/tools/src/main/java/org/apache/rocketmq/tools/admin/DefaultMQAdminExtImpl.java b/tools/src/main/java/org/apache/rocketmq/tools/admin/DefaultMQAdminExtImpl.java
index b6d6ace..dfd3325 100644
--- a/tools/src/main/java/org/apache/rocketmq/tools/admin/DefaultMQAdminExtImpl.java
+++ b/tools/src/main/java/org/apache/rocketmq/tools/admin/DefaultMQAdminExtImpl.java
@@ -17,6 +17,7 @@
 package org.apache.rocketmq.tools.admin;
 
 import java.io.UnsupportedEncodingException;
+import java.text.MessageFormat;
 import java.util.ArrayList;
 import java.util.Arrays;
 import java.util.Collections;
@@ -29,6 +30,12 @@ import java.util.Map.Entry;
 import java.util.Properties;
 import java.util.Random;
 import java.util.Set;
+import java.util.concurrent.ConcurrentHashMap;
+import java.util.concurrent.CopyOnWriteArrayList;
+import java.util.concurrent.CountDownLatch;
+import java.util.concurrent.LinkedBlockingQueue;
+import java.util.concurrent.ThreadPoolExecutor;
+import java.util.concurrent.TimeUnit;
 import org.apache.commons.lang3.StringUtils;
 import org.apache.rocketmq.client.QueryResult;
 import org.apache.rocketmq.client.admin.MQAdminExtInner;
@@ -38,9 +45,11 @@ import org.apache.rocketmq.client.impl.MQClientManager;
 import org.apache.rocketmq.client.impl.factory.MQClientInstance;
 import org.apache.rocketmq.client.log.ClientLogger;
 import org.apache.rocketmq.common.AclConfig;
+import org.apache.rocketmq.common.KeyBuilder;
 import org.apache.rocketmq.common.MixAll;
 import org.apache.rocketmq.common.PlainAccessConfig;
 import org.apache.rocketmq.common.ServiceState;
+import org.apache.rocketmq.common.ThreadFactoryImpl;
 import org.apache.rocketmq.common.TopicConfig;
 import org.apache.rocketmq.common.UtilAll;
 import org.apache.rocketmq.common.admin.ConsumeStats;
@@ -65,6 +74,7 @@ import org.apache.rocketmq.common.protocol.body.ConsumeStatsList;
 import org.apache.rocketmq.common.protocol.body.ConsumerConnection;
 import org.apache.rocketmq.common.protocol.body.ConsumerRunningInfo;
 import org.apache.rocketmq.common.protocol.body.GroupList;
+import org.apache.rocketmq.common.protocol.body.HARuntimeInfo;
 import org.apache.rocketmq.common.protocol.body.KVTable;
 import org.apache.rocketmq.common.protocol.body.ProducerConnection;
 import org.apache.rocketmq.common.protocol.body.QueryConsumeQueueResponseBody;
@@ -73,12 +83,14 @@ import org.apache.rocketmq.common.protocol.body.SubscriptionGroupWrapper;
 import org.apache.rocketmq.common.protocol.body.TopicConfigSerializeWrapper;
 import org.apache.rocketmq.common.protocol.body.TopicList;
 import org.apache.rocketmq.common.protocol.header.UpdateConsumerOffsetRequestHeader;
+import org.apache.rocketmq.common.protocol.header.UpdateGroupForbiddenRequestHeader;
 import org.apache.rocketmq.common.protocol.heartbeat.SubscriptionData;
 import org.apache.rocketmq.common.protocol.route.BrokerData;
 import org.apache.rocketmq.common.protocol.route.QueueData;
 import org.apache.rocketmq.common.protocol.route.TopicRouteData;
 import org.apache.rocketmq.common.statictopic.TopicConfigAndQueueMapping;
 import org.apache.rocketmq.common.statictopic.TopicQueueMappingDetail;
+import org.apache.rocketmq.common.subscription.GroupForbidden;
 import org.apache.rocketmq.common.subscription.SubscriptionGroupConfig;
 import org.apache.rocketmq.logging.InternalLogger;
 import org.apache.rocketmq.remoting.RPCHook;
@@ -89,8 +101,13 @@ import org.apache.rocketmq.remoting.exception.RemotingConnectException;
 import org.apache.rocketmq.remoting.exception.RemotingException;
 import org.apache.rocketmq.remoting.exception.RemotingSendRequestException;
 import org.apache.rocketmq.remoting.exception.RemotingTimeoutException;
+import org.apache.rocketmq.tools.admin.api.BrokerOperatorResult;
 import org.apache.rocketmq.tools.admin.api.MessageTrack;
 import org.apache.rocketmq.tools.admin.api.TrackType;
+import org.apache.rocketmq.tools.admin.common.AdminToolHandler;
+import org.apache.rocketmq.tools.admin.common.AdminToolResult;
+import org.apache.rocketmq.tools.admin.common.AdminToolsResultCodeEnum;
+import org.apache.rocketmq.tools.command.CommandUtil;
 
 public class DefaultMQAdminExtImpl implements MQAdminExt, MQAdminExtInner {
 
@@ -120,6 +137,9 @@ public class DefaultMQAdminExtImpl implements MQAdminExt, MQAdminExtInner {
     private long timeoutMillis = 20000;
     private Random random = new Random();
 
+    protected final List<String> kvNamespaceToDeleteList = Arrays.asList(NamesrvUtil.NAMESPACE_ORDER_TOPIC_CONFIG);
+    protected ThreadPoolExecutor threadPoolExecutor;
+
     public DefaultMQAdminExtImpl(DefaultMQAdminExt defaultMQAdminExt, long timeoutMillis) {
         this(defaultMQAdminExt, null, timeoutMillis);
     }
@@ -130,8 +150,7 @@ public class DefaultMQAdminExtImpl implements MQAdminExt, MQAdminExtInner {
         this.timeoutMillis = timeoutMillis;
     }
 
-    @Override
-    public void start() throws MQClientException {
+    @Override public void start() throws MQClientException {
         switch (this.serviceState) {
             case CREATE_JUST:
                 this.serviceState = ServiceState.START_FAILED;
@@ -143,9 +162,7 @@ public class DefaultMQAdminExtImpl implements MQAdminExt, MQAdminExtInner {
                 boolean registerOK = mqClientInstance.registerAdminExt(this.defaultMQAdminExt.getAdminExtGroup(), this);
                 if (!registerOK) {
                     this.serviceState = ServiceState.CREATE_JUST;
-                    throw new MQClientException("The adminExt group[" + this.defaultMQAdminExt.getAdminExtGroup()
-                        + "] has created already, specifed another name please."
-                        + FAQUrl.suggestTodo(FAQUrl.GROUP_NAME_DUPLICATE_URL), null);
+                    throw new MQClientException("The adminExt group[" + this.defaultMQAdminExt.getAdminExtGroup() + "] has created already, specifed another name please." + FAQUrl.suggestTodo(FAQUrl.GROUP_NAME_DUPLICATE_URL), null);
                 }
 
                 mqClientInstance.start();
@@ -153,20 +170,22 @@ public class DefaultMQAdminExtImpl implements MQAdminExt, MQAdminExtInner {
                 log.info("the adminExt [{}] start OK", this.defaultMQAdminExt.getAdminExtGroup());
 
                 this.serviceState = ServiceState.RUNNING;
+
+                int theadPoolCoreSize = Integer.parseInt(System.getProperty("rocketmq.admin.threadpool.coresize", "20"));
+
+                this.threadPoolExecutor = new ThreadPoolExecutor(theadPoolCoreSize, 100, 5, TimeUnit.MINUTES, new LinkedBlockingQueue<Runnable>(), new ThreadFactoryImpl("DefaultMQAdminExtImpl_"));
+
                 break;
             case RUNNING:
             case START_FAILED:
             case SHUTDOWN_ALREADY:
-                throw new MQClientException("The AdminExt service state not OK, maybe started once, "
-                    + this.serviceState
-                    + FAQUrl.suggestTodo(FAQUrl.CLIENT_SERVICE_NOT_OK), null);
+                throw new MQClientException("The AdminExt service state not OK, maybe started once, " + this.serviceState + FAQUrl.suggestTodo(FAQUrl.CLIENT_SERVICE_NOT_OK), null);
             default:
                 break;
         }
     }
 
-    @Override
-    public void shutdown() {
+    @Override public void shutdown() {
         switch (this.serviceState) {
             case CREATE_JUST:
                 break;
@@ -176,6 +195,7 @@ public class DefaultMQAdminExtImpl implements MQAdminExt, MQAdminExtInner {
 
                 log.info("the adminExt [{}] shutdown OK", this.defaultMQAdminExt.getAdminExtGroup());
                 this.serviceState = ServiceState.SHUTDOWN_ALREADY;
+                this.threadPoolExecutor.shutdown();
                 break;
             case SHUTDOWN_ALREADY:
                 break;
@@ -184,22 +204,46 @@ public class DefaultMQAdminExtImpl implements MQAdminExt, MQAdminExtInner {
         }
     }
 
-    @Override
-    public void updateBrokerConfig(String brokerAddr,
-        Properties properties) throws RemotingConnectException, RemotingSendRequestException,
-        RemotingTimeoutException, UnsupportedEncodingException, InterruptedException, MQBrokerException {
+    @Override public void addBrokerToContainer(String brokerContainerAddr,
+        String brokerConfig) throws InterruptedException, MQBrokerException, RemotingTimeoutException, RemotingSendRequestException, RemotingConnectException {
+        this.mqClientInstance.getMQClientAPIImpl().addBroker(brokerContainerAddr, brokerConfig, timeoutMillis);
+    }
+
+    @Override public void removeBrokerFromContainer(String brokerContainerAddr, String clusterName, String brokerName,
+        long brokerId) throws InterruptedException, MQBrokerException, RemotingTimeoutException, RemotingSendRequestException, RemotingConnectException {
+        this.mqClientInstance.getMQClientAPIImpl().removeBroker(brokerContainerAddr, clusterName, brokerName, brokerId, 20000);
+    }
+
+    public AdminToolResult adminToolExecute(AdminToolHandler handler) {
+        try {
+            return handler.doExecute();
+        } catch (RemotingException e) {
+            log.error("", e);
+            return AdminToolResult.failure(AdminToolsResultCodeEnum.REMOTING_ERROR, e.getMessage());
+        } catch (MQClientException e) {
+            if (ResponseCode.TOPIC_NOT_EXIST == e.getResponseCode()) {
+                return AdminToolResult.failure(AdminToolsResultCodeEnum.TOPIC_ROUTE_INFO_NOT_EXIST, e.getErrorMessage());
+            }
+            return AdminToolResult.failure(AdminToolsResultCodeEnum.MQ_CLIENT_ERROR, e.getMessage());
+        } catch (InterruptedException e) {
+            return AdminToolResult.failure(AdminToolsResultCodeEnum.INTERRUPT_ERROR, e.getMessage());
+        } catch (Exception e) {
+            return AdminToolResult.failure(AdminToolsResultCodeEnum.MQ_BROKER_ERROR, e.getMessage());
+        }
+    }
+
+    @Override public void updateBrokerConfig(String brokerAddr,
+        Properties properties) throws RemotingConnectException, RemotingSendRequestException, RemotingTimeoutException, UnsupportedEncodingException, InterruptedException, MQBrokerException {
         this.mqClientInstance.getMQClientAPIImpl().updateBrokerConfig(brokerAddr, properties, timeoutMillis);
     }
 
-    @Override
-    public Properties getBrokerConfig(final String brokerAddr) throws RemotingConnectException,
-        RemotingSendRequestException, RemotingTimeoutException, UnsupportedEncodingException, InterruptedException, MQBrokerException {
+    @Override public Properties getBrokerConfig(
+        final String brokerAddr) throws RemotingConnectException, RemotingSendRequestException, RemotingTimeoutException, UnsupportedEncodingException, InterruptedException, MQBrokerException {
         return this.mqClientInstance.getMQClientAPIImpl().getBrokerConfig(brokerAddr, timeoutMillis);
     }
 
-    @Override
-    public void createAndUpdateTopicConfig(String addr, TopicConfig config) throws RemotingException, MQBrokerException,
-        InterruptedException, MQClientException {
+    @Override public void createAndUpdateTopicConfig(String addr,
+        TopicConfig config) throws RemotingException, MQBrokerException, InterruptedException, MQClientException {
         this.mqClientInstance.getMQClientAPIImpl().createTopic(addr, this.defaultMQAdminExt.getCreateTopicKey(), config, timeoutMillis);
     }
 
@@ -218,42 +262,34 @@ public class DefaultMQAdminExtImpl implements MQAdminExt, MQAdminExtInner {
         this.mqClientInstance.getMQClientAPIImpl().updateGlobalWhiteAddrsConfig(addr, globalWhiteAddrs, timeoutMillis);
     }
 
-    @Override
-    public ClusterAclVersionInfo examineBrokerClusterAclVersionInfo(
+    @Override public ClusterAclVersionInfo examineBrokerClusterAclVersionInfo(
         String addr) throws RemotingException, MQBrokerException, InterruptedException, MQClientException {
         return this.mqClientInstance.getMQClientAPIImpl().getBrokerClusterAclInfo(addr, timeoutMillis);
     }
 
-    @Override
-    public AclConfig examineBrokerClusterAclConfig(
+    @Override public AclConfig examineBrokerClusterAclConfig(
         String addr) throws RemotingException, MQBrokerException, InterruptedException, MQClientException {
         return this.mqClientInstance.getMQClientAPIImpl().getBrokerClusterConfig(addr, timeoutMillis);
     }
 
-    @Override
-    public void createAndUpdateSubscriptionGroupConfig(String addr,
-        SubscriptionGroupConfig config) throws RemotingException,
-        MQBrokerException, InterruptedException, MQClientException {
+    @Override public void createAndUpdateSubscriptionGroupConfig(String addr,
+        SubscriptionGroupConfig config) throws RemotingException, MQBrokerException, InterruptedException, MQClientException {
         this.mqClientInstance.getMQClientAPIImpl().createSubscriptionGroup(addr, config, timeoutMillis);
     }
 
-    @Override
-    public SubscriptionGroupConfig examineSubscriptionGroupConfig(String addr, String group)
-        throws InterruptedException, RemotingException, MQClientException, MQBrokerException {
+    @Override public SubscriptionGroupConfig examineSubscriptionGroupConfig(String addr,
+        String group) throws InterruptedException, RemotingException, MQClientException, MQBrokerException {
         SubscriptionGroupWrapper wrapper = this.mqClientInstance.getMQClientAPIImpl().getAllSubscriptionGroup(addr, timeoutMillis);
         return wrapper.getSubscriptionGroupTable().get(group);
     }
 
-    @Override
-    public TopicConfig examineTopicConfig(String addr,
+    @Override public TopicConfig examineTopicConfig(String addr,
         String topic) throws InterruptedException, MQBrokerException, RemotingTimeoutException, RemotingSendRequestException, RemotingConnectException {
         return this.mqClientInstance.getMQClientAPIImpl().getTopicConfig(addr, topic, timeoutMillis);
     }
 
-    @Override
-    public TopicStatsTable examineTopicStats(
-        String topic) throws RemotingException, MQClientException, InterruptedException,
-        MQBrokerException {
+    @Override public TopicStatsTable examineTopicStats(
+        String topic) throws RemotingException, MQClientException, InterruptedException, MQBrokerException {
         TopicRouteData topicRouteData = this.examineTopicRouteInfo(topic);
         TopicStatsTable topicStatsTable = new TopicStatsTable();
 
@@ -276,51 +312,91 @@ public class DefaultMQAdminExtImpl implements MQAdminExt, MQAdminExtInner {
         return topicStatsTable;
     }
 
-    @Override
-    public TopicStatsTable examineTopicStats(String brokerAddr,
-        String topic) throws RemotingException, MQClientException, InterruptedException,
-        MQBrokerException {
+    @Override public AdminToolResult<TopicStatsTable> examineTopicStatsConcurrent(final String topic) {
+        return adminToolExecute(new AdminToolHandler() {
+            @Override public AdminToolResult doExecute() throws Exception {
+                final TopicStatsTable topicStatsTable = new TopicStatsTable();
+                TopicRouteData topicRouteData = examineTopicRouteInfo(topic);
+
+                if (topicRouteData == null || topicRouteData.getBrokerDatas() == null || topicRouteData.getBrokerDatas().size() == 0) {
+                    return AdminToolResult.success(topicStatsTable);
+                }
+                final CountDownLatch latch = new CountDownLatch(topicRouteData.getBrokerDatas().size());
+                for (final BrokerData bd : topicRouteData.getBrokerDatas()) {
+                    threadPoolExecutor.submit(new Runnable() {
+                        @Override public void run() {
+                            try {
+                                String addr = bd.selectBrokerAddr();
+                                if (addr != null) {
+                                    TopicStatsTable tst = mqClientInstance.getMQClientAPIImpl().getTopicStatsInfo(addr, topic, timeoutMillis);
+                                    topicStatsTable.getOffsetTable().putAll(tst.getOffsetTable());
+                                }
+                            } catch (Exception e) {
+                                log.error("getTopicStatsInfo error. topic=" + topic, e);
+                            } finally {
+                                latch.countDown();
+                            }
+                        }
+                    });
+                }
+                latch.await(timeoutMillis, TimeUnit.MILLISECONDS);
+
+                return AdminToolResult.success(topicStatsTable);
+            }
+        });
+    }
+
+    @Override public TopicStatsTable examineTopicStats(String brokerAddr,
+        String topic) throws RemotingException, MQClientException, InterruptedException, MQBrokerException {
         return this.mqClientInstance.getMQClientAPIImpl().getTopicStatsInfo(brokerAddr, topic, timeoutMillis);
     }
 
-    @Override
-    public TopicList fetchAllTopicList() throws RemotingException, MQClientException, InterruptedException {
+    @Override public TopicList fetchAllTopicList() throws RemotingException, MQClientException, InterruptedException {
         return this.mqClientInstance.getMQClientAPIImpl().getTopicListFromNameServer(timeoutMillis);
     }
 
-    @Override
-    public TopicList fetchTopicsByCLuster(
+    @Override public TopicList fetchTopicsByCLuster(
         String clusterName) throws RemotingException, MQClientException, InterruptedException {
         return this.mqClientInstance.getMQClientAPIImpl().getTopicsByCluster(clusterName, timeoutMillis);
     }
 
-    @Override
-    public KVTable fetchBrokerRuntimeStats(
-        final String brokerAddr) throws RemotingConnectException, RemotingSendRequestException,
-        RemotingTimeoutException, InterruptedException, MQBrokerException {
+    @Override public KVTable fetchBrokerRuntimeStats(
+        final String brokerAddr) throws RemotingConnectException, RemotingSendRequestException, RemotingTimeoutException, InterruptedException, MQBrokerException {
         return this.mqClientInstance.getMQClientAPIImpl().getBrokerRuntimeInfo(brokerAddr, timeoutMillis);
     }
 
-    @Override
-    public ConsumeStats examineConsumeStats(
-        String consumerGroup) throws RemotingException, MQClientException, InterruptedException,
-        MQBrokerException {
+    @Override public ConsumeStats examineConsumeStats(
+        String consumerGroup) throws RemotingException, MQClientException, InterruptedException, MQBrokerException {
         return examineConsumeStats(consumerGroup, null);
     }
 
-    @Override
-    public ConsumeStats examineConsumeStats(String consumerGroup,
-        String topic) throws RemotingException, MQClientException,
-        InterruptedException, MQBrokerException {
-        String retryTopic = MixAll.getRetryTopic(consumerGroup);
-        TopicRouteData topicRouteData = this.examineTopicRouteInfo(retryTopic);
+    @Override public ConsumeStats examineConsumeStats(String consumerGroup,
+        String topic) throws RemotingException, MQClientException, InterruptedException, MQBrokerException {
+        TopicRouteData topicRouteData = null;
+        List<String> routeTopics = new ArrayList<>();
+        routeTopics.add(MixAll.getRetryTopic(consumerGroup));
+        if (topic != null) {
+            routeTopics.add(topic);
+            routeTopics.add(KeyBuilder.buildPopRetryTopic(topic, consumerGroup));
+        }
+        for (int i = 0; i < routeTopics.size(); i++) {
+            try {
+                topicRouteData = this.examineTopicRouteInfo(routeTopics.get(i));
+                if (topicRouteData != null) {
+                    break;
+                }
+            } catch (Throwable e) {
+                if (i == routeTopics.size() - 1) {
+                    throw e;
+                }
+            }
+        }
         ConsumeStats result = new ConsumeStats();
 
         for (BrokerData bd : topicRouteData.getBrokerDatas()) {
             String addr = bd.selectBrokerAddr();
             if (addr != null) {
-                ConsumeStats consumeStats =
-                    this.mqClientInstance.getMQClientAPIImpl().getConsumeStats(addr, consumerGroup, topic, timeoutMillis * 3);
+                ConsumeStats consumeStats = this.mqClientInstance.getMQClientAPIImpl().getConsumeStats(addr, consumerGroup, topic, timeoutMillis * 3);
                 result.getOffsetTable().putAll(consumeStats.getOffsetTable());
                 double value = result.getConsumeTps() + consumeStats.getConsumeTps();
                 result.setConsumeTps(value);
@@ -328,7 +404,7 @@ public class DefaultMQAdminExtImpl implements MQAdminExt, MQAdminExtInner {
         }
 
         Set<String> topics = new HashSet<>();
-        for (MessageQueue messageQueue: result.getOffsetTable().keySet()) {
+        for (MessageQueue messageQueue : result.getOffsetTable().keySet()) {
             topics.add(messageQueue.getTopic());
         }
 
@@ -337,7 +413,7 @@ public class DefaultMQAdminExtImpl implements MQAdminExt, MQAdminExtInner {
         // for topic, we put the physical stats, how about group?
         // staticResult.getOffsetTable().putAll(result.getOffsetTable());
 
-        for (String currentTopic: topics) {
+        for (String currentTopic : topics) {
             TopicRouteData currentRoute = this.examineTopicRouteInfo(currentTopic);
             Map<String, TopicConfigAndQueueMapping> brokerConfigMap = MQAdminUtils.examineTopicConfigFromRoute(currentTopic, currentRoute, defaultMQAdminExt);
             ConsumeStats consumeStats = MQAdminUtils.convertPhysicalConsumeStats(brokerConfigMap, result);
@@ -345,27 +421,84 @@ public class DefaultMQAdminExtImpl implements MQAdminExt, MQAdminExtInner {
         }
 
         if (staticResult.getOffsetTable().isEmpty()) {
-            throw new MQClientException(ResponseCode.CONSUMER_NOT_ONLINE,
-                "Not found the consumer group consume stats, because return offset table is empty, maybe the consumer not consume any message");
+            throw new MQClientException(ResponseCode.CONSUMER_NOT_ONLINE, "Not found the consumer group consume stats, because return offset table is empty, maybe the consumer not consume any message");
         }
 
         return staticResult;
     }
 
     @Override
-    public ClusterInfo examineBrokerClusterInfo() throws InterruptedException, MQBrokerException, RemotingTimeoutException,
-        RemotingSendRequestException, RemotingConnectException {
-        return this.mqClientInstance.getMQClientAPIImpl().getBrokerClusterInfo(timeoutMillis);
+    public AdminToolResult<ConsumeStats> examineConsumeStatsConcurrent(final String consumerGroup, final String topic) {
+
+        return adminToolExecute(new AdminToolHandler() {
+            @Override public AdminToolResult doExecute() throws Exception {
+                TopicRouteData topicRouteData = null;
+                List<String> routeTopics = new ArrayList<>();
+                routeTopics.add(MixAll.getRetryTopic(consumerGroup));
+                if (topic != null) {
+                    routeTopics.add(topic);
+                    routeTopics.add(KeyBuilder.buildPopRetryTopic(topic, consumerGroup));
+                }
+                for (int i = 0; i < routeTopics.size(); i++) {
+                    try {
+                        topicRouteData = examineTopicRouteInfo(routeTopics.get(i));
+                        if (topicRouteData != null) {
+                            break;
+                        }
+                    } catch (Throwable e) {
+                        continue;
+                    }
+                }
+                if (topicRouteData == null || topicRouteData.getBrokerDatas() == null || topicRouteData.getBrokerDatas().size() == 0) {
+                    return AdminToolResult.failure(AdminToolsResultCodeEnum.TOPIC_ROUTE_INFO_NOT_EXIST, "topic router info not found");
+                }
+
+                final ConsumeStats result = new ConsumeStats();
+                final CountDownLatch latch = new CountDownLatch(topicRouteData.getBrokerDatas().size());
+                final Map<String, Double> consumerTpsMap = new ConcurrentHashMap<>(topicRouteData.getBrokerDatas().size());
+                for (final BrokerData bd : topicRouteData.getBrokerDatas()) {
+                    threadPoolExecutor.submit(new Runnable() {
+                        @Override public void run() {
+                            try {
+                                String addr = bd.selectBrokerAddr();
+                                if (addr != null) {
+                                    ConsumeStats consumeStats = mqClientInstance.getMQClientAPIImpl().getConsumeStats(addr, consumerGroup, topic, timeoutMillis);
+                                    result.getOffsetTable().putAll(consumeStats.getOffsetTable());
+                                    consumerTpsMap.put(addr, consumeStats.getConsumeTps());
+                                }
+                            } catch (Exception e) {
+                                log.error("getTopicStatsInfo error. topic=" + topic, e);
+                            } finally {
+                                latch.countDown();
+                            }
+                        }
+                    });
+                }
+                latch.await(timeoutMillis, TimeUnit.MILLISECONDS);
+
+                for (Double tps : consumerTpsMap.values()) {
+                    result.setConsumeTps(result.getConsumeTps() + tps);
+                }
+
+                if (result.getOffsetTable().isEmpty()) {
+                    AdminToolResult.failure(AdminToolsResultCodeEnum.CONSUMER_NOT_ONLINE, "Not found the consumer group consume stats, because return offset table is empty, maybe the consumer not consume any message");
+                }
+                return AdminToolResult.success(result);
+            }
+        });
     }
 
     @Override
-    public TopicRouteData examineTopicRouteInfo(
+    public ClusterInfo examineBrokerClusterInfo() throws InterruptedException, MQBrokerException, RemotingTimeoutException, RemotingSendRequestException, RemotingConnectException {
+        return this.mqClientInstance.getMQClientAPIImpl().getBrokerClusterInfo(timeoutMillis);
+    }
+
+    @Override public TopicRouteData examineTopicRouteInfo(
         String topic) throws RemotingException, MQClientException, InterruptedException {
         return this.mqClientInstance.getMQClientAPIImpl().getTopicRouteInfoFromNameServer(topic, timeoutMillis);
     }
 
-    @Override
-    public MessageExt viewMessage(String topic,
+    @Override public MessageExt viewMessage(String topic,
         String msgId) throws RemotingException, MQBrokerException, InterruptedException, MQClientException {
         try {
             MessageDecoder.decodeMessageId(msgId);
@@ -376,8 +509,18 @@ public class DefaultMQAdminExtImpl implements MQAdminExt, MQAdminExtInner {
         return this.mqClientInstance.getMQAdminImpl().queryMessageByUniqKey(topic, msgId);
     }
 
-    @Override
-    public ConsumerConnection examineConsumerConnectionInfo(
+    @Override public MessageExt queryMessage(String clusterName, String topic,
+        String msgId) throws RemotingException, MQBrokerException, InterruptedException, MQClientException {
+        try {
+            MessageDecoder.decodeMessageId(msgId);
+            return this.viewMessage(msgId);
+        } catch (Exception e) {
+            log.warn("the msgId maybe created by new client. msgId={}", msgId, e);
+        }
+        return this.mqClientInstance.getMQAdminImpl().queryMessageByUniqKey(clusterName, topic, msgId);
+    }
+
+    @Override public ConsumerConnection examineConsumerConnectionInfo(
         String consumerGroup) throws InterruptedException, MQBrokerException,
         RemotingException, MQClientException {
         ConsumerConnection result = new ConsumerConnection();
@@ -415,10 +558,8 @@ public class DefaultMQAdminExtImpl implements MQAdminExt, MQAdminExtInner {
         return result;
     }
 
-    @Override
-    public ProducerConnection examineProducerConnectionInfo(String producerGroup,
-        final String topic) throws RemotingException,
-        MQClientException, InterruptedException, MQBrokerException {
+    @Override public ProducerConnection examineProducerConnectionInfo(String producerGroup,
+        final String topic) throws RemotingException, MQClientException, InterruptedException, MQBrokerException {
         ProducerConnection result = new ProducerConnection();
         List<BrokerData> brokers = this.examineTopicRouteInfo(topic).getBrokerDatas();
         BrokerData brokerData = brokers.get(random.nextInt(brokers.size()));
@@ -438,14 +579,12 @@ public class DefaultMQAdminExtImpl implements MQAdminExt, MQAdminExtInner {
         return result;
     }
 
-    @Override
-    public List<String> getNameServerAddressList() {
+    @Override public List<String> getNameServerAddressList() {
         return this.mqClientInstance.getMQClientAPIImpl().getNameServerAddressList();
     }
 
-    @Override
-    public int wipeWritePermOfBroker(final String namesrvAddr, String brokerName) throws RemotingCommandException,
-        RemotingConnectException, RemotingSendRequestException, RemotingTimeoutException, InterruptedException, MQClientException {
+    @Override public int wipeWritePermOfBroker(final String namesrvAddr,
+        String brokerName) throws RemotingCommandException, RemotingConnectException, RemotingSendRequestException, RemotingTimeoutException, InterruptedException, MQClientException {
         return this.mqClientInstance.getMQClientAPIImpl().wipeWritePermOfBroker(namesrvAddr, brokerName, timeoutMillis);
     }
 
@@ -455,35 +594,71 @@ public class DefaultMQAdminExtImpl implements MQAdminExt, MQAdminExtInner {
         return this.mqClientInstance.getMQClientAPIImpl().addWritePermOfBroker(namesrvAddr, brokerName, timeoutMillis);
     }
 
-    @Override
-    public void putKVConfig(String namespace, String key, String value) {
+    @Override public void putKVConfig(String namespace, String key, String value) {
     }
 
-    @Override
-    public String getKVConfig(String namespace,
+    @Override public String getKVConfig(String namespace,
         String key) throws RemotingException, MQClientException, InterruptedException {
         return this.mqClientInstance.getMQClientAPIImpl().getKVConfigValue(namespace, key, timeoutMillis);
     }
 
-    @Override
-    public KVTable getKVListByNamespace(
+    @Override public KVTable getKVListByNamespace(
         String namespace) throws RemotingException, MQClientException, InterruptedException {
         return this.mqClientInstance.getMQClientAPIImpl().getKVListByNamespace(namespace, timeoutMillis);
     }
 
-    @Override
-    public void deleteTopicInBroker(Set<String> addrs,
-        String topic) throws RemotingException, MQBrokerException, InterruptedException,
-        MQClientException {
+    @Override public void deleteTopic(String topicName,
+        String clusterName) throws RemotingException, MQBrokerException, InterruptedException, MQClientException {
+        Set<String> brokerAddressSet = CommandUtil.fetchMasterAndSlaveAddrByClusterName(this.defaultMQAdminExt, clusterName);
+        this.deleteTopicInBroker(brokerAddressSet, topicName);
+        List<String> nameServerList = this.getNameServerAddressList();
+        Set<String> nameServerSet = new HashSet<String>(nameServerList);
+        this.deleteTopicInNameServer(nameServerSet, topicName);
+        for (String namespace : this.kvNamespaceToDeleteList) {
+            this.deleteKvConfig(namespace, topicName);
+        }
+    }
+
+    @Override public void deleteTopicInBroker(Set<String> addrs,
+        String topic) throws RemotingException, MQBrokerException, InterruptedException, MQClientException {
         for (String addr : addrs) {
             this.mqClientInstance.getMQClientAPIImpl().deleteTopicInBroker(addr, topic, timeoutMillis);
         }
     }
 
-    @Override
-    public void deleteTopicInNameServer(Set<String> addrs,
-        String topic) throws RemotingException, MQBrokerException, InterruptedException,
-        MQClientException {
+    @Override public AdminToolResult<BrokerOperatorResult> deleteTopicInBrokerConcurrent(final Set<String> addrs,
+        final String topic) {
+        final List<String> successList = new CopyOnWriteArrayList<>();
+        final List<String> failureList = new CopyOnWriteArrayList<>();
+        final CountDownLatch latch = new CountDownLatch(addrs.size());
+        for (final String addr : addrs) {
+            threadPoolExecutor.submit(new Runnable() {
+                @Override public void run() {
+                    try {
+                        mqClientInstance.getMQClientAPIImpl().deleteTopicInBroker(addr, topic, timeoutMillis);
+                        successList.add(addr);
+                    } catch (Exception e) {
+                        log.error("deleteTopicInBrokerConcurrent error. topic=" + topic + ", host=" + addr, e);
+                        failureList.add(addr);
+                    } finally {
+                        latch.countDown();
+                    }
+                }
+            });
+        }
+        try {
+            latch.await(timeoutMillis, TimeUnit.MILLISECONDS);
+        } catch (Exception e) {
+        }
+
+        BrokerOperatorResult result = new BrokerOperatorResult();
+        result.setSuccessList(successList);
+        result.setFailureList(failureList);
+        return AdminToolResult.success(result);
+    }
+
+    @Override public void deleteTopicInNameServer(Set<String> addrs,
+        String topic) throws RemotingException, MQBrokerException, InterruptedException, MQClientException {
         if (addrs == null) {
             String ns = this.mqClientInstance.getMQClientAPIImpl().fetchNameServerAddr();
             addrs = new HashSet(Arrays.asList(ns.split(";")));
@@ -493,92 +668,82 @@ public class DefaultMQAdminExtImpl implements MQAdminExt, MQAdminExtInner {
         }
     }
 
-    @Override
-    public void deleteSubscriptionGroup(String addr,
-        String groupName) throws RemotingException, MQBrokerException, InterruptedException,
-        MQClientException {
+    @Override public void deleteSubscriptionGroup(String addr,
+        String groupName) throws RemotingException, MQBrokerException, InterruptedException, MQClientException {
         this.mqClientInstance.getMQClientAPIImpl().deleteSubscriptionGroup(addr, groupName, false, timeoutMillis);
     }
 
-    @Override
-    public void deleteSubscriptionGroup(String addr,
-        String groupName, boolean removeOffset) throws RemotingException, MQBrokerException, InterruptedException,
-        MQClientException {
+    @Override public void deleteSubscriptionGroup(String addr, String groupName,
+        boolean removeOffset) throws RemotingException, MQBrokerException, InterruptedException, MQClientException {
         this.mqClientInstance.getMQClientAPIImpl().deleteSubscriptionGroup(addr, groupName, removeOffset, timeoutMillis);
     }
 
-    @Override
-    public void createAndUpdateKvConfig(String namespace, String key,
-        String value) throws RemotingException, MQBrokerException,
-        InterruptedException, MQClientException {
+    @Override public void createAndUpdateKvConfig(String namespace, String key,
+        String value) throws RemotingException, MQBrokerException, InterruptedException, MQClientException {
         this.mqClientInstance.getMQClientAPIImpl().putKVConfigValue(namespace, key, value, timeoutMillis);
     }
 
-    @Override
-    public void deleteKvConfig(String namespace,
-        String key) throws RemotingException, MQBrokerException, InterruptedException,
-        MQClientException {
+    @Override public void deleteKvConfig(String namespace,
+        String key) throws RemotingException, MQBrokerException, InterruptedException, MQClientException {
         this.mqClientInstance.getMQClientAPIImpl().deleteKVConfigValue(namespace, key, timeoutMillis);
     }
 
-    @Override
-    public List<RollbackStats> resetOffsetByTimestampOld(String consumerGroup, String topic, long timestamp,
-        boolean force)
-        throws RemotingException, MQBrokerException, InterruptedException, MQClientException {
+    @Override public List<RollbackStats> resetOffsetByTimestampOld(String consumerGroup, String topic, long timestamp,
+        boolean force) throws RemotingException, MQBrokerException, InterruptedException, MQClientException {
         TopicRouteData topicRouteData = this.examineTopicRouteInfo(topic);
         List<RollbackStats> rollbackStatsList = new ArrayList<RollbackStats>();
-        Map<String, Integer> topicRouteMap = new HashMap<String, Integer>();
-        for (BrokerData bd : topicRouteData.getBrokerDatas()) {
-            for (QueueData queueData : topicRouteData.getQueueDatas()) {
-                if (StringUtils.equals(queueData.getBrokerName(), bd.getBrokerName())) {
-                    topicRouteMap.put(bd.selectBrokerAddr(), queueData.getReadQueueNums());
-                }
-            }
+        Map<String, QueueData> topicRouteMap = new HashMap<String, QueueData>();
+        for (QueueData queueData : topicRouteData.getQueueDatas()) {
+            topicRouteMap.put(queueData.getBrokerName(), queueData);
         }
         for (BrokerData bd : topicRouteData.getBrokerDatas()) {
             String addr = bd.selectBrokerAddr();
             if (addr != null) {
-                ConsumeStats consumeStats = this.mqClientInstance.getMQClientAPIImpl().getConsumeStats(addr, consumerGroup, timeoutMillis);
-
-                boolean hasConsumed = false;
-                for (Map.Entry<MessageQueue, OffsetWrapper> entry : consumeStats.getOffsetTable().entrySet()) {
-                    MessageQueue queue = entry.getKey();
-                    OffsetWrapper offsetWrapper = entry.getValue();
-                    if (topic.equals(queue.getTopic())) {
-                        hasConsumed = true;
-                        RollbackStats rollbackStats = resetOffsetConsumeOffset(addr, consumerGroup, queue, offsetWrapper, timestamp, force);
-                        rollbackStatsList.add(rollbackStats);
-                    }
-                }
+                rollbackStatsList.addAll(resetOffsetByTimestampOld(addr, topicRouteMap.get(bd.getBrokerName()), consumerGroup, topic, timestamp, force));
+            }
+        }
+        return rollbackStatsList;
+    }
 
-                if (!hasConsumed) {
-                    HashMap<MessageQueue, TopicOffset> topicStatus =
-                        this.mqClientInstance.getMQClientAPIImpl().getTopicStatsInfo(addr, topic, timeoutMillis).getOffsetTable();
-                    for (int i = 0; i < topicRouteMap.get(addr); i++) {
-                        MessageQueue queue = new MessageQueue(topic, bd.getBrokerName(), i);
-                        OffsetWrapper offsetWrapper = new OffsetWrapper();
-                        offsetWrapper.setBrokerOffset(topicStatus.get(queue).getMaxOffset());
-                        offsetWrapper.setConsumerOffset(topicStatus.get(queue).getMinOffset());
-
-                        RollbackStats rollbackStats = resetOffsetConsumeOffset(addr, consumerGroup, queue, offsetWrapper, timestamp, force);
-                        rollbackStatsList.add(rollbackStats);
-                    }
-                }
+    private List<RollbackStats> resetOffsetByTimestampOld(String brokerAddr, QueueData queueData, String consumerGroup,
+        String topic, long timestamp,
+        boolean force) throws RemotingException, MQBrokerException, InterruptedException, MQClientException {
+        List<RollbackStats> rollbackStatsList = new ArrayList<RollbackStats>();
+        ConsumeStats consumeStats = this.mqClientInstance.getMQClientAPIImpl().getConsumeStats(brokerAddr, consumerGroup, timeoutMillis);
+
+        boolean hasConsumed = false;
+        for (Map.Entry<MessageQueue, OffsetWrapper> entry : consumeStats.getOffsetTable().entrySet()) {
+            MessageQueue queue = entry.getKey();
+            OffsetWrapper offsetWrapper = entry.getValue();
+            if (topic.equals(queue.getTopic())) {
+                hasConsumed = true;
+                RollbackStats rollbackStats = resetOffsetConsumeOffset(brokerAddr, consumerGroup, queue, offsetWrapper, timestamp, force);
+                rollbackStatsList.add(rollbackStats);
+            }
+        }
+
+        if (!hasConsumed) {
+            HashMap<MessageQueue, TopicOffset> topicStatus = this.mqClientInstance.getMQClientAPIImpl().getTopicStatsInfo(brokerAddr, topic, timeoutMillis).getOffsetTable();
+            for (int i = 0; i < queueData.getReadQueueNums(); i++) {
+                MessageQueue queue = new MessageQueue(topic, queueData.getBrokerName(), i);
+                OffsetWrapper offsetWrapper = new OffsetWrapper();
+                offsetWrapper.setBrokerOffset(topicStatus.get(queue).getMaxOffset());
+                offsetWrapper.setConsumerOffset(topicStatus.get(queue).getMinOffset());
+
+                RollbackStats rollbackStats = resetOffsetConsumeOffset(brokerAddr, consumerGroup, queue, offsetWrapper, timestamp, force);
+                rollbackStatsList.add(rollbackStats);
             }
         }
         return rollbackStatsList;
     }
 
-    @Override
-    public Map<MessageQueue, Long> resetOffsetByTimestamp(String topic, String group, long timestamp, boolean isForce)
-        throws RemotingException, MQBrokerException, InterruptedException, MQClientException {
+    @Override public Map<MessageQueue, Long> resetOffsetByTimestamp(String topic, String group, long timestamp,
+        boolean isForce) throws RemotingException, MQBrokerException, InterruptedException, MQClientException {
         return resetOffsetByTimestamp(topic, group, timestamp, isForce, false);
     }
 
-    @Override
-    public void resetOffsetNew(String consumerGroup, String topic,
-        long timestamp) throws RemotingException, MQBrokerException,
-        InterruptedException, MQClientException {
+    @Override public void resetOffsetNew(String consumerGroup, String topic,
+        long timestamp) throws RemotingException, MQBrokerException, InterruptedException, MQClientException {
         try {
             this.resetOffsetByTimestamp(topic, consumerGroup, timestamp, true);
         } catch (MQClientException e) {
@@ -590,9 +755,76 @@ public class DefaultMQAdminExtImpl implements MQAdminExt, MQAdminExtInner {
         }
     }
 
+    @Override
+    public AdminToolResult<BrokerOperatorResult> resetOffsetNewConcurrent(final String group, final String topic,
+        final long timestamp) {
+        return adminToolExecute(new AdminToolHandler() {
+            @Override public AdminToolResult doExecute() throws Exception {
+                TopicRouteData topicRouteData = examineTopicRouteInfo(topic);
+                if (topicRouteData == null || topicRouteData.getBrokerDatas() == null || topicRouteData.getBrokerDatas().size() == 0) {
+                    return AdminToolResult.failure(AdminToolsResultCodeEnum.TOPIC_ROUTE_INFO_NOT_EXIST, "topic router info not found");
+                }
+                final Map<String, QueueData> topicRouteMap = new HashMap<String, QueueData>();
+                for (QueueData queueData : topicRouteData.getQueueDatas()) {
+                    topicRouteMap.put(queueData.getBrokerName(), queueData);
+                }
+
+                final CopyOnWriteArrayList successList = new CopyOnWriteArrayList();
+                final CopyOnWriteArrayList failureList = new CopyOnWriteArrayList();
+                final CountDownLatch latch = new CountDownLatch(topicRouteData.getBrokerDatas().size());
+                for (final BrokerData bd : topicRouteData.getBrokerDatas()) {
+                    threadPoolExecutor.submit(new Runnable() {
+                        @Override public void run() {
+                            String addr = bd.selectBrokerAddr();
+                            try {
+                                if (addr != null) {
+                                    Map<MessageQueue, Long> offsetTable = mqClientInstance.getMQClientAPIImpl().invokeBrokerToResetOffset(addr, topic, group, timestamp, true, timeoutMillis, false);
+                                    if (offsetTable != null) {
+                                        successList.add(addr);
+                                    } else {
+                                        failureList.add(addr);
+                                    }
+                                }
+                            } catch (MQClientException e) {
+                                if (ResponseCode.CONSUMER_NOT_ONLINE == e.getResponseCode()) {
+                                    try {
+                                        resetOffsetByTimestampOld(addr, topicRouteMap.get(bd.getBrokerName()), group, topic, timestamp, true);
+                                        successList.add(addr);
+                                    } catch (Exception e2) {
+                                        log.error(MessageFormat.format("resetOffsetByTimestampOld error. addr={0}, topic={1}, group={2},timestamp={3}", addr, topic, group, timestamp), e);
+                                        failureList.add(addr);
+                                    }
+                                } else if (ResponseCode.SYSTEM_ERROR == e.getResponseCode()) {
+                                    // CODE: 1  DESC: THe consumer group <GID_newggghh> not exist, never online
+                                    successList.add(addr);
+                                } else {
+                                    failureList.add(addr);
+                                    log.error(MessageFormat.format("resetOffsetNewConcurrent error. addr={0}, topic={1}, group={2},timestamp={3}", addr, topic, group, timestamp), e);
+                                }
+                            } catch (Exception e) {
+                                failureList.add(addr);
+                                log.error(MessageFormat.format("resetOffsetNewConcurrent error. addr={0}, topic={1}, group={2},timestamp={3}", addr, topic, group, timestamp), e);
+                            } finally {
+                                latch.countDown();
+                            }
+                        }
+                    });
+                }
+                latch.await(timeoutMillis, TimeUnit.MILLISECONDS);
+                BrokerOperatorResult result = new BrokerOperatorResult();
+                result.setSuccessList(successList);
+                result.setFailureList(failureList);
+                if (successList.size() == topicRouteData.getBrokerDatas().size()) {
+                    return AdminToolResult.success(result);
+                } else {
+                    return AdminToolResult.failure(AdminToolsResultCodeEnum.MQ_BROKER_ERROR, "operator failure", result);
+                }
+            }
+        });
+    }
+
     public Map<MessageQueue, Long> resetOffsetByTimestamp(String topic, String group, long timestamp, boolean isForce,
-        boolean isC)
-        throws RemotingException, MQBrokerException, InterruptedException, MQClientException {
+        boolean isC) throws RemotingException, MQBrokerException, InterruptedException, MQClientException {
         TopicRouteData topicRouteData = this.examineTopicRouteInfo(topic);
         List<BrokerData> brokerDatas = topicRouteData.getBrokerDatas();
         Map<MessageQueue, Long> allOffsetTable = new HashMap<MessageQueue, Long>();
@@ -600,9 +832,7 @@ public class DefaultMQAdminExtImpl implements MQAdminExt, MQAdminExtInner {
             for (BrokerData brokerData : brokerDatas) {
                 String addr = brokerData.selectBrokerAddr();
                 if (addr != null) {
-                    Map<MessageQueue, Long> offsetTable =
-                        this.mqClientInstance.getMQClientAPIImpl().invokeBrokerToResetOffset(addr, topic, group, timestamp, isForce,
-                            timeoutMillis, isC);
+                    Map<MessageQueue, Long> offsetTable = this.mqClientInstance.getMQClientAPIImpl().invokeBrokerToResetOffset(addr, topic, group, timestamp, isForce, timeoutMillis, isC);
                     if (offsetTable != null) {
                         allOffsetTable.putAll(offsetTable);
                     }
@@ -613,16 +843,14 @@ public class DefaultMQAdminExtImpl implements MQAdminExt, MQAdminExtInner {
     }
 
     private RollbackStats resetOffsetConsumeOffset(String brokerAddr, String consumeGroup, MessageQueue queue,
-        OffsetWrapper offsetWrapper,
-        long timestamp, boolean force) throws RemotingException, InterruptedException, MQBrokerException {
+        OffsetWrapper offsetWrapper, long timestamp,
+        boolean force) throws RemotingException, InterruptedException, MQBrokerException {
         long resetOffset;
         if (timestamp == -1) {
 
             resetOffset = this.mqClientInstance.getMQClientAPIImpl().getMaxOffset(brokerAddr, queue.getTopic(), queue.getQueueId(), timeoutMillis);
         } else {
-            resetOffset =
-                this.mqClientInstance.getMQClientAPIImpl().searchOffset(brokerAddr, queue.getTopic(), queue.getQueueId(), timestamp,
-                    timeoutMillis);
+            resetOffset = this.mqClientInstance.getMQClientAPIImpl().searchOffset(brokerAddr, queue.getTopic(), queue.getQueueId(), timestamp, timeoutMillis);
         }
 
         RollbackStats rollbackStats = new RollbackStats();
@@ -645,36 +873,28 @@ public class DefaultMQAdminExtImpl implements MQAdminExt, MQAdminExtInner {
         return rollbackStats;
     }
 
-    @Override
-    public Map<String, Map<MessageQueue, Long>> getConsumeStatus(String topic, String group,
-        String clientAddr) throws RemotingException,
-        MQBrokerException, InterruptedException, MQClientException {
+    @Override public Map<String, Map<MessageQueue, Long>> getConsumeStatus(String topic, String group,
+        String clientAddr) throws RemotingException, MQBrokerException, InterruptedException, MQClientException {
         TopicRouteData topicRouteData = this.examineTopicRouteInfo(topic);
         List<BrokerData> brokerDatas = topicRouteData.getBrokerDatas();
         if (brokerDatas != null && brokerDatas.size() > 0) {
             String addr = brokerDatas.get(0).selectBrokerAddr();
             if (addr != null) {
-                return this.mqClientInstance.getMQClientAPIImpl().invokeBrokerToGetConsumerStatus(addr, topic, group, clientAddr,
-                    timeoutMillis);
+                return this.mqClientInstance.getMQClientAPIImpl().invokeBrokerToGetConsumerStatus(addr, topic, group, clientAddr, timeoutMillis);
             }
         }
         return Collections.EMPTY_MAP;
     }
 
-    @Override
-    public void createOrUpdateOrderConf(String key, String value,
-        boolean isCluster) throws RemotingException, MQBrokerException,
-        InterruptedException, MQClientException {
+    @Override public void createOrUpdateOrderConf(String key, String value,
+        boolean isCluster) throws RemotingException, MQBrokerException, InterruptedException, MQClientException {
 
         if (isCluster) {
-            this.mqClientInstance.getMQClientAPIImpl()
-                .putKVConfigValue(NamesrvUtil.NAMESPACE_ORDER_TOPIC_CONFIG, key, value, timeoutMillis);
+            this.mqClientInstance.getMQClientAPIImpl().putKVConfigValue(NamesrvUtil.NAMESPACE_ORDER_TOPIC_CONFIG, key, value, timeoutMillis);
         } else {
             String oldOrderConfs = null;
             try {
-                oldOrderConfs =
-                    this.mqClientInstance.getMQClientAPIImpl().getKVConfigValue(NamesrvUtil.NAMESPACE_ORDER_TOPIC_CONFIG, key,
-                        timeoutMillis);
+                oldOrderConfs = this.mqClientInstance.getMQClientAPIImpl().getKVConfigValue(NamesrvUtil.NAMESPACE_ORDER_TOPIC_CONFIG, key, timeoutMillis);
             } catch (Exception e) {
                 e.printStackTrace();
             }
@@ -696,15 +916,12 @@ public class DefaultMQAdminExtImpl implements MQAdminExt, MQAdminExtInner {
                 newOrderConf.append(splitor).append(entry.getValue());
                 splitor = ";";
             }
-            this.mqClientInstance.getMQClientAPIImpl().putKVConfigValue(NamesrvUtil.NAMESPACE_ORDER_TOPIC_CONFIG, key,
-                newOrderConf.toString(), timeoutMillis);
+            this.mqClientInstance.getMQClientAPIImpl().putKVConfigValue(NamesrvUtil.NAMESPACE_ORDER_TOPIC_CONFIG, key, newOrderConf.toString(), timeoutMillis);
         }
     }
 
-    @Override
-    public GroupList queryTopicConsumeByWho(
-        String topic) throws InterruptedException, MQBrokerException, RemotingException,
-        MQClientException {
+    @Override public GroupList queryTopicConsumeByWho(
+        String topic) throws InterruptedException, MQBrokerException, RemotingException, MQClientException {
         TopicRouteData topicRouteData = this.examineTopicRouteInfo(topic);
 
         for (BrokerData bd : topicRouteData.getBrokerDatas()) {
@@ -712,6 +929,19 @@ public class DefaultMQAdminExtImpl implements MQAdminExt, MQAdminExtInner {
             if (addr != null) {
                 return this.mqClientInstance.getMQClientAPIImpl().queryTopicConsumeByWho(addr, topic, timeoutMillis);
             }
+        }
+        return null;
+    }
+
+    @Override public SubscriptionData querySubscription(String group,
+        String topic) throws InterruptedException, MQBrokerException, RemotingException, MQClientException {
+        TopicRouteData topicRouteData = this.examineTopicRouteInfo(topic);
+
+        for (BrokerData bd : topicRouteData.getBrokerDatas()) {
+            String addr = bd.selectBrokerAddr();
+            if (addr != null) {
+                return this.mqClientInstance.getMQClientAPIImpl().querySubscriptionByConsumer(addr, group, topic, timeoutMillis);
+            }
 
             break;
         }
@@ -719,10 +949,61 @@ public class DefaultMQAdminExtImpl implements MQAdminExt, MQAdminExtInner {
         return null;
     }
 
-    @Override
-    public List<QueueTimeSpan> queryConsumeTimeSpan(final String topic,
-        final String group) throws InterruptedException, MQBrokerException,
-        RemotingException, MQClientException {
+    @Override public TopicList queryTopicsByConsumer(
+        String group) throws InterruptedException, MQBrokerException, RemotingException, MQClientException {
+        String retryTopic = MixAll.getRetryTopic(group);
+        TopicRouteData topicRouteData = this.examineTopicRouteInfo(retryTopic);
+        TopicList result = new TopicList();
+
+        //Query all brokers
+        for (BrokerData bd : topicRouteData.getBrokerDatas()) {
+            String addr = bd.selectBrokerAddr();
+            if (addr != null) {
+                TopicList topicList = this.mqClientInstance.getMQClientAPIImpl().queryTopicsByConsumer(addr, group, timeoutMillis);
+                result.getTopicList().addAll(topicList.getTopicList());
+            }
+        }
+
+        return result;
+    }
+
+    @Override public AdminToolResult<TopicList> queryTopicsByConsumerConcurrent(final String group) {
+        return adminToolExecute(new AdminToolHandler() {
+            @Override public AdminToolResult doExecute() throws Exception {
+                String retryTopic = MixAll.getRetryTopic(group);
+                TopicRouteData topicRouteData = examineTopicRouteInfo(retryTopic);
+
+                if (topicRouteData == null || topicRouteData.getBrokerDatas() == null || topicRouteData.getBrokerDatas().size() == 0) {
+                    return AdminToolResult.failure(AdminToolsResultCodeEnum.TOPIC_ROUTE_INFO_NOT_EXIST, "router info not found.");
+                }
+                final TopicList result = new TopicList();
+                final CountDownLatch latch = new CountDownLatch(topicRouteData.getBrokerDatas().size());
+                for (final BrokerData bd : topicRouteData.getBrokerDatas()) {
+                    threadPoolExecutor.submit(new Runnable() {
+                        @Override public void run() {
+                            try {
+                                String addr = bd.selectBrokerAddr();
+                                if (addr != null) {
+                                    TopicList topicList = mqClientInstance.getMQClientAPIImpl().queryTopicsByConsumer(addr, group, timeoutMillis);
+                                    result.getTopicList().addAll(topicList.getTopicList());
+                                }
+                            } catch (Exception e) {
+                                log.error("getTopicStatsInfo error. groupId=" + group, e);
+                            } finally {
+                                latch.countDown();
+                            }
+                        }
+                    });
+                }
+                latch.await(timeoutMillis, TimeUnit.MILLISECONDS);
+
+                return AdminToolResult.success(result);
+            }
+        });
+    }
+
+    @Override public List<QueueTimeSpan> queryConsumeTimeSpan(final String topic,
+        final String group) throws InterruptedException, MQBrokerException, RemotingException, MQClientException {
         List<QueueTimeSpan> spanSet = new ArrayList<QueueTimeSpan>();
         TopicRouteData topicRouteData = this.examineTopicRouteInfo(topic);
         for (BrokerData bd : topicRouteData.getBrokerDatas()) {
@@ -735,9 +1016,41 @@ public class DefaultMQAdminExtImpl implements MQAdminExt, MQAdminExtInner {
     }
 
     @Override
-    public boolean cleanExpiredConsumerQueue(
-        String cluster) throws RemotingConnectException, RemotingSendRequestException,
-        RemotingTimeoutException, MQClientException, InterruptedException {
+    public AdminToolResult<List<QueueTimeSpan>> queryConsumeTimeSpanConcurrent(final String topic, final String group) {
+        return adminToolExecute(new AdminToolHandler() {
+            @Override public AdminToolResult doExecute() throws Exception {
+                final List<QueueTimeSpan> spanSet = new ArrayList<QueueTimeSpan>();
+                TopicRouteData topicRouteData = examineTopicRouteInfo(topic);
+
+                if (topicRouteData == null || topicRouteData.getBrokerDatas() == null || topicRouteData.getBrokerDatas().size() == 0) {
+                    return AdminToolResult.success(spanSet);
+                }
+                final CountDownLatch latch = new CountDownLatch(topicRouteData.getBrokerDatas().size());
+                for (final BrokerData bd : topicRouteData.getBrokerDatas()) {
+                    threadPoolExecutor.submit(new Runnable() {
+                        @Override public void run() {
+                            try {
+                                String addr = bd.selectBrokerAddr();
+                                if (addr != null) {
+                                    spanSet.addAll(mqClientInstance.getMQClientAPIImpl().queryConsumeTimeSpan(addr, topic, group, timeoutMillis));
+                                }
+                            } catch (Exception e) {
+                                log.error("queryConsumeTimeSpan error. topic=" + topic, e);
+                            } finally {
+                                latch.countDown();
+                            }
+                        }
+                    });
+                }
+                latch.await(timeoutMillis, TimeUnit.MILLISECONDS);
+
+                return AdminToolResult.success(spanSet);
+            }
+        });
+    }
+
+    @Override public boolean cleanExpiredConsumerQueue(
+        String cluster) throws RemotingConnectException, RemotingSendRequestException, RemotingTimeoutException, MQClientException, InterruptedException {
         boolean result = false;
         try {
             ClusterInfo clusterInfo = examineBrokerClusterInfo();
@@ -756,8 +1069,7 @@ public class DefaultMQAdminExtImpl implements MQAdminExt, MQAdminExtInner {
     }
 
     public boolean cleanExpiredConsumerQueueByCluster(ClusterInfo clusterInfo,
-        String cluster) throws RemotingConnectException,
-        RemotingSendRequestException, RemotingTimeoutException, MQClientException, InterruptedException {
+        String cluster) throws RemotingConnectException, RemotingSendRequestException, RemotingTimeoutException, MQClientException, InterruptedException {
         boolean result = false;
         String[] addrs = clusterInfo.retrieveAllAddrByCluster(cluster);
         for (String addr : addrs) {
@@ -766,18 +1078,15 @@ public class DefaultMQAdminExtImpl implements MQAdminExt, MQAdminExtInner {
         return result;
     }
 
-    @Override
-    public boolean cleanExpiredConsumerQueueByAddr(
-        String addr) throws RemotingConnectException, RemotingSendRequestException,
-        RemotingTimeoutException, MQClientException, InterruptedException {
+    @Override public boolean cleanExpiredConsumerQueueByAddr(
+        String addr) throws RemotingConnectException, RemotingSendRequestException, RemotingTimeoutException, MQClientException, InterruptedException {
         boolean result = mqClientInstance.getMQClientAPIImpl().cleanExpiredConsumeQueue(addr, timeoutMillis);
         log.warn("clean expired ConsumeQueue on target " + addr + " broker " + result);
         return result;
     }
 
-    @Override
-    public boolean cleanUnusedTopic(String cluster) throws RemotingConnectException, RemotingSendRequestException,
-        RemotingTimeoutException, MQClientException, InterruptedException {
+    @Override public boolean cleanUnusedTopic(
+        String cluster) throws RemotingConnectException, RemotingSendRequestException, RemotingTimeoutException, MQClientException, InterruptedException {
         boolean result = false;
         try {
             ClusterInfo clusterInfo = examineBrokerClusterInfo();
@@ -795,8 +1104,8 @@ public class DefaultMQAdminExtImpl implements MQAdminExt, MQAdminExtInner {
         return result;
     }
 
-    public boolean cleanUnusedTopicByCluster(ClusterInfo clusterInfo, String cluster) throws RemotingConnectException,
-        RemotingSendRequestException, RemotingTimeoutException, MQClientException, InterruptedException {
+    public boolean cleanUnusedTopicByCluster(ClusterInfo clusterInfo,
+        String cluster) throws RemotingConnectException, RemotingSendRequestException, RemotingTimeoutException, MQClientException, InterruptedException {
         boolean result = false;
         String[] addrs = clusterInfo.retrieveAllAddrByCluster(cluster);
         for (String addr : addrs) {
@@ -805,18 +1114,20 @@ public class DefaultMQAdminExtImpl implements MQAdminExt, MQAdminExtInner {
         return result;
     }
 
-    @Override
-    public boolean cleanUnusedTopicByAddr(String addr) throws RemotingConnectException, RemotingSendRequestException,
-        RemotingTimeoutException, MQClientException, InterruptedException {
+    @Override public boolean cleanUnusedTopicByAddr(
+        String addr) throws RemotingConnectException, RemotingSendRequestException, RemotingTimeoutException, MQClientException, InterruptedException {
         boolean result = mqClientInstance.getMQClientAPIImpl().cleanUnusedTopicByAddr(addr, timeoutMillis);
         log.warn("clean expired ConsumeQueue on target " + addr + " broker " + result);
         return result;
     }
 
-    @Override
-    public ConsumerRunningInfo getConsumerRunningInfo(String consumerGroup, String clientId,
-        boolean jstack) throws RemotingException,
-        MQClientException, InterruptedException {
+    @Override public ConsumerRunningInfo getConsumerRunningInfo(String consumerGroup, String clientId,
+        boolean jstack) throws RemotingException, MQClientException, InterruptedException {
+        return this.getConsumerRunningInfo(consumerGroup, clientId, jstack, false);
+    }
+
+    @Override public ConsumerRunningInfo getConsumerRunningInfo(String consumerGroup, String clientId, boolean jstack,
+        boolean metrics) throws RemotingException, MQClientException, InterruptedException {
         String topic = MixAll.RETRY_GROUP_TOPIC_PREFIX + consumerGroup;
         TopicRouteData topicRouteData = this.examineTopicRouteInfo(topic);
         List<BrokerData> brokerDatas = topicRouteData.getBrokerDatas();
@@ -824,21 +1135,18 @@ public class DefaultMQAdminExtImpl implements MQAdminExt, MQAdminExtInner {
             for (BrokerData brokerData : brokerDatas) {
                 String addr = brokerData.selectBrokerAddr();
                 if (addr != null) {
-                    return this.mqClientInstance.getMQClientAPIImpl().getConsumerRunningInfo(addr, consumerGroup, clientId, jstack,
-                        timeoutMillis * 3);
+                    return this.mqClientInstance.getMQClientAPIImpl().getConsumerRunningInfo(addr, consumerGroup, clientId, jstack, timeoutMillis);
                 }
             }
         }
         return null;
     }
 
-    @Override
-    public ConsumeMessageDirectlyResult consumeMessageDirectly(String consumerGroup, String clientId, String msgId)
-        throws RemotingException, MQClientException, InterruptedException, MQBrokerException {
+    @Override public ConsumeMessageDirectlyResult consumeMessageDirectly(String consumerGroup, String clientId,
+        String msgId) throws RemotingException, MQClientException, InterruptedException, MQBrokerException {
         MessageExt msg = this.viewMessage(msgId);
 
-        return this.mqClientInstance.getMQClientAPIImpl().consumeMessageDirectly(RemotingUtil.socketAddress2String(msg.getStoreHost()),
-            consumerGroup, clientId, msgId, timeoutMillis * 3);
+        return this.mqClientInstance.getMQClientAPIImpl().consumeMessageDirectly(RemotingUtil.socketAddress2String(msg.getStoreHost()), consumerGroup, clientId, msg.getTopic(), msgId, timeoutMillis);
     }
 
     @Override
@@ -847,19 +1155,15 @@ public class DefaultMQAdminExtImpl implements MQAdminExt, MQAdminExtInner {
         final String msgId) throws RemotingException, MQClientException, InterruptedException, MQBrokerException {
         MessageExt msg = this.viewMessage(topic, msgId);
         if (msg.getProperty(MessageConst.PROPERTY_UNIQ_CLIENT_MESSAGE_ID_KEYIDX) == null) {
-            return this.mqClientInstance.getMQClientAPIImpl().consumeMessageDirectly(RemotingUtil.socketAddress2String(msg.getStoreHost()),
-                consumerGroup, clientId, msgId, timeoutMillis * 3);
+            return this.mqClientInstance.getMQClientAPIImpl().consumeMessageDirectly(RemotingUtil.socketAddress2String(msg.getStoreHost()), consumerGroup, clientId, topic, msgId, timeoutMillis);
         } else {
             MessageClientExt msgClient = (MessageClientExt) msg;
-            return this.mqClientInstance.getMQClientAPIImpl().consumeMessageDirectly(RemotingUtil.socketAddress2String(msg.getStoreHost()),
-                consumerGroup, clientId, msgClient.getOffsetMsgId(), timeoutMillis * 3);
+            return this.mqClientInstance.getMQClientAPIImpl().consumeMessageDirectly(RemotingUtil.socketAddress2String(msg.getStoreHost()), consumerGroup, clientId, topic, msgClient.getOffsetMsgId(), timeoutMillis);
         }
     }
 
-    @Override
-    public List<MessageTrack> messageTrackDetail(
-        MessageExt msg) throws RemotingException, MQClientException, InterruptedException,
-        MQBrokerException {
+    @Override public List<MessageTrack> messageTrackDetail(
+        MessageExt msg) throws RemotingException, MQClientException, InterruptedException, MQBrokerException {
         List<MessageTrack> result = new ArrayList<MessageTrack>();
 
         GroupList groupList = this.queryTopicConsumeByWho(msg.getTopic());
@@ -919,9 +1223,7 @@ public class DefaultMQAdminExtImpl implements MQAdminExt, MQAdminExtInner {
                         while (it.hasNext()) {
                             Entry<String, SubscriptionData> next = it.next();
                             if (next.getKey().equals(msg.getTopic())) {
-                                if (next.getValue().getTagsSet().contains(msg.getTags())
-                                    || next.getValue().getTagsSet().contains("*")
-                                    || next.getValue().getTagsSet().isEmpty()) {
+                                if (next.getValue().getTagsSet().contains(msg.getTags()) || next.getValue().getTagsSet().contains("*") || next.getValue().getTagsSet().isEmpty()) {
                                 } else {
                                     mt.setTrackType(TrackType.CONSUMED_BUT_FILTERED);
                                 }
@@ -940,9 +1242,107 @@ public class DefaultMQAdminExtImpl implements MQAdminExt, MQAdminExtInner {
         return result;
     }
 
+    @Override public List<MessageTrack> messageTrackDetailConcurrent(
+        final MessageExt msg) throws RemotingException, MQClientException, InterruptedException, MQBrokerException {
+        final List<MessageTrack> result = new ArrayList<MessageTrack>();
+
+        GroupList groupList = this.queryTopicConsumeByWho(msg.getTopic());
+
+        final CountDownLatch countDownLatch = new CountDownLatch(groupList.getGroupList().size());
+
+        for (final String group : groupList.getGroupList()) {
+
+            threadPoolExecutor.submit(new Runnable() {
+                @Override public void run() {
+                    MessageTrack mt = new MessageTrack();
+                    mt.setConsumerGroup(group);
+                    mt.setTrackType(TrackType.UNKNOWN);
+                    ConsumerConnection cc = null;
+                    try {
+                        cc = DefaultMQAdminExtImpl.this.examineConsumerConnectionInfo(group);
+                    } catch (MQBrokerException e) {
+                        if (ResponseCode.CONSUMER_NOT_ONLINE == e.getResponseCode()) {
+                            mt.setTrackType(TrackType.NOT_ONLINE);
+                        }
+                        mt.setExceptionDesc("CODE:" + e.getResponseCode() + " DESC:" + e.getErrorMessage());
+                        result.add(mt);
+                        countDownLatch.countDown();
+                        return;
+                    } catch (Exception e) {
+                        mt.setExceptionDesc(RemotingHelper.exceptionSimpleDesc(e));
+                        result.add(mt);
+                        countDownLatch.countDown();
+                        return;
+                    }
+
+                    switch (cc.getConsumeType()) {
+                        case CONSUME_ACTIVELY:
+                            mt.setTrackType(TrackType.PULL);
+                            break;
+                        case CONSUME_PASSIVELY:
+                            boolean ifConsumed = false;
+                            try {
+                                ifConsumed = DefaultMQAdminExtImpl.this.consumed(msg, group);
+                            } catch (MQClientException e) {
+                                if (ResponseCode.CONSUMER_NOT_ONLINE == e.getResponseCode()) {
+                                    mt.setTrackType(TrackType.NOT_ONLINE);
+                                }
+                                mt.setExceptionDesc("CODE:" + e.getResponseCode() + " DESC:" + e.getErrorMessage());
+                                result.add(mt);
+                                countDownLatch.countDown();
+                                return;
+                            } catch (MQBrokerException e) {
+                                if (ResponseCode.CONSUMER_NOT_ONLINE == e.getResponseCode()) {
+                                    mt.setTrackType(TrackType.NOT_ONLINE);
+                                }
+                                mt.setExceptionDesc("CODE:" + e.getResponseCode() + " DESC:" + e.getErrorMessage());
+                                result.add(mt);
+                                countDownLatch.countDown();
+                                return;
+                            } catch (Exception e) {
+                                mt.setExceptionDesc(RemotingHelper.exceptionSimpleDesc(e));
+                                result.add(mt);
+                                countDownLatch.countDown();
+                                return;
+                            }
+
+                            if (ifConsumed) {
+                                mt.setTrackType(TrackType.CONSUMED);
+                                Iterator<Entry<String, SubscriptionData>> it = cc.getSubscriptionTable().entrySet().iterator();
+                                while (it.hasNext()) {
+                                    Entry<String, SubscriptionData> next = it.next();
+                                    if (next.getKey().equals(msg.getTopic())) {
+                                        if (next.getValue().getTagsSet().contains(msg.getTags()) || next.getValue().getTagsSet().contains("*") || next.getValue().getTagsSet().isEmpty()) {
+                                        } else {
+                                            mt.setTrackType(TrackType.CONSUMED_BUT_FILTERED);
+                                        }
+                                    }
+                                }
+                            } else {
+                                mt.setTrackType(TrackType.NOT_CONSUME_YET);
+                            }
+                            break;
+                        default:
+                            break;
+                    }
+                    result.add(mt);
+                    countDownLatch.countDown();
+                    return;
+                }
+            });
+        }
+
+        countDownLatch.await(timeoutMillis, TimeUnit.MILLISECONDS);
+
+        return result;
+    }
+
+    public static void main(String[] args) {
+        Arrays.asList(null);
+    }
+
     public boolean consumed(final MessageExt msg,
-        final String group) throws RemotingException, MQClientException, InterruptedException,
-        MQBrokerException {
+        final String group) throws RemotingException, MQClientException, InterruptedException, MQBrokerException {
 
         ConsumeStats cstats = this.examineConsumeStats(group);
 
@@ -968,45 +1368,67 @@ public class DefaultMQAdminExtImpl implements MQAdminExt, MQAdminExtInner {
         return false;
     }
 
-    @Override
-    public void cloneGroupOffset(String srcGroup, String destGroup, String topic,
-        boolean isOffline) throws RemotingException,
-        MQClientException, InterruptedException, MQBrokerException {
+    public boolean consumedConcurrent(final MessageExt msg,
+        final String group) throws RemotingException, MQClientException, InterruptedException, MQBrokerException {
+
+        AdminToolResult<ConsumeStats> cstats = this.examineConsumeStatsConcurrent(group, null);
+
+        if (!cstats.isSuccess()) {
+            throw new MQClientException(cstats.getCode(), cstats.getErrorMsg());
+        }
+
+        ClusterInfo ci = this.examineBrokerClusterInfo();
+
+        if (cstats.isSuccess()) {
+            for (Entry<MessageQueue, OffsetWrapper> next : cstats.getData().getOffsetTable().entrySet()) {
+                MessageQueue mq = next.getKey();
+                if (mq.getTopic().equals(msg.getTopic()) && mq.getQueueId() == msg.getQueueId()) {
+                    BrokerData brokerData = ci.getBrokerAddrTable().get(mq.getBrokerName());
+                    if (brokerData != null) {
+                        String addr = brokerData.getBrokerAddrs().get(MixAll.MASTER_ID);
+                        if (addr.equals(RemotingUtil.socketAddress2String(msg.getStoreHost()))) {
+                            if (next.getValue().getConsumerOffset() > msg.getQueueOffset()) {
+                                return true;
+                            }
+                        }
+                    }
+                }
+            }
+        }
+
+        return false;
+    }
+
+    @Override public void cloneGroupOffset(String srcGroup, String destGroup, String topic,
+        boolean isOffline) throws RemotingException, MQClientException, InterruptedException, MQBrokerException {
         String retryTopic = MixAll.getRetryTopic(srcGroup);
         TopicRouteData topicRouteData = this.examineTopicRouteInfo(retryTopic);
 
         for (BrokerData bd : topicRouteData.getBrokerDatas()) {
             String addr = bd.selectBrokerAddr();
             if (addr != null) {
-                this.mqClientInstance.getMQClientAPIImpl().cloneGroupOffset(addr, srcGroup, destGroup, topic, isOffline, timeoutMillis * 3);
+                this.mqClientInstance.getMQClientAPIImpl().cloneGroupOffset(addr, srcGroup, destGroup, topic, isOffline, timeoutMillis);
             }
         }
     }
 
-    @Override
-    public BrokerStatsData viewBrokerStatsData(String brokerAddr, String statsName,
-        String statsKey) throws RemotingConnectException,
-        RemotingSendRequestException, RemotingTimeoutException, MQClientException, InterruptedException {
+    @Override public BrokerStatsData viewBrokerStatsData(String brokerAddr, String statsName,
+        String statsKey) throws RemotingConnectException, RemotingSendRequestException, RemotingTimeoutException, MQClientException, InterruptedException {
         return this.mqClientInstance.getMQClientAPIImpl().viewBrokerStatsData(brokerAddr, statsName, statsKey, timeoutMillis);
     }
 
-    @Override
-    public Set<String> getClusterList(String topic) throws RemotingConnectException, RemotingSendRequestException,
-        RemotingTimeoutException, MQClientException, InterruptedException {
+    @Override public Set<String> getClusterList(
+        String topic) throws RemotingConnectException, RemotingSendRequestException, RemotingTimeoutException, MQClientException, InterruptedException {
         return this.mqClientInstance.getMQClientAPIImpl().getClusterList(topic, timeoutMillis);
     }
 
-    @Override
-    public ConsumeStatsList fetchConsumeStatsInBroker(final String brokerAddr, boolean isOrder, long timeoutMillis)
-        throws RemotingConnectException, RemotingSendRequestException, RemotingTimeoutException, MQClientException,
-        InterruptedException {
+    @Override public ConsumeStatsList fetchConsumeStatsInBroker(final String brokerAddr, boolean isOrder,
+        long timeoutMillis) throws RemotingConnectException, RemotingSendRequestException, RemotingTimeoutException, MQClientException, InterruptedException {
         return this.mqClientInstance.getMQClientAPIImpl().fetchConsumeStatsInBroker(brokerAddr, isOrder, timeoutMillis);
     }
 
-    @Override
-    public Set<String> getTopicClusterList(
-        final String topic) throws InterruptedException, MQBrokerException, MQClientException,
-        RemotingException {
+    @Override public Set<String> getTopicClusterList(
+        final String topic) throws InterruptedException, MQBrokerException, MQClientException, RemotingException {
         Set<String> clusterSet = new HashSet<String>();
         ClusterInfo clusterInfo = examineBrokerClusterInfo();
         TopicRouteData topicRouteData = examineTopicRouteInfo(topic);
@@ -1022,22 +1444,16 @@ public class DefaultMQAdminExtImpl implements MQAdminExt, MQAdminExtInner {
         return clusterSet;
     }
 
-    @Override
-    public SubscriptionGroupWrapper getAllSubscriptionGroup(final String brokerAddr,
-        long timeoutMillis) throws InterruptedException,
-        RemotingTimeoutException, RemotingSendRequestException, RemotingConnectException, MQBrokerException {
+    @Override public SubscriptionGroupWrapper getAllSubscriptionGroup(final String brokerAddr,
+        long timeoutMillis) throws InterruptedException, RemotingTimeoutException, RemotingSendRequestException, RemotingConnectException, MQBrokerException {
         return this.mqClientInstance.getMQClientAPIImpl().getAllSubscriptionGroup(brokerAddr, timeoutMillis);
     }
 
-    @Override
-    public SubscriptionGroupWrapper getUserSubscriptionGroup(final String brokerAddr,
-        long timeoutMillis) throws InterruptedException,
-        RemotingTimeoutException, RemotingSendRequestException, RemotingConnectException, MQBrokerException {
-        SubscriptionGroupWrapper subscriptionGroupWrapper = this.mqClientInstance.getMQClientAPIImpl()
-            .getAllSubscriptionGroup(brokerAddr, timeoutMillis);
-
-        Iterator<Entry<String, SubscriptionGroupConfig>> iterator = subscriptionGroupWrapper.getSubscriptionGroupTable()
-            .entrySet().iterator();
+    @Override public SubscriptionGroupWrapper getUserSubscriptionGroup(final String brokerAddr,
+        long timeoutMillis) throws InterruptedException, RemotingTimeoutException, RemotingSendRequestException, RemotingConnectException, MQBrokerException {
+        SubscriptionGroupWrapper subscriptionGroupWrapper = this.mqClientInstance.getMQClientAPIImpl().getAllSubscriptionGroup(brokerAddr, timeoutMillis);
+
+        Iterator<Entry<String, SubscriptionGroupConfig>> iterator = subscriptionGroupWrapper.getSubscriptionGroupTable().entrySet().iterator();
         while (iterator.hasNext()) {
             Map.Entry<String, SubscriptionGroupConfig> configEntry = iterator.next();
             if (MixAll.isSysConsumerGroup(configEntry.getKey()) || SYSTEM_GROUP_SET.contains(configEntry.getKey())) {
@@ -1048,22 +1464,16 @@ public class DefaultMQAdminExtImpl implements MQAdminExt, MQAdminExtInner {
         return subscriptionGroupWrapper;
     }
 
-    @Override
-    public TopicConfigSerializeWrapper getAllTopicConfig(final String brokerAddr,
-        long timeoutMillis) throws InterruptedException,
-        RemotingTimeoutException, RemotingSendRequestException, RemotingConnectException, MQBrokerException {
+    @Override public TopicConfigSerializeWrapper getAllTopicConfig(final String brokerAddr,
+        long timeoutMillis) throws InterruptedException, RemotingTimeoutException, RemotingSendRequestException, RemotingConnectException, MQBrokerException {
         return this.mqClientInstance.getMQClientAPIImpl().getAllTopicConfig(brokerAddr, timeoutMillis);
     }
 
-    @Override
-    public TopicConfigSerializeWrapper getUserTopicConfig(final String brokerAddr, final boolean specialTopic,
-        long timeoutMillis) throws InterruptedException, RemotingException,
-        MQBrokerException, MQClientException {
+    @Override public TopicConfigSerializeWrapper getUserTopicConfig(final String brokerAddr, final boolean specialTopic,
+        long timeoutMillis) throws InterruptedException, RemotingException, MQBrokerException, MQClientException {
         TopicConfigSerializeWrapper topicConfigSerializeWrapper = this.getAllTopicConfig(brokerAddr, timeoutMillis);
-        TopicList topicList = this.mqClientInstance.getMQClientAPIImpl().getSystemTopicListFromBroker(brokerAddr,
-            timeoutMillis);
-        Iterator<Entry<String, TopicConfig>> iterator = topicConfigSerializeWrapper.getTopicConfigTable().entrySet()
-            .iterator();
+        TopicList topicList = this.mqClientInstance.getMQClientAPIImpl().getSystemTopicListFromBroker(brokerAddr, timeoutMillis);
+        Iterator<Entry<String, TopicConfig>> iterator = topicConfigSerializeWrapper.getTopicConfigTable().entrySet().iterator();
         while (iterator.hasNext()) {
             String topic = iterator.next().getKey();
             if (topicList.getTopicList().contains(topic) || (!specialTopic && (topic.startsWith(MixAll.RETRY_GROUP_TOPIC_PREFIX) || topic.startsWith(MixAll.DLQ_GROUP_TOPIC_PREFIX)))) {
@@ -1073,64 +1483,50 @@ public class DefaultMQAdminExtImpl implements MQAdminExt, MQAdminExtInner {
         return topicConfigSerializeWrapper;
     }
 
-    @Override
-    public void createTopic(String key, String newTopic, int queueNum, Map<String, String> attributes) throws MQClientException {
+    @Override public void createTopic(String key, String newTopic, int queueNum,
+        Map<String, String> attributes) throws MQClientException {
         createTopic(key, newTopic, queueNum, 0, null);
     }
 
-    @Override
-    public void createTopic(String key, String newTopic, int queueNum, int topicSysFlag, Map<String, String> attributes) throws MQClientException {
+    @Override public void createTopic(String key, String newTopic, int queueNum, int topicSysFlag,
+        Map<String, String> attributes) throws MQClientException {
         this.mqClientInstance.getMQAdminImpl().createTopic(key, newTopic, queueNum, topicSysFlag, attributes);
     }
 
-    @Override
-    public void createStaticTopic(final String addr, final String defaultTopic, final TopicConfig topicConfig,
+    @Override public void createStaticTopic(final String addr, final String defaultTopic, final TopicConfig topicConfig,
         final TopicQueueMappingDetail mappingDetail,
         final boolean force) throws RemotingException, InterruptedException, MQBrokerException {
         this.mqClientInstance.getMQClientAPIImpl().createStaticTopic(addr, defaultTopic, topicConfig, mappingDetail, force, timeoutMillis);
     }
 
-    @Override
-    public long searchOffset(MessageQueue mq, long timestamp) throws MQClientException {
+    @Override public long searchOffset(MessageQueue mq, long timestamp) throws MQClientException {
         return this.mqClientInstance.getMQAdminImpl().searchOffset(mq, timestamp);
     }
 
-    @Override
-    public long maxOffset(MessageQueue mq) throws MQClientException {
+    @Override public long maxOffset(MessageQueue mq) throws MQClientException {
         return this.mqClientInstance.getMQAdminImpl().maxOffset(mq);
     }
 
-    @Override
-    public long minOffset(MessageQueue mq) throws MQClientException {
+    @Override public long minOffset(MessageQueue mq) throws MQClientException {
         return this.mqClientInstance.getMQAdminImpl().minOffset(mq);
     }
 
-    @Override
-    public long earliestMsgStoreTime(MessageQueue mq) throws MQClientException {
+    @Override public long earliestMsgStoreTime(MessageQueue mq) throws MQClientException {
         return this.mqClientInstance.getMQAdminImpl().earliestMsgStoreTime(mq);
     }
 
-    @Override
-    public MessageExt viewMessage(
+    @Override public MessageExt viewMessage(
         String msgId) throws RemotingException, MQBrokerException, InterruptedException, MQClientException {
         return this.mqClientInstance.getMQAdminImpl().viewMessage(msgId);
     }
 
-    @Override
-    public QueryResult queryMessage(String topic, String key, int maxNum, long begin, long end)
-        throws MQClientException, InterruptedException {
-
-        return this.mqClientInstance.getMQAdminImpl().queryMessage(topic, key, maxNum, begin, end);
-    }
-
-    public QueryResult queryMessageByUniqKey(String topic, String key, int maxNum, long begin,
+    @Override public QueryResult queryMessage(String topic, String key, int maxNum, long begin,
         long end) throws MQClientException, InterruptedException {
 
-        return this.mqClientInstance.getMQAdminImpl().queryMessageByUniqKey(topic, key, maxNum, begin, end);
+        return this.mqClientInstance.getMQAdminImpl().queryMessage(topic, key, maxNum, begin, end);
     }
 
-    @Override
-    public void updateConsumeOffset(String brokerAddr, String consumeGroup, MessageQueue mq,
+    @Override public void updateConsumeOffset(String brokerAddr, String consumeGroup, MessageQueue mq,
         long offset) throws RemotingException, InterruptedException, MQBrokerException {
         UpdateConsumerOffsetRequestHeader requestHeader = new UpdateConsumerOffsetRequestHeader();
         requestHeader.setConsumerGroup(consumeGroup);
@@ -1140,41 +1536,31 @@ public class DefaultMQAdminExtImpl implements MQAdminExt, MQAdminExtInner {
         this.mqClientInstance.getMQClientAPIImpl().updateConsumerOffset(brokerAddr, requestHeader, timeoutMillis);
     }
 
-    @Override
-    public void updateNameServerConfig(final Properties properties, final List<String> nameServers)
-        throws InterruptedException, RemotingConnectException,
-        UnsupportedEncodingException, RemotingSendRequestException, RemotingTimeoutException,
-        MQClientException, MQBrokerException {
+    @Override public void updateNameServerConfig(final Properties properties,
+        final List<String> nameServers) throws InterruptedException, RemotingConnectException, UnsupportedEncodingException, RemotingSendRequestException, RemotingTimeoutException, MQClientException, MQBrokerException {
         this.mqClientInstance.getMQClientAPIImpl().updateNameServerConfig(properties, nameServers, timeoutMillis);
     }
 
-    @Override
-    public Map<String, Properties> getNameServerConfig(final List<String> nameServers)
-        throws InterruptedException, RemotingTimeoutException,
-        RemotingSendRequestException, RemotingConnectException, MQClientException,
-        UnsupportedEncodingException {
+    @Override public Map<String, Properties> getNameServerConfig(
+        final List<String> nameServers) throws InterruptedException, RemotingTimeoutException, RemotingSendRequestException, RemotingConnectException, MQClientException, UnsupportedEncodingException {
         return this.mqClientInstance.getMQClientAPIImpl().getNameServerConfig(nameServers, timeoutMillis);
     }
 
     @Override
     public QueryConsumeQueueResponseBody queryConsumeQueue(String brokerAddr, String topic, int queueId, long index,
-        int count, String consumerGroup)
-        throws InterruptedException, RemotingTimeoutException, RemotingSendRequestException, RemotingConnectException, MQClientException {
-        return this.mqClientInstance.getMQClientAPIImpl().queryConsumeQueue(
-            brokerAddr, topic, queueId, index, count, consumerGroup, timeoutMillis
-        );
+        int count,
+        String consumerGroup) throws InterruptedException, RemotingTimeoutException, RemotingSendRequestException, RemotingConnectException, MQClientException {
+        return this.mqClientInstance.getMQClientAPIImpl().queryConsumeQueue(brokerAddr, topic, queueId, index, count, consumerGroup, timeoutMillis);
     }
 
-    @Override
-    public boolean resumeCheckHalfMessage(String msgId)
-        throws RemotingException, MQClientException, InterruptedException, MQBrokerException {
+    @Override public boolean resumeCheckHalfMessage(
+        String msgId) throws RemotingException, MQClientException, InterruptedException, MQBrokerException {
         MessageExt msg = this.viewMessage(msgId);
 
         return this.mqClientInstance.getMQClientAPIImpl().resumeCheckHalfMessage(RemotingUtil.socketAddress2String(msg.getStoreHost()), msgId, timeoutMillis);
     }
 
-    @Override
-    public boolean resumeCheckHalfMessage(final String topic,
+    @Override public boolean resumeCheckHalfMessage(final String topic,
         final String msgId) throws RemotingException, MQClientException, InterruptedException, MQBrokerException {
         MessageExt msg = this.viewMessage(topic, msgId);
         if (msg.getProperty(MessageConst.PROPERTY_UNIQ_CLIENT_MESSAGE_ID_KEYIDX) == null) {
@@ -1185,11 +1571,67 @@ public class DefaultMQAdminExtImpl implements MQAdminExt, MQAdminExtInner {
         }
     }
 
-    @Override
-    public void setMessageRequestMode(final String brokerAddr, final String topic, final String consumerGroup, final
-        MessageRequestMode mode, final int popShareQueueNum, final long timeoutMillis)
-        throws InterruptedException, RemotingTimeoutException, RemotingSendRequestException,
-        RemotingConnectException, MQClientException {
+    @Override public void setMessageRequestMode(final String brokerAddr, final String topic, final String consumerGroup,
+        final MessageRequestMode mode, final int popShareQueueNum,
+        final long timeoutMillis) throws InterruptedException, RemotingTimeoutException, RemotingSendRequestException, RemotingConnectException, MQClientException {
         this.mqClientInstance.getMQClientAPIImpl().setMessageRequestMode(brokerAddr, topic, consumerGroup, mode, popShareQueueNum, timeoutMillis);
     }
+
+    @Override
+    public long searchOffset(final String brokerAddr, final String topicName, final int queueId, final long timestamp,
+        final long timeoutMillis) throws RemotingException, MQBrokerException, InterruptedException {
+        return this.mqClientInstance.getMQClientAPIImpl().searchOffset(brokerAddr, topicName, queueId, timestamp, timeoutMillis);
+    }
+
+    public QueryResult queryMessageByUniqKey(String topic, String key, int maxNum, long begin,
+        long end) throws MQClientException, InterruptedException {
+
+        return this.mqClientInstance.getMQAdminImpl().queryMessageByUniqKey(topic, key, maxNum, begin, end);
+    }
+
+    @Override
+    public void resetOffsetByQueueId(final String brokerAddr, final String consumeGroup, final String topicName,
+        final int queueId, final long resetOffset) throws RemotingException, InterruptedException, MQBrokerException {
+        UpdateConsumerOffsetRequestHeader requestHeader = new UpdateConsumerOffsetRequestHeader();
+        requestHeader.setConsumerGroup(consumeGroup);
+        requestHeader.setTopic(topicName);
+        requestHeader.setQueueId(queueId);
+        requestHeader.setCommitOffset(resetOffset);
+        this.mqClientInstance.getMQClientAPIImpl().updateConsumerOffset(brokerAddr, requestHeader, timeoutMillis);
+    }
+
+    @Override public HARuntimeInfo getBrokerHAStatus(
+        String brokerAddr) throws RemotingConnectException, RemotingSendRequestException, RemotingTimeoutException, InterruptedException, MQBrokerException {
+        return this.mqClientInstance.getMQClientAPIImpl().getBrokerHAStatus(brokerAddr, timeoutMillis);
+    }
+
+    @Override public void resetMasterFlushOffset(String brokerAddr,
+        long masterFlushOffset) throws InterruptedException, MQBrokerException, RemotingTimeoutException, RemotingSendRequestException, RemotingConnectException {
+        this.mqClientInstance.getMQClientAPIImpl().resetMasterFlushOffset(brokerAddr, masterFlushOffset);
+    }
+
+    @Override
+    public GroupForbidden updateAndGetGroupReadForbidden(String brokerAddr, String groupName, String topicName,
+        Boolean readable) throws RemotingException, InterruptedException, MQBrokerException {
+        UpdateGroupForbiddenRequestHeader requestHeader = new UpdateGroupForbiddenRequestHeader();
+        requestHeader.setGroup(groupName);
+        requestHeader.setTopic(topicName);
+        requestHeader.setReadable(readable);
+        return this.mqClientInstance.getMQClientAPIImpl().updateAndGetGroupForbidden(brokerAddr, requestHeader, timeoutMillis);
+    }
+
+    @Override public void deleteTopicInNameServer(Set<String> addrs, String clusterName,
+        String topic) throws RemotingException, MQBrokerException, InterruptedException, MQClientException {
+        if (addrs == null) {
+            String ns = this.mqClientInstance.getMQClientAPIImpl().fetchNameServerAddr();
+            addrs = new HashSet(Arrays.asList(ns.split(";")));
+        }
+        for (String addr : addrs) {
+            this.mqClientInstance.getMQClientAPIImpl().deleteTopicInNameServer(addr, clusterName, topic, timeoutMillis);
+        }
+    }
+
+    public MQClientInstance getMqClientInstance() {
+        return mqClientInstance;
+    }
 }
diff --git a/tools/src/main/java/org/apache/rocketmq/tools/admin/MQAdminExt.java b/tools/src/main/java/org/apache/rocketmq/tools/admin/MQAdminExt.java
index 33d580d..5ed147c 100644
--- a/tools/src/main/java/org/apache/rocketmq/tools/admin/MQAdminExt.java
+++ b/tools/src/main/java/org/apache/rocketmq/tools/admin/MQAdminExt.java
@@ -36,6 +36,7 @@ import org.apache.rocketmq.common.protocol.body.ConsumeStatsList;
 import org.apache.rocketmq.common.protocol.body.ConsumerConnection;
 import org.apache.rocketmq.common.protocol.body.ConsumerRunningInfo;
 import org.apache.rocketmq.common.protocol.body.GroupList;
+import org.apache.rocketmq.common.protocol.body.HARuntimeInfo;
 import org.apache.rocketmq.common.protocol.body.KVTable;
 import org.apache.rocketmq.common.protocol.body.ProducerConnection;
 import org.apache.rocketmq.common.protocol.body.QueryConsumeQueueResponseBody;
@@ -43,14 +44,17 @@ import org.apache.rocketmq.common.protocol.body.QueueTimeSpan;
 import org.apache.rocketmq.common.protocol.body.SubscriptionGroupWrapper;
 import org.apache.rocketmq.common.protocol.body.TopicConfigSerializeWrapper;
 import org.apache.rocketmq.common.protocol.body.TopicList;
+import org.apache.rocketmq.common.protocol.heartbeat.SubscriptionData;
 import org.apache.rocketmq.common.protocol.route.TopicRouteData;
 import org.apache.rocketmq.common.statictopic.TopicQueueMappingDetail;
+import org.apache.rocketmq.common.subscription.GroupForbidden;
 import org.apache.rocketmq.common.subscription.SubscriptionGroupConfig;
 import org.apache.rocketmq.remoting.exception.RemotingCommandException;
 import org.apache.rocketmq.remoting.exception.RemotingConnectException;
 import org.apache.rocketmq.remoting.exception.RemotingException;
 import org.apache.rocketmq.remoting.exception.RemotingSendRequestException;
 import org.apache.rocketmq.remoting.exception.RemotingTimeoutException;
+import org.apache.rocketmq.tools.admin.api.BrokerOperatorResult;
 import org.apache.rocketmq.tools.admin.api.MessageTrack;
 
 import java.io.UnsupportedEncodingException;
@@ -58,12 +62,19 @@ import java.util.List;
 import java.util.Map;
 import java.util.Properties;
 import java.util.Set;
+import org.apache.rocketmq.tools.admin.common.AdminToolResult;
 
 public interface MQAdminExt extends MQAdmin {
     void start() throws MQClientException;
 
     void shutdown();
 
+    void addBrokerToContainer(final String brokerContainerAddr, final String brokerConfig) throws InterruptedException,
+        MQBrokerException, RemotingTimeoutException, RemotingSendRequestException, RemotingConnectException;
+
+    void removeBrokerFromContainer(final String brokerContainerAddr, String clusterName, final String brokerName,
+        long brokerId) throws InterruptedException, MQBrokerException, RemotingTimeoutException, RemotingSendRequestException, RemotingConnectException;
+
     void updateBrokerConfig(final String brokerAddr, final Properties properties) throws RemotingConnectException,
         RemotingSendRequestException, RemotingTimeoutException, UnsupportedEncodingException, InterruptedException, MQBrokerException;
 
@@ -98,7 +109,6 @@ public interface MQAdminExt extends MQAdmin {
 
     SubscriptionGroupConfig examineSubscriptionGroupConfig(final String addr, final String group) throws InterruptedException, RemotingException, MQClientException, MQBrokerException;
 
-
     TopicStatsTable examineTopicStats(
         final String topic) throws RemotingException, MQClientException, InterruptedException,
         MQBrokerException;
@@ -106,6 +116,8 @@ public interface MQAdminExt extends MQAdmin {
     TopicStatsTable examineTopicStats(String brokerAddr, final String topic) throws RemotingException, MQClientException, InterruptedException,
             MQBrokerException;
 
+    AdminToolResult<TopicStatsTable> examineTopicStatsConcurrent(String topic);
+
     TopicList fetchAllTopicList() throws RemotingException, MQClientException, InterruptedException;
 
     TopicList fetchTopicsByCLuster(
@@ -123,6 +135,8 @@ public interface MQAdminExt extends MQAdmin {
         final String topic) throws RemotingException, MQClientException,
         InterruptedException, MQBrokerException;
 
+    AdminToolResult<ConsumeStats> examineConsumeStatsConcurrent(String consumerGroup, String topic);
+
     ClusterInfo examineBrokerClusterInfo() throws InterruptedException, MQBrokerException, RemotingTimeoutException,
         RemotingSendRequestException, RemotingConnectException;
 
@@ -157,10 +171,20 @@ public interface MQAdminExt extends MQAdmin {
     KVTable getKVListByNamespace(
         final String namespace) throws RemotingException, MQClientException, InterruptedException;
 
+    void deleteTopic(final String topicName,
+        final String clusterName) throws RemotingException, MQBrokerException, InterruptedException, MQClientException;
+
     void deleteTopicInBroker(final Set<String> addrs, final String topic) throws RemotingException, MQBrokerException,
         InterruptedException, MQClientException;
 
+    AdminToolResult<BrokerOperatorResult> deleteTopicInBrokerConcurrent(Set<String> addrs, String topic);
+
+    void deleteTopicInNameServer(final Set<String> addrs,
+        final String topic) throws RemotingException, MQBrokerException,
+        InterruptedException, MQClientException;
+
     void deleteTopicInNameServer(final Set<String> addrs,
+        final String clusterName,
         final String topic) throws RemotingException, MQBrokerException,
         InterruptedException, MQClientException;
 
@@ -186,6 +210,9 @@ public interface MQAdminExt extends MQAdmin {
     void resetOffsetNew(String consumerGroup, String topic, long timestamp) throws RemotingException, MQBrokerException,
         InterruptedException, MQClientException;
 
+    AdminToolResult<BrokerOperatorResult> resetOffsetNewConcurrent(final String group, final String topic,
+        final long timestamp);
+
     Map<String, Map<MessageQueue, Long>> getConsumeStatus(String topic, String group,
         String clientAddr) throws RemotingException,
         MQBrokerException, InterruptedException, MQClientException;
@@ -197,10 +224,18 @@ public interface MQAdminExt extends MQAdmin {
     GroupList queryTopicConsumeByWho(final String topic) throws RemotingConnectException, RemotingSendRequestException,
         RemotingTimeoutException, InterruptedException, MQBrokerException, RemotingException, MQClientException;
 
+    TopicList queryTopicsByConsumer(final String group) throws InterruptedException, MQBrokerException, RemotingException, MQClientException;
+
+    AdminToolResult<TopicList> queryTopicsByConsumerConcurrent(final String group);
+
+    SubscriptionData querySubscription(final String group, final String topic) throws InterruptedException, MQBrokerException, RemotingException, MQClientException;
+
     List<QueueTimeSpan> queryConsumeTimeSpan(final String topic,
         final String group) throws InterruptedException, MQBrokerException,
         RemotingException, MQClientException;
 
+    AdminToolResult<List<QueueTimeSpan>> queryConsumeTimeSpanConcurrent(final String topic, final String group);
+
     boolean cleanExpiredConsumerQueue(String cluster) throws RemotingConnectException, RemotingSendRequestException,
         RemotingTimeoutException, MQClientException, InterruptedException;
 
@@ -216,6 +251,9 @@ public interface MQAdminExt extends MQAdmin {
     ConsumerRunningInfo getConsumerRunningInfo(final String consumerGroup, final String clientId, final boolean jstack)
         throws RemotingException, MQClientException, InterruptedException;
 
+    ConsumerRunningInfo getConsumerRunningInfo(final String consumerGroup, final String clientId, final boolean jstack, final boolean metrics)
+        throws RemotingException, MQClientException, InterruptedException;
+
     ConsumeMessageDirectlyResult consumeMessageDirectly(String consumerGroup,
         String clientId,
         String msgId) throws RemotingException, MQClientException, InterruptedException, MQBrokerException;
@@ -229,6 +267,10 @@ public interface MQAdminExt extends MQAdmin {
         MessageExt msg) throws RemotingException, MQClientException, InterruptedException,
         MQBrokerException;
 
+    List<MessageTrack> messageTrackDetailConcurrent(
+        MessageExt msg) throws RemotingException, MQClientException, InterruptedException,
+        MQBrokerException;
+
     void cloneGroupOffset(String srcGroup, String destGroup, String topic, boolean isOffline) throws RemotingException,
         MQClientException, InterruptedException, MQBrokerException;
 
@@ -254,6 +296,7 @@ public interface MQAdminExt extends MQAdmin {
         long timeoutMillis) throws InterruptedException, RemotingTimeoutException, RemotingSendRequestException,
         RemotingConnectException, MQBrokerException;
 
+
     TopicConfigSerializeWrapper getAllTopicConfig(final String brokerAddr,
         long timeoutMillis) throws InterruptedException, RemotingTimeoutException, RemotingSendRequestException,
         RemotingConnectException, MQBrokerException;
@@ -315,11 +358,37 @@ public interface MQAdminExt extends MQAdmin {
         throws InterruptedException, RemotingTimeoutException, RemotingSendRequestException,
         RemotingConnectException, MQClientException;
 
+    long searchOffset(final String brokerAddr, final String topicName,
+        final int queueId, final long timestamp, final long timeoutMillis)
+        throws RemotingException, MQBrokerException, InterruptedException;
 
+    void resetOffsetByQueueId(final String brokerAddr, final String consumerGroup,
+        final String topicName, final int queueId, final long resetOffset)
+        throws RemotingException, InterruptedException, MQBrokerException;
 
     TopicConfig examineTopicConfig(final String addr,
                                    final String topic) throws InterruptedException, MQBrokerException, RemotingTimeoutException, RemotingSendRequestException, RemotingConnectException;
 
     void createStaticTopic(final String addr, final String defaultTopic, final TopicConfig topicConfig, final TopicQueueMappingDetail mappingDetail, final boolean force) throws RemotingException, InterruptedException, MQBrokerException;
 
+    GroupForbidden updateAndGetGroupReadForbidden(String brokerAddr, String groupName, String topicName,
+        Boolean readable)
+        throws RemotingException, InterruptedException, MQBrokerException;
+
+
+    MessageExt queryMessage(String clusterName,
+        String topic,
+        String msgId) throws RemotingException, MQBrokerException, InterruptedException, MQClientException;
+
+    HARuntimeInfo getBrokerHAStatus(String brokerAddr) throws RemotingConnectException, RemotingSendRequestException,
+        RemotingTimeoutException, InterruptedException, MQBrokerException;
+
+    /**
+     * Reset master flush offset in slave
+     *
+     * @param brokerAddr slave broker address
+     * @param masterFlushOffset master flush offset
+     */
+    void resetMasterFlushOffset(String brokerAddr, long masterFlushOffset)
+        throws InterruptedException, MQBrokerException, RemotingTimeoutException, RemotingSendRequestException, RemotingConnectException;
 }
diff --git a/tools/src/main/java/org/apache/rocketmq/tools/admin/api/BrokerOperatorResult.java b/tools/src/main/java/org/apache/rocketmq/tools/admin/api/BrokerOperatorResult.java
new file mode 100644
index 0000000..5ec04b5
--- /dev/null
+++ b/tools/src/main/java/org/apache/rocketmq/tools/admin/api/BrokerOperatorResult.java
@@ -0,0 +1,50 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.rocketmq.tools.admin.api;
+
+import java.util.List;
+
+public class BrokerOperatorResult {
+
+    private List<String> successList;
+
+    private List<String> failureList;
+
+    public List<String> getSuccessList() {
+        return successList;
+    }
+
+    public void setSuccessList(List<String> successList) {
+        this.successList = successList;
+    }
+
+    public List<String> getFailureList() {
+        return failureList;
+    }
+
+    public void setFailureList(List<String> failureList) {
+        this.failureList = failureList;
+    }
+
+    @Override
+    public String toString() {
+        return "BrokerOperatorResult{" +
+            "successList=" + successList +
+            ", failureList=" + failureList +
+            '}';
+    }
+}
diff --git a/tools/src/main/java/org/apache/rocketmq/tools/admin/common/AdminToolHandler.java b/tools/src/main/java/org/apache/rocketmq/tools/admin/common/AdminToolHandler.java
new file mode 100644
index 0000000..1afebeb
--- /dev/null
+++ b/tools/src/main/java/org/apache/rocketmq/tools/admin/common/AdminToolHandler.java
@@ -0,0 +1,21 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.rocketmq.tools.admin.common;
+
+public interface AdminToolHandler {
+    AdminToolResult doExecute() throws Exception;
+}
diff --git a/tools/src/main/java/org/apache/rocketmq/tools/admin/common/AdminToolResult.java b/tools/src/main/java/org/apache/rocketmq/tools/admin/common/AdminToolResult.java
new file mode 100644
index 0000000..21b9652
--- /dev/null
+++ b/tools/src/main/java/org/apache/rocketmq/tools/admin/common/AdminToolResult.java
@@ -0,0 +1,76 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.rocketmq.tools.admin.common;
+
+public class AdminToolResult<T> {
+
+    private boolean success;
+    private int code;
+    private String errorMsg;
+    private T data;
+
+    public AdminToolResult(boolean success, int code, String errorMsg, T data) {
+        this.success = success;
+        this.code = code;
+        this.errorMsg = errorMsg;
+        this.data = data;
+    }
+
+    public static AdminToolResult success(Object data) {
+        return new AdminToolResult(true, AdminToolsResultCodeEnum.SUCCESS.getCode(), "success", data);
+    }
+
+    public static AdminToolResult failure(AdminToolsResultCodeEnum errorCodeEnum, String errorMsg) {
+        return new AdminToolResult(false, errorCodeEnum.getCode(), errorMsg, null);
+    }
+
+    public static AdminToolResult failure(AdminToolsResultCodeEnum errorCodeEnum, String errorMsg, Object data) {
+        return new AdminToolResult(false, errorCodeEnum.getCode(), errorMsg, data);
+    }
+
+    public boolean isSuccess() {
+        return success;
+    }
+
+    public void setSuccess(boolean success) {
+        this.success = success;
+    }
+
+    public int getCode() {
+        return code;
+    }
+
+    public void setCode(int code) {
+        this.code = code;
+    }
+
+    public String getErrorMsg() {
+        return errorMsg;
+    }
+
+    public void setErrorMsg(String errorMsg) {
+        this.errorMsg = errorMsg;
+    }
+
+    public T getData() {
+        return data;
+    }
+
+    public void setData(T data) {
+        this.data = data;
+    }
+}
diff --git a/tools/src/main/java/org/apache/rocketmq/tools/admin/common/AdminToolsResultCodeEnum.java b/tools/src/main/java/org/apache/rocketmq/tools/admin/common/AdminToolsResultCodeEnum.java
new file mode 100644
index 0000000..9598845
--- /dev/null
+++ b/tools/src/main/java/org/apache/rocketmq/tools/admin/common/AdminToolsResultCodeEnum.java
@@ -0,0 +1,43 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.rocketmq.tools.admin.common;
+
+public enum AdminToolsResultCodeEnum {
+
+    /**
+     *
+     */
+    SUCCESS(200),
+
+    REMOTING_ERROR(-1001),
+    MQ_BROKER_ERROR(-1002),
+    MQ_CLIENT_ERROR(-1003),
+    INTERRUPT_ERROR(-1004),
+
+    TOPIC_ROUTE_INFO_NOT_EXIST(-2001),
+    CONSUMER_NOT_ONLINE(-2002);
+
+    private int code;
+
+    AdminToolsResultCodeEnum(int code) {
+        this.code = code;
+    }
+
+    public int getCode() {
+        return code;
+    }
+}
diff --git a/tools/src/main/java/org/apache/rocketmq/tools/command/CommandUtil.java b/tools/src/main/java/org/apache/rocketmq/tools/command/CommandUtil.java
index cdf5f32..c2e4cb2 100644
--- a/tools/src/main/java/org/apache/rocketmq/tools/command/CommandUtil.java
+++ b/tools/src/main/java/org/apache/rocketmq/tools/command/CommandUtil.java
@@ -39,6 +39,8 @@ public class CommandUtil {
     private static final String ERROR_MESSAGE = "Make sure the specified clusterName exists or the name server " +
         "connected to is correct.";
 
+    public static final String NO_MASTER_PLACEHOLDER = "NO_MASTER";
+
     public static Map<String/*master addr*/, List<String>/*slave addr*/> fetchMasterAndSlaveDistinguish(
         final MQAdminExt adminExt, final String clusterName)
         throws InterruptedException, RemotingConnectException,
@@ -62,14 +64,23 @@ public class CommandUtil {
             }
 
             String masterAddr = brokerData.getBrokerAddrs().get(MixAll.MASTER_ID);
-            masterAndSlaveMap.put(masterAddr, new ArrayList<String>());
+
+            if (masterAddr == null) {
+                masterAndSlaveMap.putIfAbsent(NO_MASTER_PLACEHOLDER, new ArrayList<>());
+            } else {
+                masterAndSlaveMap.put(masterAddr, new ArrayList<String>());
+            }
 
             for (Entry<Long, String> brokerAddrEntry : brokerData.getBrokerAddrs().entrySet()) {
                 if (brokerAddrEntry.getValue() == null || brokerAddrEntry.getKey() == MixAll.MASTER_ID) {
                     continue;
                 }
 
-                masterAndSlaveMap.get(masterAddr).add(brokerAddrEntry.getValue());
+                if (masterAddr == null) {
+                    masterAndSlaveMap.get(NO_MASTER_PLACEHOLDER).add(brokerAddrEntry.getValue());
+                } else {
+                    masterAndSlaveMap.get(masterAddr).add(brokerAddrEntry.getValue());
+                }
             }
         }
 
@@ -136,7 +147,7 @@ public class CommandUtil {
 
     public static String fetchBrokerNameByAddr(final MQAdminExt adminExt, final String addr) throws Exception {
         ClusterInfo clusterInfoSerializeWrapper = adminExt.examineBrokerClusterInfo();
-        HashMap<String/* brokerName */, BrokerData> brokerAddrTable = clusterInfoSerializeWrapper.getBrokerAddrTable();
+        Map<String/* brokerName */, BrokerData> brokerAddrTable = clusterInfoSerializeWrapper.getBrokerAddrTable();
         Iterator<Map.Entry<String, BrokerData>> it = brokerAddrTable.entrySet().iterator();
         while (it.hasNext()) {
             Map.Entry<String, BrokerData> entry = it.next();
diff --git a/tools/src/main/java/org/apache/rocketmq/tools/command/MQAdminStartup.java b/tools/src/main/java/org/apache/rocketmq/tools/command/MQAdminStartup.java
index c8bfe66..bc72038 100644
--- a/tools/src/main/java/org/apache/rocketmq/tools/command/MQAdminStartup.java
+++ b/tools/src/main/java/org/apache/rocketmq/tools/command/MQAdminStartup.java
@@ -38,6 +38,7 @@ import org.apache.rocketmq.tools.command.broker.BrokerStatusSubCommand;
 import org.apache.rocketmq.tools.command.broker.CleanExpiredCQSubCommand;
 import org.apache.rocketmq.tools.command.broker.CleanUnusedTopicCommand;
 import org.apache.rocketmq.tools.command.broker.GetBrokerConfigCommand;
+import org.apache.rocketmq.tools.command.broker.ResetMasterFlushOffsetSubCommand;
 import org.apache.rocketmq.tools.command.broker.SendMsgStatusCommand;
 import org.apache.rocketmq.tools.command.broker.UpdateBrokerConfigSubCommand;
 import org.apache.rocketmq.tools.command.cluster.CLusterSendMsgRTCommand;
@@ -51,9 +52,12 @@ import org.apache.rocketmq.tools.command.consumer.GetConsumerConfigSubCommand;
 import org.apache.rocketmq.tools.command.consumer.SetConsumeModeSubCommand;
 import org.apache.rocketmq.tools.command.consumer.StartMonitoringSubCommand;
 import org.apache.rocketmq.tools.command.consumer.UpdateSubGroupSubCommand;
+import org.apache.rocketmq.tools.command.container.AddBrokerSubCommand;
+import org.apache.rocketmq.tools.command.container.RemoveBrokerSubCommand;
 import org.apache.rocketmq.tools.command.export.ExportConfigsCommand;
 import org.apache.rocketmq.tools.command.export.ExportMetadataCommand;
 import org.apache.rocketmq.tools.command.export.ExportMetricsCommand;
+import org.apache.rocketmq.tools.command.ha.HAStatusSubCommand;
 import org.apache.rocketmq.tools.command.message.CheckMsgSendRTCommand;
 import org.apache.rocketmq.tools.command.message.ConsumeMessageCommand;
 import org.apache.rocketmq.tools.command.message.PrintMessageByQueueCommand;
@@ -146,8 +150,11 @@ public class MQAdminStartup {
                             String namesrvAddr = commandLine.getOptionValue('n');
                             System.setProperty(MixAll.NAMESRV_ADDR_PROPERTY, namesrvAddr);
                         }
-
-                        cmd.execute(commandLine, options, AclUtils.getAclRPCHook(rocketmqHome + MixAll.ACL_CONF_TOOLS_FILE));
+                        if (rpcHook != null) {
+                            cmd.execute(commandLine, options, rpcHook);
+                        } else {
+                            cmd.execute(commandLine, options, AclUtils.getAclRPCHook(rocketmqHome + MixAll.ACL_CONF_TOOLS_FILE));
+                        }
                     } else {
                         System.out.printf("The sub command %s not exist.%n", args[0]);
                     }
@@ -171,6 +178,9 @@ public class MQAdminStartup {
         initCommand(new TopicStatusSubCommand());
         initCommand(new TopicClusterSubCommand());
 
+        initCommand(new AddBrokerSubCommand());
+        initCommand(new RemoveBrokerSubCommand());
+        initCommand(new ResetMasterFlushOffsetSubCommand());
         initCommand(new BrokerStatusSubCommand());
         initCommand(new QueryMsgByIdSubCommand());
         initCommand(new QueryMsgByKeySubCommand());
@@ -228,13 +238,14 @@ public class MQAdminStartup {
         initCommand(new UpdateGlobalWhiteAddrSubCommand());
         initCommand(new GetAccessConfigSubCommand());
 
-
         initCommand(new UpdateStaticTopicSubCommand());
         initCommand(new RemappingStaticTopicSubCommand());
 
         initCommand(new ExportMetadataCommand());
         initCommand(new ExportConfigsCommand());
         initCommand(new ExportMetricsCommand());
+
+        initCommand(new HAStatusSubCommand());
     }
 
     private static void initLogback() throws JoranException {
@@ -247,6 +258,7 @@ public class MQAdminStartup {
 
     private static void printHelp() {
         System.out.printf("The most commonly used mqadmin commands are:%n");
+
         for (SubCommand cmd : subCommandList) {
             System.out.printf("   %-20s %s%n", cmd.commandName(), cmd.commandDesc());
         }
diff --git a/tools/src/main/java/org/apache/rocketmq/tools/command/broker/GetBrokerConfigCommand.java b/tools/src/main/java/org/apache/rocketmq/tools/command/broker/GetBrokerConfigCommand.java
index 6bacd3c..b9cfdf9 100644
--- a/tools/src/main/java/org/apache/rocketmq/tools/command/broker/GetBrokerConfigCommand.java
+++ b/tools/src/main/java/org/apache/rocketmq/tools/command/broker/GetBrokerConfigCommand.java
@@ -85,19 +85,28 @@ public class GetBrokerConfigCommand implements SubCommand {
                 Map<String, List<String>> masterAndSlaveMap
                     = CommandUtil.fetchMasterAndSlaveDistinguish(defaultMQAdminExt, clusterName);
 
-                for (Entry<String, List<String>> masterAndSlaveEntry : masterAndSlaveMap.entrySet()) {
+                for (String masterAddr : masterAndSlaveMap.keySet()) {
+
+                    if (masterAddr == null) {
+                        continue;
+                    }
 
                     getAndPrint(
-                            defaultMQAdminExt,
-                            String.format("============Master: %s============\n", masterAndSlaveEntry.getKey()),
-                            masterAndSlaveEntry.getKey()
+                        defaultMQAdminExt,
+                        String.format("============Master: %s============\n", masterAddr),
+                        masterAddr
                     );
-                    for (String slaveAddr : masterAndSlaveEntry.getValue()) {
+
+                    for (String slaveAddr : masterAndSlaveMap.get(masterAddr)) {
+
+                        if (slaveAddr == null) {
+                            continue;
+                        }
 
                         getAndPrint(
-                                defaultMQAdminExt,
-                                String.format("============My Master: %s=====Slave: %s============\n", masterAndSlaveEntry.getKey(), slaveAddr),
-                                slaveAddr
+                            defaultMQAdminExt,
+                            String.format("============My Master: %s=====Slave: %s============\n", masterAddr, slaveAddr),
+                            slaveAddr
                         );
                     }
                 }
@@ -117,6 +126,10 @@ public class GetBrokerConfigCommand implements SubCommand {
 
         System.out.print(printPrefix);
 
+        if (addr.equals(CommandUtil.NO_MASTER_PLACEHOLDER)) {
+            return;
+        }
+
         Properties properties = defaultMQAdminExt.getBrokerConfig(addr);
         if (properties == null) {
             System.out.printf("Broker[%s] has no config property!\n", addr);
diff --git a/tools/src/main/java/org/apache/rocketmq/tools/command/broker/ResetMasterFlushOffsetSubCommand.java b/tools/src/main/java/org/apache/rocketmq/tools/command/broker/ResetMasterFlushOffsetSubCommand.java
new file mode 100644
index 0000000..b2ac48c
--- /dev/null
+++ b/tools/src/main/java/org/apache/rocketmq/tools/command/broker/ResetMasterFlushOffsetSubCommand.java
@@ -0,0 +1,71 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.rocketmq.tools.command.broker;
+
+import org.apache.commons.cli.CommandLine;
+import org.apache.commons.cli.Option;
+import org.apache.commons.cli.Options;
+import org.apache.rocketmq.remoting.RPCHook;
+import org.apache.rocketmq.tools.admin.DefaultMQAdminExt;
+import org.apache.rocketmq.tools.command.SubCommand;
+import org.apache.rocketmq.tools.command.SubCommandException;
+
+public class ResetMasterFlushOffsetSubCommand implements SubCommand {
+    @Override
+    public String commandName() {
+        return "resetMasterFlushOffset";
+    }
+
+    @Override
+    public String commandDesc() {
+        return "Reset master flush offset in slave";
+    }
+
+    @Override
+    public Options buildCommandlineOptions(Options options) {
+        Option opt = new Option("b", "brokerAddr", true, "which broker to reset");
+        opt.setRequired(false);
+        options.addOption(opt);
+
+        opt = new Option("o", "offset", true, "the offset to reset at");
+        opt.setRequired(false);
+        options.addOption(opt);
+
+        return options;
+    }
+
+    @Override
+    public void execute(CommandLine commandLine, Options options,
+        RPCHook rpcHook) throws SubCommandException {
+        DefaultMQAdminExt defaultMQAdminExt = new DefaultMQAdminExt(rpcHook);
+        defaultMQAdminExt.setInstanceName(Long.toString(System.currentTimeMillis()));
+
+        try {
+            defaultMQAdminExt.start();
+            String brokerAddr = commandLine.getOptionValue('b').trim();
+            long masterFlushOffset = Long.parseLong(commandLine.getOptionValue('o').trim());
+
+            defaultMQAdminExt.resetMasterFlushOffset(brokerAddr, masterFlushOffset);
+            System.out.printf("reset master flush offset to %d success%n", masterFlushOffset);
+        } catch (Exception e) {
+            throw new SubCommandException(this.getClass().getSimpleName() + " command failed", e);
+        } finally {
+            defaultMQAdminExt.shutdown();
+        }
+    }
+}
diff --git a/tools/src/main/java/org/apache/rocketmq/tools/command/broker/UpdateBrokerConfigSubCommand.java b/tools/src/main/java/org/apache/rocketmq/tools/command/broker/UpdateBrokerConfigSubCommand.java
index a94fa50..98abeb6 100644
--- a/tools/src/main/java/org/apache/rocketmq/tools/command/broker/UpdateBrokerConfigSubCommand.java
+++ b/tools/src/main/java/org/apache/rocketmq/tools/command/broker/UpdateBrokerConfigSubCommand.java
@@ -50,6 +50,10 @@ public class UpdateBrokerConfigSubCommand implements SubCommand {
         opt.setRequired(false);
         options.addOption(opt);
 
+        opt = new Option("a", "updateAllBroker", true, "update all brokers include slave");
+        opt.setRequired(false);
+        options.addOption(opt);
+
         opt = new Option("k", "key", true, "config key");
         opt.setRequired(true);
         options.addOption(opt);
@@ -87,9 +91,15 @@ public class UpdateBrokerConfigSubCommand implements SubCommand {
 
                 defaultMQAdminExt.start();
 
-                Set<String> masterSet =
-                    CommandUtil.fetchMasterAddrByClusterName(defaultMQAdminExt, clusterName);
-                for (String brokerAddr : masterSet) {
+                Set<String> brokerAddrSet;
+
+                if (commandLine.hasOption('a')) {
+                    brokerAddrSet = CommandUtil.fetchMasterAndSlaveAddrByClusterName(defaultMQAdminExt, clusterName);
+                } else {
+                    brokerAddrSet = CommandUtil.fetchMasterAddrByClusterName(defaultMQAdminExt, clusterName);
+                }
+
+                for (String brokerAddr : brokerAddrSet) {
                     try {
                         defaultMQAdminExt.updateBrokerConfig(brokerAddr, properties);
                         System.out.printf("update broker config success, %s\n", brokerAddr);
diff --git a/tools/src/main/java/org/apache/rocketmq/tools/command/cluster/CLusterSendMsgRTCommand.java b/tools/src/main/java/org/apache/rocketmq/tools/command/cluster/CLusterSendMsgRTCommand.java
index 872a130..0658ba0 100644
--- a/tools/src/main/java/org/apache/rocketmq/tools/command/cluster/CLusterSendMsgRTCommand.java
+++ b/tools/src/main/java/org/apache/rocketmq/tools/command/cluster/CLusterSendMsgRTCommand.java
@@ -20,7 +20,7 @@ package org.apache.rocketmq.tools.command.cluster;
 import java.math.BigDecimal;
 import java.text.SimpleDateFormat;
 import java.util.Date;
-import java.util.HashMap;
+import java.util.Map;
 import java.util.Set;
 import java.util.TimeZone;
 import java.util.TreeSet;
@@ -93,7 +93,7 @@ public class CLusterSendMsgRTCommand implements SubCommand {
             producer.start();
 
             ClusterInfo clusterInfoSerializeWrapper = defaultMQAdminExt.examineBrokerClusterInfo();
-            HashMap<String, Set<String>> clusterAddr = clusterInfoSerializeWrapper
+            Map<String, Set<String>> clusterAddr = clusterInfoSerializeWrapper
                 .getClusterAddrTable();
 
             Set<String> clusterNames = null;
diff --git a/tools/src/main/java/org/apache/rocketmq/tools/command/cluster/ClusterListSubCommand.java b/tools/src/main/java/org/apache/rocketmq/tools/command/cluster/ClusterListSubCommand.java
index 6a0cd71..f34d032 100644
--- a/tools/src/main/java/org/apache/rocketmq/tools/command/cluster/ClusterListSubCommand.java
+++ b/tools/src/main/java/org/apache/rocketmq/tools/command/cluster/ClusterListSubCommand.java
@@ -20,17 +20,15 @@ import java.util.Iterator;
 import java.util.Map;
 import java.util.Set;
 import java.util.TreeSet;
+
 import org.apache.commons.cli.CommandLine;
 import org.apache.commons.cli.Option;
 import org.apache.commons.cli.Options;
-import org.apache.rocketmq.client.exception.MQBrokerException;
+import org.apache.commons.lang3.StringUtils;
 import org.apache.rocketmq.common.protocol.body.ClusterInfo;
 import org.apache.rocketmq.common.protocol.body.KVTable;
 import org.apache.rocketmq.common.protocol.route.BrokerData;
 import org.apache.rocketmq.remoting.RPCHook;
-import org.apache.rocketmq.remoting.exception.RemotingConnectException;
-import org.apache.rocketmq.remoting.exception.RemotingSendRequestException;
-import org.apache.rocketmq.remoting.exception.RemotingTimeoutException;
 import org.apache.rocketmq.tools.admin.DefaultMQAdminExt;
 import org.apache.rocketmq.tools.command.SubCommand;
 import org.apache.rocketmq.tools.command.SubCommandException;
@@ -44,7 +42,7 @@ public class ClusterListSubCommand implements SubCommand {
 
     @Override
     public String commandDesc() {
-        return "List all of clusters";
+        return "List cluster infos";
     }
 
     @Override
@@ -57,6 +55,10 @@ public class ClusterListSubCommand implements SubCommand {
         opt.setRequired(false);
         options.addOption(opt);
 
+        opt = new Option("c", "clusterName", true, "which cluster");
+        opt.setRequired(false);
+        options.addOption(opt);
+
         return options;
     }
 
@@ -74,6 +76,8 @@ public class ClusterListSubCommand implements SubCommand {
             printInterval = Long.parseLong(commandLine.getOptionValue('i')) * 1000;
         }
 
+        String clusterName = commandLine.hasOption('c') ? commandLine.getOptionValue('c').trim() : "";
+
         try {
             defaultMQAdminExt.start();
             long i = 0;
@@ -82,10 +86,15 @@ public class ClusterListSubCommand implements SubCommand {
                 if (i++ > 0) {
                     Thread.sleep(printInterval);
                 }
+
+                ClusterInfo clusterInfo = defaultMQAdminExt.examineBrokerClusterInfo();
+
+                Set<String> clusterNames = getTargetClusterNames(clusterName, clusterInfo);
+
                 if (commandLine.hasOption('m')) {
-                    this.printClusterMoreStats(defaultMQAdminExt);
+                    this.printClusterMoreStats(clusterNames, defaultMQAdminExt, clusterInfo);
                 } else {
-                    this.printClusterBaseInfo(defaultMQAdminExt);
+                    this.printClusterBaseInfo(clusterNames, defaultMQAdminExt, clusterInfo);
                 }
             }
             while (enableInterval);
@@ -96,11 +105,19 @@ public class ClusterListSubCommand implements SubCommand {
         }
     }
 
-    private void printClusterMoreStats(final DefaultMQAdminExt defaultMQAdminExt) throws RemotingConnectException,
-        RemotingTimeoutException, RemotingSendRequestException, InterruptedException, MQBrokerException {
-
-        ClusterInfo clusterInfoSerializeWrapper = defaultMQAdminExt.examineBrokerClusterInfo();
+    private Set<String> getTargetClusterNames(String clusterName, ClusterInfo clusterInfo) {
+        if (StringUtils.isEmpty(clusterName)) {
+            return clusterInfo.getClusterAddrTable().keySet();
+        } else {
+            Set<String> clusterNames = new TreeSet<String>();
+            clusterNames.add(clusterName);
+            return clusterNames;
+        }
+    }
 
+    private void printClusterMoreStats(final Set<String> clusterNames,
+        final DefaultMQAdminExt defaultMQAdminExt,
+        final ClusterInfo clusterInfo) {
         System.out.printf("%-16s  %-32s %14s %14s %14s %14s%n",
             "#Cluster Name",
             "#Broker Name",
@@ -110,17 +127,16 @@ public class ClusterListSubCommand implements SubCommand {
             "#OutTotalToday"
         );
 
-        Iterator<Map.Entry<String, Set<String>>> itCluster = clusterInfoSerializeWrapper.getClusterAddrTable().entrySet().iterator();
-        while (itCluster.hasNext()) {
-            Map.Entry<String, Set<String>> next = itCluster.next();
-            String clusterName = next.getKey();
-            TreeSet<String> brokerNameSet = new TreeSet<String>();
-            brokerNameSet.addAll(next.getValue());
+        for (String clusterName : clusterNames) {
+            TreeSet<String> brokerNameTreeSet = new TreeSet<String>();
+            Set<String> brokerNameSet = clusterInfo.getClusterAddrTable().get(clusterName);
+            if (brokerNameSet != null && !brokerNameSet.isEmpty()) {
+                brokerNameTreeSet.addAll(brokerNameSet);
+            }
 
-            for (String brokerName : brokerNameSet) {
-                BrokerData brokerData = clusterInfoSerializeWrapper.getBrokerAddrTable().get(brokerName);
+            for (String brokerName : brokerNameTreeSet) {
+                BrokerData brokerData = clusterInfo.getBrokerAddrTable().get(brokerName);
                 if (brokerData != null) {
-
                     Iterator<Map.Entry<Long, String>> itAddr = brokerData.getBrokerAddrs().entrySet().iterator();
                     while (itAddr.hasNext()) {
                         Map.Entry<Long, String> next1 = itAddr.next();
@@ -144,7 +160,7 @@ public class ClusterListSubCommand implements SubCommand {
                             inTotalToday = Long.parseLong(msgPutTotalTodayNow) - Long.parseLong(msgPutTotalTodayMorning);
                             outTotalToday = Long.parseLong(msgGetTotalTodayNow) - Long.parseLong(msgGetTotalTodayMorning);
 
-                        } catch (Exception e) {
+                        } catch (Exception ignored) {
                         }
 
                         System.out.printf("%-16s  %-32s %14d %14d %14d %14d%n",
@@ -158,19 +174,12 @@ public class ClusterListSubCommand implements SubCommand {
                     }
                 }
             }
-
-            if (itCluster.hasNext()) {
-                System.out.printf("");
-            }
         }
     }
 
-    private void printClusterBaseInfo(
-        final DefaultMQAdminExt defaultMQAdminExt) throws RemotingConnectException, RemotingTimeoutException,
-        RemotingSendRequestException, InterruptedException, MQBrokerException {
-
-        ClusterInfo clusterInfoSerializeWrapper = defaultMQAdminExt.examineBrokerClusterInfo();
-
+    private void printClusterBaseInfo(final Set<String> clusterNames,
+        final DefaultMQAdminExt defaultMQAdminExt,
+        final ClusterInfo clusterInfo) {
         System.out.printf("%-16s  %-22s  %-4s  %-22s %-16s %19s %19s %10s %5s %6s%n",
             "#Cluster Name",
             "#Broker Name",
@@ -181,20 +190,20 @@ public class ClusterListSubCommand implements SubCommand {
             "#OutTPS(LOAD)",
             "#PCWait(ms)",
             "#Hour",
-            "#SPACE"
+            "#SPACE",
+            "#ACTIVATED"
         );
 
-        Iterator<Map.Entry<String, Set<String>>> itCluster = clusterInfoSerializeWrapper.getClusterAddrTable().entrySet().iterator();
-        while (itCluster.hasNext()) {
-            Map.Entry<String, Set<String>> next = itCluster.next();
-            String clusterName = next.getKey();
-            TreeSet<String> brokerNameSet = new TreeSet<String>();
-            brokerNameSet.addAll(next.getValue());
+        for (String clusterName : clusterNames) {
+            TreeSet<String> brokerNameTreeSet = new TreeSet<String>();
+            Set<String> brokerNameSet = clusterInfo.getClusterAddrTable().get(clusterName);
+            if (brokerNameSet != null && !brokerNameSet.isEmpty()) {
+                brokerNameTreeSet.addAll(brokerNameSet);
+            }
 
-            for (String brokerName : brokerNameSet) {
-                BrokerData brokerData = clusterInfoSerializeWrapper.getBrokerAddrTable().get(brokerName);
+            for (String brokerName : brokerNameTreeSet) {
+                BrokerData brokerData = clusterInfo.getBrokerAddrTable().get(brokerName);
                 if (brokerData != null) {
-
                     Iterator<Map.Entry<Long, String>> itAddr = brokerData.getBrokerAddrs().entrySet().iterator();
                     while (itAddr.hasNext()) {
                         Map.Entry<Long, String> next1 = itAddr.next();
@@ -208,8 +217,10 @@ public class ClusterListSubCommand implements SubCommand {
                         String pageCacheLockTimeMills = "";
                         String earliestMessageTimeStamp = "";
                         String commitLogDiskRatio = "";
+                        boolean isBrokerActive = false;
                         try {
                             KVTable kvTable = defaultMQAdminExt.fetchBrokerRuntimeStats(next1.getValue());
+                            isBrokerActive = Boolean.parseBoolean(kvTable.getTable().get("brokerActive"));
                             String putTps = kvTable.getTable().get("putTps");
                             String getTransferedTps = kvTable.getTable().get("getTransferedTps");
                             sendThreadPoolQueueSize = kvTable.getTable().get("sendThreadPoolQueueSize");
@@ -246,15 +257,15 @@ public class ClusterListSubCommand implements SubCommand {
                         double space = 0.0;
 
                         if (earliestMessageTimeStamp != null && earliestMessageTimeStamp.length() > 0) {
-                            long mills = System.currentTimeMillis() - Long.valueOf(earliestMessageTimeStamp);
+                            long mills = System.currentTimeMillis() - Long.parseLong(earliestMessageTimeStamp);
                             hour = mills / 1000.0 / 60.0 / 60.0;
                         }
 
                         if (commitLogDiskRatio != null && commitLogDiskRatio.length() > 0) {
-                            space = Double.valueOf(commitLogDiskRatio);
+                            space = Double.parseDouble(commitLogDiskRatio);
                         }
 
-                        System.out.printf("%-16s  %-22s  %-4s  %-22s %-16s %19s %19s %10s %5s %6s%n",
+                        System.out.printf("%-16s  %-22s  %-4s  %-22s %-16s %19s %19s %10s %5s %6s %10s%n",
                             clusterName,
                             brokerName,
                             next1.getKey(),
@@ -264,15 +275,12 @@ public class ClusterListSubCommand implements SubCommand {
                             String.format("%9.2f(%s,%sms)", out, pullThreadPoolQueueSize, pullThreadPoolQueueHeadWaitTimeMills),
                             pageCacheLockTimeMills,
                             String.format("%2.2f", hour),
-                            String.format("%.4f", space)
+                            String.format("%.4f", space),
+                            isBrokerActive
                         );
                     }
                 }
             }
-
-            if (itCluster.hasNext()) {
-                System.out.printf("");
-            }
         }
     }
 }
diff --git a/tools/src/main/java/org/apache/rocketmq/tools/command/consumer/ConsumerProgressSubCommand.java b/tools/src/main/java/org/apache/rocketmq/tools/command/consumer/ConsumerProgressSubCommand.java
index 48e0c1b..86464c2 100644
--- a/tools/src/main/java/org/apache/rocketmq/tools/command/consumer/ConsumerProgressSubCommand.java
+++ b/tools/src/main/java/org/apache/rocketmq/tools/command/consumer/ConsumerProgressSubCommand.java
@@ -64,6 +64,10 @@ public class ConsumerProgressSubCommand implements SubCommand {
         opt.setRequired(false);
         options.addOption(opt);
 
+        opt = new Option("t", "topicName", true, "topic name");
+        opt.setRequired(false);
+        options.addOption(opt);
+
         Option optionShowClientIP = new Option("s", "showClientIP", true, "Show Client IP per Queue");
         optionShowClientIP.setRequired(false);
         options.addOption(optionShowClientIP);
@@ -79,12 +83,13 @@ public class ConsumerProgressSubCommand implements SubCommand {
             for (Connection connection : consumerConnection.getConnectionSet()) {
                 String clientId = connection.getClientId();
                 ConsumerRunningInfo consumerRunningInfo = defaultMQAdminExt.getConsumerRunningInfo(groupName, clientId,
-                    false);
+                    false, false);
                 for (MessageQueue messageQueue : consumerRunningInfo.getMqTable().keySet()) {
                     results.put(messageQueue, clientId.split("@")[0]);
                 }
             }
-        } catch (Exception ignore) {
+        } catch (Exception e) {
+            log.error("getMqAllocationsResult error, ", e);
         }
         return results;
     }
@@ -102,7 +107,13 @@ public class ConsumerProgressSubCommand implements SubCommand {
 
             if (commandLine.hasOption('g')) {
                 String consumerGroup = commandLine.getOptionValue('g').trim();
-                ConsumeStats consumeStats = defaultMQAdminExt.examineConsumeStats(consumerGroup);
+                String topicName = commandLine.hasOption('t') ? commandLine.getOptionValue('t').trim() : null;
+                ConsumeStats consumeStats;
+                if (topicName == null) {
+                    consumeStats = defaultMQAdminExt.examineConsumeStats(consumerGroup);
+                } else {
+                    consumeStats = defaultMQAdminExt.examineConsumeStats(consumerGroup, topicName);
+                }
                 List<MessageQueue> mqList = new LinkedList<MessageQueue>();
                 mqList.addAll(consumeStats.getOffsetTable().keySet());
                 Collections.sort(mqList);
diff --git a/tools/src/main/java/org/apache/rocketmq/tools/command/consumer/DeleteSubscriptionGroupCommand.java b/tools/src/main/java/org/apache/rocketmq/tools/command/consumer/DeleteSubscriptionGroupCommand.java
index fb0efeb..44113e3 100644
--- a/tools/src/main/java/org/apache/rocketmq/tools/command/consumer/DeleteSubscriptionGroupCommand.java
+++ b/tools/src/main/java/org/apache/rocketmq/tools/command/consumer/DeleteSubscriptionGroupCommand.java
@@ -27,7 +27,6 @@ import org.apache.rocketmq.tools.admin.DefaultMQAdminExt;
 import org.apache.rocketmq.tools.command.CommandUtil;
 import org.apache.rocketmq.tools.command.SubCommand;
 import org.apache.rocketmq.tools.command.SubCommandException;
-import org.apache.rocketmq.tools.command.topic.DeleteTopicSubCommand;
 
 public class DeleteSubscriptionGroupCommand implements SubCommand {
     @Override
@@ -68,17 +67,19 @@ public class DeleteSubscriptionGroupCommand implements SubCommand {
         try {
             // groupName
             String groupName = commandLine.getOptionValue('g').trim();
-
-            boolean removeOffset = false;
+            boolean cleanOffset = false;
             if (commandLine.hasOption('r')) {
-                removeOffset = Boolean.valueOf(commandLine.getOptionValue("r").trim());
+                try {
+                    cleanOffset = Boolean.valueOf(commandLine.getOptionValue('r').trim());
+                } catch (Exception e) {
+                }
             }
 
             if (commandLine.hasOption('b')) {
                 String addr = commandLine.getOptionValue('b').trim();
                 adminExt.start();
 
-                adminExt.deleteSubscriptionGroup(addr, groupName, removeOffset);
+                adminExt.deleteSubscriptionGroup(addr, groupName, cleanOffset);
                 System.out.printf("delete subscription group [%s] from broker [%s] success.%n", groupName,
                     addr);
 
@@ -89,17 +90,15 @@ public class DeleteSubscriptionGroupCommand implements SubCommand {
 
                 Set<String> masterSet = CommandUtil.fetchMasterAddrByClusterName(adminExt, clusterName);
                 for (String master : masterSet) {
-                    adminExt.deleteSubscriptionGroup(master, groupName, removeOffset);
+                    adminExt.deleteSubscriptionGroup(master, groupName, cleanOffset);
                     System.out.printf(
                         "delete subscription group [%s] from broker [%s] in cluster [%s] success.%n",
                         groupName, master, clusterName);
                 }
 
                 try {
-                    DeleteTopicSubCommand.deleteTopic(adminExt, clusterName, MixAll.RETRY_GROUP_TOPIC_PREFIX
-                        + groupName);
-                    DeleteTopicSubCommand.deleteTopic(adminExt, clusterName, MixAll.DLQ_GROUP_TOPIC_PREFIX
-                        + groupName);
+                    adminExt.deleteTopic(MixAll.RETRY_GROUP_TOPIC_PREFIX + groupName, clusterName);
+                    adminExt.deleteTopic(MixAll.DLQ_GROUP_TOPIC_PREFIX + groupName, clusterName);
                 } catch (Exception e) {
                     e.printStackTrace();
                 }
diff --git a/tools/src/main/java/org/apache/rocketmq/tools/command/consumer/GetConsumerConfigSubCommand.java b/tools/src/main/java/org/apache/rocketmq/tools/command/consumer/GetConsumerConfigSubCommand.java
index 61e3611..4696b4f 100644
--- a/tools/src/main/java/org/apache/rocketmq/tools/command/consumer/GetConsumerConfigSubCommand.java
+++ b/tools/src/main/java/org/apache/rocketmq/tools/command/consumer/GetConsumerConfigSubCommand.java
@@ -134,7 +134,7 @@ class ConsumerConfigInfo {
         return brokerName;
     }
 
-    public void setBrokerName(String brokerNameList) {
+    public void setBrokerName(String brokerName) {
         this.brokerName = brokerName;
     }
 
diff --git a/tools/src/main/java/org/apache/rocketmq/tools/command/container/AddBrokerSubCommand.java b/tools/src/main/java/org/apache/rocketmq/tools/command/container/AddBrokerSubCommand.java
new file mode 100644
index 0000000..12e66ab
--- /dev/null
+++ b/tools/src/main/java/org/apache/rocketmq/tools/command/container/AddBrokerSubCommand.java
@@ -0,0 +1,66 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.rocketmq.tools.command.container;
+
+import org.apache.commons.cli.CommandLine;
+import org.apache.commons.cli.Option;
+import org.apache.commons.cli.Options;
+import org.apache.rocketmq.remoting.RPCHook;
+import org.apache.rocketmq.tools.admin.DefaultMQAdminExt;
+import org.apache.rocketmq.tools.command.SubCommand;
+import org.apache.rocketmq.tools.command.SubCommandException;
+
+public class AddBrokerSubCommand implements SubCommand {
+    @Override public String commandName() {
+        return "addBroker";
+    }
+
+    @Override public String commandDesc() {
+        return "Add a broker to specified container";
+    }
+
+    @Override public Options buildCommandlineOptions(Options options) {
+        Option opt = new Option("c", "brokerContainerAddr", true, "Broker container address");
+        opt.setRequired(true);
+        options.addOption(opt);
+
+        opt = new Option("b", "brokerConfigPath", true, "Broker config path");
+        opt.setRequired(true);
+        options.addOption(opt);
+
+        return options;
+    }
+
+    @Override public void execute(CommandLine commandLine, Options options,
+        RPCHook rpcHook) throws SubCommandException {
+        DefaultMQAdminExt defaultMQAdminExt = new DefaultMQAdminExt(rpcHook);
+        defaultMQAdminExt.setInstanceName(Long.toString(System.currentTimeMillis()));
+
+        try {
+            defaultMQAdminExt.start();
+            String brokerContainerAddr = commandLine.getOptionValue('c').trim();
+            String brokerConfigPath = commandLine.getOptionValue('b').trim();
+            defaultMQAdminExt.addBrokerToContainer(brokerContainerAddr, brokerConfigPath);
+            System.out.printf("add broker to %s success%n", brokerContainerAddr);
+        } catch (Exception e) {
+            throw new SubCommandException(this.getClass().getSimpleName() + " command failed", e);
+        } finally {
+            defaultMQAdminExt.shutdown();
+        }
+    }
+}
diff --git a/tools/src/main/java/org/apache/rocketmq/tools/command/container/RemoveBrokerSubCommand.java b/tools/src/main/java/org/apache/rocketmq/tools/command/container/RemoveBrokerSubCommand.java
new file mode 100644
index 0000000..9907d8a
--- /dev/null
+++ b/tools/src/main/java/org/apache/rocketmq/tools/command/container/RemoveBrokerSubCommand.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.tools.command.container;
+
+import org.apache.commons.cli.CommandLine;
+import org.apache.commons.cli.Option;
+import org.apache.commons.cli.Options;
+import org.apache.rocketmq.remoting.RPCHook;
+import org.apache.rocketmq.tools.admin.DefaultMQAdminExt;
+import org.apache.rocketmq.tools.command.SubCommand;
+import org.apache.rocketmq.tools.command.SubCommandException;
+
+public class RemoveBrokerSubCommand implements SubCommand {
+    @Override public String commandName() {
+        return "removeBroker";
+    }
+
+    @Override public String commandDesc() {
+        return "Remove a broker from specified container";
+    }
+
+    @Override public Options buildCommandlineOptions(Options options) {
+        Option opt = new Option("c", "brokerContainerAddr", true, "Broker container address");
+        opt.setRequired(true);
+        options.addOption(opt);
+
+        opt = new Option("b", "brokerIdentity", true, "Information to identify a broker: clusterName:brokerName:brokerId");
+        opt.setRequired(true);
+        options.addOption(opt);
+
+        return options;
+    }
+
+    @Override public void execute(CommandLine commandLine, Options options,
+        RPCHook rpcHook) throws SubCommandException {
+        DefaultMQAdminExt defaultMQAdminExt = new DefaultMQAdminExt(rpcHook);
+        defaultMQAdminExt.setInstanceName(Long.toString(System.currentTimeMillis()));
+
+        try {
+            defaultMQAdminExt.start();
+            String brokerContainerAddr = commandLine.getOptionValue('c').trim();
+            String[] brokerIdentities = commandLine.getOptionValue('b').trim().split(":");
+            String clusterName = brokerIdentities[0].trim();
+            String brokerName = brokerIdentities[1].trim();
+            long brokerId;
+            try {
+                brokerId = Long.parseLong(brokerIdentities[2].trim());
+            } catch (NumberFormatException e) {
+                e.printStackTrace();
+                return;
+            }
+            if (brokerId < 0) {
+                System.out.printf("brokerId can't be negative%n");
+                return;
+            }
+            defaultMQAdminExt.removeBrokerFromContainer(brokerContainerAddr, clusterName, brokerName, brokerId);
+            System.out.printf("remove broker from %s success%n", brokerContainerAddr);
+        } catch (Exception e) {
+            throw new SubCommandException(this.getClass().getSimpleName() + " command failed", e);
+        } finally {
+            defaultMQAdminExt.shutdown();
+        }
+    }
+}
diff --git a/tools/src/main/java/org/apache/rocketmq/tools/command/ha/HAStatusSubCommand.java b/tools/src/main/java/org/apache/rocketmq/tools/command/ha/HAStatusSubCommand.java
new file mode 100644
index 0000000..228c67d
--- /dev/null
+++ b/tools/src/main/java/org/apache/rocketmq/tools/command/ha/HAStatusSubCommand.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.tools.command.ha;
+
+import java.util.Set;
+
+import org.apache.commons.cli.CommandLine;
+import org.apache.commons.cli.Option;
+import org.apache.commons.cli.Options;
+import org.apache.rocketmq.common.UtilAll;
+import org.apache.rocketmq.common.protocol.body.HARuntimeInfo;
+import org.apache.rocketmq.common.protocol.body.HARuntimeInfo.HAClientRuntimeInfo;
+import org.apache.rocketmq.common.protocol.body.HARuntimeInfo.HAConnectionRuntimeInfo;
+import org.apache.rocketmq.remoting.RPCHook;
+import org.apache.rocketmq.srvutil.ServerUtil;
+import org.apache.rocketmq.tools.admin.DefaultMQAdminExt;
+import org.apache.rocketmq.tools.command.CommandUtil;
+import org.apache.rocketmq.tools.command.SubCommand;
+import org.apache.rocketmq.tools.command.SubCommandException;
+
+public class HAStatusSubCommand implements SubCommand {
+
+    @Override
+    public String commandName() {
+        return "haStatus";
+    }
+
+    @Override
+    public String commandDesc() {
+        return "Fetch ha runtime status data";
+    }
+
+    @Override
+    public Options buildCommandlineOptions(Options options) {
+        Option opt = new Option("c", "clusterName", true, "which cluster");
+        opt.setRequired(false);
+        options.addOption(opt);
+
+        opt = new Option("b", "brokerAddr", true, "which broker to fetch");
+        opt.setRequired(false);
+        options.addOption(opt);
+
+        opt = new Option("f", "follow", true, "the interval(second) of get info");
+        opt.setRequired(false);
+        options.addOption(opt);
+
+        return options;
+    }
+
+    @Override
+    public void execute(CommandLine commandLine, Options options, RPCHook rpcHook) throws SubCommandException {
+        DefaultMQAdminExt defaultMQAdminExt = new DefaultMQAdminExt(rpcHook);
+        defaultMQAdminExt.setInstanceName(Long.toString(System.currentTimeMillis()));
+
+        try {
+            if (commandLine.hasOption('f')) {
+                String flushSecondStr = commandLine.getOptionValue('f');
+                int flushSecond = 3;
+                if (flushSecondStr != null && !flushSecondStr.trim().equals("")) {
+                    flushSecond = Integer.parseInt(flushSecondStr);
+                }
+
+                defaultMQAdminExt.start();
+
+                while (true) {
+                    this.innerExec(commandLine, options, defaultMQAdminExt);
+                    Thread.sleep(flushSecond * 1000);
+                }
+            } else {
+                defaultMQAdminExt.start();
+
+                this.innerExec(commandLine, options, defaultMQAdminExt);
+            }
+        } catch (Exception e) {
+            throw new SubCommandException(this.getClass().getSimpleName() + " command failed", e);
+        } finally {
+            defaultMQAdminExt.shutdown();
+        }
+    }
+
+    private void innerExec(CommandLine commandLine, Options options,
+        DefaultMQAdminExt defaultMQAdminExt) throws Exception {
+        if (commandLine.hasOption('b')) {
+            String addr = commandLine.getOptionValue('b').trim();
+            this.printStatus(addr, defaultMQAdminExt);
+        } else if (commandLine.hasOption('c')) {
+
+            String clusterName = commandLine.getOptionValue('c').trim();
+            Set<String> masterSet = CommandUtil.fetchMasterAddrByClusterName(defaultMQAdminExt, clusterName);
+
+            for (String addr : masterSet) {
+                this.printStatus(addr, defaultMQAdminExt);
+            }
+        } else {
+            ServerUtil.printCommandLineHelp("mqadmin " + this.commandName(), options);
+        }
+
+    }
+
+    private void printStatus(String brokerAddr, DefaultMQAdminExt defaultMQAdminExt) throws Exception {
+        HARuntimeInfo haRuntimeInfo = defaultMQAdminExt.getBrokerHAStatus(brokerAddr);
+
+        if (haRuntimeInfo.isMaster()) {
+            System.out.printf("\n#MasterAddr\t%s\n#MasterCommitLogMaxOffset\t%d\n#SlaveNum\t%d\n#InSyncSlaveNum\t%d\n", brokerAddr,
+                haRuntimeInfo.getMasterCommitLogMaxOffset(), haRuntimeInfo.getHaConnectionInfo().size(), haRuntimeInfo.getInSyncSlaveNums());
+            System.out.printf("%-32s  %-16s %16s %16s %16s %16s\n",
+                "#SlaveAddr",
+                "#SlaveAckOffset",
+                "#Diff",
+                "#TransferSpeed(KB/s)",
+                "#Status",
+                "#TransferFromWhere"
+            );
+
+            for (HAConnectionRuntimeInfo cInfo : haRuntimeInfo.getHaConnectionInfo()) {
+                System.out.printf("%-32s  %-16d %16d %16.2f %16s %16d\n",
+                    cInfo.getAddr(),
+                    cInfo.getSlaveAckOffset(),
+                    cInfo.getDiff(),
+                    cInfo.getTransferredByteInSecond() / 1024.0,
+                    cInfo.isInSync() ? "OK" : "Fall Behind",
+                    cInfo.getTransferFromWhere());
+            }
+        } else {
+            HAClientRuntimeInfo haClientRuntimeInfo = haRuntimeInfo.getHaClientRuntimeInfo();
+
+            System.out.printf("\n#MasterAddr\t%s\n", haClientRuntimeInfo.getMasterAddr());
+            System.out.printf("#CommitLogMaxOffset\t%d\n", haClientRuntimeInfo.getMaxOffset());
+            System.out.printf("#TransferSpeed(KB/s)\t%.2f\n", haClientRuntimeInfo.getTransferredByteInSecond() / 1024.0);
+            System.out.printf("#LastReadTime\t%s\n", UtilAll.timeMillisToHumanString2(haClientRuntimeInfo.getLastReadTimestamp()));
+            System.out.printf("#LastWriteTime\t%s\n", UtilAll.timeMillisToHumanString2(haClientRuntimeInfo.getLastWriteTimestamp()));
+            System.out.printf("#MasterFlushOffset\t%s\n", haClientRuntimeInfo.getMasterFlushOffset());
+        }
+    }
+
+}
diff --git a/tools/src/main/java/org/apache/rocketmq/tools/command/offset/ResetOffsetByTimeCommand.java b/tools/src/main/java/org/apache/rocketmq/tools/command/offset/ResetOffsetByTimeCommand.java
index 266de75..6e4e570 100644
--- a/tools/src/main/java/org/apache/rocketmq/tools/command/offset/ResetOffsetByTimeCommand.java
+++ b/tools/src/main/java/org/apache/rocketmq/tools/command/offset/ResetOffsetByTimeCommand.java
@@ -64,6 +64,15 @@ public class ResetOffsetByTimeCommand implements SubCommand {
         opt = new Option("c", "cplus", false, "reset c++ client offset");
         opt.setRequired(false);
         options.addOption(opt);
+
+        opt = new Option("b", "broker", true, "broker addr");
+        opt.setRequired(false);
+        options.addOption(opt);
+
+        opt = new Option("q", "queue", true, "queue id");
+        opt.setRequired(false);
+        options.addOption(opt);
+
         return options;
     }
 
@@ -96,7 +105,32 @@ public class ResetOffsetByTimeCommand implements SubCommand {
                 isC = true;
             }
 
+            String brokerAddr = null;
+            if (commandLine.hasOption('b')) {
+                brokerAddr = commandLine.getOptionValue("b");
+            }
+            int queueId = -1;
+            if (commandLine.hasOption("q")) {
+                queueId = Integer.valueOf(commandLine.getOptionValue('q'));
+            }
+
             defaultMQAdminExt.start();
+
+            if (brokerAddr != null && queueId > -1) {
+                System.out.printf("rollback consumer offset by specified group[%s], topic[%s], queueId[%s], broker[%s], timestamp(string)[%s], timestamp(long)[%s]%n",
+                        group, topic, queueId, brokerAddr, timeStampStr, timestamp);
+                try {
+                    long resetOffset = defaultMQAdminExt.searchOffset(brokerAddr, topic, queueId, timestamp, 3000);
+                    System.out.printf("Rollback Offset is: %s", resetOffset);
+                    if (resetOffset > 0) {
+                        defaultMQAdminExt.resetOffsetByQueueId(brokerAddr, group, topic, queueId, resetOffset);
+                    }
+                } catch (Throwable e) {
+                    throw e;
+                }
+                return;
+            }
+
             Map<MessageQueue, Long> offsetTable;
             try {
                 offsetTable = defaultMQAdminExt.resetOffsetByTimestamp(topic, group, timestamp, force, isC);
diff --git a/tools/src/test/java/org/apache/rocketmq/tools/admin/DefaultMQAdminExtTest.java b/tools/src/test/java/org/apache/rocketmq/tools/admin/DefaultMQAdminExtTest.java
index a7fd514..59df145 100644
--- a/tools/src/test/java/org/apache/rocketmq/tools/admin/DefaultMQAdminExtTest.java
+++ b/tools/src/test/java/org/apache/rocketmq/tools/admin/DefaultMQAdminExtTest.java
@@ -283,7 +283,7 @@ public class DefaultMQAdminExtTest {
     @Test
     public void testExamineBrokerClusterInfo() throws InterruptedException, MQBrokerException, RemotingTimeoutException, RemotingSendRequestException, RemotingConnectException {
         ClusterInfo clusterInfo = defaultMQAdminExt.examineBrokerClusterInfo();
-        HashMap<String, BrokerData> brokerList = clusterInfo.getBrokerAddrTable();
+        Map<String, BrokerData> brokerList = clusterInfo.getBrokerAddrTable();
         assertThat(brokerList.get("default-broker").getBrokerName()).isEqualTo("default-broker");
         assertThat(brokerList.containsKey("broker-test")).isTrue();
 
@@ -294,7 +294,7 @@ public class DefaultMQAdminExtTest {
         clusterMap.put("default-cluster", brokers);
         ClusterInfo cInfo = mock(ClusterInfo.class);
         when(cInfo.getClusterAddrTable()).thenReturn(clusterMap);
-        HashMap<String, Set<String>> clusterAddress = cInfo.getClusterAddrTable();
+        Map<String, Set<String>> clusterAddress = cInfo.getClusterAddrTable();
         assertThat(clusterAddress.containsKey("default-cluster")).isTrue();
         assertThat(clusterAddress.get("default-cluster").size()).isEqualTo(2);
     }
diff --git a/tools/src/test/java/org/apache/rocketmq/tools/command/CommandUtilTest.java b/tools/src/test/java/org/apache/rocketmq/tools/command/CommandUtilTest.java
index b556e5c..a7b2143 100644
--- a/tools/src/test/java/org/apache/rocketmq/tools/command/CommandUtilTest.java
+++ b/tools/src/test/java/org/apache/rocketmq/tools/command/CommandUtilTest.java
@@ -92,7 +92,7 @@ public class CommandUtilTest {
     @Test
     public void testFetchMasterAndSlaveDistinguish() throws InterruptedException, MQBrokerException, RemotingTimeoutException, RemotingSendRequestException, RemotingConnectException {
         Map<String, List<String>> result = CommandUtil.fetchMasterAndSlaveDistinguish(defaultMQAdminExtImpl, "default-cluster");
-        assertThat(result.get(null).get(0)).isEqualTo("127.0.0.1:10911");
+        assertThat(result.get(CommandUtil.NO_MASTER_PLACEHOLDER).get(0)).isEqualTo("127.0.0.1:10911");
     }
 
     @Test
diff --git a/tools/src/test/java/org/apache/rocketmq/tools/command/message/QueryMsgByUniqueKeySubCommandTest.java b/tools/src/test/java/org/apache/rocketmq/tools/command/message/QueryMsgByUniqueKeySubCommandTest.java
index e21a66f..1298c91 100644
--- a/tools/src/test/java/org/apache/rocketmq/tools/command/message/QueryMsgByUniqueKeySubCommandTest.java
+++ b/tools/src/test/java/org/apache/rocketmq/tools/command/message/QueryMsgByUniqueKeySubCommandTest.java
@@ -89,7 +89,6 @@ public class QueryMsgByUniqueKeySubCommandTest {
         field.setAccessible(true);
         field.set(mqClientInstance, mQAdminImpl);
 
-
         field = DefaultMQAdminExt.class.getDeclaredField("defaultMQAdminExtImpl");
         field.setAccessible(true);
         field.set(defaultMQAdminExt, defaultMQAdminExtImpl);
@@ -97,7 +96,7 @@ public class QueryMsgByUniqueKeySubCommandTest {
         ConsumeMessageDirectlyResult result = new ConsumeMessageDirectlyResult();
         result.setConsumeResult(CMResult.CR_SUCCESS);
         result.setRemark("customRemark_122333444");
-        when(mQClientAPIImpl.consumeMessageDirectly(anyString(), anyString(), anyString(), anyString(), anyLong())).thenReturn(result);
+        when(mQClientAPIImpl.consumeMessageDirectly(anyString(), anyString(), anyString(), anyString(), anyString(), anyLong())).thenReturn(result);
 
         MessageExt retMsgExt = new MessageExt();
         retMsgExt.setMsgId("0A3A54F7BF7D18B4AAC28A3FA2CF0000");
@@ -135,9 +134,8 @@ public class QueryMsgByUniqueKeySubCommandTest {
         groupList.setGroupList(groupSets);
         when(mQClientAPIImpl.queryTopicConsumeByWho(anyString(), anyString(), anyLong())).thenReturn(groupList);
 
-
         ConsumeStats consumeStats = new ConsumeStats();
-        consumeStats.setConsumeTps(100*10000);
+        consumeStats.setConsumeTps(100 * 10000);
         HashMap<MessageQueue, OffsetWrapper> offsetTable = new HashMap<MessageQueue, OffsetWrapper>();
         MessageQueue messageQueue = new MessageQueue();
         messageQueue.setBrokerName("messageQueue BrokerName testing");
@@ -149,7 +147,7 @@ public class QueryMsgByUniqueKeySubCommandTest {
         offsetWrapper.setLastTimestamp(System.currentTimeMillis());
         offsetTable.put(messageQueue, offsetWrapper);
         consumeStats.setOffsetTable(offsetTable);
-        when(mQClientAPIImpl.getConsumeStats(anyString(), anyString(), (String)isNull(), anyLong())).thenReturn(consumeStats);
+        when(mQClientAPIImpl.getConsumeStats(anyString(), anyString(), (String) isNull(), anyLong())).thenReturn(consumeStats);
 
         ClusterInfo clusterInfo = new ClusterInfo();
         HashMap<String, BrokerData> brokerAddrTable = new HashMap<String, BrokerData>();
@@ -185,7 +183,7 @@ public class QueryMsgByUniqueKeySubCommandTest {
 
         Options options = ServerUtil.buildCommandlineOptions(new Options());
 
-        String[] args = new String[]{"-t myTopicTest", "-i msgId"};
+        String[] args = new String[] {"-t myTopicTest", "-i msgId"};
         CommandLine commandLine = ServerUtil.parseCmdLine("mqadmin ", args, cmd.buildCommandlineOptions(options), new PosixParser());
         cmd.execute(commandLine, options, null);
 
@@ -208,12 +206,10 @@ public class QueryMsgByUniqueKeySubCommandTest {
 
         Options options = ServerUtil.buildCommandlineOptions(new Options());
 
-        String[] args = new String[]{"-t myTopicTest", "-i 7F000001000004D20000000000000066"};
+        String[] args = new String[] {"-t myTopicTest", "-i 7F000001000004D20000000000000066"};
         CommandLine commandLine = ServerUtil.parseCmdLine("mqadmin ", args, cmd.buildCommandlineOptions(options), new PosixParser());
         cmd.execute(commandLine, options, null);
 
-
-
     }
 
     @Test
@@ -221,7 +217,7 @@ public class QueryMsgByUniqueKeySubCommandTest {
 
         Options options = ServerUtil.buildCommandlineOptions(new Options());
 
-        String[] args = new String[]{"-t myTopicTest", "-i 0A3A54F7BF7D18B4AAC28A3FA2CF0000", "-g producerGroupName", "-d clientId"};
+        String[] args = new String[] {"-t myTopicTest", "-i 0A3A54F7BF7D18B4AAC28A3FA2CF0000", "-g producerGroupName", "-d clientId"};
         CommandLine commandLine = ServerUtil.parseCmdLine("mqadmin ", args, cmd.buildCommandlineOptions(options), new PosixParser());
         cmd.execute(commandLine, options, null);
 
@@ -241,7 +237,7 @@ public class QueryMsgByUniqueKeySubCommandTest {
         CommandLine commandLine = ServerUtil.parseCmdLine("mqadmin ", args, cmd.buildCommandlineOptions(options), new PosixParser());
         cmd.execute(commandLine, options, null);
 
-        args = new String[]{"-t myTopicTest", "-i 0A3A54F7BF7D18B4AAC28A3FA2CF0000", "-g producerGroupName", "-d clientId"};
+        args = new String[] {"-t myTopicTest", "-i 0A3A54F7BF7D18B4AAC28A3FA2CF0000", "-g producerGroupName", "-d clientId"};
         commandLine = ServerUtil.parseCmdLine("mqadmin ", args, cmd.buildCommandlineOptions(options), new PosixParser());
         cmd.execute(commandLine, options, null);
 

[rocketmq] 05/07: feature(store):[RIP-34]Support quorum write and adaptive degradation in master slave architecture (#3979)

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

jinrongtong pushed a commit to branch 5.0.0-beta-tmp
in repository https://gitbox.apache.org/repos/asf/rocketmq.git

commit 5c350af174c864992d3611e8432075762c2d3bd4
Author: rongtong <ji...@163.com>
AuthorDate: Mon Mar 14 14:07:02 2022 +0800

    feature(store):[RIP-34]Support quorum write and adaptive degradation in master slave architecture (#3979)
---
 .../rocketmq/store/AllocateMappedFileService.java  |    7 +-
 .../rocketmq/store/AppendMessageCallback.java      |    2 +
 .../java/org/apache/rocketmq/store/CommitLog.java  |  500 ++++++---
 .../org/apache/rocketmq/store/ConsumeQueue.java    |   50 +-
 .../org/apache/rocketmq/store/ConsumeQueueExt.java |    4 +
 .../apache/rocketmq/store/DefaultMessageStore.java | 1060 +++++++++++++-------
 .../apache/rocketmq/store/GetMessageResult.java    |    6 +-
 .../org/apache/rocketmq/store/MappedFileQueue.java |   83 +-
 .../org/apache/rocketmq/store/MessageStore.java    |  255 ++++-
 .../rocketmq/store/MultiPathMappedFileQueue.java   |    5 +-
 .../apache/rocketmq/store/PutMessageResult.java    |   26 +-
 .../apache/rocketmq/store/PutMessageStatus.java    |    6 +-
 .../apache/rocketmq/store/QueryMessageResult.java  |    4 +
 ...sageStatus.java => SelectMappedFileResult.java} |   37 +-
 .../org/apache/rocketmq/store/StoreCheckpoint.java |   18 +-
 .../apache/rocketmq/store/StoreStatsService.java   |   11 +
 .../rocketmq/store/config/MessageStoreConfig.java  |  217 +++-
 .../rocketmq/store/dledger/DLedgerCommitLog.java   |   78 +-
 .../apache/rocketmq/store/ha/DefaultHAClient.java  |  392 ++++++++
 ...{HAConnection.java => DefaultHAConnection.java} |  164 +--
 .../apache/rocketmq/store/ha/DefaultHAService.java |  339 +++++++
 .../org/apache/rocketmq/store/ha/FlowMonitor.java  |   76 ++
 .../rocketmq/store/ha/GroupTransferService.java    |  140 +++
 .../org/apache/rocketmq/store/ha/HAClient.java     |  104 ++
 .../org/apache/rocketmq/store/ha/HAConnection.java |  436 ++------
 .../HAConnectionState.java}                        |   36 +-
 .../ha/HAConnectionStateNotificationRequest.java   |   49 +
 .../ha/HAConnectionStateNotificationService.java   |  150 +++
 .../org/apache/rocketmq/store/ha/HAService.java    |  666 ++----------
 .../apache/rocketmq/store/ha/WaitNotifyObject.java |   67 +-
 .../PutMessageHook.java}                           |   32 +-
 .../SendMessageBackHook.java}                      |   28 +-
 .../org/apache/rocketmq/store/index/IndexFile.java |   22 +-
 .../apache/rocketmq/store/index/IndexService.java  |   79 +-
 .../rocketmq/store/logfile/DefaultMappedFile.java  |  114 +--
 .../apache/rocketmq/store/logfile/MappedFile.java  |   14 +-
 .../java/org/apache/rocketmq/store/pop/AckMsg.java |   10 +
 .../apache/rocketmq/store/pop/PopCheckPoint.java   |   12 +-
 .../rocketmq/store/queue/BatchConsumeQueue.java    |    7 +-
 .../store/queue/ConsumeQueueInterface.java         |    6 +
 .../rocketmq/store/queue/ConsumeQueueStore.java    |  144 ++-
 .../rocketmq/store/queue/FileQueueLifeCycle.java   |    2 +-
 .../rocketmq/store/queue/QueueOffsetAssigner.java  |   24 +-
 .../rocketmq/store/stats/BrokerStatsManager.java   |  431 +++++++-
 .../apache/rocketmq/store/AppendCallbackTest.java  |    4 +-
 .../apache/rocketmq/store/BatchPutMessageTest.java |   43 +-
 .../apache/rocketmq/store/ConsumeQueueTest.java    |    1 +
 .../store/DefaultMessageStoreCleanFilesTest.java   |    6 +-
 .../rocketmq/store/DefaultMessageStoreTest.java    |  188 +++-
 .../java/org/apache/rocketmq/store/HATest.java     |   55 +-
 .../apache/rocketmq/store/MultiDispatchTest.java   |    7 +-
 .../store/MultiPathMappedFileQueueTest.java        |   31 +-
 .../rocketmq/store/StoreStatsServiceTest.java      |    1 -
 .../org/apache/rocketmq/store/StoreTestBase.java   |    4 +-
 .../store/dledger/DLedgerCommitlogTest.java        |   13 +-
 .../store/dledger/MessageStoreTestBase.java        |    3 +-
 .../rocketmq/store/dledger/MixCommitlogTest.java   |    2 -
 .../apache/rocketmq/store/ha/FlowMonitorTest.java  |   61 ++
 .../org/apache/rocketmq/store/ha/HAClientTest.java |   72 ++
 .../org/apache/rocketmq/store/ha/HAServerTest.java |  295 ++++++
 .../rocketmq/store/ha/WaitNotifyObjectTest.java    |    2 -
 .../store/queue/BatchConsumeMessageTest.java       |   32 +-
 .../apache/rocketmq/store/queue/QueueTestBase.java |    3 +-
 63 files changed, 4666 insertions(+), 2070 deletions(-)

diff --git a/store/src/main/java/org/apache/rocketmq/store/AllocateMappedFileService.java b/store/src/main/java/org/apache/rocketmq/store/AllocateMappedFileService.java
index 847e99d..47ee11e 100644
--- a/store/src/main/java/org/apache/rocketmq/store/AllocateMappedFileService.java
+++ b/store/src/main/java/org/apache/rocketmq/store/AllocateMappedFileService.java
@@ -44,9 +44,9 @@ public class AllocateMappedFileService extends ServiceThread {
     private PriorityBlockingQueue<AllocateRequest> requestQueue =
         new PriorityBlockingQueue<AllocateRequest>();
     private volatile boolean hasException = false;
-    private MessageStore messageStore;
+    private DefaultMessageStore messageStore;
 
-    public AllocateMappedFileService(MessageStore messageStore) {
+    public AllocateMappedFileService(DefaultMessageStore messageStore) {
         this.messageStore = messageStore;
     }
 
@@ -121,6 +121,9 @@ public class AllocateMappedFileService extends ServiceThread {
 
     @Override
     public String getServiceName() {
+        if (messageStore != null && messageStore.getBrokerConfig().isInBrokerContainer()) {
+            return messageStore.getBrokerConfig().getLoggerIdentifier() + AllocateMappedFileService.class.getSimpleName();
+        }
         return AllocateMappedFileService.class.getSimpleName();
     }
 
diff --git a/store/src/main/java/org/apache/rocketmq/store/AppendMessageCallback.java b/store/src/main/java/org/apache/rocketmq/store/AppendMessageCallback.java
index accd4e2..ad0a526 100644
--- a/store/src/main/java/org/apache/rocketmq/store/AppendMessageCallback.java
+++ b/store/src/main/java/org/apache/rocketmq/store/AppendMessageCallback.java
@@ -17,6 +17,8 @@
 package org.apache.rocketmq.store;
 
 import java.nio.ByteBuffer;
+import org.apache.rocketmq.common.message.MessageExtBatch;
+import org.apache.rocketmq.common.message.MessageExtBrokerInner;
 
 /**
  * Write messages callback interface
diff --git a/store/src/main/java/org/apache/rocketmq/store/CommitLog.java b/store/src/main/java/org/apache/rocketmq/store/CommitLog.java
index 711b314..383e2cc 100644
--- a/store/src/main/java/org/apache/rocketmq/store/CommitLog.java
+++ b/store/src/main/java/org/apache/rocketmq/store/CommitLog.java
@@ -35,14 +35,16 @@ import java.util.concurrent.TimeUnit;
 import java.util.concurrent.TimeoutException;
 import java.util.function.Supplier;
 
+import org.apache.rocketmq.common.MixAll;
 import org.apache.rocketmq.common.ServiceThread;
 import org.apache.rocketmq.common.TopicConfig;
 import org.apache.rocketmq.common.UtilAll;
 import org.apache.rocketmq.common.constant.LoggerName;
-import org.apache.rocketmq.common.message.MessageAccessor;
 import org.apache.rocketmq.common.message.MessageConst;
 import org.apache.rocketmq.common.message.MessageDecoder;
 import org.apache.rocketmq.common.message.MessageExt;
+import org.apache.rocketmq.common.message.MessageExtBatch;
+import org.apache.rocketmq.common.message.MessageExtBrokerInner;
 import org.apache.rocketmq.common.sysflag.MessageSysFlag;
 import org.apache.rocketmq.common.topic.TopicValidator;
 import org.apache.rocketmq.common.utils.QueueTypeUtils;
@@ -50,11 +52,9 @@ import org.apache.rocketmq.logging.InternalLogger;
 import org.apache.rocketmq.logging.InternalLoggerFactory;
 import org.apache.rocketmq.store.config.BrokerRole;
 import org.apache.rocketmq.store.config.FlushDiskType;
-import org.apache.rocketmq.store.config.MessageStoreConfig;
 import org.apache.rocketmq.store.ha.HAService;
 import org.apache.rocketmq.store.logfile.MappedFile;
 import org.apache.rocketmq.common.attribute.CQType;
-import org.apache.rocketmq.store.schedule.ScheduleMessageService;
 
 /**
  * Store all metadata downtime for recovery, data protection reliability
@@ -64,9 +64,9 @@ public class CommitLog implements Swappable {
     public final static int MESSAGE_MAGIC_CODE = -626843481;
     protected static final InternalLogger log = InternalLoggerFactory.getLogger(LoggerName.STORE_LOGGER_NAME);
     // End of file empty MAGIC CODE cbd43194
-    protected final static int BLANK_MAGIC_CODE = -875286124;
+    public final static int BLANK_MAGIC_CODE = -875286124;
     protected final MappedFileQueue mappedFileQueue;
-    protected final MessageStore defaultMessageStore;
+    protected final DefaultMessageStore defaultMessageStore;
 
     private final FlushManager flushManager;
 
@@ -87,36 +87,36 @@ public class CommitLog implements Swappable {
 
     protected int commitLogSize;
 
-    public CommitLog(final DefaultMessageStore defaultMessageStore) {
-        String storePath = defaultMessageStore.getMessageStoreConfig().getStorePathCommitLog();
-        if (storePath.contains(MessageStoreConfig.MULTI_PATH_SPLITTER)) {
-            this.mappedFileQueue = new MultiPathMappedFileQueue(defaultMessageStore.getMessageStoreConfig(),
-                defaultMessageStore.getMessageStoreConfig().getMappedFileSizeCommitLog(),
-                defaultMessageStore.getAllocateMappedFileService(), this::getFullStorePaths);
+    public CommitLog(final DefaultMessageStore messageStore) {
+        String storePath = messageStore.getMessageStoreConfig().getStorePathCommitLog();
+        if (storePath.contains(MixAll.MULTI_PATH_SPLITTER)) {
+            this.mappedFileQueue = new MultiPathMappedFileQueue(messageStore.getMessageStoreConfig(),
+                messageStore.getMessageStoreConfig().getMappedFileSizeCommitLog(),
+                messageStore.getAllocateMappedFileService(), this::getFullStorePaths);
         } else {
             this.mappedFileQueue = new MappedFileQueue(storePath,
-                defaultMessageStore.getMessageStoreConfig().getMappedFileSizeCommitLog(),
-                defaultMessageStore.getAllocateMappedFileService());
+                messageStore.getMessageStoreConfig().getMappedFileSizeCommitLog(),
+                messageStore.getAllocateMappedFileService());
         }
 
-        this.defaultMessageStore = defaultMessageStore;
+        this.defaultMessageStore = messageStore;
 
         this.flushManager = new DefaultFlushManager();
 
-        this.appendMessageCallback = new DefaultAppendMessageCallback(defaultMessageStore.getMessageStoreConfig().getMaxMessageSize());
+        this.appendMessageCallback = new DefaultAppendMessageCallback(messageStore.getMessageStoreConfig().getMaxMessageSize());
         putMessageThreadLocal = new ThreadLocal<PutMessageThreadLocal>() {
             @Override
             protected PutMessageThreadLocal initialValue() {
                 return new PutMessageThreadLocal(defaultMessageStore.getMessageStoreConfig().getMaxMessageSize());
             }
         };
-        this.putMessageLock = defaultMessageStore.getMessageStoreConfig().isUseReentrantLockWhenPutMessage() ? new PutMessageReentrantLock() : new PutMessageSpinLock();
+        this.putMessageLock = messageStore.getMessageStoreConfig().isUseReentrantLockWhenPutMessage() ? new PutMessageReentrantLock() : new PutMessageSpinLock();
 
-        flushDiskWatcher = new FlushDiskWatcher();
+        this.flushDiskWatcher = new FlushDiskWatcher();
 
         this.topicQueueLock = new TopicQueueLock();
 
-        this.commitLogSize = defaultMessageStore.getMessageStoreConfig().getMappedFileSizeCommitLog();
+        this.commitLogSize = messageStore.getMessageStoreConfig().getMappedFileSizeCommitLog();
     }
 
     public void setFullStorePaths(Set<String> fullStorePaths) {
@@ -127,8 +127,8 @@ public class CommitLog implements Swappable {
         return fullStorePaths;
     }
 
-    public ThreadLocal<PutMessageThreadLocal> getPutMessageThreadLocal() {
-        return putMessageThreadLocal;
+    public long getTotalSize() {
+        return this.mappedFileQueue.getTotalFileSize();
     }
 
     public boolean load() {
@@ -141,6 +141,8 @@ public class CommitLog implements Swappable {
     public void start() {
         this.flushManager.start();
         log.info("start commitLog successfully. storeRoot: {}", this.defaultMessageStore.getMessageStoreConfig().getStorePathRootDir());
+        this.flushManager.start();
+        log.info("start commitLog successfully. storeRoot: {}", this.defaultMessageStore.getMessageStoreConfig().getStorePathRootDir());
         flushDiskWatcher.setDaemon(true);
         flushDiskWatcher.start();
     }
@@ -148,6 +150,8 @@ public class CommitLog implements Swappable {
     public void shutdown() {
         this.flushManager.shutdown();
         log.info("shutdown commitLog successfully. storeRoot: {}", this.defaultMessageStore.getMessageStoreConfig().getStorePathRootDir());
+        this.flushManager.shutdown();
+        log.info("shutdown commitLog successfully. storeRoot: {}", this.defaultMessageStore.getMessageStoreConfig().getStorePathRootDir());
         flushDiskWatcher.shutdown(true);
     }
 
@@ -157,6 +161,10 @@ public class CommitLog implements Swappable {
         return this.mappedFileQueue.getFlushedWhere();
     }
 
+    public long getFlushedWhere() {
+        return this.mappedFileQueue.getFlushedWhere();
+    }
+
     public long getMaxOffset() {
         return this.mappedFileQueue.getMaxOffset();
     }
@@ -207,29 +215,98 @@ public class CommitLog implements Swappable {
         return null;
     }
 
+    public boolean getData(final long offset, final int size, final ByteBuffer byteBuffer) {
+        int mappedFileSize = this.defaultMessageStore.getMessageStoreConfig().getMappedFileSizeCommitLog();
+        MappedFile mappedFile = this.mappedFileQueue.findMappedFileByOffset(offset, offset == 0);
+        if (mappedFile != null) {
+            int pos = (int) (offset % mappedFileSize);
+            return mappedFile.getData(pos, size, byteBuffer);
+        }
+        return false;
+    }
+
+    public List<SelectMappedBufferResult> getBulkData(final long offset, final int size) {
+        List<SelectMappedBufferResult> bufferResultList = new ArrayList<>();
+
+        int mappedFileSize = this.defaultMessageStore.getMessageStoreConfig().getMappedFileSizeCommitLog();
+        int remainSize = size;
+        long startOffset = offset;
+        long maxOffset = this.getMaxOffset();
+        if (offset + size > maxOffset) {
+            remainSize = (int) (maxOffset - offset);
+            log.warn("get bulk data size out of range, correct to max offset. offset: {}, size: {}, max: {}", offset, remainSize, maxOffset);
+        }
+
+        while (remainSize > 0) {
+            MappedFile mappedFile = this.mappedFileQueue.findMappedFileByOffset(startOffset, startOffset == 0);
+            if (mappedFile != null) {
+                int pos = (int) (startOffset % mappedFileSize);
+                int readableSize = mappedFile.getReadPosition() - pos;
+                int readSize = Math.min(remainSize, readableSize);
+
+                SelectMappedBufferResult bufferResult = mappedFile.selectMappedBuffer(pos, readSize);
+                if (bufferResult == null) {
+                    break;
+                }
+                bufferResultList.add(bufferResult);
+                remainSize -= readSize;
+                startOffset += readSize;
+            }
+        }
+
+        return bufferResultList;
+    }
+
+    public SelectMappedFileResult getFile(final long offset) {
+        int mappedFileSize = this.defaultMessageStore.getMessageStoreConfig().getMappedFileSizeCommitLog();
+        MappedFile mappedFile = this.mappedFileQueue.findMappedFileByOffset(offset, offset == 0);
+        if (mappedFile != null) {
+            int size = (int) (mappedFile.getReadPosition() - offset % mappedFileSize);
+            if (size > 0) {
+                return new SelectMappedFileResult(size, mappedFile);
+            }
+        }
+        return null;
+    }
+
+    //Create new mappedFile if not exits.
+    public boolean getLastMappedFile(final long startOffset) {
+        MappedFile lastMappedFile = this.mappedFileQueue.getLastMappedFile(startOffset);
+        if (null == lastMappedFile) {
+            log.error("getLastMappedFile error. offset:{}", startOffset);
+            return false;
+        }
+
+        return true;
+    }
+
     /**
      * When the normal exit, data recovery, all memory data have been flush
      */
     public void recoverNormally(long maxPhyOffsetOfConsumeQueue) {
         boolean checkCRCOnRecover = this.defaultMessageStore.getMessageStoreConfig().isCheckCRCOnRecover();
+        boolean checkDupInfo = this.defaultMessageStore.getMessageStoreConfig().isDuplicationEnable();
         final List<MappedFile> mappedFiles = this.mappedFileQueue.getMappedFiles();
         if (!mappedFiles.isEmpty()) {
             // Began to recover from the last third file
             int index = mappedFiles.size() - 3;
-            if (index < 0)
+            if (index < 0) {
                 index = 0;
+            }
 
             MappedFile mappedFile = mappedFiles.get(index);
             ByteBuffer byteBuffer = mappedFile.sliceByteBuffer();
             long processOffset = mappedFile.getFileFromOffset();
             long mappedFileOffset = 0;
+            long lastValidMsgPhyOffset = this.getConfirmOffset();
             // normal recover doesn't require dispatching
             boolean doDispatch = false;
             while (true) {
-                DispatchRequest dispatchRequest = this.checkMessageAndReturnSize(byteBuffer, checkCRCOnRecover);
+                DispatchRequest dispatchRequest = this.checkMessageAndReturnSize(byteBuffer, checkCRCOnRecover, checkDupInfo);
                 int size = dispatchRequest.getMsgSize();
                 // Normal data
                 if (dispatchRequest.isSuccess() && size > 0) {
+                    lastValidMsgPhyOffset = processOffset + mappedFileOffset;
                     mappedFileOffset += size;
                     this.getMessageStore().onCommitLogDispatch(dispatchRequest, doDispatch, mappedFile, true, false);
                 }
@@ -253,12 +330,19 @@ public class CommitLog implements Swappable {
                 }
                 // Intermediate file read error
                 else if (!dispatchRequest.isSuccess()) {
+                    if (size > 0) {
+                        log.warn("found a half message at {}, it will be truncated.", processOffset + mappedFileOffset);
+                    }
                     log.info("recover physics file end, " + mappedFile.getFileName());
                     break;
                 }
             }
 
             processOffset += mappedFileOffset;
+            // Set a candidate confirm offset.
+            // In most cases, this value will be overwritten by confirmLog.init.
+            // It works if some confirmed messages are lost.
+            this.setConfirmOffset(lastValidMsgPhyOffset);
             this.mappedFileQueue.setFlushedWhere(processOffset);
             this.mappedFileQueue.setCommittedWhere(processOffset);
             this.mappedFileQueue.truncateDirtyFiles(processOffset);
@@ -277,8 +361,9 @@ public class CommitLog implements Swappable {
         }
     }
 
-    public DispatchRequest checkMessageAndReturnSize(java.nio.ByteBuffer byteBuffer, final boolean checkCRC) {
-        return this.checkMessageAndReturnSize(byteBuffer, checkCRC, true);
+    public DispatchRequest checkMessageAndReturnSize(java.nio.ByteBuffer byteBuffer, final boolean checkCRC,
+        final boolean checkDupInfo) {
+        return this.checkMessageAndReturnSize(byteBuffer, checkCRC, checkDupInfo, true);
     }
 
     private void doNothingForDeadCode(final Object obj) {
@@ -293,7 +378,7 @@ public class CommitLog implements Swappable {
      * @return 0 Come the end of the file // >0 Normal messages // -1 Message checksum failure
      */
     public DispatchRequest checkMessageAndReturnSize(java.nio.ByteBuffer byteBuffer, final boolean checkCRC,
-        final boolean readBody) {
+        final boolean checkDupInfo, final boolean readBody) {
         try {
             // 1 TOTAL SIZE
             int totalSize = byteBuffer.getInt();
@@ -382,6 +467,14 @@ public class CommitLog implements Swappable {
 
                 uniqKey = propertiesMap.get(MessageConst.PROPERTY_UNIQ_CLIENT_MESSAGE_ID_KEYIDX);
 
+                if (checkDupInfo) {
+                    String dupInfo = propertiesMap.get(MessageConst.DUP_INFO);
+                    if (null == dupInfo || dupInfo.split("_").length != 2) {
+                        log.warn("DupInfo in properties check failed. dupInfo={}", dupInfo);
+                        return new DispatchRequest(-1, false);
+                    }
+                }
+
                 String tags = propertiesMap.get(MessageConst.PROPERTY_TAGS);
                 if (tags != null && tags.length() > 0) {
                     tagsCode = MessageExtBrokerInner.tagsString2tagsCode(MessageExt.parseTopicFilterType(sysFlag), tags);
@@ -393,12 +486,12 @@ public class CommitLog implements Swappable {
                     if (TopicValidator.RMQ_SYS_SCHEDULE_TOPIC.equals(topic) && t != null) {
                         int delayLevel = Integer.parseInt(t);
 
-                        if (delayLevel > this.defaultMessageStore.getScheduleMessageService().getMaxDelayLevel()) {
-                            delayLevel = this.defaultMessageStore.getScheduleMessageService().getMaxDelayLevel();
+                        if (delayLevel > this.defaultMessageStore.getMaxDelayLevel()) {
+                            delayLevel = this.defaultMessageStore.getMaxDelayLevel();
                         }
 
                         if (delayLevel > 0) {
-                            tagsCode = this.defaultMessageStore.getScheduleMessageService().computeDeliverTimestamp(delayLevel,
+                            tagsCode = this.defaultMessageStore.computeDeliverTimestamp(delayLevel,
                                 storeTimestamp);
                         }
                     }
@@ -437,7 +530,6 @@ public class CommitLog implements Swappable {
 
             return dispatchRequest;
         } catch (Exception e) {
-            log.error("CheckMessageAndReturnSizeOld", e);
         }
 
         return new DispatchRequest(-1, false /* success */);
@@ -475,17 +567,33 @@ public class CommitLog implements Swappable {
     }
 
     public long getConfirmOffset() {
-        return this.confirmOffset;
+        if (this.defaultMessageStore.getMessageStoreConfig().isDuplicationEnable()) {
+            return this.confirmOffset;
+        } else {
+            return getMaxOffset();
+        }
     }
 
     public void setConfirmOffset(long phyOffset) {
         this.confirmOffset = phyOffset;
     }
 
+    public long getLastFileFromOffset() {
+        MappedFile lastMappedFile = this.mappedFileQueue.getLastMappedFile();
+        if (lastMappedFile != null) {
+            if (lastMappedFile.isAvailable()) {
+                return lastMappedFile.getFileFromOffset();
+            }
+        }
+
+        return -1;
+    }
+
     @Deprecated
     public void recoverAbnormally(long maxPhyOffsetOfConsumeQueue) {
         // recover by the minimum time stamp
         boolean checkCRCOnRecover = this.defaultMessageStore.getMessageStoreConfig().isCheckCRCOnRecover();
+        boolean checkDupInfo = this.defaultMessageStore.getMessageStoreConfig().isDuplicationEnable();
         final List<MappedFile> mappedFiles = this.mappedFileQueue.getMappedFiles();
         if (!mappedFiles.isEmpty()) {
             // Looking beginning to recover from which file
@@ -507,15 +615,17 @@ public class CommitLog implements Swappable {
             ByteBuffer byteBuffer = mappedFile.sliceByteBuffer();
             long processOffset = mappedFile.getFileFromOffset();
             long mappedFileOffset = 0;
+            long lastValidMsgPhyOffset = this.getConfirmOffset();
             // abnormal recover require dispatching
             boolean doDispatch = true;
             while (true) {
-                DispatchRequest dispatchRequest = this.checkMessageAndReturnSize(byteBuffer, checkCRCOnRecover);
+                DispatchRequest dispatchRequest = this.checkMessageAndReturnSize(byteBuffer, checkCRCOnRecover, checkDupInfo);
                 int size = dispatchRequest.getMsgSize();
 
                 if (dispatchRequest.isSuccess()) {
                     // Normal data
                     if (size > 0) {
+                        lastValidMsgPhyOffset = processOffset + mappedFileOffset;
                         mappedFileOffset += size;
 
                         if (this.defaultMessageStore.getMessageStoreConfig().isDuplicationEnable()) {
@@ -546,12 +656,21 @@ public class CommitLog implements Swappable {
                         }
                     }
                 } else {
+
+                    if (size > 0) {
+                        log.warn("found a half message at {}, it will be truncated.", processOffset + mappedFileOffset);
+                    }
+
                     log.info("recover physics file end, " + mappedFile.getFileName() + " pos=" + byteBuffer.position());
                     break;
                 }
             }
 
             processOffset += mappedFileOffset;
+            // Set a candidate confirm offset.
+            // In most cases, this value will be overwritten by confirmLog.init.
+            // It works if some confirmed messages are lost.
+            this.setConfirmOffset(lastValidMsgPhyOffset);
             this.mappedFileQueue.setFlushedWhere(processOffset);
             this.mappedFileQueue.setCommittedWhere(processOffset);
             this.mappedFileQueue.truncateDirtyFiles(processOffset);
@@ -571,6 +690,18 @@ public class CommitLog implements Swappable {
         }
     }
 
+    public void truncateDirtyFiles(long phyOffset) {
+        if (phyOffset <= this.getFlushedWhere()) {
+            this.mappedFileQueue.setFlushedWhere(phyOffset);
+        }
+
+        if (phyOffset <= this.mappedFileQueue.getCommittedWhere()) {
+            this.mappedFileQueue.setCommittedWhere(phyOffset);
+        }
+
+        this.mappedFileQueue.truncateDirtyFiles(phyOffset);
+    }
+
     protected void onCommitLogAppend(MessageExtBrokerInner msg, AppendMessageResult result, MappedFile commitLogFile) {
         this.getMessageStore().onCommitLogAppend(msg, result, commitLogFile);
     }
@@ -611,10 +742,6 @@ public class CommitLog implements Swappable {
         return false;
     }
 
-    private void notifyMessageArriving() {
-
-    }
-
     public boolean resetOffset(long offset) {
         return this.mappedFileQueue.resetOffset(offset);
     }
@@ -623,7 +750,7 @@ public class CommitLog implements Swappable {
         return beginTimeInLock;
     }
 
-    private String generateKey(StringBuilder keyBuilder, MessageExt messageExt) {
+    public String generateKey(StringBuilder keyBuilder, MessageExt messageExt) {
         keyBuilder.setLength(0);
         keyBuilder.append(messageExt.getTopic());
         keyBuilder.append('-');
@@ -631,9 +758,17 @@ public class CommitLog implements Swappable {
         return keyBuilder.toString();
     }
 
+    public void setMappedFileQueueOffset(final long phyOffset) {
+        this.mappedFileQueue.setFlushedWhere(phyOffset);
+        this.mappedFileQueue.setCommittedWhere(phyOffset);
+    }
+
     public CompletableFuture<PutMessageResult> asyncPutMessage(final MessageExtBrokerInner msg) {
         // Set the storage time
-        msg.setStoreTimestamp(System.currentTimeMillis());
+        if (!defaultMessageStore.getMessageStoreConfig().isDuplicationEnable()) {
+            msg.setStoreTimestamp(System.currentTimeMillis());
+        }
+
         // Set the message body BODY CRC (consider the most appropriate setting
         // on the client)
         msg.setBodyCRC(UtilAll.crc32(msg.getBody()));
@@ -643,28 +778,6 @@ public class CommitLog implements Swappable {
         StoreStatsService storeStatsService = this.defaultMessageStore.getStoreStatsService();
 
         String topic = msg.getTopic();
-//        int queueId msg.getQueueId();
-        final int tranType = MessageSysFlag.getTransactionValue(msg.getSysFlag());
-        if (tranType == MessageSysFlag.TRANSACTION_NOT_TYPE
-            || tranType == MessageSysFlag.TRANSACTION_COMMIT_TYPE) {
-            // Delay Delivery
-            if (msg.getDelayTimeLevel() > 0) {
-                if (msg.getDelayTimeLevel() > this.defaultMessageStore.getScheduleMessageService().getMaxDelayLevel()) {
-                    msg.setDelayTimeLevel(this.defaultMessageStore.getScheduleMessageService().getMaxDelayLevel());
-                }
-
-                topic = TopicValidator.RMQ_SYS_SCHEDULE_TOPIC;
-                int queueId = ScheduleMessageService.delayLevel2QueueId(msg.getDelayTimeLevel());
-
-                // Backup real topic, queueId
-                MessageAccessor.putProperty(msg, MessageConst.PROPERTY_REAL_TOPIC, msg.getTopic());
-                MessageAccessor.putProperty(msg, MessageConst.PROPERTY_REAL_QUEUE_ID, String.valueOf(msg.getQueueId()));
-                msg.setPropertiesString(MessageDecoder.messageProperties2String(msg.getProperties()));
-
-                msg.setTopic(topic);
-                msg.setQueueId(queueId);
-            }
-        }
 
         InetSocketAddress bornSocketAddress = (InetSocketAddress) msg.getBornHost();
         if (bornSocketAddress.getAddress() instanceof Inet6Address) {
@@ -680,10 +793,39 @@ public class CommitLog implements Swappable {
         String topicQueueKey = generateKey(putMessageThreadLocal.getKeyBuilder(), msg);
         long elapsedTimeInLock = 0;
         MappedFile unlockMappedFile = null;
+        MappedFile mappedFile = this.mappedFileQueue.getLastMappedFile();
+
+        long currOffset;
+        if (mappedFile == null) {
+            currOffset = 0;
+        } else {
+            currOffset = mappedFile.getFileFromOffset() + mappedFile.getWrotePosition();
+        }
+
+        boolean needHandleHA = needHandleHA(msg);
+        int needAckNums = 1;
+
+        if (needHandleHA) {
+            int inSyncReplicas = Math.min(this.defaultMessageStore.getAliveReplicaNumInGroup(),
+                this.defaultMessageStore.getHaService().inSyncSlaveNums(currOffset) + 1);
+            needAckNums = calcNeedAckNums(inSyncReplicas);
+            if (needAckNums > inSyncReplicas) {
+                // Tell the producer, don't have enough slaves to handle the send request
+                return CompletableFuture.completedFuture(new PutMessageResult(PutMessageStatus.IN_SYNC_REPLICAS_NOT_ENOUGH, null));
+            }
+        }
 
         topicQueueLock.lock(topicQueueKey);
         try {
-            defaultMessageStore.assignOffset(topicQueueKey, msg, getMessageNum(msg));
+
+            boolean needAssignOffset = true;
+            if (defaultMessageStore.getMessageStoreConfig().isDuplicationEnable()
+                && defaultMessageStore.getMessageStoreConfig().getBrokerRole() != BrokerRole.SLAVE) {
+                needAssignOffset = false;
+            }
+            if (needAssignOffset) {
+                defaultMessageStore.assignOffset(msg, getMessageNum(msg));
+            }
 
             PutMessageResult encodeResult = putMessageThreadLocal.getEncoder().encode(msg);
             if (encodeResult != null) {
@@ -694,13 +836,14 @@ public class CommitLog implements Swappable {
 
             putMessageLock.lock(); //spin or ReentrantLock ,depending on store config
             try {
-                MappedFile mappedFile = this.mappedFileQueue.getLastMappedFile();
                 long beginLockTimestamp = this.defaultMessageStore.getSystemClock().now();
                 this.beginTimeInLock = beginLockTimestamp;
 
                 // Here settings are stored timestamp, in order to ensure an orderly
                 // global
-                msg.setStoreTimestamp(beginLockTimestamp);
+                if (!defaultMessageStore.getMessageStoreConfig().isDuplicationEnable()) {
+                    msg.setStoreTimestamp(beginLockTimestamp);
+                }
 
                 if (null == mappedFile || mappedFile.isFull()) {
                     mappedFile = this.mappedFileQueue.getLastMappedFile(0); // Mark: NewFile may be cause noise
@@ -708,7 +851,7 @@ public class CommitLog implements Swappable {
                 if (null == mappedFile) {
                     log.error("create mapped file1 error, topic: " + msg.getTopic() + " clientAddr: " + msg.getBornHostString());
                     beginTimeInLock = 0;
-                    return CompletableFuture.completedFuture(new PutMessageResult(PutMessageStatus.CREATE_MAPEDFILE_FAILED, null));
+                    return CompletableFuture.completedFuture(new PutMessageResult(PutMessageStatus.CREATE_MAPPED_FILE_FAILED, null));
                 }
 
                 result = mappedFile.appendMessage(msg, this.appendMessageCallback, putMessageContext);
@@ -725,7 +868,7 @@ public class CommitLog implements Swappable {
                             // XXX: warn and notify me
                             log.error("create mapped file2 error, topic: " + msg.getTopic() + " clientAddr: " + msg.getBornHostString());
                             beginTimeInLock = 0;
-                            return CompletableFuture.completedFuture(new PutMessageResult(PutMessageStatus.CREATE_MAPEDFILE_FAILED, result));
+                            return CompletableFuture.completedFuture(new PutMessageResult(PutMessageStatus.CREATE_MAPPED_FILE_FAILED, result));
                         }
                         result = mappedFile.appendMessage(msg, this.appendMessageCallback, putMessageContext);
                         if (AppendMessageStatus.PUT_OK.equals(result.getStatus())) {
@@ -747,7 +890,6 @@ public class CommitLog implements Swappable {
                 elapsedTimeInLock = this.defaultMessageStore.getSystemClock().now() - beginLockTimestamp;
                 beginTimeInLock = 0;
             } finally {
-                beginTimeInLock = 0;
                 putMessageLock.unlock();
             }
         } finally {
@@ -768,17 +910,7 @@ public class CommitLog implements Swappable {
         storeStatsService.getSinglePutMessageTopicTimesTotal(msg.getTopic()).add(1);
         storeStatsService.getSinglePutMessageTopicSizeTotal(topic).add(result.getWroteBytes());
 
-        CompletableFuture<PutMessageStatus> flushResultFuture = submitFlushRequest(result, msg);
-        CompletableFuture<PutMessageStatus> replicaResultFuture = submitReplicaRequest(result, msg);
-        return flushResultFuture.thenCombine(replicaResultFuture, (flushStatus, replicaStatus) -> {
-            if (flushStatus != PutMessageStatus.PUT_OK) {
-                putMessageResult.setPutMessageStatus(flushStatus);
-            }
-            if (replicaStatus != PutMessageStatus.PUT_OK) {
-                putMessageResult.setPutMessageStatus(replicaStatus);
-            }
-            return putMessageResult;
-        });
+        return handleDiskFlushAndHA(putMessageResult, msg, needAckNums, needHandleHA);
     }
 
     public CompletableFuture<PutMessageResult> asyncPutMessages(final MessageExtBatch messageExtBatch) {
@@ -810,6 +942,26 @@ public class CommitLog implements Swappable {
         MappedFile unlockMappedFile = null;
         MappedFile mappedFile = this.mappedFileQueue.getLastMappedFile();
 
+        long currOffset;
+        if (mappedFile == null) {
+            currOffset = 0;
+        } else {
+            currOffset = mappedFile.getFileFromOffset() + mappedFile.getWrotePosition();
+        }
+
+        int needAckNums = 1;
+        boolean needHandleHA = needHandleHA(messageExtBatch);
+
+        if (needHandleHA) {
+            int inSyncReplicas = Math.min(this.defaultMessageStore.getAliveReplicaNumInGroup(),
+                this.defaultMessageStore.getHaService().inSyncSlaveNums(currOffset) + 1);
+            needAckNums = calcNeedAckNums(inSyncReplicas);
+            if (needAckNums > inSyncReplicas) {
+                // Tell the producer, don't have enough slaves to handle the send request
+                return CompletableFuture.completedFuture(new PutMessageResult(PutMessageStatus.IN_SYNC_REPLICAS_NOT_ENOUGH, null));
+            }
+        }
+
         //fine-grained lock instead of the coarse-grained
         PutMessageThreadLocal pmThreadLocal = this.putMessageThreadLocal.get();
         MessageExtEncoder batchEncoder = pmThreadLocal.getEncoder();
@@ -821,7 +973,7 @@ public class CommitLog implements Swappable {
 
         topicQueueLock.lock(topicQueueKey);
         try {
-            defaultMessageStore.assignOffset(topicQueueKey, messageExtBatch, (short) putMessageContext.getBatchSize());
+            defaultMessageStore.assignOffset(messageExtBatch, (short) putMessageContext.getBatchSize());
 
             putMessageLock.lock();
             try {
@@ -838,7 +990,7 @@ public class CommitLog implements Swappable {
                 if (null == mappedFile) {
                     log.error("Create mapped file1 error, topic: {} clientAddr: {}", messageExtBatch.getTopic(), messageExtBatch.getBornHostString());
                     beginTimeInLock = 0;
-                    return CompletableFuture.completedFuture(new PutMessageResult(PutMessageStatus.CREATE_MAPEDFILE_FAILED, null));
+                    return CompletableFuture.completedFuture(new PutMessageResult(PutMessageStatus.CREATE_MAPPED_FILE_FAILED, null));
                 }
 
                 result = mappedFile.appendMessages(messageExtBatch, this.appendMessageCallback, putMessageContext);
@@ -853,7 +1005,7 @@ public class CommitLog implements Swappable {
                             // XXX: warn and notify me
                             log.error("Create mapped file2 error, topic: {} clientAddr: {}", messageExtBatch.getTopic(), messageExtBatch.getBornHostString());
                             beginTimeInLock = 0;
-                            return CompletableFuture.completedFuture(new PutMessageResult(PutMessageStatus.CREATE_MAPEDFILE_FAILED, result));
+                            return CompletableFuture.completedFuture(new PutMessageResult(PutMessageStatus.CREATE_MAPPED_FILE_FAILED, result));
                         }
                         result = mappedFile.appendMessages(messageExtBatch, this.appendMessageCallback, putMessageContext);
                         break;
@@ -890,9 +1042,51 @@ public class CommitLog implements Swappable {
         storeStatsService.getSinglePutMessageTopicTimesTotal(messageExtBatch.getTopic()).add(result.getMsgNum());
         storeStatsService.getSinglePutMessageTopicSizeTotal(messageExtBatch.getTopic()).add(result.getWroteBytes());
 
-        CompletableFuture<PutMessageStatus> flushOKFuture = submitFlushRequest(result, messageExtBatch);
-        CompletableFuture<PutMessageStatus> replicaOKFuture = submitReplicaRequest(result, messageExtBatch);
-        return flushOKFuture.thenCombine(replicaOKFuture, (flushStatus, replicaStatus) -> {
+        return handleDiskFlushAndHA(putMessageResult, messageExtBatch, needAckNums, needHandleHA);
+    }
+
+    private int calcNeedAckNums(int inSyncReplicas) {
+        int needAckNums = this.defaultMessageStore.getMessageStoreConfig().getInSyncReplicas();
+        if (this.defaultMessageStore.getMessageStoreConfig().isEnableAutoInSyncReplicas()) {
+            needAckNums = Math.min(needAckNums, inSyncReplicas);
+            needAckNums = Math.max(needAckNums, this.defaultMessageStore.getMessageStoreConfig().getMinInSyncReplicas());
+        }
+        return needAckNums;
+    }
+
+    private boolean needHandleHA(MessageExt messageExt) {
+
+        if (!messageExt.isWaitStoreMsgOK()) {
+            /*
+              No need to sync messages that special config to extra broker slaves.
+              @see MessageConst.PROPERTY_WAIT_STORE_MSG_OK
+             */
+            return false;
+        }
+
+        if (this.defaultMessageStore.getMessageStoreConfig().isDuplicationEnable()) {
+            return false;
+        }
+
+        if (BrokerRole.SYNC_MASTER != this.defaultMessageStore.getMessageStoreConfig().getBrokerRole()) {
+            // No need to check ha in async or slave broker
+            return false;
+        }
+
+        return true;
+    }
+
+    private CompletableFuture<PutMessageResult> handleDiskFlushAndHA(PutMessageResult putMessageResult,
+        MessageExt messageExt, int needAckNums, boolean needHandleHA) {
+        CompletableFuture<PutMessageStatus> flushResultFuture = handleDiskFlush(putMessageResult.getAppendMessageResult(), messageExt);
+        CompletableFuture<PutMessageStatus> replicaResultFuture;
+        if (!needHandleHA) {
+            replicaResultFuture = CompletableFuture.completedFuture(PutMessageStatus.PUT_OK);
+        } else {
+            replicaResultFuture = handleHA(putMessageResult.getAppendMessageResult(), putMessageResult, needAckNums);
+        }
+
+        return flushResultFuture.thenCombine(replicaResultFuture, (flushStatus, replicaStatus) -> {
             if (flushStatus != PutMessageStatus.PUT_OK) {
                 putMessageResult.setPutMessageStatus(flushStatus);
             }
@@ -901,66 +1095,38 @@ public class CommitLog implements Swappable {
             }
             return putMessageResult;
         });
-
     }
 
-    public CompletableFuture<PutMessageStatus> submitFlushRequest(AppendMessageResult result, MessageExt messageExt) {
+    private CompletableFuture<PutMessageStatus> handleDiskFlush(AppendMessageResult result, MessageExt messageExt) {
         return this.flushManager.handleDiskFlush(result, messageExt);
     }
 
-    public CompletableFuture<PutMessageStatus> submitReplicaRequest(AppendMessageResult result, MessageExt messageExt) {
-        if (BrokerRole.SYNC_MASTER == this.defaultMessageStore.getMessageStoreConfig().getBrokerRole()) {
-            HAService service = this.defaultMessageStore.getHaService();
-            if (messageExt.isWaitStoreMsgOK()) {
-                if (service.isSlaveOK(result.getWroteBytes() + result.getWroteOffset())) {
-                    GroupCommitRequest request = new GroupCommitRequest(result.getWroteOffset() + result.getWroteBytes(),
-                        this.defaultMessageStore.getMessageStoreConfig().getSyncFlushTimeout());
-                    flushDiskWatcher.add(request);
-                    service.putRequest(request);
-                    service.getWaitNotifyObject().wakeupAll();
-                    return request.future();
-                } else {
-                    return CompletableFuture.completedFuture(PutMessageStatus.SLAVE_NOT_AVAILABLE);
-                }
-            }
+    private CompletableFuture<PutMessageStatus> handleHA(AppendMessageResult result, PutMessageResult putMessageResult,
+        int needAckNums) {
+        if (needAckNums <= 1) {
+            return CompletableFuture.completedFuture(PutMessageStatus.PUT_OK);
         }
-        return CompletableFuture.completedFuture(PutMessageStatus.PUT_OK);
-    }
 
-    public void handleDiskFlush(AppendMessageResult result, PutMessageResult putMessageResult, MessageExt messageExt) {
-        this.flushManager.handleDiskFlush(result, putMessageResult, messageExt);
-    }
+        HAService haService = this.defaultMessageStore.getHaService();
 
-    public void handleHA(AppendMessageResult result, PutMessageResult putMessageResult, MessageExt messageExt) {
-        if (BrokerRole.SYNC_MASTER == this.defaultMessageStore.getMessageStoreConfig().getBrokerRole()) {
-            HAService service = this.defaultMessageStore.getHaService();
-            if (messageExt.isWaitStoreMsgOK()) {
-                // Determine whether to wait
-                if (service.isSlaveOK(result.getWroteOffset() + result.getWroteBytes())) {
-                    GroupCommitRequest request = new GroupCommitRequest(result.getWroteOffset() + result.getWroteBytes(),
-                        this.defaultMessageStore.getMessageStoreConfig().getSlaveTimeout());
-                    service.putRequest(request);
-                    service.getWaitNotifyObject().wakeupAll();
-                    PutMessageStatus replicaStatus = null;
-                    try {
-                        replicaStatus = request.future().get(this.defaultMessageStore.getMessageStoreConfig().getSyncFlushTimeout(),
-                            TimeUnit.MILLISECONDS);
-                    } catch (InterruptedException | ExecutionException | TimeoutException e) {
-                    }
-                    if (replicaStatus != PutMessageStatus.PUT_OK) {
-                        log.error("do sync transfer other node, wait return, but failed, topic: " + messageExt.getTopic() + " tags: "
-                            + messageExt.getTags() + " client address: " + messageExt.getBornHostNameString());
-                        putMessageResult.setPutMessageStatus(PutMessageStatus.FLUSH_SLAVE_TIMEOUT);
-                    }
-                }
-                // Slave problem
-                else {
-                    // Tell the producer, slave not available
-                    putMessageResult.setPutMessageStatus(PutMessageStatus.SLAVE_NOT_AVAILABLE);
-                }
-            }
-        }
+        long nextOffset = result.getWroteOffset() + result.getWroteBytes();
+        // NOTE: Plus the master replicas
+//        int inSyncReplicas = haService.inSyncSlaveNums(nextOffset) + 1;
+
+//        if (needAckNums > inSyncReplicas) {
+//            /*
+//             * Tell the producer, don't have enough slaves to handle the send request.
+//             * NOTE: this may cause msg duplicate
+//             */
+//            putMessageResult.setPutMessageStatus(PutMessageStatus.IN_SYNC_REPLICAS_NOT_ENOUGH);
+//            return CompletableFuture.completedFuture(PutMessageStatus.IN_SYNC_REPLICAS_NOT_ENOUGH);
+//        }
 
+        // Wait enough acks from different slaves
+        GroupCommitRequest request = new GroupCommitRequest(nextOffset, this.defaultMessageStore.getMessageStoreConfig().getSlaveTimeout(), needAckNums - 1);
+        haService.putRequest(request);
+        haService.getWaitNotifyObject().wakeupAll();
+        return request.future();
     }
 
     /**
@@ -1083,6 +1249,9 @@ public class CommitLog implements Swappable {
 
         @Override
         public String getServiceName() {
+            if (CommitLog.this.defaultMessageStore.getBrokerConfig().isInBrokerContainer()) {
+                return CommitLog.this.defaultMessageStore.getBrokerConfig().getLoggerIdentifier() + CommitRealTimeService.class.getSimpleName();
+            }
             return CommitRealTimeService.class.getSimpleName();
         }
 
@@ -1133,6 +1302,7 @@ public class CommitLog implements Swappable {
         private long lastFlushTimestamp = 0;
         private long printTimes = 0;
 
+        @Override
         public void run() {
             CommitLog.log.info(this.getServiceName() + " service started");
 
@@ -1197,6 +1367,9 @@ public class CommitLog implements Swappable {
 
         @Override
         public String getServiceName() {
+            if (CommitLog.this.defaultMessageStore.getBrokerConfig().isInBrokerContainer()) {
+                return CommitLog.this.defaultMessageStore.getBrokerConfig().getLoggerIdentifier() + FlushRealTimeService.class.getSimpleName();
+            }
             return FlushRealTimeService.class.getSimpleName();
         }
 
@@ -1206,14 +1379,16 @@ public class CommitLog implements Swappable {
         }
 
         @Override
-        public long getJointime() {
+        public long getJoinTime() {
             return 1000 * 60 * 5;
         }
     }
 
     public static class GroupCommitRequest {
         private final long nextOffset;
-        private CompletableFuture<PutMessageStatus> flushOKFuture = new CompletableFuture<>();
+        // Indicate the GroupCommitRequest result: true or false
+        private final CompletableFuture<PutMessageStatus> flushOKFuture = new CompletableFuture<>();
+        private volatile int ackNums = 1;
         private final long deadLine;
 
         public GroupCommitRequest(long nextOffset, long timeoutMillis) {
@@ -1221,22 +1396,30 @@ public class CommitLog implements Swappable {
             this.deadLine = System.nanoTime() + (timeoutMillis * 1_000_000);
         }
 
-        public long getDeadLine() {
-            return deadLine;
+        public GroupCommitRequest(long nextOffset, long timeoutMillis, int ackNums) {
+            this(nextOffset, timeoutMillis);
+            this.ackNums = ackNums;
         }
 
         public long getNextOffset() {
             return nextOffset;
         }
 
-        public void wakeupCustomer(final PutMessageStatus putMessageStatus) {
-            this.flushOKFuture.complete(putMessageStatus);
+        public int getAckNums() {
+            return ackNums;
+        }
+
+        public long getDeadLine() {
+            return deadLine;
+        }
+
+        public void wakeupCustomer(final PutMessageStatus status) {
+            this.flushOKFuture.complete(status);
         }
 
         public CompletableFuture<PutMessageStatus> future() {
             return flushOKFuture;
         }
-
     }
 
     /**
@@ -1312,7 +1495,7 @@ public class CommitLog implements Swappable {
             try {
                 Thread.sleep(10);
             } catch (InterruptedException e) {
-                CommitLog.log.warn(this.getServiceName() + " Exception, ", e);
+                CommitLog.log.warn("GroupCommitService Exception, ", e);
             }
 
             synchronized (this) {
@@ -1331,11 +1514,14 @@ public class CommitLog implements Swappable {
 
         @Override
         public String getServiceName() {
+            if (CommitLog.this.defaultMessageStore.getBrokerConfig().isInBrokerContainer()) {
+                return CommitLog.this.defaultMessageStore.getBrokerConfig().getLoggerIdentifier() + GroupCommitService.class.getSimpleName();
+            }
             return GroupCommitService.class.getSimpleName();
         }
 
         @Override
-        public long getJointime() {
+        public long getJoinTime() {
             return 1000 * 60 * 5;
         }
     }
@@ -1344,7 +1530,7 @@ public class CommitLog implements Swappable {
         private volatile List<GroupCommitRequest> requestsWrite = new ArrayList<GroupCommitRequest>();
         private volatile List<GroupCommitRequest> requestsRead = new ArrayList<GroupCommitRequest>();
 
-        public boolean isAynscRequestsFull() {
+        public boolean isAsyncRequestsFull() {
             return requestsWrite.size() > CommitLog.this.defaultMessageStore.getMessageStoreConfig().getMaxAsyncPutMessageRequests() * 2;
         }
 
@@ -1439,11 +1625,14 @@ public class CommitLog implements Swappable {
 
         @Override
         public String getServiceName() {
-            return CommitLog.GroupCheckService.class.getSimpleName();
+            if (CommitLog.this.defaultMessageStore.getBrokerConfig().isInBrokerContainer()) {
+                return CommitLog.this.defaultMessageStore.getBrokerConfig().getLoggerIdentifier() + GroupCheckService.class.getSimpleName();
+            }
+            return GroupCheckService.class.getSimpleName();
         }
 
         @Override
-        public long getJointime() {
+        public long getJoinTime() {
             return 1000 * 60 * 5;
         }
     }
@@ -1451,16 +1640,12 @@ public class CommitLog implements Swappable {
     class DefaultAppendMessageCallback implements AppendMessageCallback {
         // File at the end of the minimum fixed length empty
         private static final int END_FILE_MIN_BLANK_LENGTH = 4 + 4;
-        private final ByteBuffer msgIdMemory;
-        private final ByteBuffer msgIdV6Memory;
         // Store the message content
         private final ByteBuffer msgStoreItemMemory;
         // The maximum length of the message
         private final int maxMessageSize;
 
         DefaultAppendMessageCallback(final int size) {
-            this.msgIdMemory = ByteBuffer.allocate(4 + 4 + 8);
-            this.msgIdV6Memory = ByteBuffer.allocate(16 + 4 + 8);
             this.msgStoreItemMemory = ByteBuffer.allocate(END_FILE_MIN_BLANK_LENGTH);
             this.maxMessageSize = size;
         }
@@ -1491,8 +1676,7 @@ public class CommitLog implements Swappable {
             // Transaction messages that require special handling
             final int tranType = MessageSysFlag.getTransactionValue(msgInner.getSysFlag());
             switch (tranType) {
-                // Prepared and Rollback message is not consumed, will not enter the
-                // consumer queuec
+                // Prepared and Rollback message is not consumed, will not enter the consume queue
                 case MessageSysFlag.TRANSACTION_PREPARED_TYPE:
                 case MessageSysFlag.TRANSACTION_ROLLBACK_TYPE:
                     queueOffset = 0L;
@@ -1869,9 +2053,6 @@ public class CommitLog implements Swappable {
             byteBuffer.limit(limit);
         }
 
-        public ByteBuffer getEncoderBuffer() {
-            return encoderBuffer;
-        }
     }
 
     interface FlushManager {
@@ -1918,8 +2099,7 @@ public class CommitLog implements Swappable {
             if (FlushDiskType.SYNC_FLUSH == CommitLog.this.defaultMessageStore.getMessageStoreConfig().getFlushDiskType()) {
                 final GroupCommitService service = (GroupCommitService) this.flushCommitLogService;
                 if (messageExt.isWaitStoreMsgOK()) {
-                    GroupCommitRequest request = new GroupCommitRequest(result.getWroteOffset() + result.getWroteBytes(),
-                        CommitLog.this.defaultMessageStore.getMessageStoreConfig().getSyncFlushTimeout());
+                    GroupCommitRequest request = new GroupCommitRequest(result.getWroteOffset() + result.getWroteBytes(), CommitLog.this.defaultMessageStore.getMessageStoreConfig().getSyncFlushTimeout());
                     service.putRequest(request);
                     CompletableFuture<PutMessageStatus> flushOkFuture = request.future();
                     PutMessageStatus flushStatus = null;
@@ -1954,8 +2134,8 @@ public class CommitLog implements Swappable {
             if (FlushDiskType.SYNC_FLUSH == CommitLog.this.defaultMessageStore.getMessageStoreConfig().getFlushDiskType()) {
                 final GroupCommitService service = (GroupCommitService) this.flushCommitLogService;
                 if (messageExt.isWaitStoreMsgOK()) {
-                    GroupCommitRequest request = new GroupCommitRequest(result.getWroteOffset() + result.getWroteBytes(),
-                        CommitLog.this.defaultMessageStore.getMessageStoreConfig().getSyncFlushTimeout());
+                    GroupCommitRequest request = new GroupCommitRequest(result.getWroteOffset() + result.getWroteBytes(), CommitLog.this.defaultMessageStore.getMessageStoreConfig().getSyncFlushTimeout());
+                    flushDiskWatcher.add(request);
                     service.putRequest(request);
                     return request.future();
                 } else {
@@ -2008,6 +2188,10 @@ public class CommitLog implements Swappable {
         this.getMappedFileQueue().swapMap(reserveNum, forceSwapIntervalMs, normalSwapIntervalMs);
     }
 
+    public boolean isMappedFilesEmpty() {
+        return this.mappedFileQueue.isMappedFilesEmpty();
+    }
+
     @Override
     public void cleanSwappedMap(long forceCleanSwapIntervalMs) {
         this.getMappedFileQueue().cleanSwappedMap(forceCleanSwapIntervalMs);
diff --git a/store/src/main/java/org/apache/rocketmq/store/ConsumeQueue.java b/store/src/main/java/org/apache/rocketmq/store/ConsumeQueue.java
index 4e33728..740377d 100644
--- a/store/src/main/java/org/apache/rocketmq/store/ConsumeQueue.java
+++ b/store/src/main/java/org/apache/rocketmq/store/ConsumeQueue.java
@@ -18,6 +18,7 @@ package org.apache.rocketmq.store;
 
 import java.io.File;
 import java.nio.ByteBuffer;
+import java.util.Collections;
 import java.util.List;
 import java.util.Map;
 
@@ -27,6 +28,7 @@ import org.apache.rocketmq.common.constant.LoggerName;
 import org.apache.rocketmq.common.message.MessageAccessor;
 import org.apache.rocketmq.common.message.MessageConst;
 import org.apache.rocketmq.common.message.MessageDecoder;
+import org.apache.rocketmq.common.message.MessageExtBrokerInner;
 import org.apache.rocketmq.logging.InternalLogger;
 import org.apache.rocketmq.logging.InternalLoggerFactory;
 import org.apache.rocketmq.store.config.BrokerRole;
@@ -106,8 +108,9 @@ public class ConsumeQueue implements ConsumeQueueInterface, FileQueueLifeCycle {
         if (!mappedFiles.isEmpty()) {
 
             int index = mappedFiles.size() - 3;
-            if (index < 0)
+            if (index < 0) {
                 index = 0;
+            }
 
             int mappedFileSizeLogics = this.mappedFileSize;
             MappedFile mappedFile = mappedFiles.get(index);
@@ -168,6 +171,14 @@ public class ConsumeQueue implements ConsumeQueueInterface, FileQueueLifeCycle {
         }
     }
 
+    public long getTotalSize() {
+        long totalSize = this.mappedFileQueue.getTotalFileSize();
+        if (isExtReadEnable()) {
+            totalSize += this.consumeQueueExt.getTotalSize();
+        }
+        return totalSize;
+    }
+
     @Override
     public long getOffsetInQueueByTime(final long timestamp) {
         MappedFile mappedFile = this.mappedFileQueue.getMappedFileByTime(timestamp);
@@ -239,12 +250,17 @@ public class ConsumeQueue implements ConsumeQueueInterface, FileQueueLifeCycle {
     }
 
     @Override
-    public void truncateDirtyLogicFiles(long phyOffet) {
+    public void truncateDirtyLogicFiles(long phyOffset) {
+        truncateDirtyLogicFiles(phyOffset, true);
+    }
+
+    public void truncateDirtyLogicFiles(long phyOffset, boolean deleteFile) {
 
         int logicFileSize = this.mappedFileSize;
 
-        this.maxPhysicOffset = phyOffet;
+        this.maxPhysicOffset = phyOffset;
         long maxExtAddr = 1;
+        boolean shouldDeleteFile = false;
         while (true) {
             MappedFile mappedFile = this.mappedFileQueue.getLastMappedFile();
             if (mappedFile != null) {
@@ -260,8 +276,8 @@ public class ConsumeQueue implements ConsumeQueueInterface, FileQueueLifeCycle {
                     long tagsCode = byteBuffer.getLong();
 
                     if (0 == i) {
-                        if (offset >= phyOffet) {
-                            this.mappedFileQueue.deleteLastMappedFile();
+                        if (offset >= phyOffset) {
+                            shouldDeleteFile = true;
                             break;
                         } else {
                             int pos = i + CQ_STORE_UNIT_SIZE;
@@ -278,7 +294,7 @@ public class ConsumeQueue implements ConsumeQueueInterface, FileQueueLifeCycle {
 
                         if (offset >= 0 && size > 0) {
 
-                            if (offset >= phyOffet) {
+                            if (offset >= phyOffset) {
                                 return;
                             }
 
@@ -299,6 +315,15 @@ public class ConsumeQueue implements ConsumeQueueInterface, FileQueueLifeCycle {
                         }
                     }
                 }
+
+                if (shouldDeleteFile) {
+                    if (deleteFile) {
+                        this.mappedFileQueue.deleteLastMappedFile();
+                    } else {
+                        this.mappedFileQueue.deleteExpiredFile(Collections.singletonList(this.mappedFileQueue.getLastMappedFile()));
+                    }
+                }
+
             } else {
                 break;
             }
@@ -399,6 +424,7 @@ public class ConsumeQueue implements ConsumeQueueInterface, FileQueueLifeCycle {
         return this.minLogicOffset / CQ_STORE_UNIT_SIZE;
     }
 
+    @Override
     public void putMessagePositionInfoWrapper(DispatchRequest request) {
         final int maxRetries = 30;
         boolean canWrite = this.defaultMessageStore.getRunningFlags().isCQWriteable();
@@ -695,7 +721,7 @@ public class ConsumeQueue implements ConsumeQueueInterface, FileQueueLifeCycle {
         private int relativePos = 0;
 
         public ConsumeQueueIterator(SelectMappedBufferResult sbr) {
-            this.sbr = sbr;
+            this.sbr =  sbr;
             if (sbr != null && sbr.getByteBuffer() != null) {
                 relativePos = sbr.getByteBuffer().position();
             }
@@ -715,11 +741,11 @@ public class ConsumeQueue implements ConsumeQueueInterface, FileQueueLifeCycle {
             if (!hasNext()) {
                 return null;
             }
-            long queueOffset = (sbr.getStartOffset() + sbr.getByteBuffer().position() - relativePos) / CQ_STORE_UNIT_SIZE;
+            long queueOffset = (sbr.getStartOffset() + sbr.getByteBuffer().position() -  relativePos) / CQ_STORE_UNIT_SIZE;
             CqUnit cqUnit = new CqUnit(queueOffset,
-                sbr.getByteBuffer().getLong(),
-                sbr.getByteBuffer().getInt(),
-                sbr.getByteBuffer().getLong());
+                    sbr.getByteBuffer().getLong(),
+                    sbr.getByteBuffer().getInt(),
+                    sbr.getByteBuffer().getLong());
 
             if (isExtAddr(cqUnit.getTagsCode())) {
                 ConsumeQueueExt.CqExtUnit cqExtUnit = new ConsumeQueueExt.CqExtUnit();
@@ -730,7 +756,7 @@ public class ConsumeQueue implements ConsumeQueueInterface, FileQueueLifeCycle {
                 } else {
                     // can't find ext content.Client will filter messages by tag also.
                     log.error("[BUG] can't find consume queue extend file content! addr={}, offsetPy={}, sizePy={}, topic={}",
-                        cqUnit.getTagsCode(), cqUnit.getPos(), cqUnit.getPos(), getTopic());
+                            cqUnit.getTagsCode(), cqUnit.getPos(), cqUnit.getPos(), getTopic());
                 }
             }
             return cqUnit;
diff --git a/store/src/main/java/org/apache/rocketmq/store/ConsumeQueueExt.java b/store/src/main/java/org/apache/rocketmq/store/ConsumeQueueExt.java
index 19c7992..f678e98 100644
--- a/store/src/main/java/org/apache/rocketmq/store/ConsumeQueueExt.java
+++ b/store/src/main/java/org/apache/rocketmq/store/ConsumeQueueExt.java
@@ -90,6 +90,10 @@ public class ConsumeQueueExt {
         }
     }
 
+    public long getTotalSize() {
+        return this.mappedFileQueue.getTotalFileSize();
+    }
+
     /**
      * Check whether {@code address} point to extend file.
      * <p>
diff --git a/store/src/main/java/org/apache/rocketmq/store/DefaultMessageStore.java b/store/src/main/java/org/apache/rocketmq/store/DefaultMessageStore.java
index 8892d61..dc6cbf5 100644
--- a/store/src/main/java/org/apache/rocketmq/store/DefaultMessageStore.java
+++ b/store/src/main/java/org/apache/rocketmq/store/DefaultMessageStore.java
@@ -16,6 +16,7 @@
  */
 package org.apache.rocketmq.store;
 
+import com.google.common.hash.Hashing;
 import java.io.File;
 import java.io.IOException;
 import java.io.RandomAccessFile;
@@ -24,6 +25,7 @@ import java.net.InetSocketAddress;
 import java.net.SocketAddress;
 import java.nio.ByteBuffer;
 import java.nio.channels.FileLock;
+import java.util.ArrayList;
 import java.util.Collections;
 import java.util.HashMap;
 import java.util.HashSet;
@@ -36,16 +38,16 @@ import java.util.Objects;
 import java.util.Optional;
 import java.util.Set;
 import java.util.concurrent.CompletableFuture;
+import java.util.concurrent.ConcurrentHashMap;
 import java.util.concurrent.ConcurrentMap;
 import java.util.concurrent.ExecutionException;
-import java.util.concurrent.CopyOnWriteArrayList;
 import java.util.concurrent.Executors;
 import java.util.concurrent.ScheduledExecutorService;
 import java.util.concurrent.TimeUnit;
 import java.util.concurrent.TimeoutException;
 import java.util.concurrent.atomic.AtomicLong;
-
 import org.apache.commons.lang3.StringUtils;
+import org.apache.rocketmq.common.AbstractBrokerRunnable;
 import org.apache.rocketmq.common.BrokerConfig;
 import org.apache.rocketmq.common.MixAll;
 import org.apache.rocketmq.common.ServiceThread;
@@ -58,10 +60,14 @@ import org.apache.rocketmq.common.constant.LoggerName;
 import org.apache.rocketmq.common.message.MessageConst;
 import org.apache.rocketmq.common.message.MessageDecoder;
 import org.apache.rocketmq.common.message.MessageExt;
+import org.apache.rocketmq.common.message.MessageExtBatch;
+import org.apache.rocketmq.common.message.MessageExtBrokerInner;
+import org.apache.rocketmq.common.protocol.body.HARuntimeInfo;
 import org.apache.rocketmq.common.running.RunningStats;
 import org.apache.rocketmq.common.sysflag.MessageSysFlag;
 import org.apache.rocketmq.common.topic.TopicValidator;
 import org.apache.rocketmq.common.utils.QueueTypeUtils;
+import org.apache.rocketmq.common.utils.ServiceProvider;
 import org.apache.rocketmq.logging.InternalLogger;
 import org.apache.rocketmq.logging.InternalLoggerFactory;
 import org.apache.rocketmq.store.config.BrokerRole;
@@ -69,23 +75,24 @@ import org.apache.rocketmq.store.config.FlushDiskType;
 import org.apache.rocketmq.store.config.MessageStoreConfig;
 import org.apache.rocketmq.store.config.StorePathConfigHelper;
 import org.apache.rocketmq.store.dledger.DLedgerCommitLog;
+import org.apache.rocketmq.store.ha.DefaultHAService;
 import org.apache.rocketmq.store.ha.HAService;
+import org.apache.rocketmq.store.hook.PutMessageHook;
+import org.apache.rocketmq.store.hook.SendMessageBackHook;
 import org.apache.rocketmq.store.index.IndexService;
 import org.apache.rocketmq.store.index.QueryOffsetResult;
-import org.apache.rocketmq.store.logfile.DefaultMappedFile;
 import org.apache.rocketmq.store.logfile.MappedFile;
 import org.apache.rocketmq.store.queue.ConsumeQueueInterface;
 import org.apache.rocketmq.store.queue.ConsumeQueueStore;
 import org.apache.rocketmq.store.queue.CqUnit;
 import org.apache.rocketmq.store.queue.ReferredIterator;
-import org.apache.rocketmq.store.schedule.ScheduleMessageService;
 import org.apache.rocketmq.store.stats.BrokerStatsManager;
 import org.apache.rocketmq.store.util.PerfCounter;
 
 public class DefaultMessageStore implements MessageStore {
-    private static final InternalLogger log = InternalLoggerFactory.getLogger(LoggerName.STORE_LOGGER_NAME);
+    private static final InternalLogger LOGGER = InternalLoggerFactory.getLogger(LoggerName.STORE_LOGGER_NAME);
 
-    public final PerfCounter.Ticks perfs = new PerfCounter.Ticks(log);
+    public final PerfCounter.Ticks perfs = new PerfCounter.Ticks(LOGGER);
 
     private final MessageStoreConfig messageStoreConfig;
     // CommitLog
@@ -105,11 +112,9 @@ public class DefaultMessageStore implements MessageStore {
 
     private final AllocateMappedFileService allocateMappedFileService;
 
-    private final ReputMessageService reputMessageService;
-
-    private final HAService haService;
+    private ReputMessageService reputMessageService;
 
-    private final ScheduleMessageService scheduleMessageService;
+    private HAService haService;
 
     private final StoreStatsService storeStatsService;
 
@@ -118,8 +123,7 @@ public class DefaultMessageStore implements MessageStore {
     private final RunningFlags runningFlags = new RunningFlags();
     private final SystemClock systemClock = new SystemClock();
 
-    private final ScheduledExecutorService scheduledExecutorService =
-        Executors.newSingleThreadScheduledExecutor(new ThreadFactoryImpl("StoreScheduledThread"));
+    private final ScheduledExecutorService scheduledExecutorService;
     private final BrokerStatsManager brokerStatsManager;
     private final MessageArrivingListener messageArrivingListener;
     private final BrokerConfig brokerConfig;
@@ -137,20 +141,36 @@ public class DefaultMessageStore implements MessageStore {
     private FileLock lock;
 
     boolean shutDownNormal = false;
+    // Max pull msg size
+    private final static int MAX_PULL_MSG_SIZE = 128 * 1024 * 1024;
 
-    private final ScheduledExecutorService diskCheckScheduledExecutorService =
-            Executors.newSingleThreadScheduledExecutor(new ThreadFactoryImpl("DiskCheckScheduledThread"));
+    private volatile int aliveReplicasNum = 1;
 
-    private final List<CleanFilesHook> cleanFilesHooks = new CopyOnWriteArrayList<>();
+    // Refer the MessageStore of MasterBroker in the same process.
+    // If current broker is master, this reference point to null or itself.
+    // If current broker is slave, this reference point to the store of master broker, and the two stores belong to
+    // different broker groups.
+    private MessageStore masterStoreInProcess = null;
 
-    // Max pull msg size
-    private final static int MAX_PULL_MSG_SIZE = 128 * 1024 * 1024;
+    private volatile long masterFlushedOffset = -1L;
+
+    private volatile long brokerInitMaxOffset = -1L;
+
+    protected List<PutMessageHook> putMessageHookList = new ArrayList<>();
+
+    private SendMessageBackHook sendMessageBackHook;
+
+    private final ConcurrentMap<Integer /* level */, Long/* delay timeMillis */> delayLevelTable =
+        new ConcurrentHashMap<Integer, Long>(32);
+
+    private int maxDelayLevel;
 
     public DefaultMessageStore(final MessageStoreConfig messageStoreConfig, final BrokerStatsManager brokerStatsManager,
         final MessageArrivingListener messageArrivingListener, final BrokerConfig brokerConfig) throws IOException {
         this.messageArrivingListener = messageArrivingListener;
         this.brokerConfig = brokerConfig;
         this.messageStoreConfig = messageStoreConfig;
+        this.aliveReplicasNum = messageStoreConfig.getTotalReplicas();
         this.brokerStatsManager = brokerStatsManager;
         this.allocateMappedFileService = new AllocateMappedFileService(this);
         if (messageStoreConfig.isEnableDLegerCommitLog()) {
@@ -164,16 +184,18 @@ public class DefaultMessageStore implements MessageStore {
         this.cleanCommitLogService = new CleanCommitLogService();
         this.cleanConsumeQueueService = new CleanConsumeQueueService();
         this.correctLogicOffsetService = new CorrectLogicOffsetService();
-        this.storeStatsService = new StoreStatsService();
+        this.storeStatsService = new StoreStatsService(brokerConfig);
         this.indexService = new IndexService(this);
-        if (!messageStoreConfig.isEnableDLegerCommitLog()) {
-            this.haService = new HAService(this);
-        } else {
-            this.haService = null;
+        if (!messageStoreConfig.isEnableDLegerCommitLog() && !this.messageStoreConfig.isDuplicationEnable()) {
+            this.haService = ServiceProvider.loadClass(ServiceProvider.HA_SERVICE_ID, HAService.class);
+            if (null == this.haService) {
+                this.haService = new DefaultHAService();
+                LOGGER.warn("Load default HA Service: {}", DefaultHAService.class.getSimpleName());
+            }
+            this.haService.init(this);
         }
-        this.reputMessageService = new ReputMessageService();
 
-        this.scheduleMessageService = new ScheduleMessageService(this);
+        this.reputMessageService = new ReputMessageService();
 
         this.transientStorePool = new TransientStorePool(messageStoreConfig);
 
@@ -185,15 +207,52 @@ public class DefaultMessageStore implements MessageStore {
 
         this.indexService.start();
 
+        this.scheduledExecutorService =
+            Executors.newSingleThreadScheduledExecutor(new ThreadFactoryImpl("StoreScheduledThread", brokerConfig));
+
         this.dispatcherList = new LinkedList<>();
         this.dispatcherList.addLast(new CommitLogDispatcherBuildConsumeQueue());
         this.dispatcherList.addLast(new CommitLogDispatcherBuildIndex());
 
         File file = new File(StorePathConfigHelper.getLockFile(messageStoreConfig.getStorePathRootDir()));
-        DefaultMappedFile.ensureDirOK(file.getParent());
-        DefaultMappedFile.ensureDirOK(getStorePathPhysic());
-        DefaultMappedFile.ensureDirOK(getStorePathLogic());
+        UtilAll.ensureDirOK(file.getParent());
+        UtilAll.ensureDirOK(getStorePathPhysic());
+        UtilAll.ensureDirOK(getStorePathLogic());
         lockFile = new RandomAccessFile(file, "rw");
+
+        parseDelayLevel();
+    }
+
+    public boolean parseDelayLevel() {
+        HashMap<String, Long> timeUnitTable = new HashMap<String, Long>();
+        timeUnitTable.put("s", 1000L);
+        timeUnitTable.put("m", 1000L * 60);
+        timeUnitTable.put("h", 1000L * 60 * 60);
+        timeUnitTable.put("d", 1000L * 60 * 60 * 24);
+
+        String levelString = messageStoreConfig.getMessageDelayLevel();
+        try {
+            String[] levelArray = levelString.split(" ");
+            for (int i = 0; i < levelArray.length; i++) {
+                String value = levelArray[i];
+                String ch = value.substring(value.length() - 1);
+                Long tu = timeUnitTable.get(ch);
+
+                int level = i + 1;
+                if (level > this.maxDelayLevel) {
+                    this.maxDelayLevel = level;
+                }
+                long num = Long.parseLong(value.substring(0, value.length() - 1));
+                long delayTimeMillis = tu * num;
+                this.delayLevelTable.put(level, delayTimeMillis);
+            }
+        } catch (Exception e) {
+            LOGGER.error("parseDelayLevel exception", e);
+            LOGGER.info("levelString String = {}", levelString);
+            return false;
+        }
+
+        return true;
     }
 
     @Override
@@ -210,7 +269,7 @@ public class DefaultMessageStore implements MessageStore {
 
         try {
             boolean lastExitOK = !this.isTempFileExist();
-            log.info("last shutdown {}, root dir: {}", lastExitOK ? "normally" : "abnormally", messageStoreConfig.getStorePathRootDir());
+            LOGGER.info("last shutdown {}, root dir: {}", lastExitOK ? "normally" : "abnormally", messageStoreConfig.getStorePathRootDir());
 
             // load Commit Log
             result = result && this.commitLog.load();
@@ -221,20 +280,19 @@ public class DefaultMessageStore implements MessageStore {
             if (result) {
                 this.storeCheckpoint =
                     new StoreCheckpoint(StorePathConfigHelper.getStoreCheckpoint(this.messageStoreConfig.getStorePathRootDir()));
-
+                this.masterFlushedOffset = this.storeCheckpoint.getMasterFlushedOffset();
                 this.indexService.load(lastExitOK);
 
                 this.recover(lastExitOK);
 
-                log.info("load over, and the max phy offset = {}", this.getMaxPhyOffset());
-
-                if (null != scheduleMessageService) {
-                    result =  this.scheduleMessageService.load();
-                }
+                LOGGER.info("load over, and the max phy offset = {}", this.getMaxPhyOffset());
             }
 
+            long maxOffset = this.getMaxPhyOffset();
+            this.setBrokerInitMaxOffset(maxOffset);
+            LOGGER.info("load over, and the max phy offset = {}", maxOffset);
         } catch (Exception e) {
-            log.error("load exception", e);
+            LOGGER.error("load exception", e);
             result = false;
         }
 
@@ -258,7 +316,10 @@ public class DefaultMessageStore implements MessageStore {
 
         lockFile.getChannel().write(ByteBuffer.wrap("lock".getBytes()));
         lockFile.getChannel().force(true);
-        {
+
+        if (this.getMessageStoreConfig().isDuplicationEnable()) {
+            this.reputMessageService.setReputFromOffset(this.commitLog.getConfirmOffset());
+        } else {
             /**
              * 1. Make sure the fast-forward messages to be truncated during the recovering according to the max physical offset of the commitlog;
              * 2. DLedger committedPos may be missing, so the maxPhysicalPosInLogicQueue maybe bigger that maxOffset returned by DLedgerCommitLog, just let it go;
@@ -286,36 +347,35 @@ public class DefaultMessageStore implements MessageStore {
                  * All the conditions has the same in common that the maxPhysicalPosInLogicQueue should be 0.
                  * If the maxPhysicalPosInLogicQueue is gt 0, there maybe something wrong.
                  */
-                log.warn("[TooSmallCqOffset] maxPhysicalPosInLogicQueue={} clMinOffset={}", maxPhysicalPosInLogicQueue, this.commitLog.getMinOffset());
+                LOGGER.warn("[TooSmallCqOffset] maxPhysicalPosInLogicQueue={} clMinOffset={}", maxPhysicalPosInLogicQueue, this.commitLog.getMinOffset());
             }
-            log.info("[SetReputOffset] maxPhysicalPosInLogicQueue={} clMinOffset={} clMaxOffset={} clConfirmedOffset={}",
+            LOGGER.info("[SetReputOffset] maxPhysicalPosInLogicQueue={} clMinOffset={} clMaxOffset={} clConfirmedOffset={}",
                 maxPhysicalPosInLogicQueue, this.commitLog.getMinOffset(), this.commitLog.getMaxOffset(), this.commitLog.getConfirmOffset());
             this.reputMessageService.setReputFromOffset(maxPhysicalPosInLogicQueue);
-            this.reputMessageService.start();
-
-            /**
-             *  1. Finish dispatching the messages fall behind, then to start other services.
-             *  2. DLedger committedPos may be missing, so here just require dispatchBehindBytes <= 0
-             */
-            while (true) {
-                if (dispatchBehindBytes() <= 0) {
-                    break;
-                }
-                Thread.sleep(1000);
-                log.info("Try to finish doing reput the messages fall behind during the starting, reputOffset={} maxOffset={} behind={}", this.reputMessageService.getReputFromOffset(), this.getMaxPhyOffset(), this.dispatchBehindBytes());
-            }
-            this.recoverTopicQueueTable();
         }
+        this.reputMessageService.start();
 
-        if (!messageStoreConfig.isEnableDLegerCommitLog()) {
-            this.haService.start();
-            this.handleScheduleMessageService(messageStoreConfig.getBrokerRole());
+        /**
+         *  1. Finish dispatching the messages fall behind, then to start other services.
+         *  2. DLedger committedPos may be missing, so here just require dispatchBehindBytes <= 0
+         */
+        while (true) {
+            if (dispatchBehindBytes() <= 0) {
+                break;
+            }
+            Thread.sleep(1000);
+            LOGGER.info("Try to finish doing reput the messages fall behind during the starting, reputOffset={} maxOffset={} behind={}", this.reputMessageService.getReputFromOffset(), this.getMaxPhyOffset(), this.dispatchBehindBytes());
         }
+        this.recoverTopicQueueTable();
 
         this.flushConsumeQueueService.start();
         this.commitLog.start();
         this.storeStatsService.start();
 
+        if (this.haService != null) {
+            this.haService.start();
+        }
+
         this.createTempFile();
         this.addScheduleTask();
         this.perfs.start();
@@ -328,17 +388,13 @@ public class DefaultMessageStore implements MessageStore {
             this.shutdown = true;
 
             this.scheduledExecutorService.shutdown();
-            this.diskCheckScheduledExecutorService.shutdown();
             try {
 
-                Thread.sleep(1000);
+                Thread.sleep(1000 * 3);
             } catch (InterruptedException e) {
-                log.error("shutdown Exception, ", e);
+                LOGGER.error("shutdown Exception, ", e);
             }
 
-            if (this.scheduleMessageService != null) {
-                this.scheduleMessageService.shutdown();
-            }
             if (this.haService != null) {
                 this.haService.shutdown();
             }
@@ -358,7 +414,7 @@ public class DefaultMessageStore implements MessageStore {
                 this.deleteFile(StorePathConfigHelper.getAbortFile(this.messageStoreConfig.getStorePathRootDir()));
                 shutDownNormal = true;
             } else {
-                log.warn("the store may be wrong, so shutdown abnormally, and keep abort file.");
+                LOGGER.warn("the store may be wrong, so shutdown abnormally, and keep abort file.");
             }
         }
 
@@ -382,91 +438,50 @@ public class DefaultMessageStore implements MessageStore {
         this.deleteFile(StorePathConfigHelper.getStoreCheckpoint(this.messageStoreConfig.getStorePathRootDir()));
     }
 
-    @Override
-    public void destroyLogics() {
-        this.consumeQueueStore.destroy();
-    }
-
-    private PutMessageStatus checkMessage(MessageExtBrokerInner msg) {
-        if (msg.getTopic().length() > Byte.MAX_VALUE) {
-            log.warn("putMessage message topic length too long " + msg.getTopic().length());
-            return PutMessageStatus.MESSAGE_ILLEGAL;
+    public long getMajorFileSize() {
+        long commitLogSize = 0;
+        if (this.commitLog != null) {
+            commitLogSize = this.commitLog.getTotalSize();
         }
 
-        if (msg.getPropertiesString() != null && msg.getPropertiesString().length() > Short.MAX_VALUE) {
-            log.warn("putMessage message properties length too long " + msg.getPropertiesString().length());
-            return PutMessageStatus.MESSAGE_ILLEGAL;
+        long consumeQueueSize = 0;
+        if (this.consumeQueueStore != null) {
+            consumeQueueSize = this.consumeQueueStore.getTotalSize();
         }
-        return PutMessageStatus.PUT_OK;
-    }
 
-    private PutMessageStatus checkMessages(MessageExtBatch messageExtBatch) {
-        if (messageExtBatch.getTopic().length() > Byte.MAX_VALUE) {
-            log.warn("putMessage message topic length too long " + messageExtBatch.getTopic().length());
-            return PutMessageStatus.MESSAGE_ILLEGAL;
+        long indexFileSize = 0;
+        if (this.indexService != null) {
+            indexFileSize = this.indexService.getTotalSize();
         }
 
-        if (messageExtBatch.getBody().length > messageStoreConfig.getMaxMessageSize()) {
-            log.warn("PutMessages body length too long " + messageExtBatch.getBody().length);
-            return PutMessageStatus.MESSAGE_ILLEGAL;
-        }
-
-        return PutMessageStatus.PUT_OK;
+        return commitLogSize + consumeQueueSize + indexFileSize;
     }
 
-    private PutMessageStatus checkStoreStatus() {
-        if (this.shutdown) {
-            log.warn("message store has shutdown, so putMessage is forbidden");
-            return PutMessageStatus.SERVICE_NOT_AVAILABLE;
-        }
-
-        if (BrokerRole.SLAVE == this.messageStoreConfig.getBrokerRole()) {
-            long value = this.printTimes.getAndIncrement();
-            if ((value % 50000) == 0) {
-                log.warn("broke role is slave, so putMessage is forbidden");
-            }
-            return PutMessageStatus.SERVICE_NOT_AVAILABLE;
-        }
-
-        if (!this.runningFlags.isWriteable()) {
-            long value = this.printTimes.getAndIncrement();
-            if ((value % 50000) == 0) {
-                log.warn("the message store is not writable. It may be caused by one of the following reasons: " +
-                    "the broker's disk is full, write to logic queue error, write to index file error, etc");
-            }
-            return PutMessageStatus.SERVICE_NOT_AVAILABLE;
-        } else {
-            this.printTimes.set(0);
-        }
-
-        if (this.isOSPageCacheBusy()) {
-            return PutMessageStatus.OS_PAGECACHE_BUSY;
-        }
-        return PutMessageStatus.PUT_OK;
+    @Override
+    public void destroyLogics() {
+        this.consumeQueueStore.destroy();
     }
 
     @Override
     public CompletableFuture<PutMessageResult> asyncPutMessage(MessageExtBrokerInner msg) {
-        PutMessageStatus checkStoreStatus = this.checkStoreStatus();
-        if (checkStoreStatus != PutMessageStatus.PUT_OK) {
-            return CompletableFuture.completedFuture(new PutMessageResult(checkStoreStatus, null));
-        }
 
-        PutMessageStatus msgCheckStatus = this.checkMessage(msg);
-        if (msgCheckStatus == PutMessageStatus.MESSAGE_ILLEGAL) {
-            return CompletableFuture.completedFuture(new PutMessageResult(msgCheckStatus, null));
+        for (PutMessageHook putMessageHook : putMessageHookList) {
+            PutMessageResult handleResult = putMessageHook.executeBeforePutMessage(msg);
+            if (handleResult != null) {
+                return CompletableFuture.completedFuture(handleResult);
+            }
         }
 
         if (msg.getProperties().containsKey(MessageConst.PROPERTY_INNER_NUM)
                 && !MessageSysFlag.check(msg.getSysFlag(), MessageSysFlag.INNER_BATCH_FLAG)) {
-            log.warn("[BUG]The message had property {} but is not an inner batch", MessageConst.PROPERTY_INNER_NUM);
+            LOGGER.warn("[BUG]The message had property {} but is not an inner batch", MessageConst.PROPERTY_INNER_NUM);
             return CompletableFuture.completedFuture(new PutMessageResult(PutMessageStatus.MESSAGE_ILLEGAL, null));
         }
 
         if (MessageSysFlag.check(msg.getSysFlag(), MessageSysFlag.INNER_BATCH_FLAG)) {
             Optional<TopicConfig> topicConfig = this.getTopicConfig(msg.getTopic());
             if (!QueueTypeUtils.isBatchCq(topicConfig)) {
-                log.error("[BUG]The message is an inner batch but cq type is not batch cq");
+                LOGGER.error("[BUG]The message is an inner batch but cq type is not batch cq");
                 return CompletableFuture.completedFuture(new PutMessageResult(PutMessageStatus.MESSAGE_ILLEGAL, null));
             }
         }
@@ -474,10 +489,11 @@ public class DefaultMessageStore implements MessageStore {
         long beginTime = this.getSystemClock().now();
         CompletableFuture<PutMessageResult> putResultFuture = this.commitLog.asyncPutMessage(msg);
 
-        putResultFuture.thenAccept((result) -> {
+        putResultFuture.thenAccept(result -> {
             long elapsedTime = this.getSystemClock().now() - beginTime;
             if (elapsedTime > 500) {
-                log.warn("putMessage not in lock elapsed time(ms)={}, bodyLength={}", elapsedTime, msg.getBody().length);
+                LOGGER.warn("DefaultMessageStore#putMessage: CommitLog#putMessage cost {}ms, topic={}, bodyLength={}",
+                    msg.getTopic(), msg.getBody().length);
             }
             this.storeStatsService.setPutMessageEntireTimeMax(elapsedTime);
 
@@ -489,34 +505,32 @@ public class DefaultMessageStore implements MessageStore {
         return putResultFuture;
     }
 
+    @Override
     public CompletableFuture<PutMessageResult> asyncPutMessages(MessageExtBatch messageExtBatch) {
-        PutMessageStatus checkStoreStatus = this.checkStoreStatus();
-        if (checkStoreStatus != PutMessageStatus.PUT_OK) {
-            return CompletableFuture.completedFuture(new PutMessageResult(checkStoreStatus, null));
-        }
 
-        PutMessageStatus msgCheckStatus = this.checkMessages(messageExtBatch);
-        if (msgCheckStatus == PutMessageStatus.MESSAGE_ILLEGAL) {
-            return CompletableFuture.completedFuture(new PutMessageResult(msgCheckStatus, null));
+        for (PutMessageHook putMessageHook : putMessageHookList) {
+            PutMessageResult handleResult = putMessageHook.executeBeforePutMessage(messageExtBatch);
+            if (handleResult != null) {
+                return CompletableFuture.completedFuture(handleResult);
+            }
         }
 
         long beginTime = this.getSystemClock().now();
-        CompletableFuture<PutMessageResult> resultFuture = this.commitLog.asyncPutMessages(messageExtBatch);
+        CompletableFuture<PutMessageResult> putResultFuture = this.commitLog.asyncPutMessages(messageExtBatch);
 
-        resultFuture.thenAccept((result) -> {
-            long elapsedTime = this.getSystemClock().now() - beginTime;
-            if (elapsedTime > 500) {
-                log.warn("not in lock elapsed time(ms)={}, bodyLength={}", elapsedTime, messageExtBatch.getBody().length);
+        putResultFuture.thenAccept(result -> {
+            long eclipseTime = this.getSystemClock().now() - beginTime;
+            if (eclipseTime > 500) {
+                LOGGER.warn("not in lock eclipse time(ms)={}, bodyLength={}", eclipseTime, messageExtBatch.getBody().length);
             }
-
-            this.storeStatsService.setPutMessageEntireTimeMax(elapsedTime);
+            this.storeStatsService.setPutMessageEntireTimeMax(eclipseTime);
 
             if (null == result || !result.isOk()) {
                 this.storeStatsService.getPutMessageFailedTimes().add(1);
             }
         });
 
-        return resultFuture;
+        return putResultFuture;
     }
 
     @Override
@@ -538,7 +552,7 @@ public class DefaultMessageStore implements MessageStore {
         } catch (ExecutionException | InterruptedException e) {
             return new PutMessageResult(PutMessageStatus.UNKNOWN_ERROR, null);
         } catch (TimeoutException e) {
-            log.error("usually it will never timeout, putMessageTimeout is much bigger than slaveTimeout and "
+            LOGGER.error("usually it will never timeout, putMessageTimeout is much bigger than slaveTimeout and "
                     + "flushTimeout so the result can be got anyway, but in some situations timeout will happen like full gc "
                     + "process hangs or other unexpected situations.");
             return new PutMessageResult(PutMessageStatus.UNKNOWN_ERROR, null);
@@ -560,6 +574,26 @@ public class DefaultMessageStore implements MessageStore {
     }
 
     @Override
+    public long getMasterFlushedOffset() {
+        return this.masterFlushedOffset;
+    }
+
+    @Override
+    public void setMasterFlushedOffset(long masterFlushedOffset) {
+        this.masterFlushedOffset = masterFlushedOffset;
+        this.storeCheckpoint.setMasterFlushedOffset(masterFlushedOffset);
+    }
+
+    @Override
+    public long getBrokerInitMaxOffset() {
+        return this.brokerInitMaxOffset;
+    }
+
+    @Override
+    public void setBrokerInitMaxOffset(long brokerInitMaxOffset) {
+        this.brokerInitMaxOffset = brokerInitMaxOffset;
+    }
+
     public SystemClock getSystemClock() {
         return systemClock;
     }
@@ -569,6 +603,52 @@ public class DefaultMessageStore implements MessageStore {
         return commitLog;
     }
 
+    public void truncateDirtyFiles(long offsetToTruncate) {
+        if (offsetToTruncate >= this.getMaxPhyOffset()) {
+            return;
+        }
+
+        this.reputMessageService.shutdown();
+
+        // truncate commitLog
+        this.commitLog.truncateDirtyFiles(offsetToTruncate);
+
+        // truncate consume queue
+        this.truncateDirtyLogicFiles(offsetToTruncate);
+
+        recoverTopicQueueTable();
+
+        this.reputMessageService = new ReputMessageService();
+        this.reputMessageService.setReputFromOffset(offsetToTruncate);
+        this.reputMessageService.start();
+    }
+
+    @Override
+    public boolean truncateFiles(long offsetToTruncate) {
+        if (offsetToTruncate >= this.getMaxPhyOffset()) {
+            return true;
+        }
+
+        if (!isOffsetAligned(offsetToTruncate)) {
+            LOGGER.error("Offset {} not align, truncate failed, need manual fix");
+            return false;
+        }
+        truncateDirtyFiles(offsetToTruncate);
+        return true;
+    }
+
+    @Override
+    public boolean isOffsetAligned(long offset) {
+        SelectMappedBufferResult mappedBufferResult = this.getCommitLogData(offset);
+
+        if (mappedBufferResult == null) {
+            return true;
+        }
+
+        DispatchRequest dispatchRequest = this.commitLog.checkMessageAndReturnSize(mappedBufferResult.getByteBuffer(), true, false);
+        return dispatchRequest.isSuccess();
+    }
+
     @Override
     public GetMessageResult getMessage(final String group, final String topic, final int queueId, final long offset,
         final int maxMsgNums,
@@ -582,12 +662,12 @@ public class DefaultMessageStore implements MessageStore {
         final int maxTotalMsgSize,
         final MessageFilter messageFilter) {
         if (this.shutdown) {
-            log.warn("message store has shutdown, so getMessage is forbidden");
+            LOGGER.warn("message store has shutdown, so getMessage is forbidden");
             return null;
         }
 
         if (!this.runningFlags.isReadable()) {
-            log.warn("message store is not readable, so getMessage is forbidden " + this.runningFlags.getFlagBits());
+            LOGGER.warn("message store is not readable, so getMessage is forbidden " + this.runningFlags.getFlagBits());
             return null;
         }
 
@@ -618,18 +698,14 @@ public class DefaultMessageStore implements MessageStore {
                 nextBeginOffset = nextOffsetCorrection(offset, offset);
             } else if (offset > maxOffset) {
                 status = GetMessageStatus.OFFSET_OVERFLOW_BADLY;
-                if (0 == minOffset) {
-                    nextBeginOffset = nextOffsetCorrection(offset, minOffset);
-                } else {
-                    nextBeginOffset = nextOffsetCorrection(offset, maxOffset);
-                }
+                nextBeginOffset = nextOffsetCorrection(offset, maxOffset);
             } else {
                 final int maxFilterMessageCount = Math.max(16000, maxMsgNums * ConsumeQueue.CQ_STORE_UNIT_SIZE);
                 final boolean diskFallRecorded = this.messageStoreConfig.isDiskFallRecorded();
 
                 long maxPullSize = Math.max(maxTotalMsgSize, 100);
                 if (maxPullSize > MAX_PULL_MSG_SIZE) {
-                    log.warn("The max pull size is too large maxPullSize={} topic={} queueId={}", maxPullSize, topic, queueId);
+                    LOGGER.warn("The max pull size is too large maxPullSize={} topic={} queueId={}", maxPullSize, topic, queueId);
                     maxPullSize = MAX_PULL_MSG_SIZE;
                 }
                 status = GetMessageStatus.NO_MATCHED_MESSAGE;
@@ -637,22 +713,22 @@ public class DefaultMessageStore implements MessageStore {
                 int cqFileNum = 0;
 
                 while (getResult.getBufferTotalSize() <= 0
-                        && nextBeginOffset < maxOffset
-                        && cqFileNum++ < this.messageStoreConfig.getTravelCqFileNumWhenGetMessage()) {
+                    && nextBeginOffset < maxOffset
+                    && cqFileNum++ < this.messageStoreConfig.getTravelCqFileNumWhenGetMessage()) {
                     ReferredIterator<CqUnit> bufferConsumeQueue = consumeQueue.iterateFrom(nextBeginOffset);
 
                     if (bufferConsumeQueue == null) {
                         status = GetMessageStatus.OFFSET_FOUND_NULL;
                         nextBeginOffset = nextOffsetCorrection(nextBeginOffset, this.consumeQueueStore.rollNextFile(consumeQueue, nextBeginOffset));
-                        log.warn("consumer request topic: " + topic + "offset: " + offset + " minOffset: " + minOffset + " maxOffset: "
-                                + maxOffset + ", but access logic queue failed. Correct nextBeginOffset to " + nextBeginOffset);
+                        LOGGER.warn("consumer request topic: " + topic + "offset: " + offset + " minOffset: " + minOffset + " maxOffset: "
+                            + maxOffset + ", but access logic queue failed. Correct nextBeginOffset to " + nextBeginOffset);
                         break;
                     }
 
                     try {
                         long nextPhyFileStartOffset = Long.MIN_VALUE;
                         while (bufferConsumeQueue.hasNext()
-                                && nextBeginOffset < maxOffset) {
+                            && nextBeginOffset < maxOffset) {
                             CqUnit cqUnit = bufferConsumeQueue.next();
                             long offsetPy = cqUnit.getPos();
                             int sizePy = cqUnit.getSize();
@@ -683,7 +759,7 @@ public class DefaultMessageStore implements MessageStore {
                             }
 
                             if (messageFilter != null
-                                    && !messageFilter.isMatchedByConsumeQueue(cqUnit.getValidTagsCodeAsLong(), cqUnit.getCqExtUnit())) {
+                                && !messageFilter.isMatchedByConsumeQueue(cqUnit.getValidTagsCodeAsLong(), cqUnit.getCqExtUnit())) {
                                 if (getResult.getBufferTotalSize() == 0) {
                                     status = GetMessageStatus.NO_MATCHED_MESSAGE;
                                 }
@@ -702,7 +778,7 @@ public class DefaultMessageStore implements MessageStore {
                             }
 
                             if (messageFilter != null
-                                    && !messageFilter.isMatchedByCommitLog(selectResult.getByteBuffer().slice(), null)) {
+                                && !messageFilter.isMatchedByCommitLog(selectResult.getByteBuffer().slice(), null)) {
                                 if (getResult.getBufferTotalSize() == 0) {
                                     status = GetMessageStatus.NO_MATCHED_MESSAGE;
                                 }
@@ -728,7 +804,7 @@ public class DefaultMessageStore implements MessageStore {
 
                 long diff = maxOffsetPy - maxPhyOffsetPulling;
                 long memory = (long) (StoreUtil.TOTAL_PHYSICAL_MEMORY_SIZE
-                        * (this.messageStoreConfig.getAccessMessageInMemoryMaxRatio() / 100.0));
+                    * (this.messageStoreConfig.getAccessMessageInMemoryMaxRatio() / 100.0));
                 getResult.setSuggestPullingFromSlave(diff > memory);
             }
         } else {
@@ -868,7 +944,7 @@ public class DefaultMessageStore implements MessageStore {
     public String getStorePathPhysic() {
         String storePathPhysic;
         if (DefaultMessageStore.this.getMessageStoreConfig().isEnableDLegerCommitLog()) {
-            storePathPhysic = ((DLedgerCommitLog)DefaultMessageStore.this.getCommitLog()).getdLedgerServer().getdLedgerConfig().getDataStorePath();
+            storePathPhysic = ((DLedgerCommitLog) DefaultMessageStore.this.getCommitLog()).getdLedgerServer().getdLedgerConfig().getDataStorePath();
         } else {
             storePathPhysic = DefaultMessageStore.this.getMessageStoreConfig().getStorePathCommitLog();
         }
@@ -886,10 +962,10 @@ public class DefaultMessageStore implements MessageStore {
         {
             double minPhysicsUsedRatio = Double.MAX_VALUE;
             String commitLogStorePath = getStorePathPhysic();
-            String[] paths = commitLogStorePath.trim().split(MessageStoreConfig.MULTI_PATH_SPLITTER);
+            String[] paths = commitLogStorePath.trim().split(MixAll.MULTI_PATH_SPLITTER);
             for (String clPath : paths) {
                 double physicRatio = UtilAll.isPathExists(clPath) ?
-                        UtilAll.getDiskPartitionSpaceUsedPercent(clPath) : -1;
+                    UtilAll.getDiskPartitionSpaceUsedPercent(clPath) : -1;
                 result.put(RunningStats.commitLogDiskRatio.name() + "_" + clPath, String.valueOf(physicRatio));
                 minPhysicsUsedRatio = Math.min(minPhysicsUsedRatio, physicRatio);
             }
@@ -901,12 +977,6 @@ public class DefaultMessageStore implements MessageStore {
             result.put(RunningStats.consumeQueueDiskRatio.name(), String.valueOf(logicsRatio));
         }
 
-        {
-            if (this.scheduleMessageService != null) {
-                this.scheduleMessageService.buildRunningStats(result);
-            }
-        }
-
         result.put(RunningStats.commitLogMinOffset.name(), String.valueOf(DefaultMessageStore.this.getMinPhyOffset()));
         result.put(RunningStats.commitLogMaxOffset.name(), String.valueOf(DefaultMessageStore.this.getMaxPhyOffset()));
 
@@ -924,6 +994,16 @@ public class DefaultMessageStore implements MessageStore {
     }
 
     @Override
+    public long getLastFileFromOffset() {
+        return this.commitLog.getLastFileFromOffset();
+    }
+
+    @Override
+    public boolean getLastMappedFile(long startOffset) {
+        return this.commitLog.getLastMappedFile(startOffset);
+    }
+
+    @Override
     public long getEarliestMessageTime(String topic, int queueId) {
         ConsumeQueueInterface logicQueue = this.findConsumeQueue(topic, queueId);
         if (logicQueue != null) {
@@ -976,7 +1056,7 @@ public class DefaultMessageStore implements MessageStore {
     @Override
     public SelectMappedBufferResult getCommitLogData(final long offset) {
         if (this.shutdown) {
-            log.warn("message store has shutdown, so getPhyQueueData is forbidden");
+            LOGGER.warn("message store has shutdown, so getPhyQueueData is forbidden");
             return null;
         }
 
@@ -984,9 +1064,19 @@ public class DefaultMessageStore implements MessageStore {
     }
 
     @Override
+    public List<SelectMappedBufferResult> getBulkCommitLogData(final long offset, final int size) {
+        if (this.shutdown) {
+            LOGGER.warn("message store has shutdown, so getBulkCommitLogData is forbidden");
+            return null;
+        }
+
+        return this.commitLog.getBulkData(offset, size);
+    }
+
+    @Override
     public boolean appendToCommitLog(long startOffset, byte[] data, int dataStart, int dataLength) {
         if (this.shutdown) {
-            log.warn("message store has shutdown, so appendToPhyQueue is forbidden");
+            LOGGER.warn("message store has shutdown, so appendToCommitLog is forbidden");
             return false;
         }
 
@@ -994,7 +1084,9 @@ public class DefaultMessageStore implements MessageStore {
         if (result) {
             this.reputMessageService.wakeup();
         } else {
-            log.error("appendToPhyQueue failed " + startOffset + " " + data.length);
+            LOGGER.error(
+                "DefaultMessageStore#appendToCommitLog: failed to append data to commitLog, physical offset={}, data "
+                    + "length={}", startOffset, data.length);
         }
 
         return result;
@@ -1052,10 +1144,10 @@ public class DefaultMessageStore implements MessageStore {
                             queryMessageResult.addMessage(result);
                         }
                     } else {
-                        log.warn("queryMessage hash duplicate, {} {}", topic, key);
+                        LOGGER.warn("queryMessage hash duplicate, topic={}, key={}", topic, key);
                     }
                 } catch (Exception e) {
-                    log.error("queryMessage exception", e);
+                    LOGGER.error("queryMessage exception", e);
                 }
             }
 
@@ -1073,12 +1165,44 @@ public class DefaultMessageStore implements MessageStore {
 
     @Override
     public void updateHaMasterAddress(String newAddr) {
-        this.haService.updateMasterAddress(newAddr);
+        if (this.haService != null) {
+            this.haService.updateHaMasterAddress(newAddr);
+        }
+    }
+
+    @Override
+    public void updateMasterAddress(String newAddr) {
+        if (this.haService != null) {
+            this.haService.updateMasterAddress(newAddr);
+        }
+    }
+
+    @Override
+    public void setAliveReplicaNumInGroup(int aliveReplicaNums) {
+        this.aliveReplicasNum = aliveReplicaNums;
+    }
+
+    @Override
+    public void wakeupHAClient() {
+        if (this.haService != null) {
+            this.haService.getHAClient().wakeup();
+        }
+    }
+
+    @Override
+    public int getAliveReplicaNumInGroup() {
+        return this.aliveReplicasNum;
     }
 
     @Override
     public long slaveFallBehindMuch() {
-        return this.commitLog.getMaxOffset() - this.haService.getPush2SlaveMaxOffset().get();
+        if (this.haService == null || this.messageStoreConfig.isDuplicationEnable() || this.messageStoreConfig.isEnableDLegerCommitLog()) {
+            LOGGER.warn("haServer is null or duplication is enable or enableDLegerCommitLog is true");
+            return -1;
+        } else {
+            return this.commitLog.getMaxOffset() - this.haService.getPush2SlaveMaxOffset().get();
+        }
+
     }
 
     @Override
@@ -1093,13 +1217,11 @@ public class DefaultMessageStore implements MessageStore {
             Entry<String, ConcurrentMap<Integer, ConsumeQueueInterface>> next = it.next();
             String topic = next.getKey();
 
-            if (!topics.contains(topic) && !topic.equals(TopicValidator.RMQ_SYS_SCHEDULE_TOPIC)
-                    && !topic.equals(TopicValidator.RMQ_SYS_TRANS_OP_HALF_TOPIC)
-                    && !MixAll.isLmq(topic)) {
+            if (!topics.contains(topic) && !TopicValidator.isSystemTopic(topic) && !MixAll.isLmq(topic)) {
                 ConcurrentMap<Integer, ConsumeQueueInterface> queueTable = next.getValue();
                 for (ConsumeQueueInterface cq : queueTable.values()) {
                     this.consumeQueueStore.destroy(cq);
-                    log.info("cleanUnusedTopic: {} {} ConsumeQueue cleaned",
+                    LOGGER.info("cleanUnusedTopic: {} {} ConsumeQueue cleaned",
                         cq.getTopic(),
                         cq.getQueueId()
                     );
@@ -1112,7 +1234,7 @@ public class DefaultMessageStore implements MessageStore {
                     this.brokerStatsManager.onTopicDeleted(topic);
                 }
 
-                log.info("cleanUnusedTopic: {},topic destroyed", topic);
+                LOGGER.info("cleanUnusedTopic: {},topic destroyed", topic);
             }
         }
 
@@ -1204,8 +1326,63 @@ public class DefaultMessageStore implements MessageStore {
     }
 
     @Override
+    public long getFlushedWhere() {
+        return this.commitLog.getFlushedWhere();
+    }
+
+    @Override
     public boolean resetWriteOffset(long phyOffset) {
-        return this.commitLog.resetOffset(phyOffset);
+        //copy a new map
+        ConcurrentHashMap<String, Long> newMap = new ConcurrentHashMap<>(consumeQueueStore.getTopicQueueTable());
+        SelectMappedBufferResult lastBuffer = null;
+        long startReadOffset = phyOffset == -1 ? 0 : phyOffset;
+        while ((lastBuffer = selectOneMessageByOffset(startReadOffset)) != null) {
+            try {
+                if (lastBuffer.getStartOffset() > startReadOffset) {
+                    startReadOffset = lastBuffer.getStartOffset();
+                    continue;
+                }
+
+                ByteBuffer bb = lastBuffer.getByteBuffer();
+                int magicCode = bb.getInt(bb.position() + 4);
+                if (magicCode == CommitLog.BLANK_MAGIC_CODE) {
+                    startReadOffset += bb.getInt(bb.position());
+                    continue;
+                } else if (magicCode != MessageDecoder.MESSAGE_MAGIC_CODE) {
+                    throw new RuntimeException("Unknown magicCode: " + magicCode);
+                }
+
+                lastBuffer.getByteBuffer().mark();
+
+                DispatchRequest dispatchRequest = checkMessageAndReturnSize(lastBuffer.getByteBuffer(), true, messageStoreConfig.isDuplicationEnable(), true);
+                if (!dispatchRequest.isSuccess())
+                    break;
+
+                lastBuffer.getByteBuffer().reset();
+
+                MessageExt msg = MessageDecoder.decode(lastBuffer.getByteBuffer(), true, false, false, false, true);
+                if (msg == null) {
+                    break;
+                }
+                String key = msg.getTopic() + "-" + msg.getQueueId();
+                Long cur = newMap.get(key);
+                if (cur != null && cur > msg.getQueueOffset()) {
+                    newMap.put(key, msg.getQueueOffset());
+                }
+                startReadOffset += msg.getStoreSize();
+            } catch (Throwable e) {
+                LOGGER.error("resetWriteOffset error.", e);
+            } finally {
+                if (lastBuffer != null)
+                    lastBuffer.release();
+            }
+        }
+        if (this.commitLog.resetOffset(phyOffset)) {
+            this.consumeQueueStore.setTopicQueueTable(newMap);
+            return true;
+        } else {
+            return false;
+        }
     }
 
     @Override
@@ -1219,6 +1396,55 @@ public class DefaultMessageStore implements MessageStore {
     }
 
     @Override
+    public byte[] calcDeltaChecksum(long from, long to) {
+        if (from < 0 || to <= from) {
+            return new byte[0];
+        }
+
+        int size = (int) (to - from);
+
+        if (size > this.messageStoreConfig.getMaxChecksumRange()) {
+            LOGGER.error("Checksum range from {}, size {} exceeds threshold {}", from, size, this.messageStoreConfig.getMaxChecksumRange());
+            return null;
+        }
+
+        List<MessageExt> msgList = new ArrayList<>();
+        List<SelectMappedBufferResult> bufferResultList = this.getBulkCommitLogData(from, size);
+        if (bufferResultList.isEmpty()) {
+            return new byte[0];
+        }
+
+        for (SelectMappedBufferResult bufferResult : bufferResultList) {
+            msgList.addAll(MessageDecoder.decodesBatch(bufferResult.getByteBuffer(), true, false, false));
+            bufferResult.release();
+        }
+
+        if (msgList.isEmpty()) {
+            return new byte[0];
+        }
+
+        ByteBuffer byteBuffer = ByteBuffer.allocate(size);
+        for (MessageExt msg : msgList) {
+            try {
+                byteBuffer.put(MessageDecoder.encodeUniquely(msg, false));
+            } catch (IOException ignore) {
+            }
+        }
+
+        return Hashing.murmur3_128().hashBytes(byteBuffer.array()).asBytes();
+    }
+
+    @Override
+    public void setPhysicalOffset(long phyOffset) {
+        this.commitLog.setMappedFileQueueOffset(phyOffset);
+    }
+
+    @Override
+    public boolean isMappedFilesEmpty() {
+        return this.commitLog.isMappedFilesEmpty();
+    }
+
+    @Override
     public MessageExt lookMessageByOffset(long commitLogOffset, int size) {
         SelectMappedBufferResult sbr = this.commitLog.getMessage(commitLogOffset, size);
         if (null != sbr) {
@@ -1238,7 +1464,8 @@ public class DefaultMessageStore implements MessageStore {
 
     private long nextOffsetCorrection(long oldOffset, long newOffset) {
         long nextOffset = oldOffset;
-        if (this.getMessageStoreConfig().getBrokerRole() != BrokerRole.SLAVE || this.getMessageStoreConfig().isOffsetCheckInSlave()) {
+        if (this.getMessageStoreConfig().getBrokerRole() != BrokerRole.SLAVE ||
+            this.getMessageStoreConfig().isOffsetCheckInSlave()) {
             nextOffset = newOffset;
         }
         return nextOffset;
@@ -1249,7 +1476,8 @@ public class DefaultMessageStore implements MessageStore {
         return (maxOffsetPy - offsetPy) > memory;
     }
 
-    private boolean isTheBatchFull(int sizePy, int unitBatchNum, int maxMsgNums, long maxMsgSize, int bufferTotal, int messageTotal, boolean isInDisk) {
+    private boolean isTheBatchFull(int sizePy, int unitBatchNum, int maxMsgNums, long maxMsgSize, int bufferTotal,
+        int messageTotal, boolean isInDisk) {
 
         if (0 == bufferTotal || 0 == messageTotal) {
             return false;
@@ -1287,7 +1515,7 @@ public class DefaultMessageStore implements MessageStore {
     private void deleteFile(final String fileName) {
         File file = new File(fileName);
         boolean result = file.delete();
-        log.info(fileName + (result ? " delete OK" : " delete Failed"));
+        LOGGER.info(fileName + (result ? " delete OK" : " delete Failed"));
     }
 
     /**
@@ -1296,42 +1524,31 @@ public class DefaultMessageStore implements MessageStore {
     private void createTempFile() throws IOException {
         String fileName = StorePathConfigHelper.getAbortFile(this.messageStoreConfig.getStorePathRootDir());
         File file = new File(fileName);
-        DefaultMappedFile.ensureDirOK(file.getParent());
+        UtilAll.ensureDirOK(file.getParent());
         boolean result = file.createNewFile();
-        log.info(fileName + (result ? " create OK" : " already exists"));
-    }
-
-    @Override
-    public void registerCleanFileHook(CleanFilesHook hook) {
-        this.cleanFilesHooks.add(hook);
+        LOGGER.info(fileName + (result ? " create OK" : " already exists"));
+        MixAll.string2File(Long.toString(MixAll.getPID()), file.getAbsolutePath());
     }
 
     private void addScheduleTask() {
 
-        this.scheduledExecutorService.scheduleAtFixedRate(new Runnable() {
+        this.scheduledExecutorService.scheduleAtFixedRate(new AbstractBrokerRunnable(this.brokerConfig) {
             @Override
-            public void run() {
-                long deleteCount = DefaultMessageStore.this.cleanFilesPeriodically();
-                DefaultMessageStore.this.cleanFilesHooks.forEach(hook -> {
-                    try {
-                        hook.execute(DefaultMessageStore.this, deleteCount);
-                    } catch (Throwable t) {
-                        log.error("execute CleanFilesHook[{}] error", hook.getName(), t);
-                    }
-                });
+            public void run2() {
+                DefaultMessageStore.this.cleanFilesPeriodically();
             }
         }, 1000 * 60, this.messageStoreConfig.getCleanResourceInterval(), TimeUnit.MILLISECONDS);
 
-        this.scheduledExecutorService.scheduleAtFixedRate(new Runnable() {
+        this.scheduledExecutorService.scheduleAtFixedRate(new AbstractBrokerRunnable(this.brokerConfig) {
             @Override
-            public void run() {
+            public void run2() {
                 DefaultMessageStore.this.checkSelf();
             }
         }, 1, 10, TimeUnit.MINUTES);
 
-        this.scheduledExecutorService.scheduleAtFixedRate(new Runnable() {
+        this.scheduledExecutorService.scheduleAtFixedRate(new AbstractBrokerRunnable(this.brokerConfig) {
             @Override
-            public void run() {
+            public void run2() {
                 if (DefaultMessageStore.this.getMessageStoreConfig().isDebugLockEnable()) {
                     try {
                         if (DefaultMessageStore.this.commitLog.getBeginTimeInLock() != 0) {
@@ -1350,26 +1567,25 @@ public class DefaultMessageStore implements MessageStore {
             }
         }, 1, 1, TimeUnit.SECONDS);
 
+        this.scheduledExecutorService.scheduleAtFixedRate(new AbstractBrokerRunnable(this.brokerConfig) {
+            @Override
+            public void run2() {
+                DefaultMessageStore.this.storeCheckpoint.flush();
+            }
+        }, 1, 1, TimeUnit.SECONDS);
+
         // this.scheduledExecutorService.scheduleAtFixedRate(new Runnable() {
         // @Override
         // public void run() {
         // DefaultMessageStore.this.cleanExpiredConsumerQueue();
         // }
         // }, 1, 1, TimeUnit.HOURS);
-        this.diskCheckScheduledExecutorService.scheduleAtFixedRate(new Runnable() {
-            public void run() {
-                DefaultMessageStore.this.cleanCommitLogService.isSpaceFull();
-            }
-        }, 1000L, 10000L, TimeUnit.MILLISECONDS);
     }
 
-    private long cleanFilesPeriodically() {
-        long deleteCount = 0L;
-        deleteCount += this.cleanCommitLogService.run();
-        deleteCount += this.cleanConsumeQueueService.run();
-
+    private void cleanFilesPeriodically() {
+        this.cleanCommitLogService.run();
+        this.cleanConsumeQueueService.run();
         this.correctLogicOffsetService.run();
-        return deleteCount;
     }
 
     private void checkSelf() {
@@ -1397,8 +1613,8 @@ public class DefaultMessageStore implements MessageStore {
         this.recoverTopicQueueTable();
         long recoverOffsetEnd = System.currentTimeMillis();
 
-        log.info("Recover end total:{} recoverCq:{} recoverClog:{} recoverOffset:{}",
-                recoverOffsetEnd - recoverCqStart, recoverCqEnd - recoverCqStart, recoverClogEnd - recoverCqEnd, recoverOffsetEnd - recoverClogEnd);
+        LOGGER.info("Recover end total:{} recoverCq:{} recoverClog:{} recoverOffset:{}",
+            recoverOffsetEnd - recoverCqStart, recoverCqEnd - recoverCqStart, recoverClogEnd - recoverCqEnd, recoverOffsetEnd - recoverClogEnd);
     }
 
     @Override
@@ -1449,11 +1665,6 @@ public class DefaultMessageStore implements MessageStore {
     }
 
     @Override
-    public ScheduleMessageService getScheduleMessageService() {
-        return scheduleMessageService;
-    }
-
-    @Override
     public RunningFlags getRunningFlags() {
         return runningFlags;
     }
@@ -1469,19 +1680,22 @@ public class DefaultMessageStore implements MessageStore {
     }
 
     @Override
+    public DispatchRequest checkMessageAndReturnSize(final ByteBuffer byteBuffer, final boolean checkCRC,
+        final boolean checkDupInfo, final boolean readBody) {
+        return this.commitLog.checkMessageAndReturnSize(byteBuffer, checkCRC, checkDupInfo, readBody);
+    }
+
+    @Override
+    public long getStateMachineVersion() {
+        return 0L;
+    }
+
     public BrokerStatsManager getBrokerStatsManager() {
         return brokerStatsManager;
     }
 
-    @Override
-    public void handleScheduleMessageService(final BrokerRole brokerRole) {
-        if (this.scheduleMessageService != null) {
-            if (brokerRole == BrokerRole.SLAVE) {
-                this.scheduleMessageService.shutdown();
-            } else {
-                this.scheduleMessageService.start();
-            }
-        }
+    public BrokerConfig getBrokerConfig() {
+        return brokerConfig;
     }
 
     public int remainTransientStoreBufferNumbs() {
@@ -1494,11 +1708,36 @@ public class DefaultMessageStore implements MessageStore {
     }
 
     @Override
+    public long remainHowManyDataToCommit() {
+        return this.commitLog.remainHowManyDataToCommit();
+    }
+
+    @Override
+    public long remainHowManyDataToFlush() {
+        return this.commitLog.remainHowManyDataToFlush();
+    }
+
+    @Override
     public LinkedList<CommitLogDispatcher> getDispatcherList() {
         return this.dispatcherList;
     }
 
     @Override
+    public void setMasterStoreInProcess(MessageStore masterStoreInProcess) {
+        this.masterStoreInProcess = masterStoreInProcess;
+    }
+
+    @Override
+    public MessageStore getMasterStoreInProcess() {
+        return this.masterStoreInProcess;
+    }
+
+    @Override
+    public boolean getData(long offset, int size, ByteBuffer byteBuffer) {
+        return this.commitLog.getData(offset, size, byteBuffer);
+    }
+
+    @Override
     public ConsumeQueueInterface getConsumeQueue(String topic, int queueId) {
         ConcurrentMap<Integer, ConsumeQueueInterface> map = this.getConsumeQueueTable().get(topic);
         if (map == null) {
@@ -1533,7 +1772,8 @@ public class DefaultMessageStore implements MessageStore {
     }
 
     @Override
-    public void onCommitLogDispatch(DispatchRequest dispatchRequest, boolean doDispatch, MappedFile commitLogFile, boolean isRecover, boolean isFileEnd) {
+    public void onCommitLogDispatch(DispatchRequest dispatchRequest, boolean doDispatch, MappedFile commitLogFile,
+        boolean isRecover, boolean isFileEnd) {
         if (doDispatch && !isFileEnd) {
             this.doDispatch(dispatchRequest);
         }
@@ -1550,7 +1790,7 @@ public class DefaultMessageStore implements MessageStore {
     }
 
     @Override
-    public void assignOffset(String topicQueueKey, MessageExtBrokerInner msg, short messageNum) {
+    public void assignOffset(MessageExtBrokerInner msg, short messageNum) {
         final int tranType = MessageSysFlag.getTransactionValue(msg.getSysFlag());
 
         if (tranType == MessageSysFlag.TRANSACTION_NOT_TYPE || tranType == MessageSysFlag.TRANSACTION_COMMIT_TYPE) {
@@ -1598,16 +1838,18 @@ public class DefaultMessageStore implements MessageStore {
 
         private final static int MAX_MANUAL_DELETE_FILE_TIMES = 20;
         private final String diskSpaceWarningLevelRatio =
-                System.getProperty("rocketmq.broker.diskSpaceWarningLevelRatio", "");
+            System.getProperty("rocketmq.broker.diskSpaceWarningLevelRatio", "");
 
         private final String diskSpaceCleanForciblyRatio =
-                System.getProperty("rocketmq.broker.diskSpaceCleanForciblyRatio", "");
+            System.getProperty("rocketmq.broker.diskSpaceCleanForciblyRatio", "");
         private long lastRedeleteTimestamp = 0;
 
         private volatile int manualDeleteFileSeveralTimes = 0;
 
         private volatile boolean cleanImmediately = false;
 
+        private int forceCleanFailedTimes = 0;
+
         double getDiskSpaceWarningLevelRatio() {
             double finalDiskSpaceWarningLevelRatio;
             if ("".equals(diskSpaceWarningLevelRatio)) {
@@ -1646,77 +1888,76 @@ public class DefaultMessageStore implements MessageStore {
 
         public void executeDeleteFilesManually() {
             this.manualDeleteFileSeveralTimes = MAX_MANUAL_DELETE_FILE_TIMES;
-            DefaultMessageStore.log.info("executeDeleteFilesManually was invoked");
+            DefaultMessageStore.LOGGER.info("executeDeleteFilesManually was invoked");
         }
 
-        public long run() {
-            int deleteCount = 0;
+        public void run() {
             try {
-                deleteCount = this.deleteExpiredFiles();
-
-                this.redeleteHangedFile();
+                this.deleteExpiredFiles();
+                this.reDeleteHangedFile();
             } catch (Throwable e) {
-                DefaultMessageStore.log.warn(this.getServiceName() + " service has exception. ", e);
+                DefaultMessageStore.LOGGER.warn(this.getServiceName() + " service has exception. ", e);
             }
-            return deleteCount;
         }
 
-        private int deleteExpiredFiles() {
+        private void deleteExpiredFiles() {
             int deleteCount = 0;
             long fileReservedTime = DefaultMessageStore.this.getMessageStoreConfig().getFileReservedTime();
             int deletePhysicFilesInterval = DefaultMessageStore.this.getMessageStoreConfig().getDeleteCommitLogFilesInterval();
-            int destroyMapedFileIntervalForcibly = DefaultMessageStore.this.getMessageStoreConfig().getDestroyMapedFileIntervalForcibly();
+            int destroyMappedFileIntervalForcibly = DefaultMessageStore.this.getMessageStoreConfig().getDestroyMapedFileIntervalForcibly();
+            int deleteFileBatchMax = DefaultMessageStore.this.getMessageStoreConfig().getDeleteFileBatchMax();
 
-            boolean timeup = this.isTimeToDelete();
-            boolean spacefull = this.isSpaceToDelete();
-            boolean manualDelete = this.manualDeleteFileSeveralTimes > 0;
+            boolean isTimeUp = this.isTimeToDelete();
+            boolean isUsageExceedsThreshold = this.isSpaceToDelete();
+            boolean isManualDelete = this.manualDeleteFileSeveralTimes > 0;
 
-            if (timeup || spacefull || manualDelete) {
+            if (isTimeUp || isUsageExceedsThreshold || isManualDelete) {
 
-                if (manualDelete)
+                if (isManualDelete) {
                     this.manualDeleteFileSeveralTimes--;
+                }
 
                 boolean cleanAtOnce = DefaultMessageStore.this.getMessageStoreConfig().isCleanFileForciblyEnable() && this.cleanImmediately;
 
-                log.info("begin to delete before {} hours file. timeup: {} spacefull: {} manualDeleteFileSeveralTimes: {} cleanAtOnce: {}",
+                LOGGER.info("begin to delete before {} hours file. isTimeUp: {} isUsageExceedsThreshold: {} manualDeleteFileSeveralTimes: {} cleanAtOnce: {} deleteFileBatchMax: {}",
                     fileReservedTime,
-                    timeup,
-                    spacefull,
+                    isTimeUp,
+                    isUsageExceedsThreshold,
                     manualDeleteFileSeveralTimes,
-                    cleanAtOnce);
+                    cleanAtOnce,
+                    deleteFileBatchMax);
 
                 fileReservedTime *= 60 * 60 * 1000;
 
                 deleteCount = DefaultMessageStore.this.commitLog.deleteExpiredFile(fileReservedTime, deletePhysicFilesInterval,
-                    destroyMapedFileIntervalForcibly, cleanAtOnce);
+                    destroyMappedFileIntervalForcibly, cleanAtOnce, deleteFileBatchMax);
                 if (deleteCount > 0) {
-                } else if (spacefull) {
-                    log.warn("disk space will be full soon, but delete file failed.");
+                } else if (isUsageExceedsThreshold) {
+                    LOGGER.warn("disk space will be full soon, but delete file failed.");
                 }
             }
-            return deleteCount;
         }
 
-        private void redeleteHangedFile() {
+        private void reDeleteHangedFile() {
             int interval = DefaultMessageStore.this.getMessageStoreConfig().getRedeleteHangedFileInterval();
             long currentTimestamp = System.currentTimeMillis();
             if ((currentTimestamp - this.lastRedeleteTimestamp) > interval) {
                 this.lastRedeleteTimestamp = currentTimestamp;
-                int destroyMapedFileIntervalForcibly =
+                int destroyMappedFileIntervalForcibly =
                     DefaultMessageStore.this.getMessageStoreConfig().getDestroyMapedFileIntervalForcibly();
-                if (DefaultMessageStore.this.commitLog.retryDeleteFirstFile(destroyMapedFileIntervalForcibly)) {
+                if (DefaultMessageStore.this.commitLog.retryDeleteFirstFile(destroyMappedFileIntervalForcibly)) {
                 }
             }
         }
 
         public String getServiceName() {
-            return CleanCommitLogService.class.getSimpleName();
+            return DefaultMessageStore.this.brokerConfig.getLoggerIdentifier() + CleanCommitLogService.class.getSimpleName();
         }
 
         private boolean isTimeToDelete() {
             String when = DefaultMessageStore.this.getMessageStoreConfig().getDeleteWhen();
             if (UtilAll.isItTimeToDo(when)) {
-                DefaultMessageStore.log.info("it's time to reclaim disk space, " + when);
+                DefaultMessageStore.LOGGER.info("it's time to reclaim disk space, " + when);
                 return true;
             }
 
@@ -1724,78 +1965,99 @@ public class DefaultMessageStore implements MessageStore {
         }
 
         private boolean isSpaceToDelete() {
-            double ratio = DefaultMessageStore.this.getMessageStoreConfig().getDiskMaxUsedSpaceRatio() / 100.0;
-
             cleanImmediately = false;
 
-            {
-                String commitLogStorePath = DefaultMessageStore.this.getStorePathPhysic();
-                String[] storePaths = commitLogStorePath.trim().split(MessageStoreConfig.MULTI_PATH_SPLITTER);
-                Set<String> fullStorePath = new HashSet<>();
-                double minPhysicRatio = 100;
-                String minStorePath = null;
-                for (String storePathPhysic : storePaths) {
-                    double physicRatio = UtilAll.getDiskPartitionSpaceUsedPercent(storePathPhysic);
-                    if (minPhysicRatio > physicRatio) {
-                        minPhysicRatio =  physicRatio;
-                        minStorePath = storePathPhysic;
-                    }
-                    if (physicRatio > getDiskSpaceCleanForciblyRatio()) {
-                        fullStorePath.add(storePathPhysic);
-                    }
+            String commitLogStorePath = DefaultMessageStore.this.getMessageStoreConfig().getStorePathCommitLog();
+            String[] storePaths = commitLogStorePath.trim().split(MixAll.MULTI_PATH_SPLITTER);
+            Set<String> fullStorePath = new HashSet<>();
+            double minPhysicRatio = 100;
+            String minStorePath = null;
+            for (String storePathPhysic : storePaths) {
+                double physicRatio = UtilAll.getDiskPartitionSpaceUsedPercent(storePathPhysic);
+                if (minPhysicRatio > physicRatio) {
+                    minPhysicRatio = physicRatio;
+                    minStorePath = storePathPhysic;
                 }
-                DefaultMessageStore.this.commitLog.setFullStorePaths(fullStorePath);
-                if (minPhysicRatio > getDiskSpaceWarningLevelRatio()) {
-                    boolean diskok = DefaultMessageStore.this.runningFlags.getAndMakeDiskFull();
-                    if (diskok) {
-                        DefaultMessageStore.log.error("physic disk maybe full soon " + minPhysicRatio +
-                                ", so mark disk full, storePathPhysic=" + minStorePath);
-                    }
-
-                    cleanImmediately = true;
-                } else if (minPhysicRatio > getDiskSpaceCleanForciblyRatio()) {
-                    cleanImmediately = true;
-                } else {
-                    boolean diskok = DefaultMessageStore.this.runningFlags.getAndMakeDiskOK();
-                    if (!diskok) {
-                        DefaultMessageStore.log.info("physic disk space OK " + minPhysicRatio +
-                                ", so mark disk ok, storePathPhysic=" + minStorePath);
-                    }
+                if (physicRatio > getDiskSpaceCleanForciblyRatio()) {
+                    fullStorePath.add(storePathPhysic);
+                }
+            }
+            DefaultMessageStore.this.commitLog.setFullStorePaths(fullStorePath);
+            if (minPhysicRatio > getDiskSpaceWarningLevelRatio()) {
+                boolean diskFull = DefaultMessageStore.this.runningFlags.getAndMakeDiskFull();
+                if (diskFull) {
+                    DefaultMessageStore.LOGGER.error("physic disk maybe full soon " + minPhysicRatio +
+                        ", so mark disk full, storePathPhysic=" + minStorePath);
                 }
 
-                if (minPhysicRatio < 0 || minPhysicRatio > ratio) {
-                    DefaultMessageStore.log.info("physic disk maybe full soon, so reclaim space, "
-                            + minPhysicRatio + ", storePathPhysic=" + minStorePath);
-                    return true;
+                cleanImmediately = true;
+                return true;
+            } else if (minPhysicRatio > getDiskSpaceCleanForciblyRatio()) {
+                cleanImmediately = true;
+                return true;
+            } else {
+                boolean diskOK = DefaultMessageStore.this.runningFlags.getAndMakeDiskOK();
+                if (!diskOK) {
+                    DefaultMessageStore.LOGGER.info("physic disk space OK " + minPhysicRatio +
+                        ", so mark disk ok, storePathPhysic=" + minStorePath);
                 }
             }
 
-            {
-                String storePathLogics = DefaultMessageStore.this.getStorePathLogic();
-                double logicsRatio = UtilAll.getDiskPartitionSpaceUsedPercent(storePathLogics);
-                if (logicsRatio > getDiskSpaceWarningLevelRatio()) {
-                    boolean diskok = DefaultMessageStore.this.runningFlags.getAndMakeDiskFull();
-                    if (diskok) {
-                        DefaultMessageStore.log.error("logics disk maybe full soon " + logicsRatio + ", so mark disk full");
-                    }
+            String storePathLogics = StorePathConfigHelper
+                .getStorePathConsumeQueue(DefaultMessageStore.this.getMessageStoreConfig().getStorePathRootDir());
+            double logicsRatio = UtilAll.getDiskPartitionSpaceUsedPercent(storePathLogics);
+            if (logicsRatio > getDiskSpaceWarningLevelRatio()) {
+                boolean diskOK = DefaultMessageStore.this.runningFlags.getAndMakeDiskFull();
+                if (diskOK) {
+                    DefaultMessageStore.LOGGER.error("logics disk maybe full soon " + logicsRatio + ", so mark disk full");
+                }
 
-                    cleanImmediately = true;
-                } else if (logicsRatio > getDiskSpaceCleanForciblyRatio()) {
-                    cleanImmediately = true;
-                } else {
-                    boolean diskok = DefaultMessageStore.this.runningFlags.getAndMakeDiskOK();
-                    if (!diskok) {
-                        DefaultMessageStore.log.info("logics disk space OK " + logicsRatio + ", so mark disk ok");
-                    }
+                cleanImmediately = true;
+                return true;
+            } else if (logicsRatio > getDiskSpaceCleanForciblyRatio()) {
+                cleanImmediately = true;
+                return true;
+            } else {
+                boolean diskOK = DefaultMessageStore.this.runningFlags.getAndMakeDiskOK();
+                if (!diskOK) {
+                    DefaultMessageStore.LOGGER.info("logics disk space OK " + logicsRatio + ", so mark disk ok");
+                }
+            }
+
+            double ratio = DefaultMessageStore.this.getMessageStoreConfig().getDiskMaxUsedSpaceRatio() / 100.0;
+            int replicasPerPartition = DefaultMessageStore.this.getMessageStoreConfig().getReplicasPerDiskPartition();
+            // Only one commitLog in node
+            if (replicasPerPartition <= 1) {
+                if (minPhysicRatio < 0 || minPhysicRatio > ratio) {
+                    DefaultMessageStore.LOGGER.info("commitLog disk maybe full soon, so reclaim space, " + minPhysicRatio);
+                    return true;
                 }
 
                 if (logicsRatio < 0 || logicsRatio > ratio) {
-                    DefaultMessageStore.log.info("logics disk maybe full soon, so reclaim space, " + logicsRatio);
+                    DefaultMessageStore.LOGGER.info("consumeQueue disk maybe full soon, so reclaim space, " + logicsRatio);
+                    return true;
+                }
+                return false;
+            } else {
+                long majorFileSize = DefaultMessageStore.this.getMajorFileSize();
+                long partitionLogicalSize = UtilAll.getDiskPartitionTotalSpace(minStorePath) / replicasPerPartition;
+                double logicalRatio = 1.0 * majorFileSize / partitionLogicalSize;
+
+                if (logicalRatio > DefaultMessageStore.this.getMessageStoreConfig().getLogicalDiskSpaceCleanForciblyThreshold()) {
+                    // if logical ratio exceeds 0.80, then clean immediately
+                    DefaultMessageStore.LOGGER.info("Logical disk usage {} exceeds logical disk space clean forcibly threshold {}, forcibly: {}",
+                        logicalRatio, minPhysicRatio, cleanImmediately);
+                    cleanImmediately = true;
                     return true;
                 }
-            }
 
-            return false;
+                boolean isUsageExceedsThreshold = logicalRatio > ratio;
+                if (isUsageExceedsThreshold) {
+                    DefaultMessageStore.LOGGER.info("Logical disk usage {} exceeds clean threshold {}, forcibly: {}",
+                        logicalRatio, ratio, cleanImmediately);
+                }
+                return isUsageExceedsThreshold;
+            }
         }
 
         public int getManualDeleteFileSeveralTimes() {
@@ -1809,11 +2071,11 @@ public class DefaultMessageStore implements MessageStore {
         public double calcStorePathPhysicRatio() {
             Set<String> fullStorePath = new HashSet<>();
             String storePath = getStorePathPhysic();
-            String[] paths = storePath.trim().split(MessageStoreConfig.MULTI_PATH_SPLITTER);
+            String[] paths = storePath.trim().split(MixAll.MULTI_PATH_SPLITTER);
             double minPhysicRatio = 100;
             for (String path : paths) {
                 double physicRatio = UtilAll.isPathExists(path) ?
-                        UtilAll.getDiskPartitionSpaceUsedPercent(path) : -1;
+                    UtilAll.getDiskPartitionSpaceUsedPercent(path) : -1;
                 minPhysicRatio = Math.min(minPhysicRatio, physicRatio);
                 if (physicRatio > getDiskSpaceCleanForciblyRatio()) {
                     fullStorePath.add(path);
@@ -1828,12 +2090,12 @@ public class DefaultMessageStore implements MessageStore {
             double physicRatio = calcStorePathPhysicRatio();
             double ratio = DefaultMessageStore.this.getMessageStoreConfig().getDiskMaxUsedSpaceRatio() / 100.0;
             if (physicRatio > ratio) {
-                DefaultMessageStore.log.info("physic disk of commitLog used: " + physicRatio);
+                DefaultMessageStore.LOGGER.info("physic disk of commitLog used: " + physicRatio);
             }
             if (physicRatio > this.getDiskSpaceWarningLevelRatio()) {
                 boolean diskok = DefaultMessageStore.this.runningFlags.getAndMakeDiskFull();
                 if (diskok) {
-                    DefaultMessageStore.log.error("physic disk of commitLog maybe full soon, used " + physicRatio + ", so mark disk full");
+                    DefaultMessageStore.LOGGER.error("physic disk of commitLog maybe full soon, used " + physicRatio + ", so mark disk full");
                 }
 
                 return true;
@@ -1841,7 +2103,7 @@ public class DefaultMessageStore implements MessageStore {
                 boolean diskok = DefaultMessageStore.this.runningFlags.getAndMakeDiskOK();
 
                 if (!diskok) {
-                    DefaultMessageStore.log.info("physic disk space of commitLog OK " + physicRatio + ", so mark disk ok");
+                    DefaultMessageStore.LOGGER.info("physic disk space of commitLog OK " + physicRatio + ", so mark disk ok");
                 }
 
                 return false;
@@ -1852,20 +2114,17 @@ public class DefaultMessageStore implements MessageStore {
     class CleanConsumeQueueService {
         private long lastPhysicalMinOffset = 0;
 
-        public long run() {
-            long deleteCount = 0;
+        public void run() {
             try {
-                deleteCount = this.deleteExpiredFiles();
+                this.deleteExpiredFiles();
             } catch (Throwable e) {
-                DefaultMessageStore.log.warn(this.getServiceName() + " service has exception. ", e);
+                DefaultMessageStore.LOGGER.warn(this.getServiceName() + " service has exception. ", e);
             }
-            return deleteCount;
         }
 
-        private long deleteExpiredFiles() {
+        private void deleteExpiredFiles() {
             int deleteLogicsFilesInterval = DefaultMessageStore.this.getMessageStoreConfig().getDeleteConsumeQueueFilesInterval();
 
-            long deleteCountSum = 0L;
             long minOffset = DefaultMessageStore.this.commitLog.getMinOffset();
             if (minOffset > this.lastPhysicalMinOffset) {
                 this.lastPhysicalMinOffset = minOffset;
@@ -1875,7 +2134,6 @@ public class DefaultMessageStore implements MessageStore {
                 for (ConcurrentMap<Integer, ConsumeQueueInterface> maps : tables.values()) {
                     for (ConsumeQueueInterface logic : maps.values()) {
                         int deleteCount = DefaultMessageStore.this.consumeQueueStore.deleteExpiredFile(logic, minOffset);
-                        deleteCountSum += deleteCount;
                         if (deleteCount > 0 && deleteLogicsFilesInterval > 0) {
                             try {
                                 Thread.sleep(deleteLogicsFilesInterval);
@@ -1887,11 +2145,10 @@ public class DefaultMessageStore implements MessageStore {
 
                 DefaultMessageStore.this.indexService.deleteExpiredFile(minOffset);
             }
-            return deleteCountSum;
         }
 
         public String getServiceName() {
-            return CleanConsumeQueueService.class.getSimpleName();
+            return DefaultMessageStore.this.brokerConfig.getLoggerIdentifier() + CleanConsumeQueueService.class.getSimpleName();
         }
     }
 
@@ -1902,7 +2159,7 @@ public class DefaultMessageStore implements MessageStore {
             try {
                 this.correctLogicMinOffset();
             } catch (Throwable e) {
-                log.warn(this.getServiceName() + " service has exception. ", e);
+                LOGGER.warn(this.getServiceName() + " service has exception. ", e);
             }
         }
 
@@ -1912,11 +2169,11 @@ public class DefaultMessageStore implements MessageStore {
             }
             // If first exist and not available, it means first file may destroy failed, delete it.
             if (DefaultMessageStore.this.consumeQueueStore.isFirstFileExist(logic) && !DefaultMessageStore.this.consumeQueueStore.isFirstFileAvailable(logic)) {
-                log.error("CorrectLogicOffsetService.needCorrect. first file not available, trigger correct." +
-                                " topic:{}, queue:{}, maxPhyOffset in queue:{}, minPhyOffset " +
-                                "in commit log:{}, minOffset in queue:{}, maxOffset in queue:{}, cqType:{}"
-                        , logic.getTopic(), logic.getQueueId(), logic.getMaxPhysicOffset()
-                        , minPhyOffset, logic.getMinOffsetInQueue(), logic.getMaxOffsetInQueue(), logic.getCQType());
+                LOGGER.error("CorrectLogicOffsetService.needCorrect. first file not available, trigger correct." +
+                        " topic:{}, queue:{}, maxPhyOffset in queue:{}, minPhyOffset " +
+                        "in commit log:{}, minOffset in queue:{}, maxOffset in queue:{}, cqType:{}"
+                    , logic.getTopic(), logic.getQueueId(), logic.getMaxPhysicOffset()
+                    , minPhyOffset, logic.getMinOffsetInQueue(), logic.getMaxOffsetInQueue(), logic.getCQType());
                 return true;
             }
 
@@ -1928,18 +2185,18 @@ public class DefaultMessageStore implements MessageStore {
 
             if (logic.getMaxPhysicOffset() < minPhyOffset) {
                 if (logic.getMinOffsetInQueue() < logic.getMaxOffsetInQueue()) {
-                    log.error("CorrectLogicOffsetService.needCorrect. logic max phy offset: {} is less than min phy offset: {}, " +
-                                    "but min offset: {} is less than max offset: {}. topic:{}, queue:{}, cqType:{}."
-                            , logic.getMaxPhysicOffset(), minPhyOffset, logic.getMinOffsetInQueue()
-                            , logic.getMaxOffsetInQueue(), logic.getTopic(), logic.getQueueId(), logic.getCQType());
+                    LOGGER.error("CorrectLogicOffsetService.needCorrect. logic max phy offset: {} is less than min phy offset: {}, " +
+                            "but min offset: {} is less than max offset: {}. topic:{}, queue:{}, cqType:{}."
+                        , logic.getMaxPhysicOffset(), minPhyOffset, logic.getMinOffsetInQueue()
+                        , logic.getMaxOffsetInQueue(), logic.getTopic(), logic.getQueueId(), logic.getCQType());
                     return true;
                 } else if (logic.getMinOffsetInQueue() == logic.getMaxOffsetInQueue()) {
                     return false;
                 } else {
-                    log.error("CorrectLogicOffsetService.needCorrect. It should not happen, logic max phy offset: {} is less than min phy offset: {}," +
-                                    " but min offset: {} is larger than max offset: {}. topic:{}, queue:{}, cqType:{}"
-                            , logic.getMaxPhysicOffset(), minPhyOffset, logic.getMinOffsetInQueue()
-                            , logic.getMaxOffsetInQueue(), logic.getTopic(), logic.getQueueId(), logic.getCQType());
+                    LOGGER.error("CorrectLogicOffsetService.needCorrect. It should not happen, logic max phy offset: {} is less than min phy offset: {}," +
+                            " but min offset: {} is larger than max offset: {}. topic:{}, queue:{}, cqType:{}"
+                        , logic.getMaxPhysicOffset(), minPhyOffset, logic.getMinOffsetInQueue()
+                        , logic.getMaxOffsetInQueue(), logic.getTopic(), logic.getQueueId(), logic.getCQType());
                     return false;
                 }
             }
@@ -1952,19 +2209,19 @@ public class DefaultMessageStore implements MessageStore {
                     if (logic.getMinOffsetInQueue() == logic.getMaxOffsetInQueue()) {
                         return false;
                     } else {
-                        log.error("CorrectLogicOffsetService.needCorrect. cqUnit is null, logic max phy offset: {} is greater than min phy offset: {}, " +
-                                        "but min offset: {} is not equal to max offset: {}. topic:{}, queue:{}, cqType:{}."
-                                , logic.getMaxPhysicOffset(), minPhyOffset, logic.getMinOffsetInQueue()
-                                , logic.getMaxOffsetInQueue(), logic.getTopic(), logic.getQueueId(), logic.getCQType());
+                        LOGGER.error("CorrectLogicOffsetService.needCorrect. cqUnit is null, logic max phy offset: {} is greater than min phy offset: {}, " +
+                                "but min offset: {} is not equal to max offset: {}. topic:{}, queue:{}, cqType:{}."
+                            , logic.getMaxPhysicOffset(), minPhyOffset, logic.getMinOffsetInQueue()
+                            , logic.getMaxOffsetInQueue(), logic.getTopic(), logic.getQueueId(), logic.getCQType());
                         return true;
                     }
                 }
 
                 if (cqUnit.getPos() < minPhyOffset) {
-                    log.error("CorrectLogicOffsetService.needCorrect. logic max phy offset: {} is greater than min phy offset: {}, " +
-                                    "but minPhyPos in cq is: {}. min offset in queue: {}, max offset in queue: {}, topic:{}, queue:{}, cqType:{}."
-                            , logic.getMaxPhysicOffset(), minPhyOffset, cqUnit.getPos(), logic.getMinOffsetInQueue()
-                            , logic.getMaxOffsetInQueue(), logic.getTopic(), logic.getQueueId(), logic.getCQType());
+                    LOGGER.error("CorrectLogicOffsetService.needCorrect. logic max phy offset: {} is greater than min phy offset: {}, " +
+                            "but minPhyPos in cq is: {}. min offset in queue: {}, max offset in queue: {}, topic:{}, queue:{}, cqType:{}."
+                        , logic.getMaxPhysicOffset(), minPhyOffset, cqUnit.getPos(), logic.getMinOffsetInQueue()
+                        , logic.getMaxOffsetInQueue(), logic.getTopic(), logic.getQueueId(), logic.getCQType());
                     return true;
                 }
 
@@ -2008,6 +2265,9 @@ public class DefaultMessageStore implements MessageStore {
         }
 
         public String getServiceName() {
+            if (brokerConfig.isInBrokerContainer()) {
+                return brokerConfig.getLoggerIdentifier() + CorrectLogicOffsetService.class.getSimpleName();
+            }
             return CorrectLogicOffsetService.class.getSimpleName();
         }
     }
@@ -2054,7 +2314,7 @@ public class DefaultMessageStore implements MessageStore {
 
         @Override
         public void run() {
-            DefaultMessageStore.log.info(this.getServiceName() + " service started");
+            DefaultMessageStore.LOGGER.info(this.getServiceName() + " service started");
 
             while (!this.isStopped()) {
                 try {
@@ -2062,22 +2322,25 @@ public class DefaultMessageStore implements MessageStore {
                     this.waitForRunning(interval);
                     this.doFlush(1);
                 } catch (Exception e) {
-                    DefaultMessageStore.log.warn(this.getServiceName() + " service has exception. ", e);
+                    DefaultMessageStore.LOGGER.warn(this.getServiceName() + " service has exception. ", e);
                 }
             }
 
             this.doFlush(RETRY_TIMES_OVER);
 
-            DefaultMessageStore.log.info(this.getServiceName() + " service end");
+            DefaultMessageStore.LOGGER.info(this.getServiceName() + " service end");
         }
 
         @Override
         public String getServiceName() {
+            if (DefaultMessageStore.this.brokerConfig.isInBrokerContainer()) {
+                return DefaultMessageStore.this.brokerConfig.getLoggerIdentifier() + FlushConsumeQueueService.class.getSimpleName();
+            }
             return FlushConsumeQueueService.class.getSimpleName();
         }
 
         @Override
-        public long getJointime() {
+        public long getJoinTime() {
             return 1000 * 60;
         }
     }
@@ -2104,24 +2367,28 @@ public class DefaultMessageStore implements MessageStore {
             }
 
             if (this.isCommitLogAvailable()) {
-                log.warn("shutdown ReputMessageService, but commitlog have not finish to be dispatched, CL: {} reputFromOffset: {}",
-                    DefaultMessageStore.this.commitLog.getMaxOffset(), this.reputFromOffset);
+                LOGGER.warn("shutdown ReputMessageService, but CommitLog have not finish to be dispatched, CommitLog max" +
+                        " offset={}, reputFromOffset={}", DefaultMessageStore.this.commitLog.getMaxOffset(),
+                    this.reputFromOffset);
             }
 
             super.shutdown();
         }
 
         public long behind() {
-            return DefaultMessageStore.this.commitLog.getMaxOffset() - this.reputFromOffset;
+            return DefaultMessageStore.this.commitLog.getConfirmOffset() - this.reputFromOffset;
         }
 
         private boolean isCommitLogAvailable() {
+            if (DefaultMessageStore.this.getMessageStoreConfig().isDuplicationEnable()) {
+                return this.reputFromOffset <= DefaultMessageStore.this.commitLog.getConfirmOffset();
+            }
             return this.reputFromOffset < DefaultMessageStore.this.commitLog.getMaxOffset();
         }
 
         private void doReput() {
             if (this.reputFromOffset < DefaultMessageStore.this.commitLog.getMinOffset()) {
-                log.warn("The reputFromOffset={} is smaller than minPyOffset={}, this usually indicate that the dispatch behind too much and the commitlog has expired.",
+                LOGGER.warn("The reputFromOffset={} is smaller than minPyOffset={}, this usually indicate that the dispatch behind too much and the commitlog has expired.",
                     this.reputFromOffset, DefaultMessageStore.this.commitLog.getMinOffset());
                 this.reputFromOffset = DefaultMessageStore.this.commitLog.getMinOffset();
             }
@@ -2137,18 +2404,17 @@ public class DefaultMessageStore implements MessageStore {
                     try {
                         this.reputFromOffset = result.getStartOffset();
 
-                        for (int readSize = 0; readSize < result.getSize() && doNext; ) {
+                        for (int readSize = 0; readSize < result.getSize() && reputFromOffset <= DefaultMessageStore.this.getConfirmOffset() && doNext; ) {
                             DispatchRequest dispatchRequest =
-                                DefaultMessageStore.this.commitLog.checkMessageAndReturnSize(result.getByteBuffer(), false, false);
+                                DefaultMessageStore.this.commitLog.checkMessageAndReturnSize(result.getByteBuffer(), false, false, false);
                             int size = dispatchRequest.getBufferSize() == -1 ? dispatchRequest.getMsgSize() : dispatchRequest.getBufferSize();
 
                             if (dispatchRequest.isSuccess()) {
                                 if (size > 0) {
                                     DefaultMessageStore.this.doDispatch(dispatchRequest);
 
-                                    if (BrokerRole.SLAVE != DefaultMessageStore.this.getMessageStoreConfig().getBrokerRole()
-                                            && DefaultMessageStore.this.brokerConfig.isLongPollingEnable()
-                                            && DefaultMessageStore.this.messageArrivingListener != null) {
+                                    if (DefaultMessageStore.this.brokerConfig.isLongPollingEnable()
+                                        && DefaultMessageStore.this.messageArrivingListener != null) {
                                         DefaultMessageStore.this.messageArrivingListener.arriving(dispatchRequest.getTopic(),
                                             dispatchRequest.getQueueId(), dispatchRequest.getConsumeQueueOffset() + 1,
                                             dispatchRequest.getTagsCode(), dispatchRequest.getStoreTimestamp(),
@@ -2158,7 +2424,8 @@ public class DefaultMessageStore implements MessageStore {
 
                                     this.reputFromOffset += size;
                                     readSize += size;
-                                    if (DefaultMessageStore.this.getMessageStoreConfig().getBrokerRole() == BrokerRole.SLAVE) {
+                                    if (!DefaultMessageStore.this.getMessageStoreConfig().isDuplicationEnable() &&
+                                        DefaultMessageStore.this.getMessageStoreConfig().getBrokerRole() == BrokerRole.SLAVE) {
                                         DefaultMessageStore.this.storeStatsService
                                             .getSinglePutMessageTopicTimesTotal(dispatchRequest.getTopic()).add(1);
                                         DefaultMessageStore.this.storeStatsService
@@ -2172,7 +2439,7 @@ public class DefaultMessageStore implements MessageStore {
                             } else if (!dispatchRequest.isSuccess()) {
 
                                 if (size > 0) {
-                                    log.error("[BUG]read total count not equals msg total size. reputFromOffset={}", reputFromOffset);
+                                    LOGGER.error("[BUG]read total count not equals msg total size. reputFromOffset={}", reputFromOffset);
                                     this.reputFromOffset += size;
                                 } else {
                                     doNext = false;
@@ -2180,7 +2447,7 @@ public class DefaultMessageStore implements MessageStore {
                                     // it will not ignore the exception and fix the reputFromOffset variable
                                     if (DefaultMessageStore.this.getMessageStoreConfig().isEnableDLegerCommitLog() ||
                                         DefaultMessageStore.this.brokerConfig.getBrokerId() == MixAll.MASTER_ID) {
-                                        log.error("[BUG]dispatch message to consume queue error, COMMITLOG OFFSET: {}",
+                                        LOGGER.error("[BUG]dispatch message to consume queue error, COMMITLOG OFFSET: {}",
                                             this.reputFromOffset);
                                         this.reputFromOffset += result.getSize() - readSize;
                                     }
@@ -2226,24 +2493,65 @@ public class DefaultMessageStore implements MessageStore {
 
         @Override
         public void run() {
-            DefaultMessageStore.log.info(this.getServiceName() + " service started");
+            DefaultMessageStore.LOGGER.info(this.getServiceName() + " service started");
 
             while (!this.isStopped()) {
                 try {
                     Thread.sleep(1);
                     this.doReput();
                 } catch (Exception e) {
-                    DefaultMessageStore.log.warn(this.getServiceName() + " service has exception. ", e);
+                    DefaultMessageStore.LOGGER.warn(this.getServiceName() + " service has exception. ", e);
                 }
             }
 
-            DefaultMessageStore.log.info(this.getServiceName() + " service end");
+            DefaultMessageStore.LOGGER.info(this.getServiceName() + " service end");
         }
 
         @Override
         public String getServiceName() {
+            if (DefaultMessageStore.this.getBrokerConfig().isInBrokerContainer()) {
+                return DefaultMessageStore.this.getBrokerConfig().getLoggerIdentifier() + ReputMessageService.class.getSimpleName();
+            }
             return ReputMessageService.class.getSimpleName();
         }
 
     }
+
+    @Override
+    public HARuntimeInfo getHARuntimeInfo() {
+        if (haService != null) {
+            return this.haService.getRuntimeInfo(this.commitLog.getMaxOffset());
+        } else {
+            return null;
+        }
+    }
+
+    public int getMaxDelayLevel() {
+        return maxDelayLevel;
+    }
+
+    public long computeDeliverTimestamp(final int delayLevel, final long storeTimestamp) {
+        Long time = this.delayLevelTable.get(delayLevel);
+        if (time != null) {
+            return time + storeTimestamp;
+        }
+
+        return storeTimestamp + 1000;
+    }
+
+    public List<PutMessageHook> getPutMessageHookList() {
+        return putMessageHookList;
+    }
+
+    @Override public void setSendMessageBackHook(SendMessageBackHook sendMessageBackHook) {
+        this.sendMessageBackHook = sendMessageBackHook;
+    }
+
+    @Override public SendMessageBackHook getSendMessageBackHook() {
+        return sendMessageBackHook;
+    }
+
+    @Override public boolean isShutdown() {
+        return shutdown;
+    }
 }
diff --git a/store/src/main/java/org/apache/rocketmq/store/GetMessageResult.java b/store/src/main/java/org/apache/rocketmq/store/GetMessageResult.java
index b027914..0f75a48 100644
--- a/store/src/main/java/org/apache/rocketmq/store/GetMessageResult.java
+++ b/store/src/main/java/org/apache/rocketmq/store/GetMessageResult.java
@@ -19,7 +19,6 @@ package org.apache.rocketmq.store;
 import java.nio.ByteBuffer;
 import java.util.ArrayList;
 import java.util.List;
-import org.apache.rocketmq.store.stats.BrokerStatsManager;
 
 public class GetMessageResult {
 
@@ -39,6 +38,7 @@ public class GetMessageResult {
     private boolean suggestPullingFromSlave = false;
 
     private int msgCount4Commercial = 0;
+    private int commercialSizePerMsg = 4 * 1024;
 
     public GetMessageResult() {
         messageMapedList = new ArrayList<>(100);
@@ -97,7 +97,7 @@ public class GetMessageResult {
         this.messageBufferList.add(mapedBuffer.getByteBuffer());
         this.bufferTotalSize += mapedBuffer.getSize();
         this.msgCount4Commercial += (int) Math.ceil(
-            mapedBuffer.getSize() / BrokerStatsManager.SIZE_PER_COUNT);
+            mapedBuffer.getSize() /  (double)commercialSizePerMsg);
     }
 
     public void addMessage(final SelectMappedBufferResult mapedBuffer, final long queueOffset) {
@@ -105,7 +105,7 @@ public class GetMessageResult {
         this.messageBufferList.add(mapedBuffer.getByteBuffer());
         this.bufferTotalSize += mapedBuffer.getSize();
         this.msgCount4Commercial += (int) Math.ceil(
-            mapedBuffer.getSize() / BrokerStatsManager.SIZE_PER_COUNT);
+            mapedBuffer.getSize() /  (double)commercialSizePerMsg);
         this.messageQueueOffset.add(queueOffset);
     }
 
diff --git a/store/src/main/java/org/apache/rocketmq/store/MappedFileQueue.java b/store/src/main/java/org/apache/rocketmq/store/MappedFileQueue.java
index 3c86a65..7ba7de4 100644
--- a/store/src/main/java/org/apache/rocketmq/store/MappedFileQueue.java
+++ b/store/src/main/java/org/apache/rocketmq/store/MappedFileQueue.java
@@ -28,6 +28,7 @@ import java.util.concurrent.CopyOnWriteArrayList;
 import java.util.stream.Stream;
 
 import com.google.common.collect.Lists;
+import org.apache.rocketmq.common.MixAll;
 import org.apache.rocketmq.common.UtilAll;
 import org.apache.rocketmq.common.constant.LoggerName;
 import org.apache.rocketmq.logging.InternalLogger;
@@ -39,8 +40,6 @@ public class MappedFileQueue implements Swappable {
     private static final InternalLogger log = InternalLoggerFactory.getLogger(LoggerName.STORE_LOGGER_NAME);
     private static final InternalLogger LOG_ERROR = InternalLoggerFactory.getLogger(LoggerName.STORE_ERROR_LOGGER_NAME);
 
-    private static final int DELETE_FILES_BATCH_MAX = 10;
-
     protected final String storePath;
 
     protected final int mappedFileSize;
@@ -221,6 +220,10 @@ public class MappedFileQueue implements Swappable {
         return mappedFileLast;
     }
 
+    public boolean isMappedFilesEmpty() {
+        return this.mappedFiles.isEmpty();
+    }
+
     protected MappedFile tryCreateMappedFile(long createOffset) {
         String nextFilePath = this.storePath + File.separator + UtilAll.offset2FileName(createOffset);
         String nextNextFilePath = this.storePath + File.separator + UtilAll.offset2FileName(createOffset
@@ -343,10 +346,7 @@ public class MappedFileQueue implements Swappable {
         final int deleteFilesInterval,
         final long intervalForcibly,
         final boolean cleanImmediately,
-        int deleteFileBatchMax) {
-        if (deleteFileBatchMax == 0) {
-            deleteFileBatchMax = DELETE_FILES_BATCH_MAX;
-        }
+        final int deleteFileBatchMax) {
         Object[] mfs = this.copyMappedFiles(0);
 
         if (null == mfs)
@@ -355,11 +355,17 @@ public class MappedFileQueue implements Swappable {
         int mfsLength = mfs.length - 1;
         int deleteCount = 0;
         List<MappedFile> files = new ArrayList<MappedFile>();
+        int skipFileNum = 0;
         if (null != mfs) {
+            //do check before deleting
+            checkSelf();
             for (int i = 0; i < mfsLength; i++) {
                 MappedFile mappedFile = (MappedFile) mfs[i];
                 long liveMaxTimestamp = mappedFile.getLastModifiedTimestamp() + expiredTime;
                 if (System.currentTimeMillis() >= liveMaxTimestamp || cleanImmediately) {
+                    if (skipFileNum > 0) {
+                        log.info("Delete CommitLog {} but skip {} files", mappedFile.getFileName(), skipFileNum);
+                    }
                     if (mappedFile.destroy(intervalForcibly)) {
                         files.add(mappedFile);
                         deleteCount++;
@@ -378,6 +384,7 @@ public class MappedFileQueue implements Swappable {
                         break;
                     }
                 } else {
+                    skipFileNum++;
                     //avoid deleting files in the middle
                     break;
                 }
@@ -432,6 +439,64 @@ public class MappedFileQueue implements Swappable {
         return deleteCount;
     }
 
+    public int deleteExpiredFileByOffsetForTimerLog(long offset, int checkOffset, int unitSize) {
+        Object[] mfs = this.copyMappedFiles(0);
+
+        List<MappedFile> files = new ArrayList<MappedFile>();
+        int deleteCount = 0;
+        if (null != mfs) {
+
+            int mfsLength = mfs.length - 1;
+
+            for (int i = 0; i < mfsLength; i++) {
+                boolean destroy = false;
+                MappedFile mappedFile = (MappedFile) mfs[i];
+                SelectMappedBufferResult result = mappedFile.selectMappedBuffer(checkOffset);
+                try {
+                    if (result != null) {
+                        int position = result.getByteBuffer().position();
+                        int size = result.getByteBuffer().getInt();//size
+                        result.getByteBuffer().getLong(); //prev pos
+                        int magic = result.getByteBuffer().getInt();
+                        if (size == unitSize && (magic | 0xF) == 0xF) {
+                            result.getByteBuffer().position(position + MixAll.UNIT_PRE_SIZE_FOR_MSG);
+                            long maxOffsetPy = result.getByteBuffer().getLong();
+                            destroy = maxOffsetPy < offset;
+                            if (destroy) {
+                                log.info("physic min commitlog offset " + offset + ", current mappedFile's max offset "
+                                    + maxOffsetPy + ", delete it");
+                            }
+                        } else {
+                            log.warn("Found error data in [{}] checkOffset:{} unitSize:{}", mappedFile.getFileName(),
+                                checkOffset, unitSize);
+                        }
+                    } else if (!mappedFile.isAvailable()) { // Handle hanged file.
+                        log.warn("Found a hanged consume queue file, attempting to delete it.");
+                        destroy = true;
+                    } else {
+                        log.warn("this being not executed forever.");
+                        break;
+                    }
+                } finally {
+                    if (null != result) {
+                        result.release();
+                    }
+                }
+
+                if (destroy && mappedFile.destroy(1000 * 60)) {
+                    files.add(mappedFile);
+                    deleteCount++;
+                } else {
+                    break;
+                }
+            }
+        }
+
+        deleteExpiredFile(files);
+
+        return deleteCount;
+    }
+
     public boolean flush(final int flushLeastPages) {
         boolean result = true;
         MappedFile mappedFile = this.findMappedFileByOffset(this.flushedWhere, this.flushedWhere == 0);
@@ -449,7 +514,7 @@ public class MappedFileQueue implements Swappable {
         return result;
     }
 
-    public boolean commit(final int commitLeastPages) {
+    public synchronized boolean commit(final int commitLeastPages) {
         boolean result = true;
         MappedFile mappedFile = this.findMappedFileByOffset(this.committedWhere, this.committedWhere == 0);
         if (mappedFile != null) {
@@ -681,4 +746,8 @@ public class MappedFileQueue implements Swappable {
     public void setCommittedWhere(final long committedWhere) {
         this.committedWhere = committedWhere;
     }
+
+    public long getTotalFileSize() {
+        return (long) mappedFileSize * mappedFiles.size();
+    }
 }
diff --git a/store/src/main/java/org/apache/rocketmq/store/MessageStore.java b/store/src/main/java/org/apache/rocketmq/store/MessageStore.java
index 341a29f..9dc27fa 100644
--- a/store/src/main/java/org/apache/rocketmq/store/MessageStore.java
+++ b/store/src/main/java/org/apache/rocketmq/store/MessageStore.java
@@ -16,22 +16,27 @@
  */
 package org.apache.rocketmq.store;
 
+import java.nio.ByteBuffer;
 import java.util.HashMap;
 import java.util.LinkedList;
 import java.util.Optional;
+import java.util.List;
 import java.util.Set;
 import java.util.concurrent.CompletableFuture;
 
 import org.apache.rocketmq.common.SystemClock;
 import org.apache.rocketmq.common.TopicConfig;
 import org.apache.rocketmq.common.message.MessageExt;
-import org.apache.rocketmq.store.config.BrokerRole;
+import org.apache.rocketmq.common.message.MessageExtBatch;
+import org.apache.rocketmq.common.message.MessageExtBrokerInner;
+import org.apache.rocketmq.common.protocol.body.HARuntimeInfo;
 import org.apache.rocketmq.store.config.MessageStoreConfig;
 import org.apache.rocketmq.store.ha.HAService;
+import org.apache.rocketmq.store.hook.PutMessageHook;
+import org.apache.rocketmq.store.hook.SendMessageBackHook;
 import org.apache.rocketmq.store.logfile.MappedFile;
 import org.apache.rocketmq.store.queue.ConsumeQueueInterface;
 import org.apache.rocketmq.store.queue.ConsumeQueueStore;
-import org.apache.rocketmq.store.schedule.ScheduleMessageService;
 import org.apache.rocketmq.store.stats.BrokerStatsManager;
 import org.apache.rocketmq.store.util.PerfCounter;
 
@@ -145,7 +150,8 @@ public interface MessageStore {
      *
      * @param topic Topic name.
      * @param queueId Queue ID.
-     * @param committed If only count committed
+     * @param committed return the max offset in ConsumeQueue if true,
+     *                  or the max offset in CommitLog if false
      * @return Maximum offset at present.
      */
     long getMaxOffsetInQueue(final String topic, final int queueId, final boolean committed);
@@ -228,6 +234,12 @@ public interface MessageStore {
     HashMap<String, String> getRuntimeInfo();
 
     /**
+     * HA runtime information
+     * @return runtime information of ha
+     */
+    HARuntimeInfo getHARuntimeInfo();
+
+    /**
      * Get the maximum commit log offset.
      *
      * @return maximum commit log offset.
@@ -285,6 +297,16 @@ public interface MessageStore {
     SelectMappedBufferResult getCommitLogData(final long offset);
 
     /**
+     * Get the raw commit log data starting from the given offset, across multiple mapped files.
+     *
+     * @param offset starting offset.
+     * @param size size of data to get
+     * @return commit log data.
+     */
+    List<SelectMappedBufferResult> getBulkCommitLogData(final long offset, final int size);
+
+
+    /**
      * Append data to commit log.
      *
      * @param startOffset starting offset.
@@ -319,6 +341,14 @@ public interface MessageStore {
      */
     void updateHaMasterAddress(final String newAddr);
 
+
+    /**
+     * Update master address.
+     *
+     * @param newAddr new address.
+     */
+    void updateMasterAddress(final String newAddr);
+
     /**
      * Return how much the slave falls behind.
      *
@@ -371,6 +401,13 @@ public interface MessageStore {
     long flush();
 
     /**
+     * Get the current flushed offset.
+     *
+     * @return flushed offset
+     */
+    long getFlushedWhere();
+
+    /**
      * Reset written offset.
      *
      * @param phyOffset new offset.
@@ -421,7 +458,7 @@ public interface MessageStore {
     LinkedList<CommitLogDispatcher> getDispatcherList();
 
     /**
-     * Get consume queue of the topic/queue.
+     * Get consume queue of the topic/queue. If consume queue not exist, will return null
      *
      * @param topic Topic.
      * @param queueId Queue ID.
@@ -429,7 +466,6 @@ public interface MessageStore {
      */
     ConsumeQueueInterface getConsumeQueue(String topic, int queueId);
 
-    ScheduleMessageService getScheduleMessageService();
 
     /**
      * Get BrokerStatsManager of the messageStore.
@@ -439,12 +475,6 @@ public interface MessageStore {
     BrokerStatsManager getBrokerStatsManager();
 
     /**
-     * handle
-     * @param brokerRole
-     */
-    void handleScheduleMessageService(BrokerRole brokerRole);
-
-    /**
      * Will be triggered when a new message is appended to commit log.
      * @param msg the msg that is appended to commit log
      * @param result append message result
@@ -511,12 +541,6 @@ public interface MessageStore {
     HAService getHaService();
 
     /**
-     * Register clean file hook
-     * @param logicalQueueCleanHook logical queue clean hook
-     */
-    void registerCleanFileHook(CleanFilesHook logicalQueueCleanHook);
-
-    /**
      * Get the allocate-mappedFile service
      * @return the allocate-mappedFile service
      */
@@ -567,11 +591,10 @@ public interface MessageStore {
      * Assign an queue offset and increase it.
      * If there is a race condition, you need to lock/unlock this method yourself.
      *
-     * @param topicQueueKey topic-queue key
      * @param msg message
      * @param messageNum message num
      */
-    void assignOffset(String topicQueueKey, MessageExtBrokerInner msg, short messageNum);
+    void assignOffset(MessageExtBrokerInner msg, short messageNum);
 
     /**
      * get topic config
@@ -579,4 +602,198 @@ public interface MessageStore {
      * @return topic config info
      */
     Optional<TopicConfig> getTopicConfig(String topic);
+
+    /**
+     * Get master broker message store in process in broker container
+     *
+     * @return
+     */
+    MessageStore getMasterStoreInProcess();
+
+    /**
+     * Set master broker message store in process
+     *
+     * @param masterStoreInProcess
+     */
+    void setMasterStoreInProcess(MessageStore masterStoreInProcess);
+
+    /**
+     * Use FileChannel to get data
+     * @param offset
+     * @param size
+     * @param byteBuffer
+     * @return
+     */
+    boolean getData(long offset, int size, ByteBuffer byteBuffer);
+
+    /**
+     * Set the number of alive replicas in group.
+     *
+     * @param aliveReplicaNums number of alive replicas
+     */
+    void setAliveReplicaNumInGroup(int aliveReplicaNums);
+
+    /**
+     * Get the number of alive replicas in group.
+     *
+     * @return number of alive replicas
+     */
+    int getAliveReplicaNumInGroup();
+
+    /**
+     * Wake up AutoRecoverHAClient to start HA connection.
+     */
+    void wakeupHAClient();
+
+
+    /**
+     * Get master flushed offset.
+     *
+     * @return master flushed offset
+     */
+    long getMasterFlushedOffset();
+
+    /**
+     * Get broker init max offset.
+     *
+     * @return broker max offset in startup
+     */
+    long getBrokerInitMaxOffset();
+
+    /**
+     * Set master flushed offset.
+     *
+     * @param masterFlushedOffset master flushed offset
+     */
+    void setMasterFlushedOffset(long masterFlushedOffset);
+
+    /**
+     * Set broker init max offset.
+     *
+     * @param brokerInitMaxOffset broker init max offset
+     */
+    void setBrokerInitMaxOffset(long brokerInitMaxOffset);
+
+    /**
+     * Calculate the checksum of a certain range of data.
+     *
+     * @param from begin offset
+     * @param to end offset
+     * @return checksum
+     */
+    byte[] calcDeltaChecksum(long from, long to);
+
+    /**
+     * Truncate commitLog and consume queue to certain offset.
+     *
+     * @param offsetToTruncate offset to truncate
+     * @return true if truncate succeed, false otherwise
+     */
+    boolean truncateFiles(long offsetToTruncate);
+
+    /**
+     * Check if the offset is align with one message.
+     *
+     * @param offset offset to check
+     * @return true if align, false otherwise
+     */
+    boolean isOffsetAligned(long offset);
+
+    /**
+     * Get put message hook list
+     *
+     * @return List of PutMessageHook
+     */
+    List<PutMessageHook> getPutMessageHookList();
+
+    /**
+     * Set send message back hook
+     *
+     * @param sendMessageBackHook
+     */
+    void setSendMessageBackHook(SendMessageBackHook sendMessageBackHook);
+
+    /**
+     * Get send message back hook
+     *
+     * @return SendMessageBackHook
+     */
+    SendMessageBackHook getSendMessageBackHook();
+
+    //The following interfaces are used for duplication mode
+
+    /**
+     * Get last mapped file and return lase file first Offset
+     *
+     * @return lastMappedFile first Offset
+     */
+    long getLastFileFromOffset();
+
+    /**
+     * Get last mapped file
+     * @param startOffset
+     * @return true when get the last mapped file, false when get null
+     */
+    boolean getLastMappedFile(long startOffset);
+
+    /**
+     * Set physical offset
+     *
+     * @param phyOffset
+     */
+    void setPhysicalOffset(long phyOffset);
+
+    /**
+     * Return whether mapped file is empty
+     *
+     * @return whether mapped file is empty
+     */
+    boolean isMappedFilesEmpty();
+
+    /**
+     * Get state machine version
+     *
+     * @return state machine version
+     */
+    long getStateMachineVersion();
+
+    /**
+     * Check message and return size
+     *
+     * @param byteBuffer
+     * @param checkCRC
+     * @param checkDupInfo
+     * @param readBody
+     * @return DispatchRequest
+     */
+    DispatchRequest checkMessageAndReturnSize(final ByteBuffer byteBuffer, final boolean checkCRC,
+        final boolean checkDupInfo, final boolean readBody);
+
+    /**
+     * Get remain transientStoreBuffer numbers
+     *
+     * @return remain transientStoreBuffer numbers
+     */
+    int remainTransientStoreBufferNumbs();
+
+    /**
+     * Get remain how many data to commit
+     *
+     * @return remain how many data to commit
+     */
+    long remainHowManyDataToCommit();
+
+    /**
+     * Get remain how many data to flush
+     *
+     * @return remain how many data to flush
+     */
+    long remainHowManyDataToFlush();
+
+    /**
+     * Get whether message store is shutdown
+     *
+     * @return whether shutdown
+     */
+    boolean isShutdown();
 }
diff --git a/store/src/main/java/org/apache/rocketmq/store/MultiPathMappedFileQueue.java b/store/src/main/java/org/apache/rocketmq/store/MultiPathMappedFileQueue.java
index c5b2316..de974fe 100644
--- a/store/src/main/java/org/apache/rocketmq/store/MultiPathMappedFileQueue.java
+++ b/store/src/main/java/org/apache/rocketmq/store/MultiPathMappedFileQueue.java
@@ -22,6 +22,7 @@ import java.util.HashSet;
 import java.util.Set;
 import java.util.function.Supplier;
 import org.apache.commons.lang3.StringUtils;
+import org.apache.rocketmq.common.MixAll;
 import org.apache.rocketmq.common.UtilAll;
 import org.apache.rocketmq.store.config.MessageStoreConfig;
 import org.apache.rocketmq.store.logfile.MappedFile;
@@ -45,7 +46,7 @@ public class MultiPathMappedFileQueue extends MappedFileQueue {
     }
 
     private Set<String> getPaths() {
-        String[] paths = config.getStorePathCommitLog().trim().split(MessageStoreConfig.MULTI_PATH_SPLITTER);
+        String[] paths = config.getStorePathCommitLog().trim().split(MixAll.MULTI_PATH_SPLITTER);
         return new HashSet<>(Arrays.asList(paths));
     }
 
@@ -54,7 +55,7 @@ public class MultiPathMappedFileQueue extends MappedFileQueue {
         if (StringUtils.isBlank(pathStr)) {
             return Collections.emptySet();
         }
-        String[] paths = pathStr.trim().split(MessageStoreConfig.MULTI_PATH_SPLITTER);
+        String[] paths = pathStr.trim().split(MixAll.MULTI_PATH_SPLITTER);
         return new HashSet<>(Arrays.asList(paths));
     }
 
diff --git a/store/src/main/java/org/apache/rocketmq/store/PutMessageResult.java b/store/src/main/java/org/apache/rocketmq/store/PutMessageResult.java
index e12cc0c..bcca6ae 100644
--- a/store/src/main/java/org/apache/rocketmq/store/PutMessageResult.java
+++ b/store/src/main/java/org/apache/rocketmq/store/PutMessageResult.java
@@ -19,14 +19,28 @@ package org.apache.rocketmq.store;
 public class PutMessageResult {
     private PutMessageStatus putMessageStatus;
     private AppendMessageResult appendMessageResult;
+    private boolean remotePut = false;
 
     public PutMessageResult(PutMessageStatus putMessageStatus, AppendMessageResult appendMessageResult) {
         this.putMessageStatus = putMessageStatus;
         this.appendMessageResult = appendMessageResult;
     }
 
+    public PutMessageResult(PutMessageStatus putMessageStatus, AppendMessageResult appendMessageResult,
+        boolean remotePut) {
+        this.putMessageStatus = putMessageStatus;
+        this.appendMessageResult = appendMessageResult;
+        this.remotePut = remotePut;
+    }
+
     public boolean isOk() {
-        return this.appendMessageResult != null && this.appendMessageResult.isOk();
+        if (remotePut) {
+            return putMessageStatus == PutMessageStatus.PUT_OK || putMessageStatus == PutMessageStatus.FLUSH_DISK_TIMEOUT
+                || putMessageStatus == PutMessageStatus.FLUSH_SLAVE_TIMEOUT || putMessageStatus == PutMessageStatus.SLAVE_NOT_AVAILABLE;
+        } else {
+            return this.appendMessageResult != null && this.appendMessageResult.isOk();
+        }
+
     }
 
     public AppendMessageResult getAppendMessageResult() {
@@ -45,10 +59,18 @@ public class PutMessageResult {
         this.putMessageStatus = putMessageStatus;
     }
 
+    public boolean isRemotePut() {
+        return remotePut;
+    }
+
+    public void setRemotePut(boolean remotePut) {
+        this.remotePut = remotePut;
+    }
+
     @Override
     public String toString() {
         return "PutMessageResult [putMessageStatus=" + putMessageStatus + ", appendMessageResult="
-            + appendMessageResult + "]";
+            + appendMessageResult + ", remotePut=" + remotePut + "]";
     }
 
 }
diff --git a/store/src/main/java/org/apache/rocketmq/store/PutMessageStatus.java b/store/src/main/java/org/apache/rocketmq/store/PutMessageStatus.java
index 29d0d95..7278be1 100644
--- a/store/src/main/java/org/apache/rocketmq/store/PutMessageStatus.java
+++ b/store/src/main/java/org/apache/rocketmq/store/PutMessageStatus.java
@@ -22,10 +22,12 @@ public enum PutMessageStatus {
     FLUSH_SLAVE_TIMEOUT,
     SLAVE_NOT_AVAILABLE,
     SERVICE_NOT_AVAILABLE,
-    CREATE_MAPEDFILE_FAILED,
+    CREATE_MAPPED_FILE_FAILED,
     MESSAGE_ILLEGAL,
     PROPERTIES_SIZE_EXCEEDED,
-    OS_PAGECACHE_BUSY,
+    OS_PAGE_CACHE_BUSY,
     UNKNOWN_ERROR,
+    IN_SYNC_REPLICAS_NOT_ENOUGH,
+    PUT_TO_REMOTE_BROKER_FAIL,
     LMQ_CONSUME_QUEUE_NUM_EXCEEDED,
 }
diff --git a/store/src/main/java/org/apache/rocketmq/store/QueryMessageResult.java b/store/src/main/java/org/apache/rocketmq/store/QueryMessageResult.java
index a7a6850..2bda659 100644
--- a/store/src/main/java/org/apache/rocketmq/store/QueryMessageResult.java
+++ b/store/src/main/java/org/apache/rocketmq/store/QueryMessageResult.java
@@ -66,4 +66,8 @@ public class QueryMessageResult {
     public int getBufferTotalSize() {
         return bufferTotalSize;
     }
+
+    public List<SelectMappedBufferResult> getMessageMapedList() {
+        return messageMapedList;
+    }
 }
diff --git a/store/src/main/java/org/apache/rocketmq/store/PutMessageStatus.java b/store/src/main/java/org/apache/rocketmq/store/SelectMappedFileResult.java
similarity index 62%
copy from store/src/main/java/org/apache/rocketmq/store/PutMessageStatus.java
copy to store/src/main/java/org/apache/rocketmq/store/SelectMappedFileResult.java
index 29d0d95..9655f28 100644
--- a/store/src/main/java/org/apache/rocketmq/store/PutMessageStatus.java
+++ b/store/src/main/java/org/apache/rocketmq/store/SelectMappedFileResult.java
@@ -14,18 +14,31 @@
  * See the License for the specific language governing permissions and
  * limitations under the License.
  */
+
 package org.apache.rocketmq.store;
 
-public enum PutMessageStatus {
-    PUT_OK,
-    FLUSH_DISK_TIMEOUT,
-    FLUSH_SLAVE_TIMEOUT,
-    SLAVE_NOT_AVAILABLE,
-    SERVICE_NOT_AVAILABLE,
-    CREATE_MAPEDFILE_FAILED,
-    MESSAGE_ILLEGAL,
-    PROPERTIES_SIZE_EXCEEDED,
-    OS_PAGECACHE_BUSY,
-    UNKNOWN_ERROR,
-    LMQ_CONSUME_QUEUE_NUM_EXCEEDED,
+import org.apache.rocketmq.store.logfile.MappedFile;
+
+public class SelectMappedFileResult {
+
+    protected int size;
+
+    protected MappedFile mappedFile;
+
+    public SelectMappedFileResult(int size, MappedFile mappedFile) {
+        this.size = size;
+        this.mappedFile = mappedFile;
+    }
+
+    public int getSize() {
+        return size;
+    }
+
+    public void setSize(int size) {
+        this.size = size;
+    }
+
+    public MappedFile getMappedFile() {
+        return mappedFile;
+    }
 }
diff --git a/store/src/main/java/org/apache/rocketmq/store/StoreCheckpoint.java b/store/src/main/java/org/apache/rocketmq/store/StoreCheckpoint.java
index f209ed2..07e4b79 100644
--- a/store/src/main/java/org/apache/rocketmq/store/StoreCheckpoint.java
+++ b/store/src/main/java/org/apache/rocketmq/store/StoreCheckpoint.java
@@ -36,10 +36,11 @@ public class StoreCheckpoint {
     private volatile long physicMsgTimestamp = 0;
     private volatile long logicsMsgTimestamp = 0;
     private volatile long indexMsgTimestamp = 0;
+    private volatile long masterFlushedOffset = 0;
 
     public StoreCheckpoint(final String scpPath) throws IOException {
         File file = new File(scpPath);
-        DefaultMappedFile.ensureDirOK(file.getParent());
+        UtilAll.ensureDirOK(file.getParent());
         boolean fileExists = file.exists();
 
         this.randomAccessFile = new RandomAccessFile(file, "rw");
@@ -51,6 +52,7 @@ public class StoreCheckpoint {
             this.physicMsgTimestamp = this.mappedByteBuffer.getLong(0);
             this.logicsMsgTimestamp = this.mappedByteBuffer.getLong(8);
             this.indexMsgTimestamp = this.mappedByteBuffer.getLong(16);
+            this.masterFlushedOffset = this.mappedByteBuffer.getLong(24);
 
             log.info("store checkpoint file physicMsgTimestamp " + this.physicMsgTimestamp + ", "
                 + UtilAll.timeMillisToHumanString(this.physicMsgTimestamp));
@@ -58,6 +60,7 @@ public class StoreCheckpoint {
                 + UtilAll.timeMillisToHumanString(this.logicsMsgTimestamp));
             log.info("store checkpoint file indexMsgTimestamp " + this.indexMsgTimestamp + ", "
                 + UtilAll.timeMillisToHumanString(this.indexMsgTimestamp));
+            log.info("store checkpoint file masterFlushedOffset " + this.masterFlushedOffset);
         } else {
             log.info("store checkpoint file not exists, " + scpPath);
         }
@@ -67,7 +70,7 @@ public class StoreCheckpoint {
         this.flush();
 
         // unmap mappedByteBuffer
-        DefaultMappedFile.clean(this.mappedByteBuffer);
+        UtilAll.cleanBuffer(this.mappedByteBuffer);
 
         try {
             this.fileChannel.close();
@@ -80,6 +83,7 @@ public class StoreCheckpoint {
         this.mappedByteBuffer.putLong(0, this.physicMsgTimestamp);
         this.mappedByteBuffer.putLong(8, this.logicsMsgTimestamp);
         this.mappedByteBuffer.putLong(16, this.indexMsgTimestamp);
+        this.mappedByteBuffer.putLong(24, this.masterFlushedOffset);
         this.mappedByteBuffer.force();
     }
 
@@ -107,8 +111,9 @@ public class StoreCheckpoint {
         long min = Math.min(this.physicMsgTimestamp, this.logicsMsgTimestamp);
 
         min -= 1000 * 3;
-        if (min < 0)
+        if (min < 0) {
             min = 0;
+        }
 
         return min;
     }
@@ -121,4 +126,11 @@ public class StoreCheckpoint {
         this.indexMsgTimestamp = indexMsgTimestamp;
     }
 
+    public long getMasterFlushedOffset() {
+        return masterFlushedOffset;
+    }
+
+    public void setMasterFlushedOffset(long masterFlushedOffset) {
+        this.masterFlushedOffset = masterFlushedOffset;
+    }
 }
diff --git a/store/src/main/java/org/apache/rocketmq/store/StoreStatsService.java b/store/src/main/java/org/apache/rocketmq/store/StoreStatsService.java
index 512c373..99b8bb4 100644
--- a/store/src/main/java/org/apache/rocketmq/store/StoreStatsService.java
+++ b/store/src/main/java/org/apache/rocketmq/store/StoreStatsService.java
@@ -28,6 +28,7 @@ import java.util.concurrent.ConcurrentMap;
 import java.util.concurrent.atomic.AtomicLong;
 import java.util.concurrent.atomic.LongAdder;
 import java.util.concurrent.locks.ReentrantLock;
+import org.apache.rocketmq.common.BrokerConfig;
 import org.apache.rocketmq.common.ServiceThread;
 import org.apache.rocketmq.common.constant.LoggerName;
 import org.apache.rocketmq.logging.InternalLogger;
@@ -81,6 +82,13 @@ public class StoreStatsService extends ServiceThread {
     private ReentrantLock samplingLock = new ReentrantLock();
     private long lastPrintTimestamp = System.currentTimeMillis();
 
+    private BrokerConfig brokerConfig;
+
+    public StoreStatsService(BrokerConfig brokerConfig) {
+        this();
+        this.brokerConfig = brokerConfig;
+    }
+
     public StoreStatsService() {
         PUT_MESSAGE_ENTIRE_TIME_BUCKETS.put(1,20);  //0-20
         PUT_MESSAGE_ENTIRE_TIME_BUCKETS.put(2,15);  //20-50
@@ -537,6 +545,9 @@ public class StoreStatsService extends ServiceThread {
 
     @Override
     public String getServiceName() {
+        if (this.brokerConfig != null && this.brokerConfig.isInBrokerContainer()) {
+            return brokerConfig.getLoggerIdentifier() + StoreStatsService.class.getSimpleName();
+        }
         return StoreStatsService.class.getSimpleName();
     }
 
diff --git a/store/src/main/java/org/apache/rocketmq/store/config/MessageStoreConfig.java b/store/src/main/java/org/apache/rocketmq/store/config/MessageStoreConfig.java
index 12ff598..1a6d6ae 100644
--- a/store/src/main/java/org/apache/rocketmq/store/config/MessageStoreConfig.java
+++ b/store/src/main/java/org/apache/rocketmq/store/config/MessageStoreConfig.java
@@ -24,8 +24,6 @@ import java.io.File;
 
 public class MessageStoreConfig {
 
-    public static final String MULTI_PATH_SPLITTER = System.getProperty("rocketmq.broker.multiPathSplitter", ",");
-
     //The root directory in which the log data is kept
     @ImportantField
     private String storePathRootDir = System.getProperty("user.home") + File.separator + "store";
@@ -91,6 +89,8 @@ public class MessageStoreConfig {
     // The number of hours to keep a log file before deleting it (in hours)
     @ImportantField
     private int fileReservedTime = 72;
+    @ImportantField
+    private int deleteFileBatchMax = 10;
     // Flow control for ConsumeQueue
     private int putMsgIndexHightWater = 600000;
     // The maximum size of message,default is 4M
@@ -130,15 +130,22 @@ public class MessageStoreConfig {
     private int haListenPort = 10912;
     private int haSendHeartbeatInterval = 1000 * 5;
     private int haHousekeepingInterval = 1000 * 20;
+    /**
+     * Maximum size of data to transfer to slave.
+     * NOTE: cannot be larger than HAClient.READ_MAX_BUFFER_SIZE
+     */
     private int haTransferBatchSize = 1024 * 32;
     @ImportantField
     private String haMasterAddress = null;
-    private int haSlaveFallbehindMax = 1024 * 1024 * 256;
+    private int haMaxGapNotInSync = 1024 * 1024 * 256;
     @ImportantField
     private BrokerRole brokerRole = BrokerRole.ASYNC_MASTER;
     @ImportantField
     private FlushDiskType flushDiskType = FlushDiskType.ASYNC_FLUSH;
+    // Used by GroupTransferService to sync messages from master to slave
     private int syncFlushTimeout = 1000 * 5;
+    // Used by PutMessage to wait messages be flushed to disk and synchronized in current broker member group.
+    private int putMessageTimeout = 1000 * 8;
     private int slaveTimeout = 3000;
     private String messageDelayLevel = "1s 5s 10s 30s 1m 2m 3m 4m 5m 6m 7m 8m 9m 10m 20m 30m 1h 2h";
     private long flushDelayOffsetInterval = 1000 * 10;
@@ -157,17 +164,24 @@ public class MessageStoreConfig {
     private int transientStorePoolSize = 5;
     private boolean fastFailIfNoBufferInStorePool = false;
 
+    // DLedger message store config
     private boolean enableDLegerCommitLog = false;
     private String dLegerGroup;
     private String dLegerPeers;
     private String dLegerSelfId;
-
     private String preferredLeaderId;
-
     private boolean isEnableBatchPush = false;
 
     private boolean enableScheduleMessageStats = true;
 
+    private boolean enableLmq = false;
+    private boolean enableMultiDispatch = false;
+    private int maxLmqConsumeQueueNum = 20000;
+
+    private boolean enableScheduleAsyncDeliver = false;
+    private int scheduleAsyncDeliverMaxPendingLimit = 2000;
+    private int scheduleAsyncDeliverMaxResendNum2Blocked = 3;
+
     private int maxBatchDeleteFilesNum = 50;
     //Polish dispatch
     private int dispatchCqThreads = 10;
@@ -210,13 +224,66 @@ public class MessageStoreConfig {
 
     private int pullBatchMaxMessageCount = 160;
 
-    private boolean enableLmq = false;
-    private boolean enableMultiDispatch = false;
-    private int maxLmqConsumeQueueNum = 20000;
+    @ImportantField
+    private int totalReplicas = 1;
 
-    private boolean enableScheduleAsyncDeliver = false;
-    private int scheduleAsyncDeliverMaxPendingLimit = 2000;
-    private int scheduleAsyncDeliverMaxResendNum2Blocked = 3;
+    /**
+     * Each message must be written successfully to at least in-sync replicas.
+     * The master broker is considered one of the in-sync replicas, and it's included in the count of total.
+     * If a master broker is ASYNC_MASTER, inSyncReplicas will be ignored.
+     */
+    @ImportantField
+    private int inSyncReplicas = 1;
+
+    /**
+     * Will be worked in auto multiple replicas mode, to provide minimum in-sync replicas.
+     */
+    @ImportantField
+    private int minInSyncReplicas = 1;
+
+    /**
+     * Dynamically adjust in-sync replicas to provide higher availability, the real time in-sync replicas
+     * will smaller than inSyncReplicas config.
+     */
+    @ImportantField
+    private boolean enableAutoInSyncReplicas = false;
+
+    /**
+     * Enable or not ha flow control
+     */
+    @ImportantField
+    private boolean haFlowControlEnable = false;
+
+    /**
+     * The max speed for one slave when transfer data in ha
+     */
+    private long maxHaTransferByteInSecond = 100 * 1024 * 1024;
+
+    /**
+     * Sync flush offset from master when broker startup, used in upgrading from old version broker.
+     */
+    private boolean syncMasterFlushOffsetWhenStartup = false;
+
+    /**
+     * Max checksum range.
+     */
+    private long maxChecksumRange = 1024 * 1024 * 1024;
+
+    private int replicasPerDiskPartition = 1;
+
+    private double logicalDiskSpaceCleanForciblyThreshold = 0.8;
+
+    /**
+     * 1. Register to broker after (startTime + disappearTimeAfterStart)
+     * 2. Internal msg exchange will start after (startTime + disappearTimeAfterStart)
+     *  PopReviveService
+     */
+    @ImportantField
+    private int disappearTimeAfterStart = -1;
+
+    private long maxSlaveResendLength = 256 * 1024 * 1024;
+
+    private boolean syncFromMinOffset = false;
 
     public boolean isDebugLockEnable() {
         return debugLockEnable;
@@ -602,12 +669,12 @@ public class MessageStoreConfig {
         this.haTransferBatchSize = haTransferBatchSize;
     }
 
-    public int getHaSlaveFallbehindMax() {
-        return haSlaveFallbehindMax;
+    public int getHaMaxGapNotInSync() {
+        return haMaxGapNotInSync;
     }
 
-    public void setHaSlaveFallbehindMax(int haSlaveFallbehindMax) {
-        this.haSlaveFallbehindMax = haSlaveFallbehindMax;
+    public void setHaMaxGapNotInSync(int haMaxGapNotInSync) {
+        this.haMaxGapNotInSync = haMaxGapNotInSync;
     }
 
     public FlushDiskType getFlushDiskType() {
@@ -630,6 +697,14 @@ public class MessageStoreConfig {
         this.syncFlushTimeout = syncFlushTimeout;
     }
 
+    public int getPutMessageTimeout() {
+        return putMessageTimeout;
+    }
+
+    public void setPutMessageTimeout(int putMessageTimeout) {
+        this.putMessageTimeout = putMessageTimeout;
+    }
+
     public int getSlaveTimeout() {
         return slaveTimeout;
     }
@@ -1035,6 +1110,118 @@ public class MessageStoreConfig {
         this.pullBatchMaxMessageCount = pullBatchMaxMessageCount;
     }
 
+    public int getDeleteFileBatchMax() {
+        return deleteFileBatchMax;
+    }
+
+    public void setDeleteFileBatchMax(int deleteFileBatchMax) {
+        this.deleteFileBatchMax = deleteFileBatchMax;
+    }
+
+    public int getTotalReplicas() {
+        return totalReplicas;
+    }
+
+    public void setTotalReplicas(int totalReplicas) {
+        this.totalReplicas = totalReplicas;
+    }
+
+    public int getInSyncReplicas() {
+        return inSyncReplicas;
+    }
+
+    public void setInSyncReplicas(int inSyncReplicas) {
+        this.inSyncReplicas = inSyncReplicas;
+    }
+
+    public int getMinInSyncReplicas() {
+        return minInSyncReplicas;
+    }
+
+    public void setMinInSyncReplicas(int minInSyncReplicas) {
+        this.minInSyncReplicas = minInSyncReplicas;
+    }
+
+    public boolean isEnableAutoInSyncReplicas() {
+        return enableAutoInSyncReplicas;
+    }
+
+    public void setEnableAutoInSyncReplicas(boolean enableAutoInSyncReplicas) {
+        this.enableAutoInSyncReplicas = enableAutoInSyncReplicas;
+    }
+
+    public boolean isHaFlowControlEnable() {
+        return haFlowControlEnable;
+    }
+
+    public void setHaFlowControlEnable(boolean haFlowControlEnable) {
+        this.haFlowControlEnable = haFlowControlEnable;
+    }
+
+    public long getMaxHaTransferByteInSecond() {
+        return maxHaTransferByteInSecond;
+    }
+
+    public void setMaxHaTransferByteInSecond(long maxHaTransferByteInSecond) {
+        this.maxHaTransferByteInSecond = maxHaTransferByteInSecond;
+    }
+
+    public boolean isSyncMasterFlushOffsetWhenStartup() {
+        return syncMasterFlushOffsetWhenStartup;
+    }
+
+    public void setSyncMasterFlushOffsetWhenStartup(boolean syncMasterFlushOffsetWhenStartup) {
+        this.syncMasterFlushOffsetWhenStartup = syncMasterFlushOffsetWhenStartup;
+    }
+
+    public long getMaxChecksumRange() {
+        return maxChecksumRange;
+    }
+
+    public void setMaxChecksumRange(long maxChecksumRange) {
+        this.maxChecksumRange = maxChecksumRange;
+    }
+
+    public int getReplicasPerDiskPartition() {
+        return replicasPerDiskPartition;
+    }
+
+    public void setReplicasPerDiskPartition(int replicasPerDiskPartition) {
+        this.replicasPerDiskPartition = replicasPerDiskPartition;
+    }
+
+    public double getLogicalDiskSpaceCleanForciblyThreshold() {
+        return logicalDiskSpaceCleanForciblyThreshold;
+    }
+
+    public void setLogicalDiskSpaceCleanForciblyThreshold(double logicalDiskSpaceCleanForciblyThreshold) {
+        this.logicalDiskSpaceCleanForciblyThreshold = logicalDiskSpaceCleanForciblyThreshold;
+    }
+
+    public int getDisappearTimeAfterStart() {
+        return disappearTimeAfterStart;
+    }
+
+    public void setDisappearTimeAfterStart(int disappearTimeAfterStart) {
+        this.disappearTimeAfterStart = disappearTimeAfterStart;
+    }
+
+    public long getMaxSlaveResendLength() {
+        return maxSlaveResendLength;
+    }
+
+    public void setMaxSlaveResendLength(long maxSlaveResendLength) {
+        this.maxSlaveResendLength = maxSlaveResendLength;
+    }
+
+    public boolean isSyncFromMinOffset() {
+        return syncFromMinOffset;
+    }
+
+    public void setSyncFromMinOffset(boolean syncFromMinOffset) {
+        this.syncFromMinOffset = syncFromMinOffset;
+    }
+
     public boolean isEnableLmq() {
         return enableLmq;
     }
diff --git a/store/src/main/java/org/apache/rocketmq/store/dledger/DLedgerCommitLog.java b/store/src/main/java/org/apache/rocketmq/store/dledger/DLedgerCommitLog.java
index fbf0067..42064a3 100644
--- a/store/src/main/java/org/apache/rocketmq/store/dledger/DLedgerCommitLog.java
+++ b/store/src/main/java/org/apache/rocketmq/store/dledger/DLedgerCommitLog.java
@@ -37,24 +37,20 @@ import java.util.LinkedList;
 import java.util.List;
 import java.util.concurrent.CompletableFuture;
 import org.apache.rocketmq.common.UtilAll;
-import org.apache.rocketmq.common.message.MessageAccessor;
-import org.apache.rocketmq.common.message.MessageConst;
 import org.apache.rocketmq.common.message.MessageDecoder;
+import org.apache.rocketmq.common.message.MessageExtBatch;
 import org.apache.rocketmq.common.sysflag.MessageSysFlag;
-import org.apache.rocketmq.common.topic.TopicValidator;
 import org.apache.rocketmq.store.AppendMessageResult;
 import org.apache.rocketmq.store.AppendMessageStatus;
 import org.apache.rocketmq.store.CommitLog;
 import org.apache.rocketmq.store.DefaultMessageStore;
 import org.apache.rocketmq.store.DispatchRequest;
-import org.apache.rocketmq.store.MessageExtBatch;
-import org.apache.rocketmq.store.MessageExtBrokerInner;
+import org.apache.rocketmq.common.message.MessageExtBrokerInner;
 import org.apache.rocketmq.store.PutMessageResult;
 import org.apache.rocketmq.store.PutMessageStatus;
 import org.apache.rocketmq.store.SelectMappedBufferResult;
 import org.apache.rocketmq.store.StoreStatsService;
 import org.apache.rocketmq.store.logfile.MappedFile;
-import org.apache.rocketmq.store.schedule.ScheduleMessageService;
 
 /**
  * Store all metadata downtime for recovery, data protection reliability
@@ -322,15 +318,10 @@ public class DLedgerCommitLog extends CommitLog {
     }
 
     @Override
-    public DispatchRequest checkMessageAndReturnSize(ByteBuffer byteBuffer, final boolean checkCRC) {
-        return this.checkMessageAndReturnSize(byteBuffer, checkCRC, true);
-    }
-
-    @Override
     public DispatchRequest checkMessageAndReturnSize(ByteBuffer byteBuffer, final boolean checkCRC,
-        final boolean readBody) {
+        final boolean checkDupInfo, final boolean readBody) {
         if (isInrecoveringOldCommitlog) {
-            return super.checkMessageAndReturnSize(byteBuffer, checkCRC, readBody);
+            return super.checkMessageAndReturnSize(byteBuffer, checkCRC, checkDupInfo, readBody);
         }
         try {
             int bodyOffset = DLedgerEntry.BODY_OFFSET;
@@ -340,13 +331,13 @@ public class DLedgerCommitLog extends CommitLog {
             int magicOld = byteBuffer.getInt();
             if (magicOld == CommitLog.BLANK_MAGIC_CODE || magicOld == CommitLog.MESSAGE_MAGIC_CODE) {
                 byteBuffer.position(pos);
-                return super.checkMessageAndReturnSize(byteBuffer, checkCRC, readBody);
+                return super.checkMessageAndReturnSize(byteBuffer, checkCRC, checkDupInfo, readBody);
             }
             if (magic == MmapFileList.BLANK_MAGIC_CODE) {
                 return new DispatchRequest(0, true);
             }
             byteBuffer.position(pos + bodyOffset);
-            DispatchRequest dispatchRequest = super.checkMessageAndReturnSize(byteBuffer, checkCRC, readBody);
+            DispatchRequest dispatchRequest = super.checkMessageAndReturnSize(byteBuffer, checkCRC, checkDupInfo, readBody);
             if (dispatchRequest.isSuccess()) {
                 dispatchRequest.setBufferSize(dispatchRequest.getMsgSize() + bodyOffset);
             } else if (dispatchRequest.getMsgSize() > 0) {
@@ -377,29 +368,6 @@ public class DLedgerCommitLog extends CommitLog {
         // on the client)
         msg.setBodyCRC(UtilAll.crc32(msg.getBody()));
 
-        //should be consistent with the old version
-        if (tranType == MessageSysFlag.TRANSACTION_NOT_TYPE
-            || tranType == MessageSysFlag.TRANSACTION_COMMIT_TYPE) {
-            // Delay Delivery
-            if (msg.getDelayTimeLevel() > 0) {
-                if (msg.getDelayTimeLevel() > this.defaultMessageStore.getScheduleMessageService().getMaxDelayLevel()) {
-                    msg.setDelayTimeLevel(this.defaultMessageStore.getScheduleMessageService().getMaxDelayLevel());
-                }
-
-
-                String topic = TopicValidator.RMQ_SYS_SCHEDULE_TOPIC;
-                int queueId = ScheduleMessageService.delayLevel2QueueId(msg.getDelayTimeLevel());
-
-                // Backup real topic, queueId
-                MessageAccessor.putProperty(msg, MessageConst.PROPERTY_REAL_TOPIC, msg.getTopic());
-                MessageAccessor.putProperty(msg, MessageConst.PROPERTY_REAL_QUEUE_ID, String.valueOf(msg.getQueueId()));
-                msg.setPropertiesString(MessageDecoder.messageProperties2String(msg.getProperties()));
-
-                msg.setTopic(topic);
-                msg.setQueueId(queueId);
-            }
-        }
-
         InetSocketAddress bornSocketAddress = (InetSocketAddress) msg.getBornHost();
         if (bornSocketAddress.getAddress() instanceof Inet6Address) {
             msg.setBornHostV6Flag();
@@ -430,7 +398,7 @@ public class DLedgerCommitLog extends CommitLog {
         String topicQueueKey = msg.getTopic() + "-" + msg.getQueueId();
         topicQueueLock.lock(topicQueueKey);
         try {
-            defaultMessageStore.assignOffset(topicQueueKey, msg, getMessageNum(msg));
+            defaultMessageStore.assignOffset(msg, getMessageNum(msg));
 
             encodeResult = this.messageSerializer.serialize(msg);
             if (encodeResult.status != AppendMessageStatus.PUT_OK) {
@@ -449,7 +417,7 @@ public class DLedgerCommitLog extends CommitLog {
                 request.setBody(encodeResult.getData());
                 dledgerFuture = (AppendFuture<AppendEntryResponse>) dLedgerServer.handleAppend(request);
                 if (dledgerFuture.getPos() == -1) {
-                    return CompletableFuture.completedFuture(new PutMessageResult(PutMessageStatus.OS_PAGECACHE_BUSY, new AppendMessageResult(AppendMessageStatus.UNKNOWN_ERROR)));
+                    return CompletableFuture.completedFuture(new PutMessageResult(PutMessageStatus.OS_PAGE_CACHE_BUSY, new AppendMessageResult(AppendMessageStatus.UNKNOWN_ERROR)));
                 }
                 long wroteOffset = dledgerFuture.getPos() + DLedgerEntry.BODY_OFFSET;
 
@@ -487,11 +455,11 @@ public class DLedgerCommitLog extends CommitLog {
                     putMessageStatus = PutMessageStatus.SERVICE_NOT_AVAILABLE;
                     break;
                 case WAIT_QUORUM_ACK_TIMEOUT:
-                    //Do not return flush_slave_timeout to the client, for the ons client will ignore it.
-                    putMessageStatus = PutMessageStatus.OS_PAGECACHE_BUSY;
+                    //Do not return flush_slave_timeout to the client, for the client will ignore it.
+                    putMessageStatus = PutMessageStatus.IN_SYNC_REPLICAS_NOT_ENOUGH;
                     break;
                 case LEADER_PENDING_FULL:
-                    putMessageStatus = PutMessageStatus.OS_PAGECACHE_BUSY;
+                    putMessageStatus = PutMessageStatus.OS_PAGE_CACHE_BUSY;
                     break;
             }
             PutMessageResult putMessageResult = new PutMessageResult(putMessageStatus, appendResult);
@@ -538,13 +506,13 @@ public class DLedgerCommitLog extends CommitLog {
         encodeResult = this.messageSerializer.serialize(messageExtBatch);
         if (encodeResult.status != AppendMessageStatus.PUT_OK) {
             return CompletableFuture.completedFuture(new PutMessageResult(PutMessageStatus.MESSAGE_ILLEGAL, new AppendMessageResult(encodeResult
-                    .status)));
+                .status)));
         }
 
         int batchNum = encodeResult.batchData.size();
         topicQueueLock.lock(encodeResult.queueOffsetKey);
         try {
-            defaultMessageStore.assignOffset(encodeResult.queueOffsetKey, messageExtBatch, (short) batchNum);
+            defaultMessageStore.assignOffset(messageExtBatch, (short) batchNum);
 
             putMessageLock.lock(); //spin or ReentrantLock ,depending on store config
             msgIdBuilder.setLength(0);
@@ -562,7 +530,7 @@ public class DLedgerCommitLog extends CommitLog {
                 AppendFuture<AppendEntryResponse> appendFuture = (AppendFuture<AppendEntryResponse>) dLedgerServer.handleAppend(request);
                 if (appendFuture.getPos() == -1) {
                     log.warn("HandleAppend return false due to error code {}", appendFuture.get().getCode());
-                    return CompletableFuture.completedFuture(new PutMessageResult(PutMessageStatus.OS_PAGECACHE_BUSY, new AppendMessageResult(AppendMessageStatus.UNKNOWN_ERROR)));
+                    return CompletableFuture.completedFuture(new PutMessageResult(PutMessageStatus.OS_PAGE_CACHE_BUSY, new AppendMessageResult(AppendMessageStatus.UNKNOWN_ERROR)));
                 }
                 dledgerFuture = (BatchAppendFuture<AppendEntryResponse>) appendFuture;
 
@@ -590,7 +558,7 @@ public class DLedgerCommitLog extends CommitLog {
 
                 elapsedTimeInLock = this.defaultMessageStore.getSystemClock().now() - beginTimeInDledgerLock;
                 appendResult = new AppendMessageResult(AppendMessageStatus.PUT_OK, firstWroteOffset, encodeResult.totalMsgLen,
-                        msgIdBuilder.toString(), System.currentTimeMillis(), queueOffset, elapsedTimeInLock);
+                    msgIdBuilder.toString(), System.currentTimeMillis(), queueOffset, elapsedTimeInLock);
                 appendResult.setMsgNum(msgNum);
             } catch (Exception e) {
                 log.error("Put message error", e);
@@ -602,7 +570,7 @@ public class DLedgerCommitLog extends CommitLog {
 
             if (elapsedTimeInLock > 500) {
                 log.warn("[NOTIFYME]putMessage in lock cost time(ms)={}, bodyLength={} AppendMessageResult={}",
-                        elapsedTimeInLock, messageExtBatch.getBody().length, appendResult);
+                    elapsedTimeInLock, messageExtBatch.getBody().length, appendResult);
             }
         } finally {
             topicQueueLock.unlock(encodeResult.queueOffsetKey);
@@ -621,11 +589,11 @@ public class DLedgerCommitLog extends CommitLog {
                     putMessageStatus = PutMessageStatus.SERVICE_NOT_AVAILABLE;
                     break;
                 case WAIT_QUORUM_ACK_TIMEOUT:
-                    //Do not return flush_slave_timeout to the client, for the ons client will ignore it.
-                    putMessageStatus = PutMessageStatus.OS_PAGECACHE_BUSY;
+                    //Do not return flush_slave_timeout to the client, for the client will ignore it.
+                    putMessageStatus = PutMessageStatus.IN_SYNC_REPLICAS_NOT_ENOUGH;
                     break;
                 case LEADER_PENDING_FULL:
-                    putMessageStatus = PutMessageStatus.OS_PAGECACHE_BUSY;
+                    putMessageStatus = PutMessageStatus.OS_PAGE_CACHE_BUSY;
                     break;
             }
             PutMessageResult putMessageResult = new PutMessageResult(putMessageStatus, appendResult);
@@ -709,7 +677,6 @@ public class DLedgerCommitLog extends CommitLog {
         return queueOffset;
     }
 
-
     class EncodeResult {
         private String queueOffsetKey;
         private ByteBuffer data;
@@ -738,7 +705,8 @@ public class DLedgerCommitLog extends CommitLog {
             return data.array();
         }
 
-        public EncodeResult(AppendMessageStatus status, String queueOffsetKey, List<byte[]> batchData, int totalMsgLen) {
+        public EncodeResult(AppendMessageStatus status, String queueOffsetKey, List<byte[]> batchData,
+            int totalMsgLen) {
             this.batchData = batchData;
             this.status = status;
             this.queueOffsetKey = queueOffsetKey;
@@ -891,8 +859,8 @@ public class DLedgerCommitLog extends CommitLog {
                 // Exceeds the maximum message
                 if (msgLen > this.maxMessageSize) {
                     CommitLog.log.warn("message size exceeded, msg total size: " + msgLen + ", msg body size: " +
-                            bodyLen
-                            + ", maxMessageSize: " + this.maxMessageSize);
+                        bodyLen
+                        + ", maxMessageSize: " + this.maxMessageSize);
                     throw new RuntimeException("message size exceeded");
                 }
 
diff --git a/store/src/main/java/org/apache/rocketmq/store/ha/DefaultHAClient.java b/store/src/main/java/org/apache/rocketmq/store/ha/DefaultHAClient.java
new file mode 100644
index 0000000..5f651da
--- /dev/null
+++ b/store/src/main/java/org/apache/rocketmq/store/ha/DefaultHAClient.java
@@ -0,0 +1,392 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.rocketmq.store.ha;
+
+import java.io.IOException;
+import java.net.SocketAddress;
+import java.nio.ByteBuffer;
+import java.nio.channels.ClosedChannelException;
+import java.nio.channels.SelectionKey;
+import java.nio.channels.Selector;
+import java.nio.channels.SocketChannel;
+import java.util.concurrent.atomic.AtomicReference;
+import org.apache.rocketmq.common.ServiceThread;
+import org.apache.rocketmq.common.constant.LoggerName;
+import org.apache.rocketmq.logging.InternalLogger;
+import org.apache.rocketmq.logging.InternalLoggerFactory;
+import org.apache.rocketmq.remoting.common.RemotingUtil;
+import org.apache.rocketmq.store.DefaultMessageStore;
+
+public class DefaultHAClient extends ServiceThread implements HAClient {
+
+    private static final InternalLogger log = InternalLoggerFactory.getLogger(LoggerName.STORE_LOGGER_NAME);
+    private static final int READ_MAX_BUFFER_SIZE = 1024 * 1024 * 4;
+    private final AtomicReference<String> masterHaAddress = new AtomicReference<>();
+    private final AtomicReference<String> masterAddress = new AtomicReference<>();
+    private final ByteBuffer reportOffset = ByteBuffer.allocate(8);
+    private SocketChannel socketChannel;
+    private Selector selector;
+    /**
+     * last time that slave reads date from master.
+     */
+    private long lastReadTimestamp = System.currentTimeMillis();
+    /**
+     * last time that slave reports offset to master.
+     */
+    private long lastWriteTimestamp = System.currentTimeMillis();
+
+    private long currentReportedOffset = 0;
+    private int dispatchPosition = 0;
+    private ByteBuffer byteBufferRead = ByteBuffer.allocate(READ_MAX_BUFFER_SIZE);
+    private ByteBuffer byteBufferBackup = ByteBuffer.allocate(READ_MAX_BUFFER_SIZE);
+    private DefaultMessageStore defaultMessageStore;
+    private volatile HAConnectionState currentState = HAConnectionState.READY;
+    private FlowMonitor flowMonitor;
+
+    public DefaultHAClient(DefaultMessageStore defaultMessageStore) throws IOException {
+        this.selector = RemotingUtil.openSelector();
+        this.defaultMessageStore = defaultMessageStore;
+        this.flowMonitor = new FlowMonitor(defaultMessageStore.getMessageStoreConfig());
+    }
+
+    public void updateHaMasterAddress(final String newAddr) {
+        String currentAddr = this.masterHaAddress.get();
+        if (masterHaAddress.compareAndSet(currentAddr, newAddr)) {
+            log.info("update master ha address, OLD: " + currentAddr + " NEW: " + newAddr);
+        }
+    }
+
+    public void updateMasterAddress(final String newAddr) {
+        String currentAddr = this.masterAddress.get();
+        if (masterAddress.compareAndSet(currentAddr, newAddr)) {
+            log.info("update master address, OLD: " + currentAddr + " NEW: " + newAddr);
+        }
+    }
+
+    public String getHaMasterAddress() {
+        return this.masterHaAddress.get();
+    }
+
+    public String getMasterAddress() {
+        return this.masterAddress.get();
+    }
+
+    private boolean isTimeToReportOffset() {
+        long interval = defaultMessageStore.now() - this.lastWriteTimestamp;
+        return interval > defaultMessageStore.getMessageStoreConfig().getHaSendHeartbeatInterval();
+    }
+
+    private boolean reportSlaveMaxOffset(final long maxOffset) {
+        this.reportOffset.position(0);
+        this.reportOffset.limit(8);
+        this.reportOffset.putLong(maxOffset);
+        this.reportOffset.position(0);
+        this.reportOffset.limit(8);
+
+        for (int i = 0; i < 3 && this.reportOffset.hasRemaining(); i++) {
+            try {
+                this.socketChannel.write(this.reportOffset);
+            } catch (IOException e) {
+                log.error(this.getServiceName()
+                    + "reportSlaveMaxOffset this.socketChannel.write exception", e);
+                return false;
+            }
+        }
+        lastWriteTimestamp = this.defaultMessageStore.getSystemClock().now();
+        return !this.reportOffset.hasRemaining();
+    }
+
+    private void reallocateByteBuffer() {
+        int remain = READ_MAX_BUFFER_SIZE - this.dispatchPosition;
+        if (remain > 0) {
+            this.byteBufferRead.position(this.dispatchPosition);
+
+            this.byteBufferBackup.position(0);
+            this.byteBufferBackup.limit(READ_MAX_BUFFER_SIZE);
+            this.byteBufferBackup.put(this.byteBufferRead);
+        }
+
+        this.swapByteBuffer();
+
+        this.byteBufferRead.position(remain);
+        this.byteBufferRead.limit(READ_MAX_BUFFER_SIZE);
+        this.dispatchPosition = 0;
+    }
+
+    private void swapByteBuffer() {
+        ByteBuffer tmp = this.byteBufferRead;
+        this.byteBufferRead = this.byteBufferBackup;
+        this.byteBufferBackup = tmp;
+    }
+
+    private boolean processReadEvent() {
+        int readSizeZeroTimes = 0;
+        while (this.byteBufferRead.hasRemaining()) {
+            try {
+                int readSize = this.socketChannel.read(this.byteBufferRead);
+                if (readSize > 0) {
+                    flowMonitor.addByteCountTransferred(readSize);
+                    readSizeZeroTimes = 0;
+                    boolean result = this.dispatchReadRequest();
+                    if (!result) {
+                        log.error("HAClient, dispatchReadRequest error");
+                        return false;
+                    }
+                    lastReadTimestamp = System.currentTimeMillis();
+                } else if (readSize == 0) {
+                    if (++readSizeZeroTimes >= 3) {
+                        break;
+                    }
+                } else {
+                    log.info("HAClient, processReadEvent read socket < 0");
+                    return false;
+                }
+            } catch (IOException e) {
+                log.info("HAClient, processReadEvent read socket exception", e);
+                return false;
+            }
+        }
+
+        return true;
+    }
+
+    private boolean dispatchReadRequest() {
+        final int msgHeaderSize = 8 + 4; // phyoffset + size
+        int readSocketPos = this.byteBufferRead.position();
+
+        while (true) {
+            int diff = this.byteBufferRead.position() - this.dispatchPosition;
+            if (diff >= msgHeaderSize) {
+                long masterPhyOffset = this.byteBufferRead.getLong(this.dispatchPosition);
+                int bodySize = this.byteBufferRead.getInt(this.dispatchPosition + 8);
+
+                long slavePhyOffset = this.defaultMessageStore.getMaxPhyOffset();
+
+                if (slavePhyOffset != 0) {
+                    if (slavePhyOffset != masterPhyOffset) {
+                        log.error("master pushed offset not equal the max phy offset in slave, SLAVE: "
+                            + slavePhyOffset + " MASTER: " + masterPhyOffset);
+                        return false;
+                    }
+                }
+
+                if (diff >= (msgHeaderSize + bodySize)) {
+                    byte[] bodyData = byteBufferRead.array();
+                    int dataStart = this.dispatchPosition + msgHeaderSize;
+
+                    this.defaultMessageStore.appendToCommitLog(
+                        masterPhyOffset, bodyData, dataStart, bodySize);
+
+                    this.byteBufferRead.position(readSocketPos);
+                    this.dispatchPosition += msgHeaderSize + bodySize;
+
+                    if (!reportSlaveMaxOffsetPlus()) {
+                        return false;
+                    }
+
+                    continue;
+                }
+            }
+
+            if (!this.byteBufferRead.hasRemaining()) {
+                this.reallocateByteBuffer();
+            }
+
+            break;
+        }
+
+        return true;
+    }
+
+    private boolean reportSlaveMaxOffsetPlus() {
+        boolean result = true;
+        long currentPhyOffset = this.defaultMessageStore.getMaxPhyOffset();
+        if (currentPhyOffset > this.currentReportedOffset) {
+            this.currentReportedOffset = currentPhyOffset;
+            result = this.reportSlaveMaxOffset(this.currentReportedOffset);
+            if (!result) {
+                this.closeMaster();
+                log.error("HAClient, reportSlaveMaxOffset error, " + this.currentReportedOffset);
+            }
+        }
+
+        return result;
+    }
+
+    public void changeCurrentState(HAConnectionState currentState) {
+        log.info("change state to {}", currentState);
+        this.currentState = currentState;
+    }
+
+    public boolean connectMaster() throws ClosedChannelException {
+        if (null == socketChannel) {
+            String addr = this.masterHaAddress.get();
+            if (addr != null) {
+                SocketAddress socketAddress = RemotingUtil.string2SocketAddress(addr);
+                this.socketChannel = RemotingUtil.connect(socketAddress);
+                if (this.socketChannel != null) {
+                    this.socketChannel.register(this.selector, SelectionKey.OP_READ);
+                    log.info("HAClient connect to master {}", addr);
+                    this.changeCurrentState(HAConnectionState.TRANSFER);
+                }
+            }
+
+            this.currentReportedOffset = this.defaultMessageStore.getMaxPhyOffset();
+
+            this.lastReadTimestamp = System.currentTimeMillis();
+        }
+
+        return this.socketChannel != null;
+    }
+
+    public void closeMaster() {
+        if (null != this.socketChannel) {
+            try {
+
+                SelectionKey sk = this.socketChannel.keyFor(this.selector);
+                if (sk != null) {
+                    sk.cancel();
+                }
+
+                this.socketChannel.close();
+
+                this.socketChannel = null;
+
+                log.info("HAClient close connection with master {}", this.masterHaAddress.get());
+                this.changeCurrentState(HAConnectionState.READY);
+            } catch (IOException e) {
+                log.warn("closeMaster exception. ", e);
+            }
+
+            this.lastReadTimestamp = 0;
+            this.dispatchPosition = 0;
+
+            this.byteBufferBackup.position(0);
+            this.byteBufferBackup.limit(READ_MAX_BUFFER_SIZE);
+
+            this.byteBufferRead.position(0);
+            this.byteBufferRead.limit(READ_MAX_BUFFER_SIZE);
+        }
+    }
+
+    @Override
+    public void run() {
+        log.info(this.getServiceName() + " service started");
+
+        this.flowMonitor.start();
+
+        while (!this.isStopped()) {
+            try {
+                switch (this.currentState) {
+                    case SHUTDOWN:
+                        return;
+                    case READY:
+                        if (!this.connectMaster()) {
+                            log.warn("HAClient connect to master {} failed", this.masterHaAddress.get());
+                            this.waitForRunning(1000 * 5);
+                        }
+                        continue;
+                    case TRANSFER:
+                        if (!transferFromMaster()) {
+                            closeMasterAndWait();
+                            continue;
+                        }
+                        break;
+                    default:
+                        this.waitForRunning(1000 * 2);
+                        continue;
+                }
+                long interval = this.defaultMessageStore.now() - this.lastReadTimestamp;
+                if (interval > this.defaultMessageStore.getMessageStoreConfig().getHaHousekeepingInterval()) {
+                    log.warn("AutoRecoverHAClient, housekeeping, found this connection[" + this.masterHaAddress
+                        + "] expired, " + interval);
+                    this.closeMaster();
+                    log.warn("AutoRecoverHAClient, master not response some time, so close connection");
+                }
+            } catch (Exception e) {
+                log.warn(this.getServiceName() + " service has exception. ", e);
+                this.closeMasterAndWait();
+            }
+        }
+
+        log.info(this.getServiceName() + " service end");
+    }
+
+    private boolean transferFromMaster() throws IOException {
+        boolean result;
+        if (this.isTimeToReportOffset()) {
+            log.info("Slave report current offset {}", this.currentReportedOffset);
+            result = this.reportSlaveMaxOffset(this.currentReportedOffset);
+            if (!result) {
+                return false;
+            }
+        }
+
+        this.selector.select(1000);
+
+        result = this.processReadEvent();
+        if (!result) {
+            return false;
+        }
+
+        return reportSlaveMaxOffsetPlus();
+    }
+
+    public void closeMasterAndWait() {
+        this.closeMaster();
+        this.waitForRunning(1000 * 5);
+    }
+
+    public long getLastWriteTimestamp() {
+        return this.lastWriteTimestamp;
+    }
+
+    public long getLastReadTimestamp() {
+        return lastReadTimestamp;
+    }
+
+    @Override public HAConnectionState getCurrentState() {
+        return currentState;
+    }
+
+    @Override public long getTransferredByteInSecond() {
+        return flowMonitor.getTransferredByteInSecond();
+    }
+
+    @Override
+    public void shutdown() {
+        this.changeCurrentState(HAConnectionState.SHUTDOWN);
+        this.flowMonitor.shutdown();
+        super.shutdown();
+
+        closeMaster();
+        try {
+            this.selector.close();
+        } catch (IOException e) {
+            log.warn("Close the selector of AutoRecoverHAClient error, ", e);
+        }
+    }
+
+    @Override
+    public String getServiceName() {
+        if (this.defaultMessageStore != null && this.defaultMessageStore.getBrokerConfig().isInBrokerContainer()) {
+            return this.defaultMessageStore.getBrokerConfig().getLoggerIdentifier() + DefaultHAClient.class.getSimpleName();
+        }
+        return DefaultHAClient.class.getSimpleName();
+    }
+}
diff --git a/store/src/main/java/org/apache/rocketmq/store/ha/HAConnection.java b/store/src/main/java/org/apache/rocketmq/store/ha/DefaultHAConnection.java
similarity index 63%
copy from store/src/main/java/org/apache/rocketmq/store/ha/HAConnection.java
copy to store/src/main/java/org/apache/rocketmq/store/ha/DefaultHAConnection.java
index 4c26971..4c16959 100644
--- a/store/src/main/java/org/apache/rocketmq/store/ha/HAConnection.java
+++ b/store/src/main/java/org/apache/rocketmq/store/ha/DefaultHAConnection.java
@@ -14,6 +14,7 @@
  * See the License for the specific language governing permissions and
  * limitations under the License.
  */
+
 package org.apache.rocketmq.store.ha;
 
 import java.io.IOException;
@@ -26,46 +27,47 @@ import org.apache.rocketmq.common.constant.LoggerName;
 import org.apache.rocketmq.logging.InternalLogger;
 import org.apache.rocketmq.logging.InternalLoggerFactory;
 import org.apache.rocketmq.remoting.common.RemotingUtil;
-import org.apache.rocketmq.remoting.netty.NettySystemConfig;
 import org.apache.rocketmq.store.SelectMappedBufferResult;
 
-public class HAConnection {
+public class DefaultHAConnection implements HAConnection {
     private static final InternalLogger log = InternalLoggerFactory.getLogger(LoggerName.STORE_LOGGER_NAME);
-    private final HAService haService;
+    private final DefaultHAService haService;
     private final SocketChannel socketChannel;
-    private final String clientAddr;
+    private final String clientAddress;
     private WriteSocketService writeSocketService;
     private ReadSocketService readSocketService;
-
+    private volatile HAConnectionState currentState = HAConnectionState.TRANSFER;
     private volatile long slaveRequestOffset = -1;
     private volatile long slaveAckOffset = -1;
+    private FlowMonitor flowMonitor;
 
-    public HAConnection(final HAService haService, final SocketChannel socketChannel) throws IOException {
+    public DefaultHAConnection(final DefaultHAService haService, final SocketChannel socketChannel) throws IOException {
         this.haService = haService;
         this.socketChannel = socketChannel;
-        this.clientAddr = this.socketChannel.socket().getRemoteSocketAddress().toString();
+        this.clientAddress = this.socketChannel.socket().getRemoteSocketAddress().toString();
         this.socketChannel.configureBlocking(false);
         this.socketChannel.socket().setSoLinger(false, -1);
         this.socketChannel.socket().setTcpNoDelay(true);
-        if (NettySystemConfig.socketSndbufSize > 0) {
-            this.socketChannel.socket().setReceiveBufferSize(NettySystemConfig.socketSndbufSize);
-        }
-        if (NettySystemConfig.socketRcvbufSize > 0) {
-            this.socketChannel.socket().setSendBufferSize(NettySystemConfig.socketRcvbufSize);
-        }
+        this.socketChannel.socket().setReceiveBufferSize(1024 * 64);
+        this.socketChannel.socket().setSendBufferSize(1024 * 64);
         this.writeSocketService = new WriteSocketService(this.socketChannel);
         this.readSocketService = new ReadSocketService(this.socketChannel);
         this.haService.getConnectionCount().incrementAndGet();
+        this.flowMonitor = new FlowMonitor(haService.getDefaultMessageStore().getMessageStoreConfig());
     }
 
     public void start() {
+        changeCurrentState(HAConnectionState.TRANSFER);
+        this.flowMonitor.start();
         this.readSocketService.start();
         this.writeSocketService.start();
     }
 
     public void shutdown() {
+        changeCurrentState(HAConnectionState.SHUTDOWN);
         this.writeSocketService.shutdown(true);
         this.readSocketService.shutdown(true);
+        this.flowMonitor.shutdown(true);
         this.close();
     }
 
@@ -74,7 +76,7 @@ public class HAConnection {
             try {
                 this.socketChannel.close();
             } catch (IOException e) {
-                HAConnection.log.error("", e);
+                log.error("", e);
             }
         }
     }
@@ -83,6 +85,31 @@ public class HAConnection {
         return socketChannel;
     }
 
+    public void changeCurrentState(HAConnectionState currentState) {
+        log.info("change state to {}", currentState);
+        this.currentState = currentState;
+    }
+
+    @Override public HAConnectionState getCurrentState() {
+        return currentState;
+    }
+
+    @Override public String getClientAddress() {
+        return this.clientAddress;
+    }
+
+    @Override public long getSlaveAckOffset() {
+        return slaveAckOffset;
+    }
+
+    public long getTransferredByteInSecond() {
+        return this.flowMonitor.getTransferredByteInSecond();
+    }
+
+    public long getTransferFromWhere() {
+        return writeSocketService.getNextTransferFromWhere();
+    }
+
     class ReadSocketService extends ServiceThread {
         private static final int READ_MAX_BUFFER_SIZE = 1024 * 1024;
         private final Selector selector;
@@ -100,35 +127,37 @@ public class HAConnection {
 
         @Override
         public void run() {
-            HAConnection.log.info(this.getServiceName() + " service started");
+            log.info(this.getServiceName() + " service started");
 
             while (!this.isStopped()) {
                 try {
                     this.selector.select(1000);
                     boolean ok = this.processReadEvent();
                     if (!ok) {
-                        HAConnection.log.error("processReadEvent error");
+                        log.error("processReadEvent error");
                         break;
                     }
 
-                    long interval = HAConnection.this.haService.getDefaultMessageStore().getSystemClock().now() - this.lastReadTimestamp;
-                    if (interval > HAConnection.this.haService.getDefaultMessageStore().getMessageStoreConfig().getHaHousekeepingInterval()) {
-                        log.warn("ha housekeeping, found this connection[" + HAConnection.this.clientAddr + "] expired, " + interval);
+                    long interval = DefaultHAConnection.this.haService.getDefaultMessageStore().getSystemClock().now() - this.lastReadTimestamp;
+                    if (interval > DefaultHAConnection.this.haService.getDefaultMessageStore().getMessageStoreConfig().getHaHousekeepingInterval()) {
+                        log.warn("ha housekeeping, found this connection[" + DefaultHAConnection.this.clientAddress + "] expired, " + interval);
                         break;
                     }
                 } catch (Exception e) {
-                    HAConnection.log.error(this.getServiceName() + " service has exception.", e);
+                    log.error(this.getServiceName() + " service has exception.", e);
                     break;
                 }
             }
 
+            changeCurrentState(HAConnectionState.SHUTDOWN);
+
             this.makeStop();
 
             writeSocketService.makeStop();
 
-            haService.removeConnection(HAConnection.this);
+            haService.removeConnection(DefaultHAConnection.this);
 
-            HAConnection.this.haService.getConnectionCount().decrementAndGet();
+            DefaultHAConnection.this.haService.getConnectionCount().decrementAndGet();
 
             SelectionKey sk = this.socketChannel.keyFor(this.selector);
             if (sk != null) {
@@ -139,14 +168,17 @@ public class HAConnection {
                 this.selector.close();
                 this.socketChannel.close();
             } catch (IOException e) {
-                HAConnection.log.error("", e);
+                log.error("", e);
             }
 
-            HAConnection.log.info(this.getServiceName() + " service end");
+            log.info(this.getServiceName() + " service end");
         }
 
         @Override
         public String getServiceName() {
+            if (haService.getDefaultMessageStore().getBrokerConfig().isInBrokerContainer()) {
+                return haService.getDefaultMessageStore().getBrokerConfig().getLoggerIdentifier() + ReadSocketService.class.getSimpleName();
+            }
             return ReadSocketService.class.getSimpleName();
         }
 
@@ -163,32 +195,26 @@ public class HAConnection {
                     int readSize = this.socketChannel.read(this.byteBufferRead);
                     if (readSize > 0) {
                         readSizeZeroTimes = 0;
-                        this.lastReadTimestamp = HAConnection.this.haService.getDefaultMessageStore().getSystemClock().now();
+                        this.lastReadTimestamp = DefaultHAConnection.this.haService.getDefaultMessageStore().getSystemClock().now();
                         if ((this.byteBufferRead.position() - this.processPosition) >= 8) {
                             int pos = this.byteBufferRead.position() - (this.byteBufferRead.position() % 8);
                             long readOffset = this.byteBufferRead.getLong(pos - 8);
                             this.processPosition = pos;
 
-                            HAConnection.this.slaveAckOffset = readOffset;
-                            if (HAConnection.this.slaveRequestOffset < 0) {
-                                HAConnection.this.slaveRequestOffset = readOffset;
-                                log.info("slave[" + HAConnection.this.clientAddr + "] request offset " + readOffset);
-                            } else if (HAConnection.this.slaveAckOffset > HAConnection.this.haService.getDefaultMessageStore().getMaxPhyOffset()) {
-                                log.warn("slave[{}] request offset={} greater than local commitLog offset={}. ",
-                                        HAConnection.this.clientAddr,
-                                        HAConnection.this.slaveAckOffset,
-                                        HAConnection.this.haService.getDefaultMessageStore().getMaxPhyOffset());
-                                return false;
+                            DefaultHAConnection.this.slaveAckOffset = readOffset;
+                            if (DefaultHAConnection.this.slaveRequestOffset < 0) {
+                                DefaultHAConnection.this.slaveRequestOffset = readOffset;
+                                log.info("slave[" + DefaultHAConnection.this.clientAddress + "] request offset " + readOffset);
                             }
 
-                            HAConnection.this.haService.notifyTransferSome(HAConnection.this.slaveAckOffset);
+                            DefaultHAConnection.this.haService.notifyTransferSome(DefaultHAConnection.this.slaveAckOffset);
                         }
                     } else if (readSize == 0) {
                         if (++readSizeZeroTimes >= 3) {
                             break;
                         }
                     } else {
-                        log.error("read socket[" + HAConnection.this.clientAddr + "] < 0");
+                        log.error("read socket[" + DefaultHAConnection.this.clientAddress + "] < 0");
                         return false;
                     }
                 } catch (IOException e) {
@@ -210,6 +236,7 @@ public class HAConnection {
         private long nextTransferFromWhere = -1;
         private SelectMappedBufferResult selectMappedBufferResult;
         private boolean lastWriteOver = true;
+        private long lastPrintTimestamp = System.currentTimeMillis();
         private long lastWriteTimestamp = System.currentTimeMillis();
 
         public WriteSocketService(final SocketChannel socketChannel) throws IOException {
@@ -221,23 +248,23 @@ public class HAConnection {
 
         @Override
         public void run() {
-            HAConnection.log.info(this.getServiceName() + " service started");
+            log.info(this.getServiceName() + " service started");
 
             while (!this.isStopped()) {
                 try {
                     this.selector.select(1000);
 
-                    if (-1 == HAConnection.this.slaveRequestOffset) {
+                    if (-1 == DefaultHAConnection.this.slaveRequestOffset) {
                         Thread.sleep(10);
                         continue;
                     }
 
                     if (-1 == this.nextTransferFromWhere) {
-                        if (0 == HAConnection.this.slaveRequestOffset) {
-                            long masterOffset = HAConnection.this.haService.getDefaultMessageStore().getCommitLog().getMaxOffset();
+                        if (0 == DefaultHAConnection.this.slaveRequestOffset) {
+                            long masterOffset = DefaultHAConnection.this.haService.getDefaultMessageStore().getCommitLog().getMaxOffset();
                             masterOffset =
                                 masterOffset
-                                    - (masterOffset % HAConnection.this.haService.getDefaultMessageStore().getMessageStoreConfig()
+                                    - (masterOffset % DefaultHAConnection.this.haService.getDefaultMessageStore().getMessageStoreConfig()
                                     .getMappedFileSizeCommitLog());
 
                             if (masterOffset < 0) {
@@ -246,19 +273,19 @@ public class HAConnection {
 
                             this.nextTransferFromWhere = masterOffset;
                         } else {
-                            this.nextTransferFromWhere = HAConnection.this.slaveRequestOffset;
+                            this.nextTransferFromWhere = DefaultHAConnection.this.slaveRequestOffset;
                         }
 
-                        log.info("master transfer data from " + this.nextTransferFromWhere + " to slave[" + HAConnection.this.clientAddr
-                            + "], and slave request " + HAConnection.this.slaveRequestOffset);
+                        log.info("master transfer data from " + this.nextTransferFromWhere + " to slave[" + DefaultHAConnection.this.clientAddress
+                            + "], and slave request " + DefaultHAConnection.this.slaveRequestOffset);
                     }
 
                     if (this.lastWriteOver) {
 
                         long interval =
-                            HAConnection.this.haService.getDefaultMessageStore().getSystemClock().now() - this.lastWriteTimestamp;
+                            DefaultHAConnection.this.haService.getDefaultMessageStore().getSystemClock().now() - this.lastWriteTimestamp;
 
-                        if (interval > HAConnection.this.haService.getDefaultMessageStore().getMessageStoreConfig()
+                        if (interval > DefaultHAConnection.this.haService.getDefaultMessageStore().getMessageStoreConfig()
                             .getHaSendHeartbeatInterval()) {
 
                             // Build Header
@@ -279,11 +306,22 @@ public class HAConnection {
                     }
 
                     SelectMappedBufferResult selectResult =
-                        HAConnection.this.haService.getDefaultMessageStore().getCommitLogData(this.nextTransferFromWhere);
+                        DefaultHAConnection.this.haService.getDefaultMessageStore().getCommitLogData(this.nextTransferFromWhere);
                     if (selectResult != null) {
                         int size = selectResult.getSize();
-                        if (size > HAConnection.this.haService.getDefaultMessageStore().getMessageStoreConfig().getHaTransferBatchSize()) {
-                            size = HAConnection.this.haService.getDefaultMessageStore().getMessageStoreConfig().getHaTransferBatchSize();
+                        if (size > DefaultHAConnection.this.haService.getDefaultMessageStore().getMessageStoreConfig().getHaTransferBatchSize()) {
+                            size = DefaultHAConnection.this.haService.getDefaultMessageStore().getMessageStoreConfig().getHaTransferBatchSize();
+                        }
+
+                        int canTransferMaxBytes = flowMonitor.canTransferMaxByteNum();
+                        if (size > canTransferMaxBytes) {
+                            if (System.currentTimeMillis() - lastPrintTimestamp > 1000) {
+                                log.warn("Trigger HA flow control, max transfer speed {}KB/s, current speed: {}KB/s",
+                                    String.format("%.2f", flowMonitor.maxTransferByteInSecond() / 1024.0),
+                                    String.format("%.2f", flowMonitor.getTransferredByteInSecond() / 1024.0));
+                                lastPrintTimestamp = System.currentTimeMillis();
+                            }
+                            size = canTransferMaxBytes;
                         }
 
                         long thisOffset = this.nextTransferFromWhere;
@@ -302,26 +340,28 @@ public class HAConnection {
                         this.lastWriteOver = this.transferData();
                     } else {
 
-                        HAConnection.this.haService.getWaitNotifyObject().allWaitForRunning(100);
+                        DefaultHAConnection.this.haService.getWaitNotifyObject().allWaitForRunning(100);
                     }
                 } catch (Exception e) {
 
-                    HAConnection.log.error(this.getServiceName() + " service has exception.", e);
+                    DefaultHAConnection.log.error(this.getServiceName() + " service has exception.", e);
                     break;
                 }
             }
 
-            HAConnection.this.haService.getWaitNotifyObject().removeFromWaitingThreadTable();
+            DefaultHAConnection.this.haService.getWaitNotifyObject().removeFromWaitingThreadTable();
 
             if (this.selectMappedBufferResult != null) {
                 this.selectMappedBufferResult.release();
             }
 
+            changeCurrentState(HAConnectionState.SHUTDOWN);
+
             this.makeStop();
 
             readSocketService.makeStop();
 
-            haService.removeConnection(HAConnection.this);
+            haService.removeConnection(DefaultHAConnection.this);
 
             SelectionKey sk = this.socketChannel.keyFor(this.selector);
             if (sk != null) {
@@ -332,10 +372,10 @@ public class HAConnection {
                 this.selector.close();
                 this.socketChannel.close();
             } catch (IOException e) {
-                HAConnection.log.error("", e);
+                DefaultHAConnection.log.error("", e);
             }
 
-            HAConnection.log.info(this.getServiceName() + " service end");
+            DefaultHAConnection.log.info(this.getServiceName() + " service end");
         }
 
         private boolean transferData() throws Exception {
@@ -344,8 +384,9 @@ public class HAConnection {
             while (this.byteBufferHeader.hasRemaining()) {
                 int writeSize = this.socketChannel.write(this.byteBufferHeader);
                 if (writeSize > 0) {
+                    flowMonitor.addByteCountTransferred(writeSize);
                     writeSizeZeroTimes = 0;
-                    this.lastWriteTimestamp = HAConnection.this.haService.getDefaultMessageStore().getSystemClock().now();
+                    this.lastWriteTimestamp = DefaultHAConnection.this.haService.getDefaultMessageStore().getSystemClock().now();
                 } else if (writeSize == 0) {
                     if (++writeSizeZeroTimes >= 3) {
                         break;
@@ -367,7 +408,7 @@ public class HAConnection {
                     int writeSize = this.socketChannel.write(this.selectMappedBufferResult.getByteBuffer());
                     if (writeSize > 0) {
                         writeSizeZeroTimes = 0;
-                        this.lastWriteTimestamp = HAConnection.this.haService.getDefaultMessageStore().getSystemClock().now();
+                        this.lastWriteTimestamp = DefaultHAConnection.this.haService.getDefaultMessageStore().getSystemClock().now();
                     } else if (writeSize == 0) {
                         if (++writeSizeZeroTimes >= 3) {
                             break;
@@ -390,6 +431,9 @@ public class HAConnection {
 
         @Override
         public String getServiceName() {
+            if (haService.getDefaultMessageStore().getBrokerConfig().isInBrokerContainer()) {
+                return haService.getDefaultMessageStore().getBrokerConfig().getLoggerIdentifier() + WriteSocketService.class.getSimpleName();
+            }
             return WriteSocketService.class.getSimpleName();
         }
 
@@ -397,5 +441,9 @@ public class HAConnection {
         public void shutdown() {
             super.shutdown();
         }
+
+        public long getNextTransferFromWhere() {
+            return nextTransferFromWhere;
+        }
     }
 }
diff --git a/store/src/main/java/org/apache/rocketmq/store/ha/DefaultHAService.java b/store/src/main/java/org/apache/rocketmq/store/ha/DefaultHAService.java
new file mode 100644
index 0000000..8c480a0
--- /dev/null
+++ b/store/src/main/java/org/apache/rocketmq/store/ha/DefaultHAService.java
@@ -0,0 +1,339 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.rocketmq.store.ha;
+
+import java.io.IOException;
+import java.net.InetSocketAddress;
+import java.net.SocketAddress;
+import java.nio.channels.SelectionKey;
+import java.nio.channels.Selector;
+import java.nio.channels.ServerSocketChannel;
+import java.nio.channels.SocketChannel;
+import java.util.LinkedList;
+import java.util.List;
+import java.util.Set;
+import java.util.concurrent.atomic.AtomicInteger;
+import java.util.concurrent.atomic.AtomicLong;
+import org.apache.rocketmq.common.ServiceThread;
+import org.apache.rocketmq.common.constant.LoggerName;
+import org.apache.rocketmq.common.protocol.body.HARuntimeInfo;
+import org.apache.rocketmq.logging.InternalLogger;
+import org.apache.rocketmq.logging.InternalLoggerFactory;
+import org.apache.rocketmq.remoting.common.RemotingUtil;
+import org.apache.rocketmq.store.CommitLog;
+import org.apache.rocketmq.store.DefaultMessageStore;
+import org.apache.rocketmq.store.config.BrokerRole;
+
+public class DefaultHAService implements HAService {
+
+    private static final InternalLogger log = InternalLoggerFactory.getLogger(LoggerName.STORE_LOGGER_NAME);
+
+    private final AtomicInteger connectionCount = new AtomicInteger(0);
+
+    private final List<HAConnection> connectionList = new LinkedList<>();
+
+    private AcceptSocketService acceptSocketService;
+
+    private DefaultMessageStore defaultMessageStore;
+
+    private WaitNotifyObject waitNotifyObject = new WaitNotifyObject();
+    private AtomicLong push2SlaveMaxOffset = new AtomicLong(0);
+
+    private GroupTransferService groupTransferService;
+
+    private DefaultHAClient haClient;
+
+    private HAConnectionStateNotificationService haConnectionStateNotificationService;
+
+    public DefaultHAService() {
+    }
+
+    public void init(final DefaultMessageStore defaultMessageStore) throws IOException {
+        this.defaultMessageStore = defaultMessageStore;
+        this.acceptSocketService =
+            new AcceptSocketService(defaultMessageStore.getMessageStoreConfig().getHaListenPort());
+        this.groupTransferService = new GroupTransferService(this, defaultMessageStore);
+        if (this.defaultMessageStore.getMessageStoreConfig().getBrokerRole() == BrokerRole.SLAVE) {
+            this.haClient = new DefaultHAClient(this.defaultMessageStore);
+        }
+        this.haConnectionStateNotificationService = new HAConnectionStateNotificationService(this, defaultMessageStore);
+    }
+
+    public void updateMasterAddress(final String newAddr) {
+        if (this.haClient != null) {
+            this.haClient.updateMasterAddress(newAddr);
+        }
+    }
+
+    public void updateHaMasterAddress(String newAddr) {
+        if (this.haClient != null) {
+            this.haClient.updateHaMasterAddress(newAddr);
+        }
+    }
+
+    public void putRequest(final CommitLog.GroupCommitRequest request) {
+        this.groupTransferService.putRequest(request);
+    }
+
+    public boolean isSlaveOK(final long masterPutWhere) {
+        boolean result = this.connectionCount.get() > 0;
+        result =
+            result
+                && ((masterPutWhere - this.push2SlaveMaxOffset.get()) < this.defaultMessageStore
+                .getMessageStoreConfig().getHaMaxGapNotInSync());
+        return result;
+    }
+
+    public void notifyTransferSome(final long offset) {
+        for (long value = this.push2SlaveMaxOffset.get(); offset > value; ) {
+            boolean ok = this.push2SlaveMaxOffset.compareAndSet(value, offset);
+            if (ok) {
+                this.groupTransferService.notifyTransferSome();
+                break;
+            } else {
+                value = this.push2SlaveMaxOffset.get();
+            }
+        }
+    }
+
+    public AtomicInteger getConnectionCount() {
+        return connectionCount;
+    }
+
+    public void start() throws Exception {
+        this.acceptSocketService.beginAccept();
+        this.acceptSocketService.start();
+        this.groupTransferService.start();
+        this.haConnectionStateNotificationService.start();
+        if (haClient != null) {
+            this.haClient.start();
+        }
+    }
+
+    public void addConnection(final DefaultHAConnection conn) {
+        synchronized (this.connectionList) {
+            this.connectionList.add(conn);
+        }
+    }
+
+    public void removeConnection(final DefaultHAConnection conn) {
+        this.haConnectionStateNotificationService.checkConnectionStateAndNotify(conn);
+        synchronized (this.connectionList) {
+            this.connectionList.remove(conn);
+        }
+    }
+
+    public void shutdown() {
+        if (this.haClient != null) {
+            this.haClient.shutdown();
+        }
+        this.acceptSocketService.shutdown(true);
+        this.destroyConnections();
+        this.groupTransferService.shutdown();
+        this.haConnectionStateNotificationService.shutdown();
+    }
+
+    public void destroyConnections() {
+        synchronized (this.connectionList) {
+            for (HAConnection c : this.connectionList) {
+                c.shutdown();
+            }
+
+            this.connectionList.clear();
+        }
+    }
+
+    public DefaultMessageStore getDefaultMessageStore() {
+        return defaultMessageStore;
+    }
+
+    public WaitNotifyObject getWaitNotifyObject() {
+        return waitNotifyObject;
+    }
+
+    public AtomicLong getPush2SlaveMaxOffset() {
+        return push2SlaveMaxOffset;
+    }
+
+    public int inSyncSlaveNums(final long masterPutWhere) {
+        int inSyncNums = 0;
+        for (HAConnection conn : this.connectionList) {
+            if (this.isInSyncSlave(masterPutWhere, conn)) {
+                inSyncNums++;
+            }
+        }
+        return inSyncNums;
+    }
+
+    private boolean isInSyncSlave(final long masterPutWhere, HAConnection conn) {
+        if (masterPutWhere - conn.getSlaveAckOffset() < this.defaultMessageStore.getMessageStoreConfig()
+            .getHaMaxGapNotInSync()) {
+            return true;
+        }
+        return false;
+    }
+
+    @Override public void putGroupConnectionStateRequest(HAConnectionStateNotificationRequest request) {
+        this.haConnectionStateNotificationService.setRequest(request);
+    }
+
+    @Override public List<HAConnection> getConnectionList() {
+        return connectionList;
+    }
+
+    @Override public HAClient getHAClient() {
+        return this.haClient;
+    }
+
+    @Override public HARuntimeInfo getRuntimeInfo(long masterPutWhere) {
+        HARuntimeInfo info = new HARuntimeInfo();
+
+        if (BrokerRole.SLAVE.equals(this.getDefaultMessageStore().getMessageStoreConfig().getBrokerRole())) {
+            info.setMaster(false);
+
+            info.getHaClientRuntimeInfo().setMasterAddr(this.haClient.getHaMasterAddress());
+            info.getHaClientRuntimeInfo().setMaxOffset(this.getDefaultMessageStore().getMaxPhyOffset());
+            info.getHaClientRuntimeInfo().setLastReadTimestamp(this.haClient.getLastReadTimestamp());
+            info.getHaClientRuntimeInfo().setLastWriteTimestamp(this.haClient.getLastWriteTimestamp());
+            info.getHaClientRuntimeInfo().setTransferredByteInSecond(this.haClient.getTransferredByteInSecond());
+            info.getHaClientRuntimeInfo().setMasterFlushOffset(this.defaultMessageStore.getMasterFlushedOffset());
+        } else {
+            info.setMaster(true);
+            int inSyncNums = 0;
+
+            info.setMasterCommitLogMaxOffset(masterPutWhere);
+
+            for (HAConnection conn : this.connectionList) {
+                HARuntimeInfo.HAConnectionRuntimeInfo cInfo = new HARuntimeInfo.HAConnectionRuntimeInfo();
+
+                long slaveAckOffset = conn.getSlaveAckOffset();
+                cInfo.setSlaveAckOffset(slaveAckOffset);
+                cInfo.setDiff(masterPutWhere - slaveAckOffset);
+                cInfo.setAddr(conn.getClientAddress().substring(1));
+                cInfo.setTransferredByteInSecond(conn.getTransferredByteInSecond());
+                cInfo.setTransferFromWhere(conn.getTransferFromWhere());
+
+                boolean isInSync = this.isInSyncSlave(masterPutWhere, conn);
+                if (isInSync) {
+                    inSyncNums++;
+                }
+                cInfo.setInSync(isInSync);
+
+                info.getHaConnectionInfo().add(cInfo);
+            }
+            info.setInSyncSlaveNums(inSyncNums);
+        }
+        return info;
+    }
+
+    /**
+     * Listens to slave connections to create {@link HAConnection}.
+     */
+    class AcceptSocketService extends ServiceThread {
+        private final SocketAddress socketAddressListen;
+        private ServerSocketChannel serverSocketChannel;
+        private Selector selector;
+
+        public AcceptSocketService(final int port) {
+            this.socketAddressListen = new InetSocketAddress(port);
+        }
+
+        /**
+         * Starts listening to slave connections.
+         *
+         * @throws Exception If fails.
+         */
+        public void beginAccept() throws Exception {
+            this.serverSocketChannel = ServerSocketChannel.open();
+            this.selector = RemotingUtil.openSelector();
+            this.serverSocketChannel.socket().setReuseAddress(true);
+            this.serverSocketChannel.socket().bind(this.socketAddressListen);
+            this.serverSocketChannel.configureBlocking(false);
+            this.serverSocketChannel.register(this.selector, SelectionKey.OP_ACCEPT);
+        }
+
+        /**
+         * {@inheritDoc}
+         */
+        @Override
+        public void shutdown(final boolean interrupt) {
+            super.shutdown(interrupt);
+            try {
+                this.serverSocketChannel.close();
+                this.selector.close();
+            } catch (IOException e) {
+                log.error("AcceptSocketService shutdown exception", e);
+            }
+        }
+
+        /**
+         * {@inheritDoc}
+         */
+        @Override
+        public void run() {
+            log.info(this.getServiceName() + " service started");
+
+            while (!this.isStopped()) {
+                try {
+                    this.selector.select(1000);
+                    Set<SelectionKey> selected = this.selector.selectedKeys();
+
+                    if (selected != null) {
+                        for (SelectionKey k : selected) {
+                            if ((k.readyOps() & SelectionKey.OP_ACCEPT) != 0) {
+                                SocketChannel sc = ((ServerSocketChannel) k.channel()).accept();
+
+                                if (sc != null) {
+                                    DefaultHAService.log.info("HAService receive new connection, "
+                                        + sc.socket().getRemoteSocketAddress());
+                                    try {
+                                        DefaultHAConnection conn = new DefaultHAConnection(DefaultHAService.this, sc);
+                                        conn.start();
+                                        DefaultHAService.this.addConnection(conn);
+                                    } catch (Exception e) {
+                                        log.error("new HAConnection exception", e);
+                                        sc.close();
+                                    }
+                                }
+                            } else {
+                                log.warn("Unexpected ops in select " + k.readyOps());
+                            }
+                        }
+
+                        selected.clear();
+                    }
+                } catch (Exception e) {
+                    log.error(this.getServiceName() + " service has exception.", e);
+                }
+            }
+
+            log.info(this.getServiceName() + " service end");
+        }
+
+        /**
+         * {@inheritDoc}
+         */
+        @Override
+        public String getServiceName() {
+            if (defaultMessageStore.getBrokerConfig().isInBrokerContainer()) {
+                return defaultMessageStore.getBrokerConfig().getLoggerIdentifier() + AcceptSocketService.class.getSimpleName();
+            }
+            return AcceptSocketService.class.getSimpleName();
+        }
+    }
+}
diff --git a/store/src/main/java/org/apache/rocketmq/store/ha/FlowMonitor.java b/store/src/main/java/org/apache/rocketmq/store/ha/FlowMonitor.java
new file mode 100644
index 0000000..f64fbf3
--- /dev/null
+++ b/store/src/main/java/org/apache/rocketmq/store/ha/FlowMonitor.java
@@ -0,0 +1,76 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.rocketmq.store.ha;
+
+import java.util.concurrent.atomic.AtomicLong;
+
+import org.apache.rocketmq.common.ServiceThread;
+import org.apache.rocketmq.store.config.MessageStoreConfig;
+
+public class FlowMonitor extends ServiceThread {
+    private final AtomicLong transferredByte = new AtomicLong(0L);
+    private volatile long transferredByteInSecond;
+    protected MessageStoreConfig messageStoreConfig;
+
+    public FlowMonitor(MessageStoreConfig messageStoreConfig) {
+        this.messageStoreConfig = messageStoreConfig;
+    }
+
+    @Override
+    public void run() {
+        while (!this.isStopped()) {
+            this.waitForRunning(1 * 1000);
+            this.calculateSpeed();
+        }
+    }
+
+    public void calculateSpeed() {
+        this.transferredByteInSecond = this.transferredByte.get();
+        this.transferredByte.set(0);
+    }
+
+    public int canTransferMaxByteNum() {
+        //Flow control is not started at present
+        if (this.isFlowControlEnable()) {
+            long res = Math.max(this.maxTransferByteInSecond() - this.transferredByte.get(), 0);
+            return res > Integer.MAX_VALUE ? Integer.MAX_VALUE : (int) res;
+        }
+        return Integer.MAX_VALUE;
+    }
+
+    public void addByteCountTransferred(long count) {
+        this.transferredByte.addAndGet(count);
+    }
+
+    public long getTransferredByteInSecond() {
+        return this.transferredByteInSecond;
+    }
+
+    @Override
+    public String getServiceName() {
+        return FlowMonitor.class.getSimpleName();
+    }
+
+    protected boolean isFlowControlEnable() {
+        return this.messageStoreConfig.isHaFlowControlEnable();
+    }
+
+    public long maxTransferByteInSecond() {
+        return this.messageStoreConfig.getMaxHaTransferByteInSecond();
+    }
+}
\ No newline at end of file
diff --git a/store/src/main/java/org/apache/rocketmq/store/ha/GroupTransferService.java b/store/src/main/java/org/apache/rocketmq/store/ha/GroupTransferService.java
new file mode 100644
index 0000000..4bfb406
--- /dev/null
+++ b/store/src/main/java/org/apache/rocketmq/store/ha/GroupTransferService.java
@@ -0,0 +1,140 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.rocketmq.store.ha;
+
+import java.util.ArrayList;
+import java.util.List;
+import org.apache.rocketmq.common.ServiceThread;
+import org.apache.rocketmq.common.constant.LoggerName;
+import org.apache.rocketmq.logging.InternalLogger;
+import org.apache.rocketmq.logging.InternalLoggerFactory;
+import org.apache.rocketmq.store.CommitLog;
+import org.apache.rocketmq.store.DefaultMessageStore;
+import org.apache.rocketmq.store.PutMessageStatus;
+
+/**
+ * GroupTransferService Service
+ */
+public class GroupTransferService extends ServiceThread {
+
+    private static final InternalLogger log = InternalLoggerFactory.getLogger(LoggerName.STORE_LOGGER_NAME);
+
+    private final WaitNotifyObject notifyTransferObject = new WaitNotifyObject();
+    private volatile List<CommitLog.GroupCommitRequest> requestsWrite = new ArrayList<>();
+    private volatile List<CommitLog.GroupCommitRequest> requestsRead = new ArrayList<>();
+    private HAService haService;
+    private DefaultMessageStore defaultMessageStore;
+
+    public GroupTransferService(final HAService haService, final DefaultMessageStore defaultMessageStore) {
+        this.haService = haService;
+        this.defaultMessageStore = defaultMessageStore;
+    }
+
+    public synchronized void putRequest(final CommitLog.GroupCommitRequest request) {
+        synchronized (this.requestsWrite) {
+            this.requestsWrite.add(request);
+        }
+        if (hasNotified.compareAndSet(false, true)) {
+            waitPoint.countDown(); // notify
+        }
+    }
+
+    public void notifyTransferSome() {
+        this.notifyTransferObject.wakeup();
+    }
+
+    private void swapRequests() {
+        List<CommitLog.GroupCommitRequest> tmp = this.requestsWrite;
+        this.requestsWrite = this.requestsRead;
+        this.requestsRead = tmp;
+    }
+
+    private void doWaitTransfer() {
+        synchronized (this.requestsRead) {
+            if (!this.requestsRead.isEmpty()) {
+                for (CommitLog.GroupCommitRequest req : this.requestsRead) {
+                    boolean transferOK = false;
+
+                    long deadLine = req.getDeadLine();
+
+                    for (int i = 0; !transferOK && deadLine - System.nanoTime() > 0; i++) {
+                        if (i > 0) {
+                            this.notifyTransferObject.waitForRunning(1000);
+                        }
+
+                        if (req.getAckNums() <= 1) {
+                            transferOK = haService.getPush2SlaveMaxOffset().get() >= req.getNextOffset();
+                            continue;
+                        }
+
+                        int ackNums = 0;
+                        for (HAConnection conn : haService.getConnectionList()) {
+                            // TODO: We must ensure every AutoRecoverHAConnection represents a different slave
+                            // Solution: Consider assign a unique and fixed IP:ADDR for each different slave
+                            if (conn.getSlaveAckOffset() >= req.getNextOffset()) {
+                                ackNums++;
+                            }
+                            if (ackNums >= req.getAckNums()) {
+                                transferOK = true;
+                                break;
+                            }
+                        }
+                    }
+
+                    if (!transferOK) {
+                        log.warn("transfer message to slave timeout, offset : {}, request acks: {}",
+                            req.getNextOffset(), req.getAckNums());
+                    }
+
+                    req.wakeupCustomer(transferOK ? PutMessageStatus.PUT_OK : PutMessageStatus.FLUSH_SLAVE_TIMEOUT);
+                }
+
+                this.requestsRead.clear();
+            }
+        }
+    }
+
+    @Override
+    public void run() {
+        log.info(this.getServiceName() + " service started");
+
+        while (!this.isStopped()) {
+            try {
+                this.waitForRunning(10);
+                this.doWaitTransfer();
+            } catch (Exception e) {
+                log.warn(this.getServiceName() + " service has exception. ", e);
+            }
+        }
+
+        log.info(this.getServiceName() + " service end");
+    }
+
+    @Override
+    protected void onWaitEnd() {
+        this.swapRequests();
+    }
+
+    @Override
+    public String getServiceName() {
+        if (defaultMessageStore != null && defaultMessageStore.getBrokerConfig().isInBrokerContainer()) {
+            return defaultMessageStore.getBrokerConfig().getLoggerIdentifier() + GroupTransferService.class.getSimpleName();
+        }
+        return GroupTransferService.class.getSimpleName();
+    }
+}
diff --git a/store/src/main/java/org/apache/rocketmq/store/ha/HAClient.java b/store/src/main/java/org/apache/rocketmq/store/ha/HAClient.java
new file mode 100644
index 0000000..0449e01
--- /dev/null
+++ b/store/src/main/java/org/apache/rocketmq/store/ha/HAClient.java
@@ -0,0 +1,104 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.rocketmq.store.ha;
+
+public interface HAClient {
+
+    /**
+     * Start HAClient
+     */
+    void start();
+
+    /**
+     * Shutdown HAClient
+     */
+    void shutdown();
+
+    /**
+     * Wakeup HAClient
+     */
+    void wakeup();
+
+    /**
+     * Update master address
+     *
+     * @param newAddress
+     */
+    void updateMasterAddress(String newAddress);
+
+    /**
+     * Update master ha address
+     *
+     * @param newAddress
+     */
+    void updateHaMasterAddress(String newAddress);
+
+    /**
+     * Get master address
+     *
+     * @return master address
+     */
+    String getMasterAddress();
+
+    /**
+     * Get master ha address
+     *
+     * @return master ha address
+     */
+    String getHaMasterAddress();
+
+    /**
+     * Get HAClient last read timestamp
+     *
+     * @return last read timestamp
+     */
+    long getLastReadTimestamp();
+
+    /**
+     * Get HAClient last write timestamp
+     *
+     * @return last write timestamp
+     */
+    long getLastWriteTimestamp();
+
+    /**
+     * Get current state for ha connection
+     *
+     * @return HAConnectionState
+     */
+    HAConnectionState getCurrentState();
+
+    /**
+     * Change the current state for ha connection for testing
+     *
+     * @param haConnectionState
+     */
+    void changeCurrentState(HAConnectionState haConnectionState);
+
+    /**
+     * Disconnecting from the master for testing
+     */
+    void closeMaster();
+
+    /**
+     * Get the transfer rate per second
+     *
+     *  @return transfer bytes in second
+     */
+    long getTransferredByteInSecond();
+}
diff --git a/store/src/main/java/org/apache/rocketmq/store/ha/HAConnection.java b/store/src/main/java/org/apache/rocketmq/store/ha/HAConnection.java
index 4c26971..8e1e922 100644
--- a/store/src/main/java/org/apache/rocketmq/store/ha/HAConnection.java
+++ b/store/src/main/java/org/apache/rocketmq/store/ha/HAConnection.java
@@ -14,388 +14,64 @@
  * See the License for the specific language governing permissions and
  * limitations under the License.
  */
+
 package org.apache.rocketmq.store.ha;
 
-import java.io.IOException;
-import java.nio.ByteBuffer;
-import java.nio.channels.SelectionKey;
-import java.nio.channels.Selector;
 import java.nio.channels.SocketChannel;
-import org.apache.rocketmq.common.ServiceThread;
-import org.apache.rocketmq.common.constant.LoggerName;
-import org.apache.rocketmq.logging.InternalLogger;
-import org.apache.rocketmq.logging.InternalLoggerFactory;
-import org.apache.rocketmq.remoting.common.RemotingUtil;
-import org.apache.rocketmq.remoting.netty.NettySystemConfig;
-import org.apache.rocketmq.store.SelectMappedBufferResult;
-
-public class HAConnection {
-    private static final InternalLogger log = InternalLoggerFactory.getLogger(LoggerName.STORE_LOGGER_NAME);
-    private final HAService haService;
-    private final SocketChannel socketChannel;
-    private final String clientAddr;
-    private WriteSocketService writeSocketService;
-    private ReadSocketService readSocketService;
-
-    private volatile long slaveRequestOffset = -1;
-    private volatile long slaveAckOffset = -1;
-
-    public HAConnection(final HAService haService, final SocketChannel socketChannel) throws IOException {
-        this.haService = haService;
-        this.socketChannel = socketChannel;
-        this.clientAddr = this.socketChannel.socket().getRemoteSocketAddress().toString();
-        this.socketChannel.configureBlocking(false);
-        this.socketChannel.socket().setSoLinger(false, -1);
-        this.socketChannel.socket().setTcpNoDelay(true);
-        if (NettySystemConfig.socketSndbufSize > 0) {
-            this.socketChannel.socket().setReceiveBufferSize(NettySystemConfig.socketSndbufSize);
-        }
-        if (NettySystemConfig.socketRcvbufSize > 0) {
-            this.socketChannel.socket().setSendBufferSize(NettySystemConfig.socketRcvbufSize);
-        }
-        this.writeSocketService = new WriteSocketService(this.socketChannel);
-        this.readSocketService = new ReadSocketService(this.socketChannel);
-        this.haService.getConnectionCount().incrementAndGet();
-    }
-
-    public void start() {
-        this.readSocketService.start();
-        this.writeSocketService.start();
-    }
-
-    public void shutdown() {
-        this.writeSocketService.shutdown(true);
-        this.readSocketService.shutdown(true);
-        this.close();
-    }
-
-    public void close() {
-        if (this.socketChannel != null) {
-            try {
-                this.socketChannel.close();
-            } catch (IOException e) {
-                HAConnection.log.error("", e);
-            }
-        }
-    }
-
-    public SocketChannel getSocketChannel() {
-        return socketChannel;
-    }
-
-    class ReadSocketService extends ServiceThread {
-        private static final int READ_MAX_BUFFER_SIZE = 1024 * 1024;
-        private final Selector selector;
-        private final SocketChannel socketChannel;
-        private final ByteBuffer byteBufferRead = ByteBuffer.allocate(READ_MAX_BUFFER_SIZE);
-        private int processPosition = 0;
-        private volatile long lastReadTimestamp = System.currentTimeMillis();
-
-        public ReadSocketService(final SocketChannel socketChannel) throws IOException {
-            this.selector = RemotingUtil.openSelector();
-            this.socketChannel = socketChannel;
-            this.socketChannel.register(this.selector, SelectionKey.OP_READ);
-            this.setDaemon(true);
-        }
-
-        @Override
-        public void run() {
-            HAConnection.log.info(this.getServiceName() + " service started");
-
-            while (!this.isStopped()) {
-                try {
-                    this.selector.select(1000);
-                    boolean ok = this.processReadEvent();
-                    if (!ok) {
-                        HAConnection.log.error("processReadEvent error");
-                        break;
-                    }
-
-                    long interval = HAConnection.this.haService.getDefaultMessageStore().getSystemClock().now() - this.lastReadTimestamp;
-                    if (interval > HAConnection.this.haService.getDefaultMessageStore().getMessageStoreConfig().getHaHousekeepingInterval()) {
-                        log.warn("ha housekeeping, found this connection[" + HAConnection.this.clientAddr + "] expired, " + interval);
-                        break;
-                    }
-                } catch (Exception e) {
-                    HAConnection.log.error(this.getServiceName() + " service has exception.", e);
-                    break;
-                }
-            }
-
-            this.makeStop();
-
-            writeSocketService.makeStop();
-
-            haService.removeConnection(HAConnection.this);
-
-            HAConnection.this.haService.getConnectionCount().decrementAndGet();
-
-            SelectionKey sk = this.socketChannel.keyFor(this.selector);
-            if (sk != null) {
-                sk.cancel();
-            }
-
-            try {
-                this.selector.close();
-                this.socketChannel.close();
-            } catch (IOException e) {
-                HAConnection.log.error("", e);
-            }
-
-            HAConnection.log.info(this.getServiceName() + " service end");
-        }
-
-        @Override
-        public String getServiceName() {
-            return ReadSocketService.class.getSimpleName();
-        }
-
-        private boolean processReadEvent() {
-            int readSizeZeroTimes = 0;
-
-            if (!this.byteBufferRead.hasRemaining()) {
-                this.byteBufferRead.flip();
-                this.processPosition = 0;
-            }
-
-            while (this.byteBufferRead.hasRemaining()) {
-                try {
-                    int readSize = this.socketChannel.read(this.byteBufferRead);
-                    if (readSize > 0) {
-                        readSizeZeroTimes = 0;
-                        this.lastReadTimestamp = HAConnection.this.haService.getDefaultMessageStore().getSystemClock().now();
-                        if ((this.byteBufferRead.position() - this.processPosition) >= 8) {
-                            int pos = this.byteBufferRead.position() - (this.byteBufferRead.position() % 8);
-                            long readOffset = this.byteBufferRead.getLong(pos - 8);
-                            this.processPosition = pos;
-
-                            HAConnection.this.slaveAckOffset = readOffset;
-                            if (HAConnection.this.slaveRequestOffset < 0) {
-                                HAConnection.this.slaveRequestOffset = readOffset;
-                                log.info("slave[" + HAConnection.this.clientAddr + "] request offset " + readOffset);
-                            } else if (HAConnection.this.slaveAckOffset > HAConnection.this.haService.getDefaultMessageStore().getMaxPhyOffset()) {
-                                log.warn("slave[{}] request offset={} greater than local commitLog offset={}. ",
-                                        HAConnection.this.clientAddr,
-                                        HAConnection.this.slaveAckOffset,
-                                        HAConnection.this.haService.getDefaultMessageStore().getMaxPhyOffset());
-                                return false;
-                            }
-
-                            HAConnection.this.haService.notifyTransferSome(HAConnection.this.slaveAckOffset);
-                        }
-                    } else if (readSize == 0) {
-                        if (++readSizeZeroTimes >= 3) {
-                            break;
-                        }
-                    } else {
-                        log.error("read socket[" + HAConnection.this.clientAddr + "] < 0");
-                        return false;
-                    }
-                } catch (IOException e) {
-                    log.error("processReadEvent exception", e);
-                    return false;
-                }
-            }
-
-            return true;
-        }
-    }
-
-    class WriteSocketService extends ServiceThread {
-        private final Selector selector;
-        private final SocketChannel socketChannel;
-
-        private final int headerSize = 8 + 4;
-        private final ByteBuffer byteBufferHeader = ByteBuffer.allocate(headerSize);
-        private long nextTransferFromWhere = -1;
-        private SelectMappedBufferResult selectMappedBufferResult;
-        private boolean lastWriteOver = true;
-        private long lastWriteTimestamp = System.currentTimeMillis();
-
-        public WriteSocketService(final SocketChannel socketChannel) throws IOException {
-            this.selector = RemotingUtil.openSelector();
-            this.socketChannel = socketChannel;
-            this.socketChannel.register(this.selector, SelectionKey.OP_WRITE);
-            this.setDaemon(true);
-        }
-
-        @Override
-        public void run() {
-            HAConnection.log.info(this.getServiceName() + " service started");
-
-            while (!this.isStopped()) {
-                try {
-                    this.selector.select(1000);
-
-                    if (-1 == HAConnection.this.slaveRequestOffset) {
-                        Thread.sleep(10);
-                        continue;
-                    }
-
-                    if (-1 == this.nextTransferFromWhere) {
-                        if (0 == HAConnection.this.slaveRequestOffset) {
-                            long masterOffset = HAConnection.this.haService.getDefaultMessageStore().getCommitLog().getMaxOffset();
-                            masterOffset =
-                                masterOffset
-                                    - (masterOffset % HAConnection.this.haService.getDefaultMessageStore().getMessageStoreConfig()
-                                    .getMappedFileSizeCommitLog());
-
-                            if (masterOffset < 0) {
-                                masterOffset = 0;
-                            }
-
-                            this.nextTransferFromWhere = masterOffset;
-                        } else {
-                            this.nextTransferFromWhere = HAConnection.this.slaveRequestOffset;
-                        }
-
-                        log.info("master transfer data from " + this.nextTransferFromWhere + " to slave[" + HAConnection.this.clientAddr
-                            + "], and slave request " + HAConnection.this.slaveRequestOffset);
-                    }
-
-                    if (this.lastWriteOver) {
-
-                        long interval =
-                            HAConnection.this.haService.getDefaultMessageStore().getSystemClock().now() - this.lastWriteTimestamp;
-
-                        if (interval > HAConnection.this.haService.getDefaultMessageStore().getMessageStoreConfig()
-                            .getHaSendHeartbeatInterval()) {
-
-                            // Build Header
-                            this.byteBufferHeader.position(0);
-                            this.byteBufferHeader.limit(headerSize);
-                            this.byteBufferHeader.putLong(this.nextTransferFromWhere);
-                            this.byteBufferHeader.putInt(0);
-                            this.byteBufferHeader.flip();
-
-                            this.lastWriteOver = this.transferData();
-                            if (!this.lastWriteOver)
-                                continue;
-                        }
-                    } else {
-                        this.lastWriteOver = this.transferData();
-                        if (!this.lastWriteOver)
-                            continue;
-                    }
-
-                    SelectMappedBufferResult selectResult =
-                        HAConnection.this.haService.getDefaultMessageStore().getCommitLogData(this.nextTransferFromWhere);
-                    if (selectResult != null) {
-                        int size = selectResult.getSize();
-                        if (size > HAConnection.this.haService.getDefaultMessageStore().getMessageStoreConfig().getHaTransferBatchSize()) {
-                            size = HAConnection.this.haService.getDefaultMessageStore().getMessageStoreConfig().getHaTransferBatchSize();
-                        }
-
-                        long thisOffset = this.nextTransferFromWhere;
-                        this.nextTransferFromWhere += size;
-
-                        selectResult.getByteBuffer().limit(size);
-                        this.selectMappedBufferResult = selectResult;
-
-                        // Build Header
-                        this.byteBufferHeader.position(0);
-                        this.byteBufferHeader.limit(headerSize);
-                        this.byteBufferHeader.putLong(thisOffset);
-                        this.byteBufferHeader.putInt(size);
-                        this.byteBufferHeader.flip();
-
-                        this.lastWriteOver = this.transferData();
-                    } else {
-
-                        HAConnection.this.haService.getWaitNotifyObject().allWaitForRunning(100);
-                    }
-                } catch (Exception e) {
-
-                    HAConnection.log.error(this.getServiceName() + " service has exception.", e);
-                    break;
-                }
-            }
-
-            HAConnection.this.haService.getWaitNotifyObject().removeFromWaitingThreadTable();
-
-            if (this.selectMappedBufferResult != null) {
-                this.selectMappedBufferResult.release();
-            }
-
-            this.makeStop();
-
-            readSocketService.makeStop();
-
-            haService.removeConnection(HAConnection.this);
-
-            SelectionKey sk = this.socketChannel.keyFor(this.selector);
-            if (sk != null) {
-                sk.cancel();
-            }
-
-            try {
-                this.selector.close();
-                this.socketChannel.close();
-            } catch (IOException e) {
-                HAConnection.log.error("", e);
-            }
-
-            HAConnection.log.info(this.getServiceName() + " service end");
-        }
-
-        private boolean transferData() throws Exception {
-            int writeSizeZeroTimes = 0;
-            // Write Header
-            while (this.byteBufferHeader.hasRemaining()) {
-                int writeSize = this.socketChannel.write(this.byteBufferHeader);
-                if (writeSize > 0) {
-                    writeSizeZeroTimes = 0;
-                    this.lastWriteTimestamp = HAConnection.this.haService.getDefaultMessageStore().getSystemClock().now();
-                } else if (writeSize == 0) {
-                    if (++writeSizeZeroTimes >= 3) {
-                        break;
-                    }
-                } else {
-                    throw new Exception("ha master write header error < 0");
-                }
-            }
-
-            if (null == this.selectMappedBufferResult) {
-                return !this.byteBufferHeader.hasRemaining();
-            }
-
-            writeSizeZeroTimes = 0;
-
-            // Write Body
-            if (!this.byteBufferHeader.hasRemaining()) {
-                while (this.selectMappedBufferResult.getByteBuffer().hasRemaining()) {
-                    int writeSize = this.socketChannel.write(this.selectMappedBufferResult.getByteBuffer());
-                    if (writeSize > 0) {
-                        writeSizeZeroTimes = 0;
-                        this.lastWriteTimestamp = HAConnection.this.haService.getDefaultMessageStore().getSystemClock().now();
-                    } else if (writeSize == 0) {
-                        if (++writeSizeZeroTimes >= 3) {
-                            break;
-                        }
-                    } else {
-                        throw new Exception("ha master write body error < 0");
-                    }
-                }
-            }
-
-            boolean result = !this.byteBufferHeader.hasRemaining() && !this.selectMappedBufferResult.getByteBuffer().hasRemaining();
-
-            if (!this.selectMappedBufferResult.getByteBuffer().hasRemaining()) {
-                this.selectMappedBufferResult.release();
-                this.selectMappedBufferResult = null;
-            }
-
-            return result;
-        }
-
-        @Override
-        public String getServiceName() {
-            return WriteSocketService.class.getSimpleName();
-        }
 
-        @Override
-        public void shutdown() {
-            super.shutdown();
-        }
-    }
+public interface HAConnection {
+    /**
+     * Start HA Connection
+     */
+    void start();
+
+    /**
+     * Shutdown HA Connection
+     */
+    void shutdown();
+
+    /**
+     * Close HA Connection
+     */
+    void close();
+
+    /**
+     * Get socket channel
+     */
+    SocketChannel getSocketChannel();
+
+    /**
+     * Get current state for ha connection
+     *
+     * @return HAConnectionState
+     */
+    HAConnectionState getCurrentState();
+
+    /**
+     * Get client address for ha connection
+     *
+     * @return client ip address
+     */
+    String getClientAddress();
+
+    /**
+     * Get the transfer rate per second
+     *
+     *  @return transfer bytes in second
+     */
+    long getTransferredByteInSecond();
+
+    /**
+     * Get the current transfer offset to the slave
+     *
+     * @return the current transfer offset to the slave
+     */
+    long getTransferFromWhere();
+
+    /**
+     * Get slave ack offset
+     *
+     * @return slave ack offset
+     */
+    long getSlaveAckOffset();
 }
diff --git a/store/src/main/java/org/apache/rocketmq/store/PutMessageStatus.java b/store/src/main/java/org/apache/rocketmq/store/ha/HAConnectionState.java
similarity index 67%
copy from store/src/main/java/org/apache/rocketmq/store/PutMessageStatus.java
copy to store/src/main/java/org/apache/rocketmq/store/ha/HAConnectionState.java
index 29d0d95..4f0c5ca 100644
--- a/store/src/main/java/org/apache/rocketmq/store/PutMessageStatus.java
+++ b/store/src/main/java/org/apache/rocketmq/store/ha/HAConnectionState.java
@@ -14,18 +14,28 @@
  * See the License for the specific language governing permissions and
  * limitations under the License.
  */
-package org.apache.rocketmq.store;
 
-public enum PutMessageStatus {
-    PUT_OK,
-    FLUSH_DISK_TIMEOUT,
-    FLUSH_SLAVE_TIMEOUT,
-    SLAVE_NOT_AVAILABLE,
-    SERVICE_NOT_AVAILABLE,
-    CREATE_MAPEDFILE_FAILED,
-    MESSAGE_ILLEGAL,
-    PROPERTIES_SIZE_EXCEEDED,
-    OS_PAGECACHE_BUSY,
-    UNKNOWN_ERROR,
-    LMQ_CONSUME_QUEUE_NUM_EXCEEDED,
+package org.apache.rocketmq.store.ha;
+
+public enum HAConnectionState {
+    /**
+     * Ready to start connection.
+     */
+    READY,
+    /**
+     * CommitLog consistency checking.
+     */
+    HANDSHAKE,
+    /**
+     * Synchronizing data.
+     */
+    TRANSFER,
+    /**
+     * Temporarily stop transferring.
+     */
+    SUSPEND,
+    /**
+     * Connection shutdown.
+     */
+    SHUTDOWN,
 }
diff --git a/store/src/main/java/org/apache/rocketmq/store/ha/HAConnectionStateNotificationRequest.java b/store/src/main/java/org/apache/rocketmq/store/ha/HAConnectionStateNotificationRequest.java
new file mode 100644
index 0000000..8a3f6aa
--- /dev/null
+++ b/store/src/main/java/org/apache/rocketmq/store/ha/HAConnectionStateNotificationRequest.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.store.ha;
+
+import java.util.concurrent.CompletableFuture;
+
+public class HAConnectionStateNotificationRequest {
+    private final CompletableFuture<Boolean> requestFuture = new CompletableFuture<>();
+    private final HAConnectionState expectState;
+    private final String remoteAddr;
+    private final boolean notifyWhenShutdown;
+
+    public HAConnectionStateNotificationRequest(HAConnectionState expectState, String remoteAddr, boolean notifyWhenShutdown) {
+        this.expectState = expectState;
+        this.remoteAddr = remoteAddr;
+        this.notifyWhenShutdown = notifyWhenShutdown;
+    }
+
+    public CompletableFuture<Boolean> getRequestFuture() {
+        return requestFuture;
+    }
+
+    public String getRemoteAddr() {
+        return remoteAddr;
+    }
+
+    public boolean isNotifyWhenShutdown() {
+        return notifyWhenShutdown;
+    }
+
+    public HAConnectionState getExpectState() {
+        return expectState;
+    }
+}
diff --git a/store/src/main/java/org/apache/rocketmq/store/ha/HAConnectionStateNotificationService.java b/store/src/main/java/org/apache/rocketmq/store/ha/HAConnectionStateNotificationService.java
new file mode 100644
index 0000000..d63d8d6
--- /dev/null
+++ b/store/src/main/java/org/apache/rocketmq/store/ha/HAConnectionStateNotificationService.java
@@ -0,0 +1,150 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.rocketmq.store.ha;
+
+import java.net.InetSocketAddress;
+import org.apache.rocketmq.common.ServiceThread;
+import org.apache.rocketmq.common.constant.LoggerName;
+import org.apache.rocketmq.logging.InternalLogger;
+import org.apache.rocketmq.logging.InternalLoggerFactory;
+import org.apache.rocketmq.store.DefaultMessageStore;
+import org.apache.rocketmq.store.config.BrokerRole;
+
+/**
+ * Service to periodically check and notify for certain connection state.
+ */
+public class HAConnectionStateNotificationService extends ServiceThread {
+
+    private static final InternalLogger LOGGER = InternalLoggerFactory.getLogger(LoggerName.STORE_LOGGER_NAME);
+
+    private static final long CONNECTION_ESTABLISH_TIMEOUT = 10 * 1000;
+
+    private volatile HAConnectionStateNotificationRequest request;
+    private volatile long lastCheckTimeStamp = -1;
+    private HAService haService;
+    private DefaultMessageStore defaultMessageStore;
+
+    public HAConnectionStateNotificationService(HAService haService, DefaultMessageStore defaultMessageStore) {
+        this.haService = haService;
+        this.defaultMessageStore = defaultMessageStore;
+    }
+
+    @Override
+    public String getServiceName() {
+        if (defaultMessageStore != null && defaultMessageStore.getBrokerConfig().isInBrokerContainer()) {
+            return defaultMessageStore.getBrokerConfig().getLoggerIdentifier() + HAConnectionStateNotificationService.class.getSimpleName();
+        }
+        return HAConnectionStateNotificationService.class.getSimpleName();
+    }
+
+    public synchronized void setRequest(HAConnectionStateNotificationRequest request) {
+        if (this.request != null) {
+            this.request.getRequestFuture().cancel(true);
+        }
+        this.request = request;
+        lastCheckTimeStamp = System.currentTimeMillis();
+    }
+
+    private synchronized void doWaitConnectionState() {
+        if (this.request == null || this.request.getRequestFuture().isDone()) {
+            return;
+        }
+
+        if (this.defaultMessageStore.getMessageStoreConfig().getBrokerRole() == BrokerRole.SLAVE) {
+            if (haService.getHAClient().getCurrentState() == this.request.getExpectState()) {
+                this.request.getRequestFuture().complete(true);
+                this.request = null;
+            } else if (haService.getHAClient().getCurrentState() == HAConnectionState.READY) {
+                if ((System.currentTimeMillis() - lastCheckTimeStamp) > CONNECTION_ESTABLISH_TIMEOUT) {
+                    LOGGER.error("Wait HA connection establish with {} timeout", this.request.getRemoteAddr());
+                    this.request.getRequestFuture().complete(false);
+                    this.request = null;
+                }
+            } else {
+                lastCheckTimeStamp = System.currentTimeMillis();
+            }
+        } else {
+            boolean connectionFound = false;
+            for (HAConnection connection : haService.getConnectionList()) {
+                if (checkConnectionStateAndNotify(connection)) {
+                    connectionFound = true;
+                }
+            }
+
+            if (connectionFound) {
+                lastCheckTimeStamp = System.currentTimeMillis();
+            }
+
+            if (!connectionFound && (System.currentTimeMillis() - lastCheckTimeStamp) > CONNECTION_ESTABLISH_TIMEOUT) {
+                LOGGER.error("Wait HA connection establish with {} timeout", this.request.getRemoteAddr());
+                this.request.getRequestFuture().complete(false);
+                this.request = null;
+            }
+        }
+    }
+
+    /**
+     * Check if connection matched and notify request.
+     *
+     * @param connection connection to check.
+     * @return if connection remote address match request.
+     */
+    public synchronized boolean checkConnectionStateAndNotify(HAConnection connection) {
+        if (this.request == null || connection == null) {
+            return false;
+        }
+
+        String remoteAddress;
+        try {
+            remoteAddress = ((InetSocketAddress) connection.getSocketChannel().getRemoteAddress())
+                .getAddress().getHostAddress();
+            if (remoteAddress.equals(request.getRemoteAddr())) {
+                HAConnectionState connState = connection.getCurrentState();
+
+                if (connState == this.request.getExpectState()) {
+                    this.request.getRequestFuture().complete(true);
+                    this.request = null;
+                } else if (this.request.isNotifyWhenShutdown() && connState == HAConnectionState.SHUTDOWN) {
+                    this.request.getRequestFuture().complete(false);
+                    this.request = null;
+                }
+                return true;
+            }
+        } catch (Exception e) {
+            LOGGER.error("Check connection address exception: {}", e);
+        }
+
+        return false;
+    }
+
+    @Override
+    public void run() {
+        LOGGER.info(this.getServiceName() + " service started");
+
+        while (!this.isStopped()) {
+            try {
+                this.waitForRunning(1000);
+                this.doWaitConnectionState();
+            } catch (Exception e) {
+                LOGGER.warn(this.getServiceName() + " service has exception. ", e);
+            }
+        }
+
+        LOGGER.info(this.getServiceName() + " service end");
+    }
+}
diff --git a/store/src/main/java/org/apache/rocketmq/store/ha/HAService.java b/store/src/main/java/org/apache/rocketmq/store/ha/HAService.java
index 7e1dda3..5f714b9 100644
--- a/store/src/main/java/org/apache/rocketmq/store/ha/HAService.java
+++ b/store/src/main/java/org/apache/rocketmq/store/ha/HAService.java
@@ -14,613 +14,115 @@
  * See the License for the specific language governing permissions and
  * limitations under the License.
  */
+
 package org.apache.rocketmq.store.ha;
 
 import java.io.IOException;
-import java.net.InetSocketAddress;
-import java.net.SocketAddress;
-import java.nio.ByteBuffer;
-import java.nio.channels.ClosedChannelException;
-import java.nio.channels.SelectionKey;
-import java.nio.channels.Selector;
-import java.nio.channels.ServerSocketChannel;
-import java.nio.channels.SocketChannel;
-import java.util.LinkedList;
 import java.util.List;
-import java.util.Set;
 import java.util.concurrent.atomic.AtomicInteger;
 import java.util.concurrent.atomic.AtomicLong;
-import java.util.concurrent.atomic.AtomicReference;
-import org.apache.rocketmq.common.ServiceThread;
-import org.apache.rocketmq.common.constant.LoggerName;
-import org.apache.rocketmq.logging.InternalLogger;
-import org.apache.rocketmq.logging.InternalLoggerFactory;
-import org.apache.rocketmq.remoting.common.RemotingUtil;
+import org.apache.rocketmq.common.protocol.body.HARuntimeInfo;
 import org.apache.rocketmq.store.CommitLog;
-import org.apache.rocketmq.store.MessageStore;
-import org.apache.rocketmq.store.PutMessageSpinLock;
-import org.apache.rocketmq.store.PutMessageStatus;
-
-public class HAService {
-    private static final InternalLogger log = InternalLoggerFactory.getLogger(LoggerName.STORE_LOGGER_NAME);
-
-    private final AtomicInteger connectionCount = new AtomicInteger(0);
-
-    private final List<HAConnection> connectionList = new LinkedList<>();
-
-    private final AcceptSocketService acceptSocketService;
-
-    private final MessageStore defaultMessageStore;
-
-    private final WaitNotifyObject waitNotifyObject = new WaitNotifyObject();
-    private final AtomicLong push2SlaveMaxOffset = new AtomicLong(0);
-
-    private final GroupTransferService groupTransferService;
-
-    private final HAClient haClient;
-
-    public HAService(final MessageStore defaultMessageStore) throws IOException {
-        this.defaultMessageStore = defaultMessageStore;
-        this.acceptSocketService =
-            new AcceptSocketService(defaultMessageStore.getMessageStoreConfig().getHaListenPort());
-        this.groupTransferService = new GroupTransferService();
-        this.haClient = new HAClient();
-    }
-
-    public void updateMasterAddress(final String newAddr) {
-        if (this.haClient != null) {
-            this.haClient.updateMasterAddress(newAddr);
-        }
-    }
-
-    public void putRequest(final CommitLog.GroupCommitRequest request) {
-        this.groupTransferService.putRequest(request);
-    }
-
-    public boolean isSlaveOK(final long masterPutWhere) {
-        boolean result = this.connectionCount.get() > 0;
-        result =
-            result
-                && ((masterPutWhere - this.push2SlaveMaxOffset.get()) < this.defaultMessageStore
-                .getMessageStoreConfig().getHaSlaveFallbehindMax());
-        return result;
-    }
-
-    public void notifyTransferSome(final long offset) {
-        for (long value = this.push2SlaveMaxOffset.get(); offset > value; ) {
-            boolean ok = this.push2SlaveMaxOffset.compareAndSet(value, offset);
-            if (ok) {
-                this.groupTransferService.notifyTransferSome();
-                break;
-            } else {
-                value = this.push2SlaveMaxOffset.get();
-            }
-        }
-    }
-
-    public AtomicInteger getConnectionCount() {
-        return connectionCount;
-    }
+import org.apache.rocketmq.store.DefaultMessageStore;
+import org.apache.rocketmq.store.config.MessageStoreConfig;
 
-    // public void notifyTransferSome() {
-    // this.groupTransferService.notifyTransferSome();
-    // }
-
-    public void start() throws Exception {
-        this.acceptSocketService.beginAccept();
-        this.acceptSocketService.start();
-        this.groupTransferService.start();
-        this.haClient.start();
-    }
-
-    public void addConnection(final HAConnection conn) {
-        synchronized (this.connectionList) {
-            this.connectionList.add(conn);
-        }
-    }
-
-    public void removeConnection(final HAConnection conn) {
-        synchronized (this.connectionList) {
-            this.connectionList.remove(conn);
-        }
-    }
-
-    public void shutdown() {
-        this.haClient.shutdown();
-        this.acceptSocketService.shutdown(true);
-        this.destroyConnections();
-        this.groupTransferService.shutdown();
-    }
-
-    public void destroyConnections() {
-        synchronized (this.connectionList) {
-            for (HAConnection c : this.connectionList) {
-                c.shutdown();
-            }
-
-            this.connectionList.clear();
-        }
-    }
-
-    public MessageStore getDefaultMessageStore() {
-        return defaultMessageStore;
-    }
-
-    public WaitNotifyObject getWaitNotifyObject() {
-        return waitNotifyObject;
-    }
-
-    public AtomicLong getPush2SlaveMaxOffset() {
-        return push2SlaveMaxOffset;
-    }
+public interface HAService {
 
     /**
-     * Listens to slave connections to create {@link HAConnection}.
+     * Init HAService, must be called before other methods.
+     *
+     * @param defaultMessageStore
+     * @throws IOException
      */
-    class AcceptSocketService extends ServiceThread {
-        private final SocketAddress socketAddressListen;
-        private ServerSocketChannel serverSocketChannel;
-        private Selector selector;
-
-        public AcceptSocketService(final int port) {
-            this.socketAddressListen = new InetSocketAddress(port);
-        }
-
-        /**
-         * Starts listening to slave connections.
-         *
-         * @throws Exception If fails.
-         */
-        public void beginAccept() throws Exception {
-            this.serverSocketChannel = ServerSocketChannel.open();
-            this.selector = RemotingUtil.openSelector();
-            this.serverSocketChannel.socket().setReuseAddress(true);
-            this.serverSocketChannel.socket().bind(this.socketAddressListen);
-            this.serverSocketChannel.configureBlocking(false);
-            this.serverSocketChannel.register(this.selector, SelectionKey.OP_ACCEPT);
-        }
-
-        /**
-         * {@inheritDoc}
-         */
-        @Override
-        public void shutdown(final boolean interrupt) {
-            super.shutdown(interrupt);
-            try {
-                this.serverSocketChannel.close();
-                this.selector.close();
-            } catch (IOException e) {
-                log.error("AcceptSocketService shutdown exception", e);
-            }
-        }
-
-        /**
-         * {@inheritDoc}
-         */
-        @Override
-        public void run() {
-            log.info(this.getServiceName() + " service started");
-
-            while (!this.isStopped()) {
-                try {
-                    this.selector.select(1000);
-                    Set<SelectionKey> selected = this.selector.selectedKeys();
-
-                    if (selected != null) {
-                        for (SelectionKey k : selected) {
-                            if ((k.readyOps() & SelectionKey.OP_ACCEPT) != 0) {
-                                SocketChannel sc = ((ServerSocketChannel) k.channel()).accept();
-
-                                if (sc != null) {
-                                    HAService.log.info("HAService receive new connection, "
-                                        + sc.socket().getRemoteSocketAddress());
-
-                                    try {
-                                        HAConnection conn = new HAConnection(HAService.this, sc);
-                                        conn.start();
-                                        HAService.this.addConnection(conn);
-                                    } catch (Exception e) {
-                                        log.error("new HAConnection exception", e);
-                                        sc.close();
-                                    }
-                                }
-                            } else {
-                                log.warn("Unexpected ops in select " + k.readyOps());
-                            }
-                        }
-
-                        selected.clear();
-                    }
-                } catch (Exception e) {
-                    log.error(this.getServiceName() + " service has exception.", e);
-                }
-            }
-
-            log.info(this.getServiceName() + " service end");
-        }
-
-        /**
-         * {@inheritDoc}
-         */
-        @Override
-        public String getServiceName() {
-            return AcceptSocketService.class.getSimpleName();
-        }
-    }
+    void init(DefaultMessageStore defaultMessageStore) throws IOException;
 
     /**
-     * GroupTransferService Service
+     * Start HA Service
+     *
+     * @throws Exception
      */
-    class GroupTransferService extends ServiceThread {
-
-        private final WaitNotifyObject notifyTransferObject = new WaitNotifyObject();
-        private final PutMessageSpinLock lock = new PutMessageSpinLock();
-        private volatile LinkedList<CommitLog.GroupCommitRequest> requestsWrite = new LinkedList<>();
-        private volatile LinkedList<CommitLog.GroupCommitRequest> requestsRead = new LinkedList<>();
-
-        public void putRequest(final CommitLog.GroupCommitRequest request) {
-            lock.lock();
-            try {
-                this.requestsWrite.add(request);
-            } finally {
-                lock.unlock();
-            }
-            this.wakeup();
-        }
-
-        public void notifyTransferSome() {
-            this.notifyTransferObject.wakeup();
-        }
-
-        private void swapRequests() {
-            lock.lock();
-            try {
-                LinkedList<CommitLog.GroupCommitRequest> tmp = this.requestsWrite;
-                this.requestsWrite = this.requestsRead;
-                this.requestsRead = tmp;
-            } finally {
-                lock.unlock();
-            }
-        }
-
-        private void doWaitTransfer() {
-            if (!this.requestsRead.isEmpty()) {
-                for (CommitLog.GroupCommitRequest req : this.requestsRead) {
-                    boolean transferOK = HAService.this.push2SlaveMaxOffset.get() >= req.getNextOffset();
-                    long deadLine = req.getDeadLine();
-                    while (!transferOK && deadLine - System.nanoTime() > 0) {
-                        this.notifyTransferObject.waitForRunning(1000);
-                        transferOK = HAService.this.push2SlaveMaxOffset.get() >= req.getNextOffset();
-                    }
-
-                    req.wakeupCustomer(transferOK ? PutMessageStatus.PUT_OK : PutMessageStatus.FLUSH_SLAVE_TIMEOUT);
-                }
-
-                this.requestsRead = new LinkedList<>();
-            }
-        }
-
-        public void run() {
-            log.info(this.getServiceName() + " service started");
-
-            while (!this.isStopped()) {
-                try {
-                    this.waitForRunning(10);
-                    this.doWaitTransfer();
-                } catch (Exception e) {
-                    log.warn(this.getServiceName() + " service has exception. ", e);
-                }
-            }
-
-            log.info(this.getServiceName() + " service end");
-        }
-
-        @Override
-        protected void onWaitEnd() {
-            this.swapRequests();
-        }
-
-        @Override
-        public String getServiceName() {
-            return GroupTransferService.class.getSimpleName();
-        }
-    }
-
-    class HAClient extends ServiceThread {
-        private static final int READ_MAX_BUFFER_SIZE = 1024 * 1024 * 4;
-        private final AtomicReference<String> masterAddress = new AtomicReference<>();
-        private final ByteBuffer reportOffset = ByteBuffer.allocate(8);
-        private SocketChannel socketChannel;
-        private Selector selector;
-        private long lastWriteTimestamp = System.currentTimeMillis();
-
-        private long currentReportedOffset = 0;
-        private int dispatchPosition = 0;
-        private ByteBuffer byteBufferRead = ByteBuffer.allocate(READ_MAX_BUFFER_SIZE);
-        private ByteBuffer byteBufferBackup = ByteBuffer.allocate(READ_MAX_BUFFER_SIZE);
-
-        public HAClient() throws IOException {
-            this.selector = RemotingUtil.openSelector();
-        }
-
-        public void updateMasterAddress(final String newAddr) {
-            String currentAddr = this.masterAddress.get();
-            if (currentAddr == null || !currentAddr.equals(newAddr)) {
-                this.masterAddress.set(newAddr);
-                log.info("update master address, OLD: " + currentAddr + " NEW: " + newAddr);
-            }
-        }
-
-        private boolean isTimeToReportOffset() {
-            long interval =
-                HAService.this.defaultMessageStore.getSystemClock().now() - this.lastWriteTimestamp;
-            boolean needHeart = interval > HAService.this.defaultMessageStore.getMessageStoreConfig()
-                .getHaSendHeartbeatInterval();
-
-            return needHeart;
-        }
-
-        private boolean reportSlaveMaxOffset(final long maxOffset) {
-            this.reportOffset.position(0);
-            this.reportOffset.limit(8);
-            this.reportOffset.putLong(maxOffset);
-            this.reportOffset.position(0);
-            this.reportOffset.limit(8);
-
-            for (int i = 0; i < 3 && this.reportOffset.hasRemaining(); i++) {
-                try {
-                    this.socketChannel.write(this.reportOffset);
-                } catch (IOException e) {
-                    log.error(this.getServiceName()
-                        + "reportSlaveMaxOffset this.socketChannel.write exception", e);
-                    return false;
-                }
-            }
-
-            lastWriteTimestamp = HAService.this.defaultMessageStore.getSystemClock().now();
-            return !this.reportOffset.hasRemaining();
-        }
-
-        private void reallocateByteBuffer() {
-            int remain = READ_MAX_BUFFER_SIZE - this.dispatchPosition;
-            if (remain > 0) {
-                this.byteBufferRead.position(this.dispatchPosition);
+    void start() throws Exception;
 
-                this.byteBufferBackup.position(0);
-                this.byteBufferBackup.limit(READ_MAX_BUFFER_SIZE);
-                this.byteBufferBackup.put(this.byteBufferRead);
-            }
-
-            this.swapByteBuffer();
-
-            this.byteBufferRead.position(remain);
-            this.byteBufferRead.limit(READ_MAX_BUFFER_SIZE);
-            this.dispatchPosition = 0;
-        }
-
-        private void swapByteBuffer() {
-            ByteBuffer tmp = this.byteBufferRead;
-            this.byteBufferRead = this.byteBufferBackup;
-            this.byteBufferBackup = tmp;
-        }
-
-        private boolean processReadEvent() {
-            int readSizeZeroTimes = 0;
-            while (this.byteBufferRead.hasRemaining()) {
-                try {
-                    int readSize = this.socketChannel.read(this.byteBufferRead);
-                    if (readSize > 0) {
-                        readSizeZeroTimes = 0;
-                        boolean result = this.dispatchReadRequest();
-                        if (!result) {
-                            log.error("HAClient, dispatchReadRequest error");
-                            return false;
-                        }
-                    } else if (readSize == 0) {
-                        if (++readSizeZeroTimes >= 3) {
-                            break;
-                        }
-                    } else {
-                        log.info("HAClient, processReadEvent read socket < 0");
-                        return false;
-                    }
-                } catch (IOException e) {
-                    log.info("HAClient, processReadEvent read socket exception", e);
-                    return false;
-                }
-            }
-
-            return true;
-        }
-
-        private boolean dispatchReadRequest() {
-            final int msgHeaderSize = 8 + 4; // phyoffset + size
-
-            while (true) {
-                int diff = this.byteBufferRead.position() - this.dispatchPosition;
-                if (diff >= msgHeaderSize) {
-                    long masterPhyOffset = this.byteBufferRead.getLong(this.dispatchPosition);
-                    int bodySize = this.byteBufferRead.getInt(this.dispatchPosition + 8);
-
-                    long slavePhyOffset = HAService.this.defaultMessageStore.getMaxPhyOffset();
-
-                    if (slavePhyOffset != 0) {
-                        if (slavePhyOffset != masterPhyOffset) {
-                            log.error("master pushed offset not equal the max phy offset in slave, SLAVE: "
-                                + slavePhyOffset + " MASTER: " + masterPhyOffset);
-                            return false;
-                        }
-                    }
-
-                    if (diff >= (msgHeaderSize + bodySize)) {
-                        byte[] bodyData = byteBufferRead.array();
-                        int dataStart = this.dispatchPosition + msgHeaderSize;
-
-                        HAService.this.defaultMessageStore.appendToCommitLog(
-                                masterPhyOffset, bodyData, dataStart, bodySize);
-
-                        this.dispatchPosition += msgHeaderSize + bodySize;
-
-                        if (!reportSlaveMaxOffsetPlus()) {
-                            return false;
-                        }
-
-                        continue;
-                    }
-                }
-
-                if (!this.byteBufferRead.hasRemaining()) {
-                    this.reallocateByteBuffer();
-                }
-
-                break;
-            }
-
-            return true;
-        }
-
-        private boolean reportSlaveMaxOffsetPlus() {
-            boolean result = true;
-            long currentPhyOffset = HAService.this.defaultMessageStore.getMaxPhyOffset();
-            if (currentPhyOffset > this.currentReportedOffset) {
-                this.currentReportedOffset = currentPhyOffset;
-                result = this.reportSlaveMaxOffset(this.currentReportedOffset);
-                if (!result) {
-                    this.closeMaster();
-                    log.error("HAClient, reportSlaveMaxOffset error, " + this.currentReportedOffset);
-                }
-            }
-
-            return result;
-        }
-
-        private boolean connectMaster() throws ClosedChannelException {
-            if (null == socketChannel) {
-                String addr = this.masterAddress.get();
-                if (addr != null) {
-
-                    SocketAddress socketAddress = RemotingUtil.string2SocketAddress(addr);
-                    if (socketAddress != null) {
-                        this.socketChannel = RemotingUtil.connect(socketAddress);
-                        if (this.socketChannel != null) {
-                            this.socketChannel.register(this.selector, SelectionKey.OP_READ);
-                        }
-                    }
-                }
-
-                this.currentReportedOffset = HAService.this.defaultMessageStore.getMaxPhyOffset();
-
-                this.lastWriteTimestamp = System.currentTimeMillis();
-            }
-
-            return this.socketChannel != null;
-        }
-
-        private void closeMaster() {
-            if (null != this.socketChannel) {
-                try {
-
-                    SelectionKey sk = this.socketChannel.keyFor(this.selector);
-                    if (sk != null) {
-                        sk.cancel();
-                    }
-
-                    this.socketChannel.close();
-
-                    this.socketChannel = null;
-                } catch (IOException e) {
-                    log.warn("closeMaster exception. ", e);
-                }
-
-                this.lastWriteTimestamp = 0;
-                this.dispatchPosition = 0;
-
-                this.byteBufferBackup.position(0);
-                this.byteBufferBackup.limit(READ_MAX_BUFFER_SIZE);
+    /**
+     * Shutdown HA Service
+     */
+    void shutdown();
 
-                this.byteBufferRead.position(0);
-                this.byteBufferRead.limit(READ_MAX_BUFFER_SIZE);
-            }
-        }
+    /**
+     * Update master address
+     *
+     * @param newAddr
+     */
+    void updateMasterAddress(String newAddr);
 
-        @Override
-        public void run() {
-            log.info(this.getServiceName() + " service started");
+    /**
+     * Update ha master address
+     *
+     * @param newAddr
+     */
+    void updateHaMasterAddress(String newAddr);
 
-            while (!this.isStopped()) {
-                try {
-                    if (this.connectMaster()) {
+    /**
+     * Returns the number of slaves those commit log are not far behind the master.
+     *
+     * @return the number of slaves
+     * @see MessageStoreConfig#getHaMaxGapNotInSync()
+     */
+    int inSyncSlaveNums(long masterPutWhere);
 
-                        if (this.isTimeToReportOffset()) {
-                            boolean result = this.reportSlaveMaxOffset(this.currentReportedOffset);
-                            if (!result) {
-                                this.closeMaster();
-                            }
-                        }
+    /**
+     * Get connection count
+     *
+     * @return the number of connection
+     */
+    AtomicInteger getConnectionCount();
 
-                        this.selector.select(1000);
+    /**
+     * Put request to handle HA
+     *
+     * @param request
+     */
+    void putRequest(final CommitLog.GroupCommitRequest request);
 
-                        boolean ok = this.processReadEvent();
-                        if (!ok) {
-                            this.closeMaster();
-                        }
+    /**
+     * Put GroupConnectionStateRequest for preOnline
+     *
+     * @param request
+     */
+    void putGroupConnectionStateRequest(HAConnectionStateNotificationRequest request);
 
-                        if (!reportSlaveMaxOffsetPlus()) {
-                            continue;
-                        }
+    /**
+     * Get ha connection list
+     *
+     * @return List<HAConnection>
+     */
+    List<HAConnection> getConnectionList();
 
-                        long interval =
-                            HAService.this.getDefaultMessageStore().getSystemClock().now()
-                                - this.lastWriteTimestamp;
-                        if (interval > HAService.this.getDefaultMessageStore().getMessageStoreConfig()
-                            .getHaHousekeepingInterval()) {
-                            log.warn("HAClient, housekeeping, found this connection[" + this.masterAddress
-                                + "] expired, " + interval);
-                            this.closeMaster();
-                            log.warn("HAClient, master not response some time, so close connection");
-                        }
-                    } else {
-                        this.waitForRunning(1000 * 5);
-                    }
-                } catch (Exception e) {
-                    log.warn(this.getServiceName() + " service has exception. ", e);
-                    this.waitForRunning(1000 * 5);
-                }
-            }
+    /**
+     * Get HAClient
+     *
+     * @return HAClient
+     */
+    HAClient getHAClient();
 
-            log.info(this.getServiceName() + " service end");
-        }
+    /**
+     * Get the max offset in all slaves
+     */
+    AtomicLong getPush2SlaveMaxOffset();
 
-        @Override
-        public void shutdown() {
-            super.shutdown();
-            closeMaster();
-        }
+    /**
+     * Get HA runtime info
+     */
+    HARuntimeInfo getRuntimeInfo(final long masterPutWhere);
 
-        // private void disableWriteFlag() {
-        // if (this.socketChannel != null) {
-        // SelectionKey sk = this.socketChannel.keyFor(this.selector);
-        // if (sk != null) {
-        // int ops = sk.interestOps();
-        // ops &= ~SelectionKey.OP_WRITE;
-        // sk.interestOps(ops);
-        // }
-        // }
-        // }
-        // private void enableWriteFlag() {
-        // if (this.socketChannel != null) {
-        // SelectionKey sk = this.socketChannel.keyFor(this.selector);
-        // if (sk != null) {
-        // int ops = sk.interestOps();
-        // ops |= SelectionKey.OP_WRITE;
-        // sk.interestOps(ops);
-        // }
-        // }
-        // }
+    /**
+     * Get WaitNotifyObject
+     */
+    WaitNotifyObject getWaitNotifyObject();
 
-        @Override
-        public String getServiceName() {
-            return HAClient.class.getSimpleName();
-        }
-    }
+    /**
+     * Judge whether the slave keeps up according to the masterPutWhere,
+     * If the offset gap exceeds haSlaveFallBehindMax, then slave is not OK
+     */
+    boolean isSlaveOK(long masterPutWhere);
 }
diff --git a/store/src/main/java/org/apache/rocketmq/store/ha/WaitNotifyObject.java b/store/src/main/java/org/apache/rocketmq/store/ha/WaitNotifyObject.java
index d5ed65f..eb6806b 100644
--- a/store/src/main/java/org/apache/rocketmq/store/ha/WaitNotifyObject.java
+++ b/store/src/main/java/org/apache/rocketmq/store/ha/WaitNotifyObject.java
@@ -20,43 +20,39 @@ import org.apache.rocketmq.common.constant.LoggerName;
 import org.apache.rocketmq.logging.InternalLogger;
 import org.apache.rocketmq.logging.InternalLoggerFactory;
 
-import java.util.Map;
-import java.util.concurrent.ConcurrentHashMap;
-import java.util.concurrent.atomic.AtomicBoolean;
+import java.util.HashMap;
 
 public class WaitNotifyObject {
     private static final InternalLogger log = InternalLoggerFactory.getLogger(LoggerName.STORE_LOGGER_NAME);
 
-    protected final ConcurrentHashMap<Long/* thread id */, AtomicBoolean/* notified */> waitingThreadTable =
-        new ConcurrentHashMap<Long, AtomicBoolean>(16);
+    protected final HashMap<Long/* thread id */, Boolean/* notified */> waitingThreadTable =
+        new HashMap<Long, Boolean>(16);
 
-    protected AtomicBoolean hasNotified = new AtomicBoolean(false);
+    protected volatile boolean hasNotified = false;
 
     public void wakeup() {
-        boolean needNotify = hasNotified.compareAndSet(false, true);
-        if (needNotify) {
-            synchronized (this) {
+        synchronized (this) {
+            if (!this.hasNotified) {
+                this.hasNotified = true;
                 this.notify();
             }
         }
     }
 
-    protected void waitForRunning(long interval) {
-        if (this.hasNotified.compareAndSet(true, false)) {
-            this.onWaitEnd();
-            return;
-        }
+    public void waitForRunning(long interval) {
         synchronized (this) {
+            if (this.hasNotified) {
+                this.hasNotified = false;
+                this.onWaitEnd();
+                return;
+            }
+
             try {
-                if (this.hasNotified.compareAndSet(true, false)) {
-                    this.onWaitEnd();
-                    return;
-                }
                 this.wait(interval);
             } catch (InterruptedException e) {
                 log.error("Interrupted", e);
             } finally {
-                this.hasNotified.set(false);
+                this.hasNotified = false;
                 this.onWaitEnd();
             }
         }
@@ -66,14 +62,15 @@ public class WaitNotifyObject {
     }
 
     public void wakeupAll() {
-        boolean needNotify = false;
-        for (Map.Entry<Long,AtomicBoolean> entry : this.waitingThreadTable.entrySet()) {
-            if (entry.getValue().compareAndSet(false, true)) {
-                needNotify = true;
+        synchronized (this) {
+            boolean needNotify = false;
+
+            for (Boolean value : this.waitingThreadTable.values()) {
+                needNotify = needNotify || !value;
+                value = true;
             }
-        }
-        if (needNotify) {
-            synchronized (this) {
+
+            if (needNotify) {
                 this.notifyAll();
             }
         }
@@ -81,22 +78,20 @@ public class WaitNotifyObject {
 
     public void allWaitForRunning(long interval) {
         long currentThreadId = Thread.currentThread().getId();
-        AtomicBoolean notified = this.waitingThreadTable.computeIfAbsent(currentThreadId, k -> new AtomicBoolean(false));
-        if (notified.compareAndSet(true, false)) {
-            this.onWaitEnd();
-            return;
-        }
         synchronized (this) {
+            Boolean notified = this.waitingThreadTable.get(currentThreadId);
+            if (notified != null && notified) {
+                this.waitingThreadTable.put(currentThreadId, false);
+                this.onWaitEnd();
+                return;
+            }
+
             try {
-                if (notified.compareAndSet(true, false)) {
-                    this.onWaitEnd();
-                    return;
-                }
                 this.wait(interval);
             } catch (InterruptedException e) {
                 log.error("Interrupted", e);
             } finally {
-                notified.set(false);
+                this.waitingThreadTable.put(currentThreadId, false);
                 this.onWaitEnd();
             }
         }
diff --git a/store/src/main/java/org/apache/rocketmq/store/PutMessageStatus.java b/store/src/main/java/org/apache/rocketmq/store/hook/PutMessageHook.java
similarity index 61%
copy from store/src/main/java/org/apache/rocketmq/store/PutMessageStatus.java
copy to store/src/main/java/org/apache/rocketmq/store/hook/PutMessageHook.java
index 29d0d95..dc47d32 100644
--- a/store/src/main/java/org/apache/rocketmq/store/PutMessageStatus.java
+++ b/store/src/main/java/org/apache/rocketmq/store/hook/PutMessageHook.java
@@ -14,18 +14,24 @@
  * See the License for the specific language governing permissions and
  * limitations under the License.
  */
-package org.apache.rocketmq.store;
+package org.apache.rocketmq.store.hook;
 
-public enum PutMessageStatus {
-    PUT_OK,
-    FLUSH_DISK_TIMEOUT,
-    FLUSH_SLAVE_TIMEOUT,
-    SLAVE_NOT_AVAILABLE,
-    SERVICE_NOT_AVAILABLE,
-    CREATE_MAPEDFILE_FAILED,
-    MESSAGE_ILLEGAL,
-    PROPERTIES_SIZE_EXCEEDED,
-    OS_PAGECACHE_BUSY,
-    UNKNOWN_ERROR,
-    LMQ_CONSUME_QUEUE_NUM_EXCEEDED,
+import org.apache.rocketmq.common.message.MessageExt;
+import org.apache.rocketmq.store.PutMessageResult;
+
+public interface PutMessageHook {
+
+    /**
+     * Name of the hook.
+     *
+     * @return name of the hook
+     */
+    String hookName();
+
+    /**
+     *  Execute before put message. For example, Message verification or special message transform
+     * @param msg
+     * @return
+     */
+    PutMessageResult executeBeforePutMessage(MessageExt msg);
 }
diff --git a/store/src/main/java/org/apache/rocketmq/store/PutMessageStatus.java b/store/src/main/java/org/apache/rocketmq/store/hook/SendMessageBackHook.java
similarity index 64%
copy from store/src/main/java/org/apache/rocketmq/store/PutMessageStatus.java
copy to store/src/main/java/org/apache/rocketmq/store/hook/SendMessageBackHook.java
index 29d0d95..0225450 100644
--- a/store/src/main/java/org/apache/rocketmq/store/PutMessageStatus.java
+++ b/store/src/main/java/org/apache/rocketmq/store/hook/SendMessageBackHook.java
@@ -14,18 +14,20 @@
  * See the License for the specific language governing permissions and
  * limitations under the License.
  */
-package org.apache.rocketmq.store;
+package org.apache.rocketmq.store.hook;
 
-public enum PutMessageStatus {
-    PUT_OK,
-    FLUSH_DISK_TIMEOUT,
-    FLUSH_SLAVE_TIMEOUT,
-    SLAVE_NOT_AVAILABLE,
-    SERVICE_NOT_AVAILABLE,
-    CREATE_MAPEDFILE_FAILED,
-    MESSAGE_ILLEGAL,
-    PROPERTIES_SIZE_EXCEEDED,
-    OS_PAGECACHE_BUSY,
-    UNKNOWN_ERROR,
-    LMQ_CONSUME_QUEUE_NUM_EXCEEDED,
+import java.util.List;
+import org.apache.rocketmq.common.message.MessageExt;
+
+public interface SendMessageBackHook {
+
+    /**
+     * Slave send message back to master at certain offset when HA handshake
+     *
+     * @param msgList
+     * @param brokerName
+     * @param brokerAddr
+     * @return
+     */
+    boolean executeSendMessageBack(List<MessageExt> msgList, String brokerName, String brokerAddr);
 }
diff --git a/store/src/main/java/org/apache/rocketmq/store/index/IndexFile.java b/store/src/main/java/org/apache/rocketmq/store/index/IndexFile.java
index e920c84..8b59f54 100644
--- a/store/src/main/java/org/apache/rocketmq/store/index/IndexFile.java
+++ b/store/src/main/java/org/apache/rocketmq/store/index/IndexFile.java
@@ -22,6 +22,7 @@ import java.nio.MappedByteBuffer;
 import java.nio.channels.FileChannel;
 import java.nio.channels.FileLock;
 import java.util.List;
+import org.apache.rocketmq.common.UtilAll;
 import org.apache.rocketmq.common.constant.LoggerName;
 import org.apache.rocketmq.logging.InternalLogger;
 import org.apache.rocketmq.logging.InternalLoggerFactory;
@@ -35,6 +36,7 @@ public class IndexFile {
     private static int invalidIndex = 0;
     private final int hashSlotNum;
     private final int indexNum;
+    private final int fileTotalSize;
     private final MappedFile mappedFile;
     private final FileChannel fileChannel;
     private final MappedByteBuffer mappedByteBuffer;
@@ -42,7 +44,7 @@ public class IndexFile {
 
     public IndexFile(final String fileName, final int hashSlotNum, final int indexNum,
         final long endPhyOffset, final long endTimestamp) throws IOException {
-        int fileTotalSize =
+        this.fileTotalSize =
             IndexHeader.INDEX_HEADER_SIZE + (hashSlotNum * hashSlotSize) + (indexNum * indexSize);
         this.mappedFile = new DefaultMappedFile(fileName, fileTotalSize);
         this.fileChannel = this.mappedFile.getFileChannel();
@@ -68,10 +70,25 @@ public class IndexFile {
         return this.mappedFile.getFileName();
     }
 
+    public int getFileSize() {
+        return this.fileTotalSize;
+    }
+
     public void load() {
         this.indexHeader.load();
     }
 
+    public void shutdown() {
+        this.flush();
+        UtilAll.cleanBuffer(this.mappedByteBuffer);
+
+        try {
+            this.fileChannel.close();
+        } catch (IOException e) {
+            log.error("Shutdown error in index file", e);
+        }
+    }
+
     public void flush() {
         long beginTime = System.currentTimeMillis();
         if (this.mappedFile.hold()) {
@@ -165,8 +182,9 @@ public class IndexFile {
     public int indexKeyHashMethod(final String key) {
         int keyHash = key.hashCode();
         int keyHashPositive = Math.abs(keyHash);
-        if (keyHashPositive < 0)
+        if (keyHashPositive < 0) {
             keyHashPositive = 0;
+        }
         return keyHashPositive;
     }
 
diff --git a/store/src/main/java/org/apache/rocketmq/store/index/IndexService.java b/store/src/main/java/org/apache/rocketmq/store/index/IndexService.java
index f2c5616..12a129a 100644
--- a/store/src/main/java/org/apache/rocketmq/store/index/IndexService.java
+++ b/store/src/main/java/org/apache/rocketmq/store/index/IndexService.java
@@ -23,35 +23,36 @@ import java.util.Arrays;
 import java.util.List;
 import java.util.concurrent.locks.ReadWriteLock;
 import java.util.concurrent.locks.ReentrantReadWriteLock;
+import org.apache.rocketmq.common.AbstractBrokerRunnable;
 import org.apache.rocketmq.common.UtilAll;
 import org.apache.rocketmq.common.constant.LoggerName;
 import org.apache.rocketmq.logging.InternalLogger;
 import org.apache.rocketmq.logging.InternalLoggerFactory;
 import org.apache.rocketmq.common.message.MessageConst;
 import org.apache.rocketmq.common.sysflag.MessageSysFlag;
+import org.apache.rocketmq.store.DefaultMessageStore;
 import org.apache.rocketmq.store.DispatchRequest;
-import org.apache.rocketmq.store.MessageStore;
 import org.apache.rocketmq.store.config.StorePathConfigHelper;
 
 public class IndexService {
-    private static final InternalLogger log = InternalLoggerFactory.getLogger(LoggerName.STORE_LOGGER_NAME);
+    private static final InternalLogger LOGGER = InternalLoggerFactory.getLogger(LoggerName.STORE_LOGGER_NAME);
     /**
      * Maximum times to attempt index file creation.
      */
     private static final int MAX_TRY_IDX_CREATE = 3;
-    private final MessageStore defaultMessageStore;
+    private final DefaultMessageStore defaultMessageStore;
     private final int hashSlotNum;
     private final int indexNum;
     private final String storePath;
     private final ArrayList<IndexFile> indexFileList = new ArrayList<IndexFile>();
     private final ReadWriteLock readWriteLock = new ReentrantReadWriteLock();
 
-    public IndexService(final MessageStore store) {
+    public IndexService(final DefaultMessageStore store) {
         this.defaultMessageStore = store;
         this.hashSlotNum = store.getMessageStoreConfig().getMaxHashSlotNum();
         this.indexNum = store.getMessageStoreConfig().getMaxIndexNum();
         this.storePath =
-            StorePathConfigHelper.getStorePathIndex(store.getMessageStoreConfig().getStorePathRootDir());
+            StorePathConfigHelper.getStorePathIndex(defaultMessageStore.getMessageStoreConfig().getStorePathRootDir());
     }
 
     public boolean load(final boolean lastExitOK) {
@@ -73,13 +74,13 @@ public class IndexService {
                         }
                     }
 
-                    log.info("load index file OK, " + f.getFileName());
+                    LOGGER.info("load index file OK, " + f.getFileName());
                     this.indexFileList.add(f);
                 } catch (IOException e) {
-                    log.error("load file {} error", file, e);
+                    LOGGER.error("load file {} error", file, e);
                     return false;
                 } catch (NumberFormatException e) {
-                    log.error("load file {} error", file, e);
+                    LOGGER.error("load file {} error", file, e);
                 }
             }
         }
@@ -87,6 +88,14 @@ public class IndexService {
         return true;
     }
 
+    public long getTotalSize() {
+        if (indexFileList.isEmpty()) {
+            return 0;
+        }
+
+        return (long) indexFileList.get(0).getFileSize() * indexFileList.size();
+    }
+
     public void deleteExpiredFile(long offset) {
         Object[] files = null;
         try {
@@ -100,7 +109,7 @@ public class IndexService {
                 files = this.indexFileList.toArray();
             }
         } catch (Exception e) {
-            log.error("destroy exception", e);
+            LOGGER.error("destroy exception", e);
         } finally {
             this.readWriteLock.readLock().unlock();
         }
@@ -128,12 +137,12 @@ public class IndexService {
                     boolean destroyed = file.destroy(3000);
                     destroyed = destroyed && this.indexFileList.remove(file);
                     if (!destroyed) {
-                        log.error("deleteExpiredFile remove failed.");
+                        LOGGER.error("deleteExpiredFile remove failed.");
                         break;
                     }
                 }
             } catch (Exception e) {
-                log.error("deleteExpiredFile has exception.", e);
+                LOGGER.error("deleteExpiredFile has exception.", e);
             } finally {
                 this.readWriteLock.writeLock().unlock();
             }
@@ -148,7 +157,7 @@ public class IndexService {
             }
             this.indexFileList.clear();
         } catch (Exception e) {
-            log.error("destroy exception", e);
+            LOGGER.error("destroy exception", e);
         } finally {
             this.readWriteLock.writeLock().unlock();
         }
@@ -186,7 +195,7 @@ public class IndexService {
                 }
             }
         } catch (Exception e) {
-            log.error("queryMsg exception", e);
+            LOGGER.error("queryMsg exception", e);
         } finally {
             this.readWriteLock.readLock().unlock();
         }
@@ -222,7 +231,7 @@ public class IndexService {
             if (req.getUniqKey() != null) {
                 indexFile = putKey(indexFile, msg, buildKey(topic, req.getUniqKey()));
                 if (indexFile == null) {
-                    log.error("putKey error commitlog {} uniqkey {}", req.getCommitLogOffset(), req.getUniqKey());
+                    LOGGER.error("putKey error commitlog {} uniqkey {}", req.getCommitLogOffset(), req.getUniqKey());
                     return;
                 }
             }
@@ -234,20 +243,20 @@ public class IndexService {
                     if (key.length() > 0) {
                         indexFile = putKey(indexFile, msg, buildKey(topic, key));
                         if (indexFile == null) {
-                            log.error("putKey error commitlog {} uniqkey {}", req.getCommitLogOffset(), req.getUniqKey());
+                            LOGGER.error("putKey error commitlog {} uniqkey {}", req.getCommitLogOffset(), req.getUniqKey());
                             return;
                         }
                     }
                 }
             }
         } else {
-            log.error("build index error, stop building index");
+            LOGGER.error("build index error, stop building index");
         }
     }
 
     private IndexFile putKey(IndexFile indexFile, DispatchRequest msg, String idxKey) {
         for (boolean ok = indexFile.putKey(idxKey, msg.getCommitLogOffset(), msg.getStoreTimestamp()); !ok; ) {
-            log.warn("Index file [" + indexFile.getFileName() + "] is full, trying to create another one");
+            LOGGER.warn("Index file [" + indexFile.getFileName() + "] is full, trying to create another one");
 
             indexFile = retryGetAndCreateIndexFile();
             if (null == indexFile) {
@@ -270,20 +279,21 @@ public class IndexService {
 
         for (int times = 0; null == indexFile && times < MAX_TRY_IDX_CREATE; times++) {
             indexFile = this.getAndCreateLastIndexFile();
-            if (null != indexFile)
+            if (null != indexFile) {
                 break;
+            }
 
             try {
-                log.info("Tried to create index file " + times + " times");
+                LOGGER.info("Tried to create index file " + times + " times");
                 Thread.sleep(1000);
             } catch (InterruptedException e) {
-                log.error("Interrupted", e);
+                LOGGER.error("Interrupted", e);
             }
         }
 
         if (null == indexFile) {
             this.defaultMessageStore.getRunningFlags().makeIndexFileError();
-            log.error("Mark index file cannot build flag");
+            LOGGER.error("Mark index file cannot build flag");
         }
 
         return indexFile;
@@ -322,16 +332,16 @@ public class IndexService {
                 this.readWriteLock.writeLock().lock();
                 this.indexFileList.add(indexFile);
             } catch (Exception e) {
-                log.error("getLastIndexFile exception ", e);
+                LOGGER.error("getLastIndexFile exception ", e);
             } finally {
                 this.readWriteLock.writeLock().unlock();
             }
 
             if (indexFile != null) {
                 final IndexFile flushThisFile = prevIndexFile;
-                Thread flushThread = new Thread(new Runnable() {
-                    @Override
-                    public void run() {
+
+                Thread flushThread = new Thread(new AbstractBrokerRunnable(defaultMessageStore.getBrokerConfig()) {
+                    @Override public void run2() {
                         IndexService.this.flush(flushThisFile);
                     }
                 }, "FlushIndexFileThread");
@@ -345,8 +355,9 @@ public class IndexService {
     }
 
     public void flush(final IndexFile f) {
-        if (null == f)
+        if (null == f) {
             return;
+        }
 
         long indexMsgTimestamp = 0;
 
@@ -367,6 +378,20 @@ public class IndexService {
     }
 
     public void shutdown() {
-
+        try {
+            this.readWriteLock.writeLock().lock();
+            for (IndexFile f : this.indexFileList) {
+                try {
+                    f.shutdown();
+                } catch (Exception e) {
+                    LOGGER.error("shutdown " + f.getFileName() + " exception", e);
+                }
+            }
+            this.indexFileList.clear();
+        } catch (Exception e) {
+            LOGGER.error("shutdown exception", e);
+        } finally {
+            this.readWriteLock.writeLock().unlock();
+        }
     }
 }
diff --git a/store/src/main/java/org/apache/rocketmq/store/logfile/DefaultMappedFile.java b/store/src/main/java/org/apache/rocketmq/store/logfile/DefaultMappedFile.java
index 4d4830b..095bb26 100644
--- a/store/src/main/java/org/apache/rocketmq/store/logfile/DefaultMappedFile.java
+++ b/store/src/main/java/org/apache/rocketmq/store/logfile/DefaultMappedFile.java
@@ -22,30 +22,26 @@ import java.io.File;
 import java.io.FileNotFoundException;
 import java.io.IOException;
 import java.io.RandomAccessFile;
-import java.lang.reflect.Method;
 import java.nio.ByteBuffer;
 import java.nio.MappedByteBuffer;
 import java.nio.channels.FileChannel;
 import java.nio.channels.FileChannel.MapMode;
-import java.security.AccessController;
-import java.security.PrivilegedAction;
 import java.util.concurrent.atomic.AtomicInteger;
 import java.util.concurrent.atomic.AtomicLong;
 import org.apache.rocketmq.common.UtilAll;
 import org.apache.rocketmq.common.constant.LoggerName;
+import org.apache.rocketmq.common.message.MessageExtBatch;
 import org.apache.rocketmq.logging.InternalLogger;
 import org.apache.rocketmq.logging.InternalLoggerFactory;
 import org.apache.rocketmq.common.message.MessageExt;
 import org.apache.rocketmq.store.AppendMessageCallback;
 import org.apache.rocketmq.store.AppendMessageResult;
 import org.apache.rocketmq.store.AppendMessageStatus;
-import org.apache.rocketmq.store.MessageExtBatch;
-import org.apache.rocketmq.store.MessageExtBrokerInner;
+import org.apache.rocketmq.common.message.MessageExtBrokerInner;
 import org.apache.rocketmq.store.PutMessageContext;
 import org.apache.rocketmq.store.SelectMappedBufferResult;
 import org.apache.rocketmq.store.TransientStorePool;
 import org.apache.rocketmq.store.config.FlushDiskType;
-import org.apache.rocketmq.store.config.MessageStoreConfig;
 import org.apache.rocketmq.store.util.LibC;
 import sun.nio.ch.DirectBuffer;
 
@@ -90,74 +86,6 @@ public class DefaultMappedFile extends AbstractMappedFile {
         init(fileName, fileSize, transientStorePool);
     }
 
-    public static void ensureDirOK(final String dirName) {
-        if (dirName != null) {
-            if (dirName.contains(MessageStoreConfig.MULTI_PATH_SPLITTER)) {
-                String[] dirs = dirName.trim().split(MessageStoreConfig.MULTI_PATH_SPLITTER);
-                for (String dir : dirs) {
-                    createDirIfNotExist(dir);
-                }
-            } else {
-                createDirIfNotExist(dirName);
-            }
-        }
-    }
-
-    private static void  createDirIfNotExist(String dirName) {
-        File f = new File(dirName);
-        if (!f.exists()) {
-            boolean result = f.mkdirs();
-            log.info(dirName + " mkdir " + (result ? "OK" : "Failed"));
-        }
-    }
-
-    public static void clean(final ByteBuffer buffer) {
-        if (buffer == null || !buffer.isDirect() || buffer.capacity() == 0)
-            return;
-        invoke(invoke(viewed(buffer), "cleaner"), "clean");
-    }
-
-    private static Object invoke(final Object target, final String methodName, final Class<?>... args) {
-        return AccessController.doPrivileged(new PrivilegedAction<Object>() {
-            @Override
-            public Object run() {
-                try {
-                    Method method = method(target, methodName, args);
-                    method.setAccessible(true);
-                    return method.invoke(target);
-                } catch (Exception e) {
-                    throw new IllegalStateException(e);
-                }
-            }
-        });
-    }
-
-    private static Method method(Object target, String methodName, Class<?>[] args)
-        throws NoSuchMethodException {
-        try {
-            return target.getClass().getMethod(methodName, args);
-        } catch (NoSuchMethodException e) {
-            return target.getClass().getDeclaredMethod(methodName, args);
-        }
-    }
-
-    private static ByteBuffer viewed(ByteBuffer buffer) {
-        String methodName = "viewedBuffer";
-        Method[] methods = buffer.getClass().getMethods();
-        for (int i = 0; i < methods.length; i++) {
-            if (methods[i].getName().equals("attachment")) {
-                methodName = "attachment";
-                break;
-            }
-        }
-
-        ByteBuffer viewedBuffer = (ByteBuffer) invoke(buffer, methodName);
-        if (viewedBuffer == null)
-            return buffer;
-        else
-            return viewed(viewedBuffer);
-    }
-
     public static int getTotalMappedFiles() {
         return TOTAL_MAPPED_FILES.get();
     }
@@ -181,7 +109,7 @@ public class DefaultMappedFile extends AbstractMappedFile {
         this.fileFromOffset = Long.parseLong(this.file.getName());
         boolean ok = false;
 
-        ensureDirOK(this.file.getParent());
+        UtilAll.ensureDirOK(this.file.getParent());
 
         try {
             this.fileChannel = new RandomAccessFile(this.file, "rw").getChannel();
@@ -207,6 +135,36 @@ public class DefaultMappedFile extends AbstractMappedFile {
         return this.file.lastModified();
     }
 
+    public boolean getData(int pos, int size, ByteBuffer byteBuffer) {
+        if (byteBuffer.remaining() < size) {
+            return false;
+        }
+
+        int readPosition = getReadPosition();
+        if ((pos + size) <= readPosition) {
+
+            if (this.hold()) {
+                try {
+                    int readNum = fileChannel.read(byteBuffer, pos);
+                    return size == readNum;
+                } catch (Throwable t) {
+                    log.warn("Get data failed pos:{} size:{} fileFromOffset:{}", pos, size, this.fileFromOffset);
+                    return false;
+                } finally {
+                    this.release();
+                }
+            } else {
+                log.debug("matched, but hold failed, request pos: " + pos + ", fileFromOffset: "
+                    + this.fileFromOffset);
+            }
+        } else {
+            log.warn("selectMappedBuffer request pos invalid, request pos: " + pos + ", size: " + size
+                + ", fileFromOffset: " + this.fileFromOffset);
+        }
+
+        return false;
+    }
+
     @Override
     public int getFileSize() {
         return fileSize;
@@ -495,8 +453,8 @@ public class DefaultMappedFile extends AbstractMappedFile {
             return true;
         }
 
-        clean(this.mappedByteBuffer);
-        clean(this.mappedByteBufferWaitToClean);
+        UtilAll.cleanBuffer(this.mappedByteBuffer);
+        UtilAll.cleanBuffer(this.mappedByteBufferWaitToClean);
         this.mappedByteBufferWaitToClean = null;
         TOTAL_MAPPED_VIRTUAL_MEMORY.addAndGet(this.fileSize * (-1));
         TOTAL_MAPPED_FILES.decrementAndGet();
@@ -636,7 +594,7 @@ public class DefaultMappedFile extends AbstractMappedFile {
             if (!force && gapTime < minGapTime) {
                 Thread.sleep(minGapTime - gapTime);
             }
-            clean(this.mappedByteBufferWaitToClean);
+            UtilAll.cleanBuffer(this.mappedByteBufferWaitToClean);
             mappedByteBufferWaitToClean = null;
             log.info("cleanSwapedMap file " + this.fileName + " success.");
         } catch (Exception e) {
diff --git a/store/src/main/java/org/apache/rocketmq/store/logfile/MappedFile.java b/store/src/main/java/org/apache/rocketmq/store/logfile/MappedFile.java
index 7d1feba..630b202 100644
--- a/store/src/main/java/org/apache/rocketmq/store/logfile/MappedFile.java
+++ b/store/src/main/java/org/apache/rocketmq/store/logfile/MappedFile.java
@@ -16,10 +16,10 @@
  */
 package org.apache.rocketmq.store.logfile;
 
+import org.apache.rocketmq.common.message.MessageExtBatch;
 import org.apache.rocketmq.store.AppendMessageCallback;
 import org.apache.rocketmq.store.AppendMessageResult;
-import org.apache.rocketmq.store.MessageExtBatch;
-import org.apache.rocketmq.store.MessageExtBrokerInner;
+import org.apache.rocketmq.common.message.MessageExtBrokerInner;
 import org.apache.rocketmq.store.PutMessageContext;
 import org.apache.rocketmq.store.SelectMappedBufferResult;
 import org.apache.rocketmq.store.TransientStorePool;
@@ -187,6 +187,16 @@ public interface MappedFile {
     long getLastModifiedTimestamp();
 
     /**
+     * Get data from a certain pos offset with size byte
+     *
+     * @param pos a certain pos offset to get data
+     * @param size the size of data
+     * @param byteBuffer the data
+     * @return true if with data; false if no data;
+     */
+    boolean getData(int pos, int size, ByteBuffer byteBuffer);
+
+    /**
      * Destroys the file and delete it from the file system.
      *
      * @param intervalForcibly If {@code true} then this method will destroy the file forcibly and ignore the reference
diff --git a/store/src/main/java/org/apache/rocketmq/store/pop/AckMsg.java b/store/src/main/java/org/apache/rocketmq/store/pop/AckMsg.java
index ab017a9..1d62e39 100644
--- a/store/src/main/java/org/apache/rocketmq/store/pop/AckMsg.java
+++ b/store/src/main/java/org/apache/rocketmq/store/pop/AckMsg.java
@@ -23,6 +23,7 @@ public class AckMsg {
     private String topic;
     private int queueId;
     private long popTime;
+    private String brokerName;
 
     public long getPopTime() {
         return popTime;
@@ -72,6 +73,14 @@ public class AckMsg {
         this.startOffset = startOffset;
     }
 
+    public String getBrokerName() {
+        return brokerName;
+    }
+
+    public void setBrokerName(String brokerName) {
+        this.brokerName = brokerName;
+    }
+
     @Override
     public String toString() {
         final StringBuilder sb = new StringBuilder("AckMsg{");
@@ -81,6 +90,7 @@ public class AckMsg {
         sb.append(", topic='").append(topic).append('\'');
         sb.append(", queueId=").append(queueId);
         sb.append(", popTime=").append(popTime);
+        sb.append(", brokerName=").append(brokerName);
         sb.append('}');
         return sb.toString();
     }
diff --git a/store/src/main/java/org/apache/rocketmq/store/pop/PopCheckPoint.java b/store/src/main/java/org/apache/rocketmq/store/pop/PopCheckPoint.java
index a4a3aac..6eccf9c 100644
--- a/store/src/main/java/org/apache/rocketmq/store/pop/PopCheckPoint.java
+++ b/store/src/main/java/org/apache/rocketmq/store/pop/PopCheckPoint.java
@@ -41,6 +41,8 @@ public class PopCheckPoint {
     private long reviveOffset;
     @JSONField(name = "d")
     private List<Integer> queueOffsetDiff;
+    @JSONField(name = "bn")
+    String brokerName;
 
     public long getReviveOffset() {
         return reviveOffset;
@@ -130,6 +132,14 @@ public class PopCheckPoint {
         this.queueOffsetDiff = queueOffsetDiff;
     }
 
+    public String getBrokerName() {
+        return brokerName;
+    }
+
+    public void setBrokerName(String brokerName) {
+        this.brokerName = brokerName;
+    }
+
     public void addDiff(int diff) {
         if (this.queueOffsetDiff == null) {
             this.queueOffsetDiff = new ArrayList<>(8);
@@ -168,7 +178,7 @@ public class PopCheckPoint {
     @Override
     public String toString() {
         return "PopCheckPoint [topic=" + topic + ", cid=" + cid + ", queueId=" + queueId + ", startOffset=" + startOffset + ", bitMap=" + bitMap + ", num=" + num + ", reviveTime=" + getReviveTime()
-            + ", reviveOffset=" + reviveOffset + ", diff=" + queueOffsetDiff + "]";
+            + ", reviveOffset=" + reviveOffset + ", diff=" + queueOffsetDiff + ", brokerName=" + brokerName + "]";
     }
 
 }
diff --git a/store/src/main/java/org/apache/rocketmq/store/queue/BatchConsumeQueue.java b/store/src/main/java/org/apache/rocketmq/store/queue/BatchConsumeQueue.java
index 3400120..b956914 100644
--- a/store/src/main/java/org/apache/rocketmq/store/queue/BatchConsumeQueue.java
+++ b/store/src/main/java/org/apache/rocketmq/store/queue/BatchConsumeQueue.java
@@ -27,7 +27,7 @@ import org.apache.rocketmq.logging.InternalLogger;
 import org.apache.rocketmq.logging.InternalLoggerFactory;
 import org.apache.rocketmq.store.DispatchRequest;
 import org.apache.rocketmq.store.MappedFileQueue;
-import org.apache.rocketmq.store.MessageExtBrokerInner;
+import org.apache.rocketmq.common.message.MessageExtBrokerInner;
 import org.apache.rocketmq.store.MessageStore;
 import org.apache.rocketmq.store.SelectMappedBufferResult;
 import org.apache.rocketmq.store.config.BrokerRole;
@@ -918,6 +918,11 @@ public class BatchConsumeQueue implements ConsumeQueueInterface, FileQueueLifeCy
     }
 
     @Override
+    public long getTotalSize() {
+        return this.mappedFileQueue.getTotalFileSize();
+    }
+
+    @Override
     public void destroy() {
         this.maxMsgPhyOffsetInCommitLog = -1;
         this.minOffsetInQueue = -1;
diff --git a/store/src/main/java/org/apache/rocketmq/store/queue/ConsumeQueueInterface.java b/store/src/main/java/org/apache/rocketmq/store/queue/ConsumeQueueInterface.java
index 5232a74..6d1b68c 100644
--- a/store/src/main/java/org/apache/rocketmq/store/queue/ConsumeQueueInterface.java
+++ b/store/src/main/java/org/apache/rocketmq/store/queue/ConsumeQueueInterface.java
@@ -111,4 +111,10 @@ public interface ConsumeQueueInterface {
      * @return cq type
      */
     CQType getCQType();
+
+    /**
+     * Gets the occupied size of CQ file on disk
+     * @return total size
+     */
+    long getTotalSize();
 }
diff --git a/store/src/main/java/org/apache/rocketmq/store/queue/ConsumeQueueStore.java b/store/src/main/java/org/apache/rocketmq/store/queue/ConsumeQueueStore.java
index 84df992..beb0532 100644
--- a/store/src/main/java/org/apache/rocketmq/store/queue/ConsumeQueueStore.java
+++ b/store/src/main/java/org/apache/rocketmq/store/queue/ConsumeQueueStore.java
@@ -16,21 +16,25 @@
  */
 package org.apache.rocketmq.store.queue;
 
+import java.nio.ByteBuffer;
 import org.apache.rocketmq.common.TopicConfig;
 import org.apache.rocketmq.common.attribute.CQType;
 import org.apache.rocketmq.common.constant.LoggerName;
+import org.apache.rocketmq.common.message.MessageDecoder;
+import org.apache.rocketmq.common.message.MessageExt;
 import org.apache.rocketmq.common.topic.TopicValidator;
 import org.apache.rocketmq.common.utils.QueueTypeUtils;
 import org.apache.rocketmq.logging.InternalLogger;
 import org.apache.rocketmq.logging.InternalLoggerFactory;
+import org.apache.rocketmq.store.CommitLog;
 import org.apache.rocketmq.store.ConsumeQueue;
 import org.apache.rocketmq.store.DefaultMessageStore;
 import org.apache.rocketmq.store.DispatchRequest;
-import org.apache.rocketmq.store.MessageExtBrokerInner;
+import org.apache.rocketmq.common.message.MessageExtBrokerInner;
+import org.apache.rocketmq.store.SelectMappedBufferResult;
 import org.apache.rocketmq.store.config.MessageStoreConfig;
 
 import java.io.File;
-import java.util.HashMap;
 import java.util.Iterator;
 import java.util.Map;
 import java.util.Objects;
@@ -79,8 +83,7 @@ public class ConsumeQueueStore {
     }
 
     /**
-     * Apply the dispatched request and build the consume queue.
-     * This function should be idempotent.
+     * Apply the dispatched request and build the consume queue. This function should be idempotent.
      *
      * @param consumeQueue consume queue
      * @param request dispatch request
@@ -144,18 +147,18 @@ public class ConsumeQueueStore {
     private ConsumeQueueInterface createConsumeQueueByType(CQType cqType, String topic, int queueId, String storePath) {
         if (Objects.equals(CQType.SimpleCQ, cqType)) {
             return new ConsumeQueue(
-                    topic,
-                    queueId,
-                    storePath,
-                    this.messageStoreConfig.getMappedFileSizeConsumeQueue(),
-                    this.messageStore);
+                topic,
+                queueId,
+                storePath,
+                this.messageStoreConfig.getMappedFileSizeConsumeQueue(),
+                this.messageStore);
         } else if (Objects.equals(CQType.BatchCQ, cqType)) {
             return new BatchConsumeQueue(
-                    topic,
-                    queueId,
-                    storePath,
-                    this.messageStoreConfig.getMapperFileSizeBatchConsumeQueue(),
-                    this.messageStore);
+                topic,
+                queueId,
+                storePath,
+                this.messageStoreConfig.getMapperFileSizeBatchConsumeQueue(),
+                this.messageStore);
         } else {
             throw new RuntimeException(format("queue type %s is not supported.", cqType.toString()));
         }
@@ -223,7 +226,8 @@ public class ConsumeQueueStore {
         fileQueueLifeCycle.truncateDirtyLogicFiles(phyOffset);
     }
 
-    public void swapMap(ConsumeQueueInterface consumeQueue, int reserveNum, long forceSwapIntervalMs, long normalSwapIntervalMs) {
+    public void swapMap(ConsumeQueueInterface consumeQueue, int reserveNum, long forceSwapIntervalMs,
+        long normalSwapIntervalMs) {
         FileQueueLifeCycle fileQueueLifeCycle = getLifeCycle(consumeQueue.getTopic(), consumeQueue.getQueueId());
         fileQueueLifeCycle.swapMap(reserveNum, forceSwapIntervalMs, normalSwapIntervalMs);
     }
@@ -270,18 +274,18 @@ public class ConsumeQueueStore {
         // TODO maybe the topic has been deleted.
         if (Objects.equals(CQType.BatchCQ, QueueTypeUtils.getCQType(topicConfig))) {
             newLogic = new BatchConsumeQueue(
-                    topic,
-                    queueId,
-                    getStorePathBatchConsumeQueue(this.messageStoreConfig.getStorePathRootDir()),
-                    this.messageStoreConfig.getMapperFileSizeBatchConsumeQueue(),
-                    this.messageStore);
+                topic,
+                queueId,
+                getStorePathBatchConsumeQueue(this.messageStoreConfig.getStorePathRootDir()),
+                this.messageStoreConfig.getMapperFileSizeBatchConsumeQueue(),
+                this.messageStore);
         } else {
             newLogic = new ConsumeQueue(
-                    topic,
-                    queueId,
-                    getStorePathConsumeQueue(this.messageStoreConfig.getStorePathRootDir()),
-                    this.messageStoreConfig.getMappedFileSizeConsumeQueue(),
-                    this.messageStore);
+                topic,
+                queueId,
+                getStorePathConsumeQueue(this.messageStoreConfig.getStorePathRootDir()),
+                this.messageStoreConfig.getMappedFileSizeConsumeQueue(),
+                this.messageStore);
         }
 
         ConsumeQueueInterface oldLogic = map.putIfAbsent(queueId, newLogic);
@@ -298,11 +302,15 @@ public class ConsumeQueueStore {
         return this.queueOffsetAssigner.currentQueueOffset(topic + "-" + queueId);
     }
 
-    public void setTopicQueueTable(HashMap<String, Long> topicQueueTable) {
+    public void setTopicQueueTable(ConcurrentMap<String, Long> topicQueueTable) {
         this.queueOffsetAssigner.setTopicQueueTable(topicQueueTable);
     }
 
-    public void setBatchTopicQueueTable(HashMap<String, Long> batchTopicQueueTable) {
+    public ConcurrentMap getTopicQueueTable() {
+        return this.queueOffsetAssigner.getTopicQueueTable();
+    }
+
+    public void setBatchTopicQueueTable(ConcurrentMap<String, Long> batchTopicQueueTable) {
         this.queueOffsetAssigner.setBatchTopicQueueTable(batchTopicQueueTable);
     }
 
@@ -311,6 +319,11 @@ public class ConsumeQueueStore {
         fileQueueLifeCycle.assignQueueOffset(this.queueOffsetAssigner, msg, messageNum);
     }
 
+    public void updateQueueOffset(String topic, int queueId, long offset) {
+        String topicQueueKey = topic + "-" + queueId;
+        this.queueOffsetAssigner.updateQueueOffset(topicQueueKey, offset);
+    }
+
     public void removeTopicQueueTable(String topic, Integer queueId) {
         this.queueOffsetAssigner.remove(topic, queueId);
     }
@@ -331,8 +344,8 @@ public class ConsumeQueueStore {
     }
 
     public void recoverOffsetTable(long minPhyOffset) {
-        HashMap<String, Long> cqOffsetTable = new HashMap<>(1024);
-        HashMap<String, Long> bcqOffsetTable = new HashMap<>(1024);
+        ConcurrentMap<String, Long> cqOffsetTable = new ConcurrentHashMap<>(1024);
+        ConcurrentMap<String, Long> bcqOffsetTable = new ConcurrentHashMap<>(1024);
 
         for (ConcurrentMap<Integer, ConsumeQueueInterface> maps : this.consumeQueueTable.values()) {
             for (ConsumeQueueInterface logic : maps.values()) {
@@ -349,6 +362,47 @@ public class ConsumeQueueStore {
             }
         }
 
+        //Correct unSubmit consumeOffset
+        if (messageStoreConfig.isDuplicationEnable()) {
+            SelectMappedBufferResult lastBuffer = null;
+            long startReadOffset = messageStore.getCommitLog().getConfirmOffset() == -1 ? 0 : messageStore.getCommitLog().getConfirmOffset();
+            while ((lastBuffer = messageStore.selectOneMessageByOffset(startReadOffset)) != null) {
+                try {
+                    if (lastBuffer.getStartOffset() > startReadOffset) {
+                        startReadOffset = lastBuffer.getStartOffset();
+                        continue;
+                    }
+
+                    ByteBuffer bb = lastBuffer.getByteBuffer();
+                    int magicCode = bb.getInt(bb.position() + 4);
+                    if (magicCode == CommitLog.BLANK_MAGIC_CODE) {
+                        startReadOffset += bb.getInt(bb.position());
+                        continue;
+                    } else if (magicCode != MessageDecoder.MESSAGE_MAGIC_CODE) {
+                        throw new RuntimeException("Unknown magicCode: " + magicCode);
+                    }
+
+                    lastBuffer.getByteBuffer().mark();
+                    DispatchRequest dispatchRequest = messageStore.getCommitLog().checkMessageAndReturnSize(lastBuffer.getByteBuffer(), true, true, true);
+                    if (!dispatchRequest.isSuccess())
+                        break;
+                    lastBuffer.getByteBuffer().reset();
+
+                    MessageExt msg = MessageDecoder.decode(lastBuffer.getByteBuffer(), true, false, false, false, true);
+                    if (msg == null)
+                        break;
+
+                    String key = msg.getTopic() + "-" + msg.getQueueId();
+                    cqOffsetTable.put(key, msg.getQueueOffset() + 1);
+                    startReadOffset += msg.getStoreSize();
+                } finally {
+                    if (lastBuffer != null)
+                        lastBuffer.release();
+                }
+
+            }
+        }
+
         this.setTopicQueueTable(cqOffsetTable);
         this.setBatchTopicQueueTable(bcqOffsetTable);
     }
@@ -366,7 +420,7 @@ public class ConsumeQueueStore {
         while (it.hasNext()) {
             Map.Entry<String, ConcurrentMap<Integer, ConsumeQueueInterface>> next = it.next();
             String topic = next.getKey();
-            if (!topic.equals(TopicValidator.RMQ_SYS_SCHEDULE_TOPIC)) {
+            if (!TopicValidator.isSystemTopic(topic)) {
                 ConcurrentMap<Integer, ConsumeQueueInterface> queueTable = next.getValue();
                 Iterator<Map.Entry<Integer, ConsumeQueueInterface>> itQT = queueTable.entrySet().iterator();
                 while (itQT.hasNext()) {
@@ -375,20 +429,20 @@ public class ConsumeQueueStore {
 
                     if (maxCLOffsetInConsumeQueue == -1) {
                         log.warn("maybe ConsumeQueue was created just now. topic={} queueId={} maxPhysicOffset={} minLogicOffset={}.",
-                                nextQT.getValue().getTopic(),
-                                nextQT.getValue().getQueueId(),
-                                nextQT.getValue().getMaxPhysicOffset(),
-                                nextQT.getValue().getMinLogicOffset());
+                            nextQT.getValue().getTopic(),
+                            nextQT.getValue().getQueueId(),
+                            nextQT.getValue().getMaxPhysicOffset(),
+                            nextQT.getValue().getMinLogicOffset());
                     } else if (maxCLOffsetInConsumeQueue < minCommitLogOffset) {
                         log.info(
-                                "cleanExpiredConsumerQueue: {} {} consumer queue destroyed, minCommitLogOffset: {} maxCLOffsetInConsumeQueue: {}",
-                                topic,
-                                nextQT.getKey(),
-                                minCommitLogOffset,
-                                maxCLOffsetInConsumeQueue);
+                            "cleanExpiredConsumerQueue: {} {} consumer queue destroyed, minCommitLogOffset: {} maxCLOffsetInConsumeQueue: {}",
+                            topic,
+                            nextQT.getKey(),
+                            minCommitLogOffset,
+                            maxCLOffsetInConsumeQueue);
 
                         removeTopicQueueTable(nextQT.getValue().getTopic(),
-                                nextQT.getValue().getQueueId());
+                            nextQT.getValue().getQueueId());
 
                         this.destroy(nextQT.getValue());
                         itQT.remove();
@@ -418,4 +472,14 @@ public class ConsumeQueueStore {
 
         return Optional.ofNullable(this.topicConfigTable.get(topic));
     }
+
+    public long getTotalSize() {
+        long totalSize = 0;
+        for (ConcurrentMap<Integer, ConsumeQueueInterface> maps : this.consumeQueueTable.values()) {
+            for (ConsumeQueueInterface logic : maps.values()) {
+                totalSize += logic.getTotalSize();
+            }
+        }
+        return totalSize;
+    }
 }
diff --git a/store/src/main/java/org/apache/rocketmq/store/queue/FileQueueLifeCycle.java b/store/src/main/java/org/apache/rocketmq/store/queue/FileQueueLifeCycle.java
index 05c217f..b2211d8 100644
--- a/store/src/main/java/org/apache/rocketmq/store/queue/FileQueueLifeCycle.java
+++ b/store/src/main/java/org/apache/rocketmq/store/queue/FileQueueLifeCycle.java
@@ -17,7 +17,7 @@
 package org.apache.rocketmq.store.queue;
 
 import org.apache.rocketmq.store.DispatchRequest;
-import org.apache.rocketmq.store.MessageExtBrokerInner;
+import org.apache.rocketmq.common.message.MessageExtBrokerInner;
 import org.apache.rocketmq.store.Swappable;
 
 public interface FileQueueLifeCycle extends Swappable {
diff --git a/store/src/main/java/org/apache/rocketmq/store/queue/QueueOffsetAssigner.java b/store/src/main/java/org/apache/rocketmq/store/queue/QueueOffsetAssigner.java
index 7e4b4ee..55614cc 100644
--- a/store/src/main/java/org/apache/rocketmq/store/queue/QueueOffsetAssigner.java
+++ b/store/src/main/java/org/apache/rocketmq/store/queue/QueueOffsetAssigner.java
@@ -17,14 +17,12 @@
 
 package org.apache.rocketmq.store.queue;
 
-import java.util.Map;
 import java.util.concurrent.ConcurrentHashMap;
+import java.util.concurrent.ConcurrentMap;
 import org.apache.rocketmq.common.constant.LoggerName;
 import org.apache.rocketmq.logging.InternalLogger;
 import org.apache.rocketmq.logging.InternalLoggerFactory;
 
-import java.util.HashMap;
-
 /**
  * QueueOffsetAssigner is a component for assigning offsets for queues.
  *
@@ -32,9 +30,9 @@ import java.util.HashMap;
 public class QueueOffsetAssigner {
     private static final InternalLogger log = InternalLoggerFactory.getLogger(LoggerName.STORE_LOGGER_NAME);
 
-    private Map<String, Long> topicQueueTable = new ConcurrentHashMap<>(1024);
-    private Map<String, Long> batchTopicQueueTable = new ConcurrentHashMap<>(1024);
-    private Map<String/* topic-queueid */, Long/* offset */> lmqTopicQueueTable = new ConcurrentHashMap<>(1024);
+    private ConcurrentMap<String, Long> topicQueueTable = new ConcurrentHashMap<>(1024);
+    private ConcurrentMap<String, Long> batchTopicQueueTable = new ConcurrentHashMap<>(1024);
+    private ConcurrentMap<String/* topic-queueid */, Long/* offset */> lmqTopicQueueTable = new ConcurrentHashMap<>(1024);
 
     public long assignQueueOffset(String topicQueueKey, short messageNum) {
         long queueOffset = this.topicQueueTable.computeIfAbsent(topicQueueKey, k -> 0L);
@@ -42,8 +40,12 @@ public class QueueOffsetAssigner {
         return queueOffset;
     }
 
+    public void updateQueueOffset(String topicQueueKey, long offset) {
+        this.topicQueueTable.put(topicQueueKey, offset);
+    }
+
     public long assignBatchQueueOffset(String topicQueueKey, short messageNum) {
-        long topicOffset = this.batchTopicQueueTable.computeIfAbsent(topicQueueKey, k -> 0L);
+        Long topicOffset = this.batchTopicQueueTable.computeIfAbsent(topicQueueKey, k -> 0L);
         this.batchTopicQueueTable.put(topicQueueKey, topicOffset + messageNum);
         return topicOffset;
     }
@@ -76,11 +78,15 @@ public class QueueOffsetAssigner {
         log.info("removeQueueFromTopicQueueTable OK Topic: {} QueueId: {}", topic, queueId);
     }
 
-    public void setTopicQueueTable(HashMap<String, Long> topicQueueTable) {
+    public void setTopicQueueTable(ConcurrentMap<String, Long> topicQueueTable) {
         this.topicQueueTable = topicQueueTable;
     }
 
-    public void setBatchTopicQueueTable(HashMap<String, Long> batchTopicQueueTable) {
+    public ConcurrentMap<String, Long> getTopicQueueTable() {
+        return topicQueueTable;
+    }
+
+    public void setBatchTopicQueueTable(ConcurrentMap<String, Long> batchTopicQueueTable) {
         this.batchTopicQueueTable = batchTopicQueueTable;
     }
 }
\ No newline at end of file
diff --git a/store/src/main/java/org/apache/rocketmq/store/stats/BrokerStatsManager.java b/store/src/main/java/org/apache/rocketmq/store/stats/BrokerStatsManager.java
index 531d3fd..1ea987a 100644
--- a/store/src/main/java/org/apache/rocketmq/store/stats/BrokerStatsManager.java
+++ b/store/src/main/java/org/apache/rocketmq/store/stats/BrokerStatsManager.java
@@ -19,8 +19,19 @@ package org.apache.rocketmq.store.stats;
 import java.util.HashMap;
 import java.util.concurrent.Executors;
 import java.util.concurrent.ScheduledExecutorService;
+import org.apache.commons.lang3.tuple.Pair;
+import org.apache.rocketmq.common.BrokerConfig;
 import org.apache.rocketmq.common.ThreadFactoryImpl;
+import org.apache.rocketmq.common.UtilAll;
 import org.apache.rocketmq.common.constant.LoggerName;
+import org.apache.rocketmq.common.statistics.StatisticsItem;
+import org.apache.rocketmq.common.statistics.StatisticsItemFormatter;
+import org.apache.rocketmq.common.statistics.StatisticsItemPrinter;
+import org.apache.rocketmq.common.statistics.StatisticsItemScheduledIncrementPrinter;
+import org.apache.rocketmq.common.statistics.StatisticsItemScheduledPrinter;
+import org.apache.rocketmq.common.statistics.StatisticsItemStateGetter;
+import org.apache.rocketmq.common.statistics.StatisticsKindMeta;
+import org.apache.rocketmq.common.statistics.StatisticsManager;
 import org.apache.rocketmq.logging.InternalLogger;
 import org.apache.rocketmq.logging.InternalLoggerFactory;
 import org.apache.rocketmq.common.stats.MomentStatsItemSet;
@@ -35,15 +46,25 @@ public class BrokerStatsManager {
     public static final String QUEUE_GET_SIZE = "QUEUE_GET_SIZE";
     public static final String TOPIC_PUT_NUMS = "TOPIC_PUT_NUMS";
     public static final String TOPIC_PUT_SIZE = "TOPIC_PUT_SIZE";
+    // Send message latency
+    public static final String TOPIC_PUT_LATENCY = "TOPIC_PUT_LATENCY";
     public static final String GROUP_GET_NUMS = "GROUP_GET_NUMS";
     public static final String GROUP_GET_SIZE = "GROUP_GET_SIZE";
+    public static final String GROUP_ACK_NUMS = "GROUP_ACK_NUMS";
+    public static final String GROUP_CK_NUMS = "GROUP_CK_NUMS";
     public static final String SNDBCK_PUT_NUMS = "SNDBCK_PUT_NUMS";
+    public static final String DLQ_PUT_NUMS = "DLQ_PUT_NUMS";
     public static final String BROKER_PUT_NUMS = "BROKER_PUT_NUMS";
     public static final String BROKER_GET_NUMS = "BROKER_GET_NUMS";
+    public static final String BROKER_ACK_NUMS = "BROKER_ACK_NUMS";
+    public static final String BROKER_CK_NUMS = "BROKER_CK_NUMS";
     public static final String GROUP_GET_FROM_DISK_NUMS = "GROUP_GET_FROM_DISK_NUMS";
     public static final String GROUP_GET_FROM_DISK_SIZE = "GROUP_GET_FROM_DISK_SIZE";
     public static final String BROKER_GET_FROM_DISK_NUMS = "BROKER_GET_FROM_DISK_NUMS";
     public static final String BROKER_GET_FROM_DISK_SIZE = "BROKER_GET_FROM_DISK_SIZE";
+
+    public static final String SNDBCK2DLQ_TIMES = "SNDBCK2DLQ_TIMES";
+
     // For commercial
     public static final String COMMERCIAL_SEND_TIMES = "COMMERCIAL_SEND_TIMES";
     public static final String COMMERCIAL_SNDBCK_TIMES = "COMMERCIAL_SNDBCK_TIMES";
@@ -53,32 +74,93 @@ public class BrokerStatsManager {
     public static final String COMMERCIAL_RCV_SIZE = "COMMERCIAL_RCV_SIZE";
     public static final String COMMERCIAL_PERM_FAILURES = "COMMERCIAL_PERM_FAILURES";
     public static final String COMMERCIAL_OWNER = "Owner";
-    // Message Size limit for one api-calling count.
-    public static final double SIZE_PER_COUNT = 64 * 1024;
+
+    public static final String ACCOUNT_OWNER_PARENT = "OWNER_PARENT";
+    public static final String ACCOUNT_OWNER_SELF = "OWNER_SELF";
+
+    public static final long ACCOUNT_STAT_INVERTAL = 60 * 1000;
+    public static final String ACCOUNT_AUTH_TYPE = "AUTH_TYPE";
+
+    public static final String ACCOUNT_SEND = "SEND";
+    public static final String ACCOUNT_RCV = "RCV";
+    public static final String ACCOUNT_SEND_BACK = "SEND_BACK";
+    public static final String ACCOUNT_SEND_BACK_TO_DLQ = "SEND_BACK_TO_DLQ";
+    public static final String ACCOUNT_AUTH_FAILED = "AUTH_FAILED";
+    public static final String ACCOUNT_SEND_REJ = "SEND_REJ";
+    public static final String ACCOUNT_REV_REJ = "RCV_REJ";
+
+    public static final String MSG_NUM = "MSG_NUM";
+    public static final String MSG_SIZE = "MSG_SIZE";
+    public static final String SUCCESS_MSG_NUM = "SUCCESS_MSG_NUM";
+    public static final String FAILURE_MSG_NUM = "FAILURE_MSG_NUM";
+    public static final String COMMERCIAL_MSG_NUM = "COMMERCIAL_MSG_NUM";
+    public static final String SUCCESS_REQ_NUM = "SUCCESS_REQ_NUM";
+    public static final String FAILURE_REQ_NUM = "FAILURE_REQ_NUM";
+    public static final String SUCCESS_MSG_SIZE = "SUCCESS_MSG_SIZE";
+    public static final String FAILURE_MSG_SIZE = "FAILURE_MSG_SIZE";
+    public static final String RT = "RT";
+    public static final String INNER_RT = "INNER_RT";
 
     public static final String GROUP_GET_FALL_SIZE = "GROUP_GET_FALL_SIZE";
     public static final String GROUP_GET_FALL_TIME = "GROUP_GET_FALL_TIME";
     // Pull Message Latency
     public static final String GROUP_GET_LATENCY = "GROUP_GET_LATENCY";
+    // Consumer Register Time
+    public static final String CONSUMER_REGISTER_TIME = "CONSUMER_REGISTER_TIME";
+    // Producer Register Time
+    public static final String PRODUCER_REGISTER_TIME = "PRODUCER_REGISTER_TIME";
+    public static final String CHANNEL_ACTIVITY = "CHANNEL_ACTIVITY";
+    public static final String CHANNEL_ACTIVITY_CONNECT = "CONNECT";
+    public static final String CHANNEL_ACTIVITY_IDLE = "IDLE";
+    public static final String CHANNEL_ACTIVITY_EXCEPTION = "EXCEPTION";
+    public static final String CHANNEL_ACTIVITY_CLOSE = "CLOSE";
 
     /**
      * read disk follow stats
      */
     private static final InternalLogger log = InternalLoggerFactory.getLogger(LoggerName.ROCKETMQ_STATS_LOGGER_NAME);
-    private static final InternalLogger COMMERCIAL_LOG = InternalLoggerFactory.getLogger(LoggerName.COMMERCIAL_LOGGER_NAME);
-    private final ScheduledExecutorService scheduledExecutorService = Executors.newSingleThreadScheduledExecutor(new ThreadFactoryImpl(
-        "BrokerStatsThread"));
-    private final ScheduledExecutorService commercialExecutor = Executors.newSingleThreadScheduledExecutor(new ThreadFactoryImpl(
-        "CommercialStatsThread"));
+    private static final InternalLogger COMMERCIAL_LOG = InternalLoggerFactory.getLogger(
+        LoggerName.COMMERCIAL_LOGGER_NAME);
+    private static final InternalLogger ACCOUNT_LOG = InternalLoggerFactory.getLogger(LoggerName.ACCOUNT_LOGGER_NAME);
+    private static final InternalLogger DLQ_STAT_LOG = InternalLoggerFactory.getLogger(
+        LoggerName.DLQ_STATS_LOGGER_NAME);
+    private ScheduledExecutorService scheduledExecutorService;
+    private ScheduledExecutorService commercialExecutor;
+    private ScheduledExecutorService accountExecutor;
+
     private final HashMap<String, StatsItemSet> statsTable = new HashMap<String, StatsItemSet>();
     private final String clusterName;
     private final boolean enableQueueStat;
-    private final MomentStatsItemSet momentStatsItemSetFallSize = new MomentStatsItemSet(GROUP_GET_FALL_SIZE, scheduledExecutorService, log);
-    private final MomentStatsItemSet momentStatsItemSetFallTime = new MomentStatsItemSet(GROUP_GET_FALL_TIME, scheduledExecutorService, log);
+    private MomentStatsItemSet momentStatsItemSetFallSize;
+    private MomentStatsItemSet momentStatsItemSetFallTime;
+
+    private final StatisticsManager accountStatManager = new StatisticsManager();
+    private StateGetter produerStateGetter;
+    private StateGetter consumerStateGetter;
+
+    private BrokerConfig brokerConfig;
+
+    public BrokerStatsManager(BrokerConfig brokerConfig) {
+        this.brokerConfig = brokerConfig;
+        this.enableQueueStat = brokerConfig.isEnableDetailStat();
+        initScheduleService();
+        this.clusterName = brokerConfig.getBrokerClusterName();
+        init();
+    }
 
     public BrokerStatsManager(String clusterName, boolean enableQueueStat) {
         this.clusterName = clusterName;
         this.enableQueueStat = enableQueueStat;
+        initScheduleService();
+        init();
+    }
+
+    public void init() {
+        momentStatsItemSetFallSize = new MomentStatsItemSet(GROUP_GET_FALL_SIZE,
+            scheduledExecutorService, log);
+
+        momentStatsItemSetFallTime = new MomentStatsItemSet(GROUP_GET_FALL_TIME,
+            scheduledExecutorService, log);
 
         if (enableQueueStat) {
             this.statsTable.put(QUEUE_PUT_NUMS, new StatsItemSet(QUEUE_PUT_NUMS, this.scheduledExecutorService, log));
@@ -90,22 +172,109 @@ public class BrokerStatsManager {
         this.statsTable.put(TOPIC_PUT_SIZE, new StatsItemSet(TOPIC_PUT_SIZE, this.scheduledExecutorService, log));
         this.statsTable.put(GROUP_GET_NUMS, new StatsItemSet(GROUP_GET_NUMS, this.scheduledExecutorService, log));
         this.statsTable.put(GROUP_GET_SIZE, new StatsItemSet(GROUP_GET_SIZE, this.scheduledExecutorService, log));
+        this.statsTable.put(GROUP_ACK_NUMS, new StatsItemSet(GROUP_ACK_NUMS, this.scheduledExecutorService, log));
+        this.statsTable.put(GROUP_CK_NUMS, new StatsItemSet(GROUP_CK_NUMS, this.scheduledExecutorService, log));
         this.statsTable.put(GROUP_GET_LATENCY, new StatsItemSet(GROUP_GET_LATENCY, this.scheduledExecutorService, log));
+        this.statsTable.put(TOPIC_PUT_LATENCY, new StatsItemSet(TOPIC_PUT_LATENCY, this.scheduledExecutorService, log));
         this.statsTable.put(SNDBCK_PUT_NUMS, new StatsItemSet(SNDBCK_PUT_NUMS, this.scheduledExecutorService, log));
+        this.statsTable.put(DLQ_PUT_NUMS, new StatsItemSet(DLQ_PUT_NUMS, this.scheduledExecutorService, log));
         this.statsTable.put(BROKER_PUT_NUMS, new StatsItemSet(BROKER_PUT_NUMS, this.scheduledExecutorService, log));
         this.statsTable.put(BROKER_GET_NUMS, new StatsItemSet(BROKER_GET_NUMS, this.scheduledExecutorService, log));
-        this.statsTable.put(GROUP_GET_FROM_DISK_NUMS, new StatsItemSet(GROUP_GET_FROM_DISK_NUMS, this.scheduledExecutorService, log));
-        this.statsTable.put(GROUP_GET_FROM_DISK_SIZE, new StatsItemSet(GROUP_GET_FROM_DISK_SIZE, this.scheduledExecutorService, log));
-        this.statsTable.put(BROKER_GET_FROM_DISK_NUMS, new StatsItemSet(BROKER_GET_FROM_DISK_NUMS, this.scheduledExecutorService, log));
-        this.statsTable.put(BROKER_GET_FROM_DISK_SIZE, new StatsItemSet(BROKER_GET_FROM_DISK_SIZE, this.scheduledExecutorService, log));
-
-        this.statsTable.put(COMMERCIAL_SEND_TIMES, new StatsItemSet(COMMERCIAL_SEND_TIMES, this.commercialExecutor, COMMERCIAL_LOG));
-        this.statsTable.put(COMMERCIAL_RCV_TIMES, new StatsItemSet(COMMERCIAL_RCV_TIMES, this.commercialExecutor, COMMERCIAL_LOG));
-        this.statsTable.put(COMMERCIAL_SEND_SIZE, new StatsItemSet(COMMERCIAL_SEND_SIZE, this.commercialExecutor, COMMERCIAL_LOG));
-        this.statsTable.put(COMMERCIAL_RCV_SIZE, new StatsItemSet(COMMERCIAL_RCV_SIZE, this.commercialExecutor, COMMERCIAL_LOG));
-        this.statsTable.put(COMMERCIAL_RCV_EPOLLS, new StatsItemSet(COMMERCIAL_RCV_EPOLLS, this.commercialExecutor, COMMERCIAL_LOG));
-        this.statsTable.put(COMMERCIAL_SNDBCK_TIMES, new StatsItemSet(COMMERCIAL_SNDBCK_TIMES, this.commercialExecutor, COMMERCIAL_LOG));
-        this.statsTable.put(COMMERCIAL_PERM_FAILURES, new StatsItemSet(COMMERCIAL_PERM_FAILURES, this.commercialExecutor, COMMERCIAL_LOG));
+        this.statsTable.put(BROKER_ACK_NUMS, new StatsItemSet(BROKER_ACK_NUMS, this.scheduledExecutorService, log));
+        this.statsTable.put(BROKER_CK_NUMS, new StatsItemSet(BROKER_CK_NUMS, this.scheduledExecutorService, log));
+        this.statsTable.put(GROUP_GET_FROM_DISK_NUMS,
+            new StatsItemSet(GROUP_GET_FROM_DISK_NUMS, this.scheduledExecutorService, log));
+        this.statsTable.put(GROUP_GET_FROM_DISK_SIZE,
+            new StatsItemSet(GROUP_GET_FROM_DISK_SIZE, this.scheduledExecutorService, log));
+        this.statsTable.put(BROKER_GET_FROM_DISK_NUMS,
+            new StatsItemSet(BROKER_GET_FROM_DISK_NUMS, this.scheduledExecutorService, log));
+        this.statsTable.put(BROKER_GET_FROM_DISK_SIZE,
+            new StatsItemSet(BROKER_GET_FROM_DISK_SIZE, this.scheduledExecutorService, log));
+
+        this.statsTable.put(SNDBCK2DLQ_TIMES,
+            new StatsItemSet(SNDBCK2DLQ_TIMES, this.scheduledExecutorService, DLQ_STAT_LOG));
+
+        this.statsTable.put(COMMERCIAL_SEND_TIMES,
+            new StatsItemSet(COMMERCIAL_SEND_TIMES, this.commercialExecutor, COMMERCIAL_LOG));
+        this.statsTable.put(COMMERCIAL_RCV_TIMES,
+            new StatsItemSet(COMMERCIAL_RCV_TIMES, this.commercialExecutor, COMMERCIAL_LOG));
+        this.statsTable.put(COMMERCIAL_SEND_SIZE,
+            new StatsItemSet(COMMERCIAL_SEND_SIZE, this.commercialExecutor, COMMERCIAL_LOG));
+        this.statsTable.put(COMMERCIAL_RCV_SIZE,
+            new StatsItemSet(COMMERCIAL_RCV_SIZE, this.commercialExecutor, COMMERCIAL_LOG));
+        this.statsTable.put(COMMERCIAL_RCV_EPOLLS,
+            new StatsItemSet(COMMERCIAL_RCV_EPOLLS, this.commercialExecutor, COMMERCIAL_LOG));
+        this.statsTable.put(COMMERCIAL_SNDBCK_TIMES,
+            new StatsItemSet(COMMERCIAL_SNDBCK_TIMES, this.commercialExecutor, COMMERCIAL_LOG));
+        this.statsTable.put(COMMERCIAL_PERM_FAILURES,
+            new StatsItemSet(COMMERCIAL_PERM_FAILURES, this.commercialExecutor, COMMERCIAL_LOG));
+
+        this.statsTable.put(CONSUMER_REGISTER_TIME,
+            new StatsItemSet(CONSUMER_REGISTER_TIME, this.scheduledExecutorService, log));
+        this.statsTable.put(PRODUCER_REGISTER_TIME,
+            new StatsItemSet(PRODUCER_REGISTER_TIME, this.scheduledExecutorService, log));
+
+        this.statsTable.put(CHANNEL_ACTIVITY, new StatsItemSet(CHANNEL_ACTIVITY, this.scheduledExecutorService, log));
+
+        StatisticsItemFormatter formatter = new StatisticsItemFormatter();
+        accountStatManager.setBriefMeta(new Pair[] {
+            Pair.of(RT, new long[][] {{50, 50}, {100, 10}, {1000, 10}}),
+            Pair.of(INNER_RT, new long[][] {{10, 10}, {100, 10}, {1000, 10}})});
+        String[] itemNames = new String[] {
+            MSG_NUM, SUCCESS_MSG_NUM, FAILURE_MSG_NUM, COMMERCIAL_MSG_NUM,
+            SUCCESS_REQ_NUM, FAILURE_REQ_NUM,
+            MSG_SIZE, SUCCESS_MSG_SIZE, FAILURE_MSG_SIZE,
+            RT, INNER_RT};
+        this.accountStatManager.addStatisticsKindMeta(createStatisticsKindMeta(
+            ACCOUNT_SEND, itemNames, this.accountExecutor, formatter, ACCOUNT_LOG, ACCOUNT_STAT_INVERTAL));
+        this.accountStatManager.addStatisticsKindMeta(createStatisticsKindMeta(
+            ACCOUNT_RCV, itemNames, this.accountExecutor, formatter, ACCOUNT_LOG, ACCOUNT_STAT_INVERTAL));
+        this.accountStatManager.addStatisticsKindMeta(createStatisticsKindMeta(
+            ACCOUNT_SEND_BACK, itemNames, this.accountExecutor, formatter, ACCOUNT_LOG, ACCOUNT_STAT_INVERTAL));
+        this.accountStatManager.addStatisticsKindMeta(createStatisticsKindMeta(
+            ACCOUNT_SEND_BACK_TO_DLQ, itemNames, this.accountExecutor, formatter, ACCOUNT_LOG, ACCOUNT_STAT_INVERTAL));
+        this.accountStatManager.addStatisticsKindMeta(createStatisticsKindMeta(
+            ACCOUNT_SEND_REJ, itemNames, this.accountExecutor, formatter, ACCOUNT_LOG, ACCOUNT_STAT_INVERTAL));
+        this.accountStatManager.addStatisticsKindMeta(createStatisticsKindMeta(
+            ACCOUNT_REV_REJ, itemNames, this.accountExecutor, formatter, ACCOUNT_LOG, ACCOUNT_STAT_INVERTAL));
+        this.accountStatManager.setStatisticsItemStateGetter(new StatisticsItemStateGetter() {
+            @Override
+            public boolean online(StatisticsItem item) {
+                String[] strArr = null;
+                try {
+                    strArr = splitAccountStatKey(item.getStatObject());
+                } catch (Exception e) {
+                    log.warn("parse account stat key failed, key: {}", item.getStatObject());
+                    return false;
+                }
+
+                // TODO ugly
+                if (strArr == null || strArr.length < 4) {
+                    return false;
+                }
+
+                String instanceId = strArr[1];
+                String topic = strArr[2];
+                String group = strArr[3];
+
+                String kind = item.getStatKind();
+                if (ACCOUNT_SEND.equals(kind) || ACCOUNT_SEND_REJ.equals(kind)) {
+                    return produerStateGetter.online(instanceId, group, topic);
+                } else if (ACCOUNT_RCV.equals(kind) || ACCOUNT_SEND_BACK.equals(kind) || ACCOUNT_SEND_BACK_TO_DLQ.equals(kind) || ACCOUNT_REV_REJ.equals(kind)) {
+                    return consumerStateGetter.online(instanceId, group, topic);
+                }
+                return false;
+            }
+        });
+    }
+
+    private void initScheduleService() {
+        this.scheduledExecutorService =
+            Executors.newSingleThreadScheduledExecutor(new ThreadFactoryImpl("BrokerStatsThread", true, brokerConfig));
+        this.commercialExecutor =
+            Executors.newSingleThreadScheduledExecutor(new ThreadFactoryImpl("CommercialStatsThread", true, brokerConfig));
+        this.accountExecutor =
+            Executors.newSingleThreadScheduledExecutor(new ThreadFactoryImpl("AccountStatsThread", true, brokerConfig));
     }
 
     public MomentStatsItemSet getMomentStatsItemSetFallSize() {
@@ -116,6 +285,22 @@ public class BrokerStatsManager {
         return momentStatsItemSetFallTime;
     }
 
+    public StateGetter getProduerStateGetter() {
+        return produerStateGetter;
+    }
+
+    public void setProduerStateGetter(StateGetter produerStateGetter) {
+        this.produerStateGetter = produerStateGetter;
+    }
+
+    public StateGetter getConsumerStateGetter() {
+        return consumerStateGetter;
+    }
+
+    public void setConsumerStateGetter(StateGetter consumerStateGetter) {
+        this.consumerStateGetter = consumerStateGetter;
+    }
+
     public void start() {
     }
 
@@ -195,6 +380,30 @@ public class BrokerStatsManager {
         }
     }
 
+    public void incConsumerRegisterTime(final int incValue) {
+        this.statsTable.get(CONSUMER_REGISTER_TIME).addValue(this.clusterName, incValue, 1);
+    }
+
+    public void incProducerRegisterTime(final int incValue) {
+        this.statsTable.get(PRODUCER_REGISTER_TIME).addValue(this.clusterName, incValue, 1);
+    }
+
+    public void incChannelConnectNum() {
+        this.statsTable.get(CHANNEL_ACTIVITY).addValue(CHANNEL_ACTIVITY_CONNECT, 1, 1);
+    }
+
+    public void incChannelCloseNum() {
+        this.statsTable.get(CHANNEL_ACTIVITY).addValue(CHANNEL_ACTIVITY_CLOSE, 1, 1);
+    }
+
+    public void incChannelExceptionNum() {
+        this.statsTable.get(CHANNEL_ACTIVITY).addValue(CHANNEL_ACTIVITY_EXCEPTION, 1, 1);
+    }
+
+    public void incChannelIdleNum() {
+        this.statsTable.get(CHANNEL_ACTIVITY).addValue(CHANNEL_ACTIVITY_IDLE, 1, 1);
+    }
+
     public void incTopicPutNums(final String topic) {
         this.statsTable.get(TOPIC_PUT_NUMS).addValue(topic, 1, 1);
     }
@@ -212,6 +421,16 @@ public class BrokerStatsManager {
         this.statsTable.get(GROUP_GET_NUMS).addValue(statsKey, incValue, 1);
     }
 
+    public void incGroupCkNums(final String group, final String topic, final int incValue) {
+        final String statsKey = buildStatsKey(topic, group);
+        this.statsTable.get(GROUP_CK_NUMS).addValue(statsKey, incValue, 1);
+    }
+
+    public void incGroupAckNums(final String group, final String topic, final int incValue) {
+        final String statsKey = buildStatsKey(topic, group);
+        this.statsTable.get(GROUP_ACK_NUMS).addValue(statsKey, incValue, 1);
+    }
+
     public String buildStatsKey(String topic, String group) {
         StringBuilder strBuilder;
         if (topic != null && group != null) {
@@ -271,6 +490,11 @@ public class BrokerStatsManager {
         this.statsTable.get(GROUP_GET_LATENCY).addRTValue(statsKey, incValue, 1);
     }
 
+    public void incTopicPutLatency(final String topic, final int queueId, final int incValue) {
+        final String statsKey = String.format("%d@%s", queueId, topic);
+        this.statsTable.get(TOPIC_PUT_LATENCY).addValue(statsKey, incValue, 1);
+    }
+
     public void incBrokerPutNums() {
         this.statsTable.get(BROKER_PUT_NUMS).getAndCreateStatsItem(this.clusterName).getValue().add(1);
     }
@@ -283,6 +507,14 @@ public class BrokerStatsManager {
         this.statsTable.get(BROKER_GET_NUMS).getAndCreateStatsItem(this.clusterName).getValue().add(incValue);
     }
 
+    public void incBrokerAckNums(final int incValue) {
+        this.statsTable.get(BROKER_ACK_NUMS).getAndCreateStatsItem(this.clusterName).getValue().add(incValue);
+    }
+
+    public void incBrokerCkNums(final int incValue) {
+        this.statsTable.get(BROKER_CK_NUMS).getAndCreateStatsItem(this.clusterName).getValue().add(incValue);
+    }
+
     public void incSendBackNums(final String group, final String topic) {
         final String statsKey = buildStatsKey(topic, group);
         this.statsTable.get(SNDBCK_PUT_NUMS).addValue(statsKey, 1, 1);
@@ -305,12 +537,48 @@ public class BrokerStatsManager {
         this.momentStatsItemSetFallSize.getAndCreateStatsItem(statsKey).getValue().set(fallBehind);
     }
 
+    public void incDLQStatValue(final String key, final String owner, final String group,
+        final String topic, final String type, final int incValue) {
+        final String statsKey = buildCommercialStatsKey(owner, topic, group, type);
+        this.statsTable.get(key).addValue(statsKey, incValue, 1);
+    }
+
     public void incCommercialValue(final String key, final String owner, final String group,
         final String topic, final String type, final int incValue) {
         final String statsKey = buildCommercialStatsKey(owner, topic, group, type);
         this.statsTable.get(key).addValue(statsKey, incValue, 1);
     }
 
+    public void incAccountValue(final String key, final String accountOwnerParent, final String accountOwnerSelf,
+        final String instanceId, final String group, final String topic,
+        final String msgType, final int incValue) {
+        final String statsKey = buildAccountStatsKey(accountOwnerParent, accountOwnerSelf, instanceId, topic, group,
+            msgType);
+        this.statsTable.get(key).addValue(statsKey, incValue, 1);
+    }
+
+    public void incAccountValue(final String key, final String accountOwnerParent, final String accountOwnerSelf,
+        final String instanceId, final String group, final String topic,
+        final String msgType, final String flowlimitThreshold, final int incValue) {
+        final String statsKey = buildAccountStatsKey(accountOwnerParent, accountOwnerSelf, instanceId, topic, group,
+            msgType, flowlimitThreshold);
+        this.statsTable.get(key).addValue(statsKey, incValue, 1);
+    }
+
+    public void incAccountValue(final String statType, final String owner, final String instanceId, final String topic,
+        final String group, final String msgType,
+        final long... incValues) {
+        final String key = buildAccountStatKey(owner, instanceId, topic, group, msgType);
+        this.accountStatManager.inc(statType, key, incValues);
+    }
+
+    public void incAccountValue(final String statType, final String owner, final String instanceId, final String topic,
+        final String group, final String msgType, final String flowlimitThreshold,
+        final long... incValues) {
+        final String key = buildAccountStatKey(owner, instanceId, topic, group, msgType, flowlimitThreshold);
+        this.accountStatManager.inc(statType, key, incValues);
+    }
+
     public String buildCommercialStatsKey(String owner, String topic, String group, String type) {
         StringBuilder strBuilder = new StringBuilder();
         strBuilder.append(owner);
@@ -323,14 +591,131 @@ public class BrokerStatsManager {
         return strBuilder.toString();
     }
 
+    public String buildAccountStatsKey(String accountOwnerParent, String accountOwnerSelf, String instanceId,
+        String topic, String group, String msgType) {
+        StringBuffer strBuilder = new StringBuffer();
+        strBuilder.append(accountOwnerParent);
+        strBuilder.append("@");
+        strBuilder.append(accountOwnerSelf);
+        strBuilder.append("@");
+        strBuilder.append(instanceId);
+        strBuilder.append("@");
+        strBuilder.append(topic);
+        strBuilder.append("@");
+        strBuilder.append(group);
+        strBuilder.append("@");
+        strBuilder.append(msgType);
+        return strBuilder.toString();
+    }
+
+    public String buildAccountStatsKey(String accountOwnerParent, String accountOwnerSelf, String instanceId,
+        String topic, String group, String msgType, String flowlimitThreshold) {
+        StringBuffer strBuilder = new StringBuffer();
+        strBuilder.append(accountOwnerParent);
+        strBuilder.append("@");
+        strBuilder.append(accountOwnerSelf);
+        strBuilder.append("@");
+        strBuilder.append(instanceId);
+        strBuilder.append("@");
+        strBuilder.append(topic);
+        strBuilder.append("@");
+        strBuilder.append(group);
+        strBuilder.append("@");
+        strBuilder.append(msgType);
+        strBuilder.append("@");
+        strBuilder.append(flowlimitThreshold);
+        return strBuilder.toString();
+    }
+
+    public String buildAccountStatKey(final String owner, final String instanceId,
+        final String topic, final String group,
+        final String msgType) {
+        final String sep = "|";
+        StringBuffer strBuilder = new StringBuffer();
+        strBuilder.append(owner).append(sep);
+        strBuilder.append(instanceId).append(sep);
+        strBuilder.append(topic).append(sep);
+        strBuilder.append(group).append(sep);
+        strBuilder.append(msgType);
+        return strBuilder.toString();
+    }
+
+    public String buildAccountStatKey(final String owner, final String instanceId,
+        final String topic, final String group,
+        final String msgType, String flowlimitThreshold) {
+        final String sep = "|";
+        StringBuffer strBuilder = new StringBuffer();
+        strBuilder.append(owner).append(sep);
+        strBuilder.append(instanceId).append(sep);
+        strBuilder.append(topic).append(sep);
+        strBuilder.append(group).append(sep);
+        strBuilder.append(msgType).append(sep);
+        strBuilder.append(flowlimitThreshold);
+        return strBuilder.toString();
+    }
+
+    public String[] splitAccountStatKey(final String accountStatKey) {
+        final String sep = "\\|";
+        return accountStatKey.split(sep);
+    }
+
+    private StatisticsKindMeta createStatisticsKindMeta(String name,
+        String[] itemNames,
+        ScheduledExecutorService executorService,
+        StatisticsItemFormatter formatter,
+        InternalLogger log,
+        long interval) {
+        final BrokerConfig brokerConfig = this.brokerConfig;
+        StatisticsItemPrinter printer = new StatisticsItemPrinter(formatter, log);
+        StatisticsKindMeta kindMeta = new StatisticsKindMeta();
+        kindMeta.setName(name);
+        kindMeta.setItemNames(itemNames);
+        kindMeta.setScheduledPrinter(
+            new StatisticsItemScheduledIncrementPrinter(
+                "Stat In One Minute: ",
+                printer,
+                executorService,
+                new StatisticsItemScheduledPrinter.InitialDelay() {
+                    @Override
+                    public long get() {
+                        return Math.abs(UtilAll.computeNextMinutesTimeMillis() - System.currentTimeMillis());
+                    }
+                },
+                interval,
+                new String[] {MSG_NUM},
+                new StatisticsItemScheduledIncrementPrinter.Valve() {
+                    @Override
+                    public boolean enabled() {
+                        return brokerConfig != null ? brokerConfig.isAccountStatsEnable() : true;
+                    }
+
+                    @Override
+                    public boolean printZeroLine() {
+                        return brokerConfig != null ? brokerConfig.isAccountStatsPrintZeroValues() : true;
+                    }
+                }
+            )
+        );
+        return kindMeta;
+    }
+
+    public interface StateGetter {
+        boolean online(String instanceId, String group, String topic);
+    }
+
     public enum StatsType {
         SEND_SUCCESS,
         SEND_FAILURE,
+
+        RCV_SUCCESS,
+        RCV_EPOLLS,
         SEND_BACK,
+        SEND_BACK_TO_DLQ,
+
+        SEND_ORDER,
         SEND_TIMER,
         SEND_TRANSACTION,
-        RCV_SUCCESS,
-        RCV_EPOLLS,
+
         PERM_FAILURE
     }
 }
diff --git a/store/src/test/java/org/apache/rocketmq/store/AppendCallbackTest.java b/store/src/test/java/org/apache/rocketmq/store/AppendCallbackTest.java
index 6e666b0..d299a47 100644
--- a/store/src/test/java/org/apache/rocketmq/store/AppendCallbackTest.java
+++ b/store/src/test/java/org/apache/rocketmq/store/AppendCallbackTest.java
@@ -25,10 +25,12 @@ import java.util.HashSet;
 import java.util.List;
 import java.util.Set;
 
+import org.apache.rocketmq.common.BrokerConfig;
 import org.apache.rocketmq.common.UtilAll;
 import org.apache.rocketmq.common.message.Message;
 import org.apache.rocketmq.common.message.MessageDecoder;
 import org.apache.rocketmq.common.message.MessageExt;
+import org.apache.rocketmq.common.message.MessageExtBatch;
 import org.apache.rocketmq.store.CommitLog.MessageExtEncoder;
 import org.apache.rocketmq.store.config.MessageStoreConfig;
 import org.junit.After;
@@ -54,7 +56,7 @@ public class AppendCallbackTest {
         messageStoreConfig.setStorePathRootDir(System.getProperty("user.home") + File.separator + "unitteststore");
         messageStoreConfig.setStorePathCommitLog(System.getProperty("user.home") + File.separator + "unitteststore" + File.separator + "commitlog");
         //too much reference
-        DefaultMessageStore messageStore = new DefaultMessageStore(messageStoreConfig, null, null, null);
+        DefaultMessageStore messageStore = new DefaultMessageStore(messageStoreConfig, null, null, new BrokerConfig());
         CommitLog commitLog = new CommitLog(messageStore);
         callback = commitLog.new DefaultAppendMessageCallback(1024);
     }
diff --git a/store/src/test/java/org/apache/rocketmq/store/BatchPutMessageTest.java b/store/src/test/java/org/apache/rocketmq/store/BatchPutMessageTest.java
index dfd5573..a9cd6c6 100644
--- a/store/src/test/java/org/apache/rocketmq/store/BatchPutMessageTest.java
+++ b/store/src/test/java/org/apache/rocketmq/store/BatchPutMessageTest.java
@@ -22,6 +22,7 @@ import org.apache.rocketmq.common.UtilAll;
 import org.apache.rocketmq.common.message.Message;
 import org.apache.rocketmq.common.message.MessageDecoder;
 import org.apache.rocketmq.common.message.MessageExt;
+import org.apache.rocketmq.common.message.MessageExtBatch;
 import org.apache.rocketmq.store.config.FlushDiskType;
 import org.apache.rocketmq.store.config.MessageStoreConfig;
 import org.apache.rocketmq.store.stats.BrokerStatsManager;
@@ -104,7 +105,7 @@ public class BatchPutMessageTest {
             short propertiesLength = (short) propertiesBytes.length;
             final byte[] topicData = msg.getTopic().getBytes(MessageDecoder.CHARSET_UTF8);
             final int topicLength = topicData.length;
-            msgLengthArr[j] = calMsgLength(msg.getBody().length, topicLength, propertiesLength+batchPropLen+1) + msgLengthArr[j - 1];
+            msgLengthArr[j] = calMsgLength(msg.getBody().length, topicLength, propertiesLength + batchPropLen + 1) + msgLengthArr[j - 1];
             j++;
         }
         byte[] batchMessageBody = MessageDecoder.encodeMessages(messages);
@@ -112,14 +113,14 @@ public class BatchPutMessageTest {
         messageExtBatch.setTopic(topic);
         messageExtBatch.setQueueId(queue);
         messageExtBatch.setBody(batchMessageBody);
-        messageExtBatch.putUserProperty(batchPropK,batchPropV);
+        messageExtBatch.putUserProperty(batchPropK, batchPropV);
         messageExtBatch.setBornTimestamp(System.currentTimeMillis());
         messageExtBatch.setStoreHost(new InetSocketAddress("127.0.0.1", 125));
         messageExtBatch.setBornHost(new InetSocketAddress("127.0.0.1", 126));
 
         PutMessageResult putMessageResult = messageStore.putMessages(messageExtBatch);
         assertThat(putMessageResult.isOk()).isTrue();
-        
+
         Thread.sleep(3 * 1000);
 
         for (long i = 0; i < 10; i++) {
@@ -187,23 +188,23 @@ public class BatchPutMessageTest {
 
     private int calMsgLength(int bodyLength, int topicLength, int propertiesLength) {
         final int msgLen = 4 //TOTALSIZE
-                + 4 //MAGICCODE
-                + 4 //BODYCRC
-                + 4 //QUEUEID
-                + 4 //FLAG
-                + 8 //QUEUEOFFSET
-                + 8 //PHYSICALOFFSET
-                + 4 //SYSFLAG
-                + 8 //BORNTIMESTAMP
-                + 8 //BORNHOST
-                + 8 //STORETIMESTAMP
-                + 8 //STOREHOSTADDRESS
-                + 4 //RECONSUMETIMES
-                + 8 //Prepared Transaction Offset
-                + 4 + (bodyLength > 0 ? bodyLength : 0) //BODY
-                + 1 + topicLength //TOPIC
-                + 2 + (propertiesLength > 0 ? propertiesLength : 0) //propertiesLength
-                + 0;
+            + 4 //MAGICCODE
+            + 4 //BODYCRC
+            + 4 //QUEUEID
+            + 4 //FLAG
+            + 8 //QUEUEOFFSET
+            + 8 //PHYSICALOFFSET
+            + 4 //SYSFLAG
+            + 8 //BORNTIMESTAMP
+            + 8 //BORNHOST
+            + 8 //STORETIMESTAMP
+            + 8 //STOREHOSTADDRESS
+            + 4 //RECONSUMETIMES
+            + 8 //Prepared Transaction Offset
+            + 4 + (bodyLength > 0 ? bodyLength : 0) //BODY
+            + 1 + topicLength //TOPIC
+            + 2 + (propertiesLength > 0 ? propertiesLength : 0) //propertiesLength
+            + 0;
         return msgLen;
     }
 
@@ -232,7 +233,7 @@ public class BatchPutMessageTest {
     private class MyMessageArrivingListener implements MessageArrivingListener {
         @Override
         public void arriving(String topic, int queueId, long logicOffset, long tagsCode, long msgStoreTime,
-                             byte[] filterBitMap, Map<String, String> properties) {
+            byte[] filterBitMap, Map<String, String> properties) {
         }
     }
 }
diff --git a/store/src/test/java/org/apache/rocketmq/store/ConsumeQueueTest.java b/store/src/test/java/org/apache/rocketmq/store/ConsumeQueueTest.java
index 998ed70..97afee8 100644
--- a/store/src/test/java/org/apache/rocketmq/store/ConsumeQueueTest.java
+++ b/store/src/test/java/org/apache/rocketmq/store/ConsumeQueueTest.java
@@ -28,6 +28,7 @@ import org.apache.rocketmq.common.BrokerConfig;
 import org.apache.rocketmq.common.UtilAll;
 import org.apache.rocketmq.common.message.MessageConst;
 import org.apache.rocketmq.common.message.MessageDecoder;
+import org.apache.rocketmq.common.message.MessageExtBrokerInner;
 import org.apache.rocketmq.store.config.MessageStoreConfig;
 import org.apache.rocketmq.store.queue.ConsumeQueueInterface;
 import org.apache.rocketmq.store.queue.CqUnit;
diff --git a/store/src/test/java/org/apache/rocketmq/store/DefaultMessageStoreCleanFilesTest.java b/store/src/test/java/org/apache/rocketmq/store/DefaultMessageStoreCleanFilesTest.java
index 2636d40..2485b2a 100644
--- a/store/src/test/java/org/apache/rocketmq/store/DefaultMessageStoreCleanFilesTest.java
+++ b/store/src/test/java/org/apache/rocketmq/store/DefaultMessageStoreCleanFilesTest.java
@@ -18,9 +18,11 @@
 package org.apache.rocketmq.store;
 
 import org.apache.rocketmq.common.BrokerConfig;
+import org.apache.rocketmq.common.MixAll;
 import org.apache.rocketmq.common.UtilAll;
 import org.apache.rocketmq.common.message.MessageConst;
 import org.apache.rocketmq.common.message.MessageDecoder;
+import org.apache.rocketmq.common.message.MessageExtBrokerInner;
 import org.apache.rocketmq.store.config.FlushDiskType;
 import org.apache.rocketmq.store.config.MessageStoreConfig;
 import org.apache.rocketmq.store.index.IndexFile;
@@ -109,10 +111,10 @@ public class DefaultMessageStoreCleanFilesTest {
         String storePath = config.getStorePathCommitLog();
         StringBuilder storePathBuilder = new StringBuilder();
         for (int i = 0; i < 3; i++) {
-            storePathBuilder.append(storePath).append(i).append(MessageStoreConfig.MULTI_PATH_SPLITTER);
+            storePathBuilder.append(storePath).append(i).append(MixAll.MULTI_PATH_SPLITTER);
         }
         config.setStorePathCommitLog(storePathBuilder.toString());
-        String[] paths = config.getStorePathCommitLog().trim().split(MessageStoreConfig.MULTI_PATH_SPLITTER);
+        String[] paths = config.getStorePathCommitLog().trim().split(MixAll.MULTI_PATH_SPLITTER);
         assertEquals(3, paths.length);
         initMessageStore(config, diskSpaceCleanForciblyRatio);
 
diff --git a/store/src/test/java/org/apache/rocketmq/store/DefaultMessageStoreTest.java b/store/src/test/java/org/apache/rocketmq/store/DefaultMessageStoreTest.java
index 2b13c5b..1e79820 100644
--- a/store/src/test/java/org/apache/rocketmq/store/DefaultMessageStoreTest.java
+++ b/store/src/test/java/org/apache/rocketmq/store/DefaultMessageStoreTest.java
@@ -28,18 +28,25 @@ import java.net.UnknownHostException;
 import java.nio.MappedByteBuffer;
 import java.nio.channels.FileChannel;
 import java.nio.channels.OverlappingFileLockException;
+import java.util.ArrayList;
+import java.util.Arrays;
+import java.util.List;
 import java.util.Map;
 import java.util.Random;
+import java.util.concurrent.TimeUnit;
 import java.util.concurrent.atomic.AtomicInteger;
 import org.apache.rocketmq.common.BrokerConfig;
 import org.apache.rocketmq.common.UtilAll;
-import org.apache.rocketmq.common.message.MessageConst;
+import org.apache.rocketmq.common.message.MessageBatch;
+import org.apache.rocketmq.common.message.MessageDecoder;
 import org.apache.rocketmq.common.message.MessageExt;
+import org.apache.rocketmq.common.message.MessageExtBatch;
+import org.apache.rocketmq.common.message.MessageExtBrokerInner;
+import org.apache.rocketmq.store.config.BrokerRole;
 import org.apache.rocketmq.store.config.FlushDiskType;
 import org.apache.rocketmq.store.config.MessageStoreConfig;
 import org.apache.rocketmq.store.config.StorePathConfigHelper;
 import org.apache.rocketmq.store.logfile.DefaultMappedFile;
-import org.apache.rocketmq.common.attribute.CQType;
 import org.apache.rocketmq.store.queue.ConsumeQueueInterface;
 import org.apache.rocketmq.store.queue.CqUnit;
 import org.apache.rocketmq.store.stats.BrokerStatsManager;
@@ -54,18 +61,19 @@ import static org.junit.Assert.assertTrue;
 
 @RunWith(MockitoJUnitRunner.class)
 public class DefaultMessageStoreTest {
-    private final String StoreMessage = "Once, there was a chance for me!";
+    private final String storeMessage = "Once, there was a chance for me!";
+    private final String messageTopic = "FooBar";
     private int QUEUE_TOTAL = 100;
-    private AtomicInteger QueueId = new AtomicInteger(0);
-    private SocketAddress BornHost;
-    private SocketAddress StoreHost;
-    private byte[] MessageBody;
+    private AtomicInteger queueId = new AtomicInteger(0);
+    private SocketAddress bornHost;
+    private SocketAddress storeHost;
+    private byte[] messageBody;
     private MessageStore messageStore;
 
     @Before
     public void init() throws Exception {
-        StoreHost = new InetSocketAddress(InetAddress.getLocalHost(), 8123);
-        BornHost = new InetSocketAddress(InetAddress.getByName("127.0.0.1"), 0);
+        storeHost = new InetSocketAddress(InetAddress.getLocalHost(), 8123);
+        bornHost = new InetSocketAddress(InetAddress.getByName("127.0.0.1"), 0);
 
         messageStore = buildMessageStore();
         boolean load = messageStore.load();
@@ -76,7 +84,7 @@ public class DefaultMessageStoreTest {
     @Test(expected = OverlappingFileLockException.class)
     public void test_repeat_restart() throws Exception {
         QUEUE_TOTAL = 1;
-        MessageBody = StoreMessage.getBytes();
+        messageBody = storeMessage.getBytes();
 
         MessageStoreConfig messageStoreConfig = new MessageStoreConfig();
         messageStoreConfig.setMappedFileSizeCommitLog(1024 * 8);
@@ -125,7 +133,7 @@ public class DefaultMessageStoreTest {
         long ipv6HostMsgs = 10;
         long totalMsgs = ipv4HostMsgs + ipv6HostMsgs;
         QUEUE_TOTAL = 1;
-        MessageBody = StoreMessage.getBytes();
+        messageBody = storeMessage.getBytes();
         for (long i = 0; i < ipv4HostMsgs; i++) {
             messageStore.putMessage(buildMessage());
         }
@@ -156,8 +164,8 @@ public class DefaultMessageStoreTest {
         MessageExt messageExt = messageStore.lookMessageByOffset(firstResult.getWroteOffset());
         MessageExt messageExt1 = getDefaultMessageStore().lookMessageByOffset(firstResult.getWroteOffset(), firstResult.getWroteBytes());
 
-        assertThat(new String(messageExt.getBody())).isEqualTo(buildMessageBodyByOffset(StoreMessage, firstOffset));
-        assertThat(new String(messageExt1.getBody())).isEqualTo(buildMessageBodyByOffset(StoreMessage, firstOffset));
+        assertThat(new String(messageExt.getBody())).isEqualTo(buildMessageBodyByOffset(storeMessage, firstOffset));
+        assertThat(new String(messageExt1.getBody())).isEqualTo(buildMessageBodyByOffset(storeMessage, firstOffset));
     }
 
     @Test
@@ -171,7 +179,7 @@ public class DefaultMessageStoreTest {
 
         MessageExt messageExt = getDefaultMessageStore().lookMessageByOffset(lastResult.getWroteOffset(), lastResult.getWroteBytes());
 
-        assertThat(new String(messageExt.getBody())).isEqualTo(buildMessageBodyByOffset(StoreMessage, lastIndex));
+        assertThat(new String(messageExt.getBody())).isEqualTo(buildMessageBodyByOffset(storeMessage, lastIndex));
     }
 
     @Test
@@ -360,7 +368,7 @@ public class DefaultMessageStoreTest {
     private AppendMessageResult[] putMessages(int totalCount, String topic, int queueId, boolean interval) {
         AppendMessageResult[] appendMessageResultArray = new AppendMessageResult[totalCount];
         for (int i = 0; i < totalCount; i++) {
-            String messageBody = buildMessageBodyByOffset(StoreMessage, i);
+            String messageBody = buildMessageBodyByOffset(storeMessage, i);
 
             MessageExtBrokerInner msgInner =
                 i < totalCount / 2 ? buildMessage(messageBody.getBytes(), topic) : buildIPv6HostMessage(messageBody.getBytes(), topic);
@@ -408,11 +416,12 @@ public class DefaultMessageStoreTest {
         msg.setKeys("Hello");
         msg.setBody(messageBody);
         msg.setKeys(String.valueOf(System.currentTimeMillis()));
-        msg.setQueueId(Math.abs(QueueId.getAndIncrement()) % QUEUE_TOTAL);
+        msg.setQueueId(Math.abs(queueId.getAndIncrement()) % QUEUE_TOTAL);
         msg.setSysFlag(0);
         msg.setBornTimestamp(System.currentTimeMillis());
-        msg.setStoreHost(StoreHost);
-        msg.setBornHost(BornHost);
+        msg.setStoreHost(storeHost);
+        msg.setBornHost(bornHost);
+        msg.setPropertiesString(MessageDecoder.messageProperties2String(msg.getProperties()));
         return msg;
     }
 
@@ -424,7 +433,7 @@ public class DefaultMessageStoreTest {
         msg.setBody(messageBody);
         msg.setMsgId("24084004018081003FAA1DDE2B3F898A00002A9F0000000000000CA0");
         msg.setKeys(String.valueOf(System.currentTimeMillis()));
-        msg.setQueueId(Math.abs(QueueId.getAndIncrement()) % QUEUE_TOTAL);
+        msg.setQueueId(Math.abs(queueId.getAndIncrement()) % QUEUE_TOTAL);
         msg.setSysFlag(0);
         msg.setBornHostV6Flag();
         msg.setStoreHostAddressV6Flag();
@@ -446,11 +455,79 @@ public class DefaultMessageStoreTest {
     }
 
     private MessageExtBrokerInner buildMessage() {
-        return buildMessage(MessageBody, "FooBar");
+        return buildMessage(messageBody, messageTopic);
+    }
+
+    public MessageExtBatch buildMessageBatch(MessageBatch msgBatch) {
+        MessageExtBatch msgExtBatch = new MessageExtBatch();
+        msgExtBatch.setTopic(messageTopic);
+        msgExtBatch.setTags("TAG1");
+        msgExtBatch.setKeys("Hello");
+        msgExtBatch.setBody(msgBatch.getBody());
+        msgExtBatch.setKeys(String.valueOf(System.currentTimeMillis()));
+        msgExtBatch.setQueueId(Math.abs(queueId.getAndIncrement()) % QUEUE_TOTAL);
+        msgExtBatch.setSysFlag(0);
+        msgExtBatch.setBornTimestamp(System.currentTimeMillis());
+        msgExtBatch.setStoreHost(storeHost);
+        msgExtBatch.setBornHost(bornHost);
+        return msgExtBatch;
+    }
+
+    @Test
+    public void testGroupCommit() throws Exception {
+        long totalMsgs = 10;
+        QUEUE_TOTAL = 1;
+        messageBody = storeMessage.getBytes();
+        for (long i = 0; i < totalMsgs; i++) {
+            messageStore.putMessage(buildMessage());
+        }
+
+        for (long i = 0; i < totalMsgs; i++) {
+            GetMessageResult result = messageStore.getMessage("GROUP_A", "TOPIC_A", 0, i, 1024 * 1024, null);
+            assertThat(result).isNotNull();
+            result.release();
+        }
+        verifyThatMasterIsFunctional(totalMsgs, messageStore);
+    }
+
+    @Test
+    public void testMaxOffset() throws InterruptedException {
+        int firstBatchMessages = 3;
+        int queueId = 0;
+        messageBody = storeMessage.getBytes();
+
+        assertThat(messageStore.getMaxOffsetInQueue(messageTopic, queueId)).isEqualTo(0);
+
+        for (int i = 0; i < firstBatchMessages; i++) {
+            final MessageExtBrokerInner msg = buildMessage();
+            msg.setQueueId(queueId);
+            messageStore.putMessage(msg);
+        }
+
+        while (messageStore.dispatchBehindBytes() != 0) {
+            TimeUnit.MILLISECONDS.sleep(1);
+        }
+
+        assertThat(messageStore.getMaxOffsetInQueue(messageTopic, queueId)).isEqualTo(firstBatchMessages);
+
+        // Disable the dispatcher
+        messageStore.getDispatcherList().clear();
+
+        int secondBatchMessages = 2;
+
+        for (int i = 0; i < secondBatchMessages; i++) {
+            final MessageExtBrokerInner msg = buildMessage();
+            msg.setQueueId(queueId);
+            messageStore.putMessage(msg);
+        }
+
+        assertThat(messageStore.getMaxOffsetInQueue(messageTopic, queueId)).isEqualTo(firstBatchMessages);
+        assertThat(messageStore.getMaxOffsetInQueue(messageTopic, queueId, true)).isEqualTo(firstBatchMessages);
+        assertThat(messageStore.getMaxOffsetInQueue(messageTopic, queueId, false)).isEqualTo(firstBatchMessages + secondBatchMessages);
     }
 
     private MessageExtBrokerInner buildIPv6HostMessage() {
-        return buildIPv6HostMessage(MessageBody, "FooBar");
+        return buildIPv6HostMessage(messageBody, "FooBar");
     }
 
     private void verifyThatMasterIsFunctional(long totalMsgs, MessageStore master) {
@@ -500,7 +577,7 @@ public class DefaultMessageStoreTest {
     @Test
     public void testRecover() throws Exception {
         String topic = "recoverTopic";
-        MessageBody = StoreMessage.getBytes();
+        messageBody = storeMessage.getBytes();
         for (int i = 0; i < 100; i++) {
             MessageExtBrokerInner messageExtBrokerInner = buildMessage();
             messageExtBrokerInner.setTopic(topic);
@@ -576,7 +653,7 @@ public class DefaultMessageStoreTest {
         //add abort file
         String fileName = StorePathConfigHelper.getAbortFile(((DefaultMessageStore) messageStore).getMessageStoreConfig().getStorePathRootDir());
         File file = new File(fileName);
-        DefaultMappedFile.ensureDirOK(file.getParent());
+        UtilAll.ensureDirOK(file.getParent());
         file.createNewFile();
 
         messageStore = buildMessageStore();
@@ -631,6 +708,71 @@ public class DefaultMessageStoreTest {
         fileChannel.close();
     }
 
+    @Test
+    public void testPutMsgExceedsMaxLength() {
+        messageBody = new byte[4 * 1024 * 1024 + 1];
+        MessageExtBrokerInner msg = buildMessage();
+
+        PutMessageResult result = messageStore.putMessage(msg);
+        assertThat(result.getPutMessageStatus()).isEqualTo(PutMessageStatus.MESSAGE_ILLEGAL);
+    }
+
+    @Test
+    public void testPutMsgBatchExceedsMaxLength() {
+        messageBody = new byte[4 * 1024 * 1024 + 1];
+        MessageExtBrokerInner msg1 = buildMessage();
+        MessageExtBrokerInner msg2 = buildMessage();
+        MessageExtBrokerInner msg3 = buildMessage();
+
+        MessageBatch msgBatch = MessageBatch.generateFromList(Arrays.asList(msg1, msg2, msg3));
+        msgBatch.setBody(msgBatch.encode());
+
+        MessageExtBatch msgExtBatch = buildMessageBatch(msgBatch);
+
+        try {
+            PutMessageResult result = this.messageStore.putMessages(msgExtBatch);
+        } catch (Exception e) {
+            assertThat(e.getMessage()).contains("message size exceeded");
+        }
+    }
+
+    @Test
+    public void testPutMsgWhenReplicasNotEnough() {
+        MessageStoreConfig messageStoreConfig = ((DefaultMessageStore) this.messageStore).getMessageStoreConfig();
+        messageStoreConfig.setBrokerRole(BrokerRole.SYNC_MASTER);
+        messageStoreConfig.setTotalReplicas(2);
+        messageStoreConfig.setInSyncReplicas(2);
+        messageStoreConfig.setEnableAutoInSyncReplicas(false);
+        this.messageStore.setAliveReplicaNumInGroup(1);
+
+        MessageExtBrokerInner msg = buildMessage();
+        PutMessageResult result = this.messageStore.putMessage(msg);
+        assertThat(result.getPutMessageStatus()).isEqualTo(PutMessageStatus.IN_SYNC_REPLICAS_NOT_ENOUGH);
+    }
+
+
+    @Test
+    public void testGetBulkCommitLogData() {
+        DefaultMessageStore defaultMessageStore = (DefaultMessageStore) messageStore;
+
+        messageBody = new byte[2 * 1024 * 1024];
+
+        for (int i = 0; i < 10; i++) {
+            MessageExtBrokerInner msg1 = buildMessage();
+            messageStore.putMessage(msg1);
+        }
+
+        System.out.printf("%d%n", defaultMessageStore.getMaxPhyOffset());
+
+        List<SelectMappedBufferResult> bufferResultList = defaultMessageStore.getBulkCommitLogData(0, (int) defaultMessageStore.getMaxPhyOffset());
+        List<MessageExt> msgList = new ArrayList<>();
+        for (SelectMappedBufferResult bufferResult : bufferResultList) {
+            msgList.addAll(MessageDecoder.decodesBatch(bufferResult.getByteBuffer(), true, false, false));
+            bufferResult.release();
+        }
+
+        assertThat(msgList.size()).isEqualTo(10);
+    }
 
     private class MyMessageArrivingListener implements MessageArrivingListener {
         @Override
diff --git a/store/src/test/java/org/apache/rocketmq/store/HATest.java b/store/src/test/java/org/apache/rocketmq/store/HATest.java
index c82a237..9ef960f 100644
--- a/store/src/test/java/org/apache/rocketmq/store/HATest.java
+++ b/store/src/test/java/org/apache/rocketmq/store/HATest.java
@@ -1,18 +1,14 @@
 /**
- * Licensed to the Apache Software Foundation (ASF) under one or more
- * contributor license agreements.  See the NOTICE file distributed with
- * this work for additional information regarding copyright ownership.
- * The ASF licenses this file to You under the Apache License, Version 2.0
- * (the "License"); you may not use this file except in compliance with
- * the License.  You may obtain a copy of the License at
+ * Licensed to the Apache Software Foundation (ASF) under one or more contributor license agreements.  See the NOTICE
+ * file distributed with this work for additional information regarding copyright ownership. The ASF licenses this file
+ * to You under the Apache License, Version 2.0 (the "License"); you may not use this file except in compliance with the
+ * License.  You may obtain a copy of the License at
  *
  * http://www.apache.org/licenses/LICENSE-2.0
  *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
+ * Unless required by applicable law or agreed to in writing, software distributed under the License is distributed on
+ * an "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. See the License for the
+ * specific language governing permissions and limitations under the License.
  */
 
 package org.apache.rocketmq.store;
@@ -21,6 +17,7 @@ import org.apache.rocketmq.common.BrokerConfig;
 import org.apache.rocketmq.common.UtilAll;
 import org.apache.rocketmq.common.message.MessageDecoder;
 import org.apache.rocketmq.common.message.MessageExt;
+import org.apache.rocketmq.common.message.MessageExtBrokerInner;
 import org.apache.rocketmq.store.config.BrokerRole;
 import org.apache.rocketmq.store.config.FlushDiskType;
 import org.apache.rocketmq.store.config.MessageStoreConfig;
@@ -62,25 +59,30 @@ public class HATest {
     private MessageStoreConfig slaveStoreConfig;
     private BrokerStatsManager brokerStatsManager = new BrokerStatsManager("simpleTest", true);
     private String storePathRootParentDir = System.getProperty("user.home") + File.separator +
-            UUID.randomUUID().toString().replace("-", "");
+        UUID.randomUUID().toString().replace("-", "");
     private String storePathRootDir = storePathRootParentDir + File.separator + "store";
+
     @Before
     public void init() throws Exception {
         StoreHost = new InetSocketAddress(InetAddress.getLocalHost(), 8123);
         BornHost = new InetSocketAddress(InetAddress.getByName("127.0.0.1"), 0);
         masterMessageStoreConfig = new MessageStoreConfig();
         masterMessageStoreConfig.setBrokerRole(BrokerRole.SYNC_MASTER);
-        masterMessageStoreConfig.setStorePathRootDir(storePathRootDir+File.separator+"master");
-        masterMessageStoreConfig.setStorePathCommitLog(storePathRootDir+File.separator+"master"+ File.separator+"commitlog");
+        masterMessageStoreConfig.setStorePathRootDir(storePathRootDir + File.separator + "master");
+        masterMessageStoreConfig.setStorePathCommitLog(storePathRootDir + File.separator + "master" + File.separator + "commitlog");
+        masterMessageStoreConfig.setTotalReplicas(2);
+        masterMessageStoreConfig.setInSyncReplicas(2);
         buildMessageStoreConfig(masterMessageStoreConfig);
         slaveStoreConfig = new MessageStoreConfig();
         slaveStoreConfig.setBrokerRole(BrokerRole.SLAVE);
-        slaveStoreConfig.setStorePathRootDir(storePathRootDir+File.separator+"slave");
-        slaveStoreConfig.setStorePathCommitLog(storePathRootDir+File.separator+"slave"+ File.separator+"commitlog");
+        slaveStoreConfig.setStorePathRootDir(storePathRootDir + File.separator + "slave");
+        slaveStoreConfig.setStorePathCommitLog(storePathRootDir + File.separator + "slave" + File.separator + "commitlog");
         slaveStoreConfig.setHaListenPort(10943);
+        slaveStoreConfig.setTotalReplicas(2);
+        slaveStoreConfig.setInSyncReplicas(2);
         buildMessageStoreConfig(slaveStoreConfig);
-        messageStore = buildMessageStore(masterMessageStoreConfig,0L);
-        slaveMessageStore = buildMessageStore(slaveStoreConfig,1L);
+        messageStore = buildMessageStore(masterMessageStoreConfig, 0L);
+        slaveMessageStore = buildMessageStore(slaveStoreConfig, 1L);
         boolean load = messageStore.load();
         boolean slaveLoad = slaveMessageStore.load();
         slaveMessageStore.updateHaMasterAddress("127.0.0.1:10912");
@@ -144,17 +146,19 @@ public class HATest {
 
         //shutdown slave, putMessage should return FLUSH_SLAVE_TIMEOUT
         slaveMessageStore.shutdown();
+        messageStore.setAliveReplicaNumInGroup(1);
+
         //wait to let master clean the slave's connection
         Thread.sleep(masterMessageStoreConfig.getHaHousekeepingInterval() + 500);
         for (long i = 0; i < totalMsgs; i++) {
             CompletableFuture<PutMessageResult> putResultFuture = messageStore.asyncPutMessage(buildMessage());
             PutMessageResult result = putResultFuture.get();
-            assertEquals(PutMessageStatus.SLAVE_NOT_AVAILABLE, result.getPutMessageStatus());
+            assertEquals(PutMessageStatus.IN_SYNC_REPLICAS_NOT_ENOUGH, result.getPutMessageStatus());
         }
     }
 
     @After
-    public void destroy() throws Exception{
+    public void destroy() throws Exception {
         Thread.sleep(5000L);
         slaveMessageStore.shutdown();
         slaveMessageStore.destroy();
@@ -164,13 +168,13 @@ public class HATest {
         UtilAll.deleteFile(file);
     }
 
-    private MessageStore buildMessageStore(MessageStoreConfig messageStoreConfig,long brokerId) throws Exception {
+    private MessageStore buildMessageStore(MessageStoreConfig messageStoreConfig, long brokerId) throws Exception {
         BrokerConfig brokerConfig = new BrokerConfig();
         brokerConfig.setBrokerId(brokerId);
         return new DefaultMessageStore(messageStoreConfig, brokerStatsManager, null, brokerConfig);
     }
 
-    private void buildMessageStoreConfig(MessageStoreConfig messageStoreConfig){
+    private void buildMessageStoreConfig(MessageStoreConfig messageStoreConfig) {
         messageStoreConfig.setMappedFileSizeCommitLog(1024 * 1024 * 10);
         messageStoreConfig.setMappedFileSizeConsumeQueue(1024 * 1024 * 10);
         messageStoreConfig.setMaxHashSlotNum(10000);
@@ -194,18 +198,17 @@ public class HATest {
         return msg;
     }
 
-    private boolean isCommitLogAvailable(DefaultMessageStore store)  {
+    private boolean isCommitLogAvailable(DefaultMessageStore store) {
         try {
-
             Field serviceField = store.getClass().getDeclaredField("reputMessageService");
             serviceField.setAccessible(true);
             DefaultMessageStore.ReputMessageService reputService =
-                    (DefaultMessageStore.ReputMessageService) serviceField.get(store);
+                (DefaultMessageStore.ReputMessageService) serviceField.get(store);
 
             Method method = DefaultMessageStore.ReputMessageService.class.getDeclaredMethod("isCommitLogAvailable");
             method.setAccessible(true);
             return (boolean) method.invoke(reputService);
-        } catch (NoSuchMethodException | IllegalAccessException | InvocationTargetException |  NoSuchFieldException e ) {
+        } catch (NoSuchMethodException | IllegalAccessException | InvocationTargetException | NoSuchFieldException e) {
             throw new RuntimeException(e);
         }
     }
diff --git a/store/src/test/java/org/apache/rocketmq/store/MultiDispatchTest.java b/store/src/test/java/org/apache/rocketmq/store/MultiDispatchTest.java
index 7f6f23e..627f9dd 100644
--- a/store/src/test/java/org/apache/rocketmq/store/MultiDispatchTest.java
+++ b/store/src/test/java/org/apache/rocketmq/store/MultiDispatchTest.java
@@ -21,9 +21,11 @@ import java.io.File;
 import java.net.InetSocketAddress;
 import java.nio.charset.Charset;
 
+import org.apache.rocketmq.common.BrokerConfig;
 import org.apache.rocketmq.common.UtilAll;
 import org.apache.rocketmq.common.message.MessageConst;
 import org.apache.rocketmq.common.message.MessageDecoder;
+import org.apache.rocketmq.common.message.MessageExtBrokerInner;
 import org.apache.rocketmq.store.config.MessageStoreConfig;
 import org.junit.After;
 import org.junit.Before;
@@ -54,8 +56,9 @@ public class MultiDispatchTest {
 
         messageStoreConfig.setEnableLmq(true);
         messageStoreConfig.setEnableMultiDispatch(true);
+        BrokerConfig brokerConfig = new BrokerConfig();
         //too much reference
-        messageStore = new DefaultMessageStore(messageStoreConfig, null, null, null);
+        messageStore = new DefaultMessageStore(messageStoreConfig, null, null, brokerConfig);
         consumeQueue = new ConsumeQueue("xxx", 0,
             getStorePathConsumeQueue(messageStoreConfig.getStorePathRootDir()), messageStoreConfig.getMappedFileSizeConsumeQueue(), messageStore);
     }
@@ -76,7 +79,7 @@ public class MultiDispatchTest {
     @Test
     public void wrapMultiDispatch() {
         MessageExtBrokerInner messageExtBrokerInner = buildMessageMultiQueue();
-        messageStore.assignOffset("test", messageExtBrokerInner, (short) 1);
+        messageStore.assignOffset( messageExtBrokerInner, (short) 1);
         assertEquals(messageExtBrokerInner.getProperty(MessageConst.PROPERTY_INNER_MULTI_QUEUE_OFFSET), "0,0");
     }
 
diff --git a/store/src/test/java/org/apache/rocketmq/store/MultiPathMappedFileQueueTest.java b/store/src/test/java/org/apache/rocketmq/store/MultiPathMappedFileQueueTest.java
index 330ca56..07037aa 100644
--- a/store/src/test/java/org/apache/rocketmq/store/MultiPathMappedFileQueueTest.java
+++ b/store/src/test/java/org/apache/rocketmq/store/MultiPathMappedFileQueueTest.java
@@ -20,6 +20,7 @@ package org.apache.rocketmq.store;
 import static org.assertj.core.api.Assertions.assertThat;
 import java.util.HashSet;
 import java.util.Set;
+import org.apache.rocketmq.common.MixAll;
 import org.apache.rocketmq.common.UtilAll;
 import org.apache.rocketmq.store.config.MessageStoreConfig;
 import org.apache.rocketmq.store.logfile.MappedFile;
@@ -33,11 +34,11 @@ public class MultiPathMappedFileQueueTest {
         final byte[] fixedMsg = new byte[1024];
 
         MessageStoreConfig config = new MessageStoreConfig();
-        config.setStorePathCommitLog("target/unit_test_store/a/" + MessageStoreConfig.MULTI_PATH_SPLITTER
-                + "target/unit_test_store/b/" + MessageStoreConfig.MULTI_PATH_SPLITTER
+        config.setStorePathCommitLog("target/unit_test_store/a/" + MixAll.MULTI_PATH_SPLITTER
+                + "target/unit_test_store/b/" + MixAll.MULTI_PATH_SPLITTER
                 + "target/unit_test_store/c/");
         MappedFileQueue mappedFileQueue = new MultiPathMappedFileQueue(config, 1024, null, null);
-        String[] storePaths = config.getStorePathCommitLog().trim().split(MessageStoreConfig.MULTI_PATH_SPLITTER);
+        String[] storePaths = config.getStorePathCommitLog().trim().split(MixAll.MULTI_PATH_SPLITTER);
         for (int i = 0; i < 1024; i++) {
             MappedFile mappedFile = mappedFileQueue.getLastMappedFile(fixedMsg.length * i);
             assertThat(mappedFile).isNotNull();
@@ -55,11 +56,11 @@ public class MultiPathMappedFileQueueTest {
             //create old mapped files
             final byte[] fixedMsg = new byte[1024];
             MessageStoreConfig config = new MessageStoreConfig();
-            config.setStorePathCommitLog("target/unit_test_store/a/" + MessageStoreConfig.MULTI_PATH_SPLITTER
-                    + "target/unit_test_store/b/" + MessageStoreConfig.MULTI_PATH_SPLITTER
+            config.setStorePathCommitLog("target/unit_test_store/a/" + MixAll.MULTI_PATH_SPLITTER
+                    + "target/unit_test_store/b/" + MixAll.MULTI_PATH_SPLITTER
                     + "target/unit_test_store/c/");
             MappedFileQueue mappedFileQueue = new MultiPathMappedFileQueue(config, 1024, null, null);
-            String[] storePaths = config.getStorePathCommitLog().trim().split(MessageStoreConfig.MULTI_PATH_SPLITTER);
+            String[] storePaths = config.getStorePathCommitLog().trim().split(MixAll.MULTI_PATH_SPLITTER);
             for (int i = 0; i < 1024; i++) {
                 MappedFile mappedFile = mappedFileQueue.getLastMappedFile(fixedMsg.length * i);
                 assertThat(mappedFile).isNotNull();
@@ -73,7 +74,7 @@ public class MultiPathMappedFileQueueTest {
         // test load and readonly
         MessageStoreConfig config = new MessageStoreConfig();
         config.setStorePathCommitLog("target/unit_test_store/b/");
-        config.setReadOnlyCommitLogStorePaths("target/unit_test_store/a" + MessageStoreConfig.MULTI_PATH_SPLITTER
+        config.setReadOnlyCommitLogStorePaths("target/unit_test_store/a" + MixAll.MULTI_PATH_SPLITTER
                 + "target/unit_test_store/c");
         MultiPathMappedFileQueue mappedFileQueue = new MultiPathMappedFileQueue(config, 1024, null, null);
 
@@ -93,11 +94,11 @@ public class MultiPathMappedFileQueueTest {
         final byte[] fixedMsg = new byte[1024];
 
         MessageStoreConfig config = new MessageStoreConfig();
-        config.setStorePathCommitLog("target/unit_test_store/a/" + MessageStoreConfig.MULTI_PATH_SPLITTER
-                + "target/unit_test_store/b/" + MessageStoreConfig.MULTI_PATH_SPLITTER
+        config.setStorePathCommitLog("target/unit_test_store/a/" + MixAll.MULTI_PATH_SPLITTER
+                + "target/unit_test_store/b/" + MixAll.MULTI_PATH_SPLITTER
                 + "target/unit_test_store/c/");
         MappedFileQueue mappedFileQueue = new MultiPathMappedFileQueue(config, 1024, null, null);
-        String[] storePaths = config.getStorePathCommitLog().trim().split(MessageStoreConfig.MULTI_PATH_SPLITTER);
+        String[] storePaths = config.getStorePathCommitLog().trim().split(MixAll.MULTI_PATH_SPLITTER);
         for (int i = 0; i < 1024; i++) {
             MappedFile mappedFile = mappedFileQueue.getLastMappedFile(fixedMsg.length * i);
             assertThat(mappedFile).isNotNull();
@@ -106,9 +107,9 @@ public class MultiPathMappedFileQueueTest {
             assertThat(mappedFile.getFileName().startsWith(storePaths[idx])).isTrue();
 
             if (i == 500) {
-                config.setStorePathCommitLog("target/unit_test_store/a/" + MessageStoreConfig.MULTI_PATH_SPLITTER
+                config.setStorePathCommitLog("target/unit_test_store/a/" + MixAll.MULTI_PATH_SPLITTER
                         + "target/unit_test_store/b/");
-                storePaths = config.getStorePathCommitLog().trim().split(MessageStoreConfig.MULTI_PATH_SPLITTER);
+                storePaths = config.getStorePathCommitLog().trim().split(MixAll.MULTI_PATH_SPLITTER);
             }
         }
         mappedFileQueue.shutdown(1000);
@@ -121,11 +122,11 @@ public class MultiPathMappedFileQueueTest {
 
         Set<String> fullStorePath = new HashSet<>();
         MessageStoreConfig config = new MessageStoreConfig();
-        config.setStorePathCommitLog("target/unit_test_store/a/" + MessageStoreConfig.MULTI_PATH_SPLITTER
-                + "target/unit_test_store/b/" + MessageStoreConfig.MULTI_PATH_SPLITTER
+        config.setStorePathCommitLog("target/unit_test_store/a/" + MixAll.MULTI_PATH_SPLITTER
+                + "target/unit_test_store/b/" + MixAll.MULTI_PATH_SPLITTER
                 + "target/unit_test_store/c/");
         MappedFileQueue mappedFileQueue = new MultiPathMappedFileQueue(config, 1024, null, () -> fullStorePath);
-        String[] storePaths = config.getStorePathCommitLog().trim().split(MessageStoreConfig.MULTI_PATH_SPLITTER);
+        String[] storePaths = config.getStorePathCommitLog().trim().split(MixAll.MULTI_PATH_SPLITTER);
         assertThat(storePaths.length).isEqualTo(3);
 
         MappedFile mappedFile = mappedFileQueue.getLastMappedFile(0);
diff --git a/store/src/test/java/org/apache/rocketmq/store/StoreStatsServiceTest.java b/store/src/test/java/org/apache/rocketmq/store/StoreStatsServiceTest.java
index 7aa17a2..ed1a0f9 100644
--- a/store/src/test/java/org/apache/rocketmq/store/StoreStatsServiceTest.java
+++ b/store/src/test/java/org/apache/rocketmq/store/StoreStatsServiceTest.java
@@ -19,7 +19,6 @@ package org.apache.rocketmq.store;
 import java.util.concurrent.BrokenBarrierException;
 import java.util.concurrent.CountDownLatch;
 import java.util.concurrent.CyclicBarrier;
-import java.util.concurrent.atomic.AtomicLong;
 import java.util.concurrent.atomic.AtomicReference;
 import java.util.concurrent.atomic.LongAdder;
 
diff --git a/store/src/test/java/org/apache/rocketmq/store/StoreTestBase.java b/store/src/test/java/org/apache/rocketmq/store/StoreTestBase.java
index 361b5e8..7a2e32a 100644
--- a/store/src/test/java/org/apache/rocketmq/store/StoreTestBase.java
+++ b/store/src/test/java/org/apache/rocketmq/store/StoreTestBase.java
@@ -19,6 +19,8 @@ package org.apache.rocketmq.store;
 import org.apache.rocketmq.common.UtilAll;
 import org.apache.rocketmq.common.message.Message;
 import org.apache.rocketmq.common.message.MessageDecoder;
+import org.apache.rocketmq.common.message.MessageExtBatch;
+import org.apache.rocketmq.common.message.MessageExtBrokerInner;
 import org.apache.rocketmq.store.logfile.DefaultMappedFile;
 import org.junit.After;
 
@@ -159,7 +161,7 @@ public class StoreTestBase {
 
     public static boolean makeSureFileExists(String fileName) throws Exception {
         File file = new File(fileName);
-        DefaultMappedFile.ensureDirOK(file.getParent());
+        UtilAll.ensureDirOK(file.getParent());
         return file.createNewFile();
     }
 
diff --git a/store/src/test/java/org/apache/rocketmq/store/dledger/DLedgerCommitlogTest.java b/store/src/test/java/org/apache/rocketmq/store/dledger/DLedgerCommitlogTest.java
index 18b9eac..a8ffbaf 100644
--- a/store/src/test/java/org/apache/rocketmq/store/dledger/DLedgerCommitlogTest.java
+++ b/store/src/test/java/org/apache/rocketmq/store/dledger/DLedgerCommitlogTest.java
@@ -30,23 +30,22 @@ import java.util.concurrent.TimeUnit;
 
 import org.apache.rocketmq.common.message.MessageDecoder;
 import org.apache.rocketmq.common.message.MessageExt;
+import org.apache.rocketmq.common.message.MessageExtBatch;
 import org.apache.rocketmq.store.DefaultMessageStore;
 import org.apache.rocketmq.store.GetMessageResult;
 import org.apache.rocketmq.store.GetMessageStatus;
-import org.apache.rocketmq.store.MessageExtBatch;
-import org.apache.rocketmq.store.MessageExtBrokerInner;
+import org.apache.rocketmq.common.message.MessageExtBrokerInner;
 import org.apache.rocketmq.store.PutMessageResult;
 import org.apache.rocketmq.store.PutMessageStatus;
 import org.junit.Assert;
 import org.junit.Test;
 
-import static org.apache.rocketmq.store.StoreTestUtil.releaseMmapFilesOnWindows;
-
 import static java.util.concurrent.TimeUnit.SECONDS;
 import static org.awaitility.Awaitility.await;
 
-public class DLedgerCommitlogTest extends MessageStoreTestBase {
+import static org.apache.rocketmq.store.StoreTestUtil.releaseMmapFilesOnWindows;
 
+public class DLedgerCommitlogTest extends MessageStoreTestBase {
 
     @Test
     public void testTruncateCQ() throws Exception {
@@ -329,7 +328,7 @@ public class DLedgerCommitlogTest extends MessageStoreTestBase {
         msgInner.setTopic(topic);
         msgInner.setQueueId(0);
         PutMessageResult putMessageResult = leaderStore.putMessage(msgInner);
-        Assert.assertEquals(PutMessageStatus.OS_PAGECACHE_BUSY, putMessageResult.getPutMessageStatus());
+        Assert.assertEquals(PutMessageStatus.IN_SYNC_REPLICAS_NOT_ENOUGH, putMessageResult.getPutMessageStatus());
 
         Thread.sleep(1000);
 
@@ -362,7 +361,7 @@ public class DLedgerCommitlogTest extends MessageStoreTestBase {
         msgInner.setTopic(topic);
         msgInner.setQueueId(0);
         PutMessageResult putMessageResult = leaderStore.putMessage(msgInner);
-        Assert.assertEquals(PutMessageStatus.OS_PAGECACHE_BUSY, putMessageResult.getPutMessageStatus());
+        Assert.assertEquals(PutMessageStatus.IN_SYNC_REPLICAS_NOT_ENOUGH, putMessageResult.getPutMessageStatus());
 
         Thread.sleep(1000);
 
diff --git a/store/src/test/java/org/apache/rocketmq/store/dledger/MessageStoreTestBase.java b/store/src/test/java/org/apache/rocketmq/store/dledger/MessageStoreTestBase.java
index 7a77e95..f1bc996 100644
--- a/store/src/test/java/org/apache/rocketmq/store/dledger/MessageStoreTestBase.java
+++ b/store/src/test/java/org/apache/rocketmq/store/dledger/MessageStoreTestBase.java
@@ -26,7 +26,7 @@ import org.apache.rocketmq.common.message.MessageDecoder;
 import org.apache.rocketmq.common.message.MessageExt;
 import org.apache.rocketmq.store.DefaultMessageStore;
 import org.apache.rocketmq.store.GetMessageResult;
-import org.apache.rocketmq.store.MessageExtBrokerInner;
+import org.apache.rocketmq.common.message.MessageExtBrokerInner;
 import org.apache.rocketmq.store.MessageStore;
 import org.apache.rocketmq.store.PutMessageResult;
 import org.apache.rocketmq.store.PutMessageStatus;
@@ -67,7 +67,6 @@ public class MessageStoreTestBase extends StoreTestBase {
             } else {
                 dLegerServer.getMemberState().changeToFollower(0, leaderId);
             }
-
         }
         if (createAbort) {
             String fileName = StorePathConfigHelper.getAbortFile(storeConfig.getStorePathRootDir());
diff --git a/store/src/test/java/org/apache/rocketmq/store/dledger/MixCommitlogTest.java b/store/src/test/java/org/apache/rocketmq/store/dledger/MixCommitlogTest.java
index 4aaa029..5fb7356 100644
--- a/store/src/test/java/org/apache/rocketmq/store/dledger/MixCommitlogTest.java
+++ b/store/src/test/java/org/apache/rocketmq/store/dledger/MixCommitlogTest.java
@@ -25,8 +25,6 @@ import org.junit.Test;
 
 public class MixCommitlogTest extends MessageStoreTestBase {
 
-
-
     @Test
     public void testFallBehindCQ() throws Exception {
         String base =  createBaseDir();
diff --git a/store/src/test/java/org/apache/rocketmq/store/ha/FlowMonitorTest.java b/store/src/test/java/org/apache/rocketmq/store/ha/FlowMonitorTest.java
new file mode 100644
index 0000000..81c9e91
--- /dev/null
+++ b/store/src/test/java/org/apache/rocketmq/store/ha/FlowMonitorTest.java
@@ -0,0 +1,61 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.rocketmq.store.ha;
+
+import org.apache.rocketmq.store.config.MessageStoreConfig;
+import org.apache.rocketmq.store.ha.FlowMonitor;
+import org.junit.Assert;
+import org.junit.Test;
+
+public class FlowMonitorTest {
+    
+    @Test
+    public void testLimit() throws Exception {
+        MessageStoreConfig messageStoreConfig = new MessageStoreConfig();
+        messageStoreConfig.setHaFlowControlEnable(true);
+        messageStoreConfig.setMaxHaTransferByteInSecond(10);
+
+        FlowMonitor flowMonitor = new FlowMonitor(messageStoreConfig);
+        flowMonitor.start();
+
+        flowMonitor.addByteCountTransferred(3);
+        Assert.assertEquals(7, flowMonitor.canTransferMaxByteNum());
+
+        Thread.sleep(2000);
+        Assert.assertEquals(10, flowMonitor.canTransferMaxByteNum());
+
+        flowMonitor.shutdown();
+    }
+
+    @Test
+    public void testSpeed() throws Exception {
+        MessageStoreConfig messageStoreConfig = new MessageStoreConfig();
+        messageStoreConfig.setHaFlowControlEnable(true);
+        messageStoreConfig.setMaxHaTransferByteInSecond(10);
+
+        FlowMonitor flowMonitor = new FlowMonitor(messageStoreConfig);
+
+        flowMonitor.addByteCountTransferred(3);
+        flowMonitor.calculateSpeed();
+        Assert.assertEquals(3, flowMonitor.getTransferredByteInSecond());
+
+        flowMonitor.addByteCountTransferred(5);
+        flowMonitor.calculateSpeed();
+        Assert.assertEquals(5, flowMonitor.getTransferredByteInSecond());
+    }
+}
... 471 lines suppressed ...

[rocketmq] 01/07: feature(namesrv&common):[RIP-29]Optimize RocketMQ NameServer (#3974)

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

jinrongtong pushed a commit to branch 5.0.0-beta-tmp
in repository https://gitbox.apache.org/repos/asf/rocketmq.git

commit 467b382777e1da8f9ecd54a2ee26aac474337708
Author: rongtong <ji...@163.com>
AuthorDate: Mon Mar 14 14:05:05 2022 +0800

    feature(namesrv&common):[RIP-29]Optimize RocketMQ NameServer (#3974)
---
 ...KeyBuilder.java => AbstractBrokerRunnable.java} |   31 +-
 .../org/apache/rocketmq/common/BrokerConfig.java   |  404 +++++--
 .../org/apache/rocketmq/common/BrokerIdentity.java |  149 +++
 .../org/apache/rocketmq/common/BrokerSyncInfo.java |   70 ++
 .../org/apache/rocketmq/common/Configuration.java  |   41 +
 .../org/apache/rocketmq/common/DataVersion.java    |   49 +-
 .../org/apache/rocketmq/common/KeyBuilder.java     |    4 +
 .../MessageType.java => LockCallback.java}         |   13 +-
 .../java/org/apache/rocketmq/common/MixAll.java    |    4 +
 .../apache/rocketmq/common/PopAckConstants.java    |    9 +
 .../org/apache/rocketmq/common/ServiceThread.java  |    8 +-
 .../apache/rocketmq/common/ThreadFactoryImpl.java  |   13 +
 .../org/apache/rocketmq/common/TopicConfig.java    |   61 +-
 .../MessageType.java => UnlockCallback.java}       |   10 +-
 .../java/org/apache/rocketmq/common/UtilAll.java   |  108 +-
 .../rocketmq/common/constant/LoggerName.java       |    7 +-
 .../apache/rocketmq/common/constant/PermName.java  |   18 +-
 .../MessageType.java => future/FutureTaskExt.java} |   26 +-
 .../rocketmq/common/message/MessageConst.java      |   26 +-
 .../rocketmq/common/message/MessageDecoder.java    |  156 ++-
 .../apache/rocketmq/common/message/MessageExt.java |   55 +
 .../rocketmq/common/message/MessageExtBatch.java   |   51 +
 .../common/message/MessageExtBrokerInner.java      |   63 ++
 .../rocketmq/common/message/MessageType.java       |   28 +-
 .../rocketmq/common/message/MessageVersion.java    |   70 ++
 .../common/namesrv/DefaultTopAddressing.java       |  165 +++
 .../NameServerUpdateCallback.java}                 |   10 +-
 .../rocketmq/common/namesrv/NamesrvConfig.java     |  117 +++
 .../rocketmq/common/namesrv/TopAddressing.java     |   88 +-
 ...amesrvRequestHeader.java => ForbiddenType.java} |   44 +-
 .../rocketmq/common/protocol/RequestCode.java      |   34 +
 .../rocketmq/common/protocol/ResponseCode.java     |    7 +-
 .../common/protocol/body/BrokerMemberGroup.java    |   90 ++
 .../protocol/body/CheckClientRequestBody.java      |    9 +
 .../rocketmq/common/protocol/body/ClusterInfo.java |   14 +-
 .../body/ConsumerOffsetSerializeWrapper.java       |   10 +
 .../common/protocol/body/ConsumerRunningInfo.java  |   31 +-
 ....java => GetBrokerMemberGroupResponseBody.java} |   16 +-
 .../body/GetRemoteClientConfigBody.java}           |   21 +-
 .../common/protocol/body/HARuntimeInfo.java        |  188 ++++
 .../common/protocol/body/LockBatchRequestBody.java |    9 +
 ...ody.java => QuerySubscriptionResponseBody.java} |   22 +-
 .../protocol/body/UnlockBatchRequestBody.java      |    9 +
 ...uestHeader.java => AddBrokerRequestHeader.java} |   24 +-
 .../ConsumeMessageDirectlyResultRequestHeader.java |   30 +
 .../DeleteSubscriptionGroupRequestHeader.java      |   10 +-
 ...eader.java => ExchangeHAInfoRequestHeader.java} |   44 +-
 ...ader.java => ExchangeHAInfoResponseHeader.java} |   44 +-
 ...java => GetBrokerMemberGroupRequestHeader.java} |   28 +-
 .../protocol/header/GetMaxOffsetRequestHeader.java |   17 +
 ...> GetSubscriptionGroupConfigRequestHeader.java} |   32 +-
 ...r.java => InitConsumerOffsetRequestHeader.java} |   22 +-
 ...tHeader.java => NotificationRequestHeader.java} |   56 +-
 ...Header.java => NotificationResponseHeader.java} |   20 +-
 ...a => NotifyMinBrokerIdChangeRequestHeader.java} |   52 +-
 ...stHeader.java => PeekMessageRequestHeader.java} |   42 +-
 ...stHeader.java => PollingInfoRequestHeader.java} |   31 +-
 ...tHeader.java => PollingInfoResponseHeader.java} |   21 +-
 .../protocol/header/PullMessageResponseHeader.java |   33 +
 ... QuerySubscriptionByConsumerRequestHeader.java} |   16 +-
 ...ava => QueryTopicsByConsumerRequestHeader.java} |   17 +-
 ...tHeader.java => RemoveBrokerRequestHeader.java} |   39 +-
 ...ader.java => ResetMasterFlushOffsetHeader.java} |   16 +-
 ...r.java => StatisticsMessagesRequestHeader.java} |   53 +-
 ...java => UpdateGroupForbiddenRequestHeader.java} |   33 +-
 ...ader.java => BrokerHeartbeatRequestHeader.java} |   58 +-
 .../DeleteTopicFromNamesrvRequestHeader.java       |   10 +
 .../header/namesrv/GetRouteInfoRequestHeader.java  |    2 +
 .../namesrv/RegisterBrokerRequestHeader.java       |   22 +
 ...Header.java => RegisterTopicRequestHeader.java} |    2 +-
 .../rocketmq/common/protocol/route/BrokerData.java |   52 +-
 .../rocketmq/common/protocol/route/QueueData.java  |   13 +
 .../common/protocol/route/TopicRouteData.java      |   53 +
 .../route/TopicRouteDatas.java}                    |   23 +-
 .../rocketmq/common/statistics/FutureHolder.java   |   53 +
 .../Interceptor.java}                              |   18 +-
 .../common/statistics/StatisticsBrief.java         |  184 ++++
 .../statistics/StatisticsBriefInterceptor.java     |   76 ++
 .../rocketmq/common/statistics/StatisticsItem.java |  175 ++++
 .../StatisticsItemFormatter.java}                  |   31 +-
 .../common/statistics/StatisticsItemPrinter.java   |   50 +
 .../StatisticsItemScheduledIncrementPrinter.java   |  290 ++++++
 .../statistics/StatisticsItemScheduledPrinter.java |   97 ++
 .../StatisticsItemStateGetter.java}                |   10 +-
 .../StatisticsKindMeta.java}                       |   38 +-
 .../common/statistics/StatisticsManager.java       |  157 +++
 .../common/subscription/GroupForbidden.java        |   86 ++
 .../subscription/SubscriptionGroupConfig.java      |   47 +-
 .../rocketmq/common/topic/TopicValidator.java      |    2 +
 .../apache/rocketmq/common/utils/MessageUtils.java |   49 +
 .../PositiveAtomicCounter.java}                    |   27 +-
 .../rocketmq/common/utils/ServiceProvider.java     |  211 ++++
 .../apache/rocketmq/common/utils/ThreadUtils.java  |   17 +-
 .../apache/rocketmq/common/DataVersionTest.java    |    7 +
 .../rocketmq/common/RegisterBrokerBodyTest.java    |    2 +-
 .../org/apache/rocketmq/common/UtilAllTest.java    |   21 +-
 .../common/message/MessageDecoderTest.java         |    9 +-
 .../rocketmq/common/message/MessageTest.java       |    1 -
 .../rocketmq/common/protocol/body/KVTableTest.java |    1 -
 .../common/statictopic/TopicQueueMappingTest.java  |    2 -
 namesrv/pom.xml                                    |   15 +-
 .../apache/rocketmq/namesrv/NamesrvController.java |  156 ++-
 .../apache/rocketmq/namesrv/NamesrvStartup.java    |   11 +-
 .../namesrv/processor/ClientRequestProcessor.java  |  106 ++
 .../processor/ClusterTestRequestProcessor.java     |    2 +-
 .../namesrv/processor/DefaultRequestProcessor.java |  287 ++---
 .../namesrv/routeinfo/BatchUnRegisterService.java  |   85 ++
 .../routeinfo/BrokerHousekeepingService.java       |    6 +-
 .../namesrv/routeinfo/RouteInfoManager.java        | 1099 ++++++++++++++------
 .../namesrv/processor/RequestProcessorTest.java    |  563 ++++++++++
 .../namesrv/routeinfo/GetRouteInfoBenchmark.java   |  148 +++
 .../namesrv/routeinfo/RegisterBrokerBenchmark.java |  177 ++++
 .../namesrv/routeinfo/RouteInfoManagerTest.java    |   85 +-
 .../routeinfo/RouteInfoManager_NewTest.java        |  783 ++++++++++++++
 114 files changed, 7214 insertions(+), 1243 deletions(-)

diff --git a/common/src/main/java/org/apache/rocketmq/common/KeyBuilder.java b/common/src/main/java/org/apache/rocketmq/common/AbstractBrokerRunnable.java
similarity index 56%
copy from common/src/main/java/org/apache/rocketmq/common/KeyBuilder.java
copy to common/src/main/java/org/apache/rocketmq/common/AbstractBrokerRunnable.java
index d30789f..89b780a 100644
--- a/common/src/main/java/org/apache/rocketmq/common/KeyBuilder.java
+++ b/common/src/main/java/org/apache/rocketmq/common/AbstractBrokerRunnable.java
@@ -14,24 +14,29 @@
  * See the License for the specific language governing permissions and
  * limitations under the License.
  */
+
 package org.apache.rocketmq.common;
 
-public class KeyBuilder {
-    public static final int POP_ORDER_REVIVE_QUEUE = 999;
+import org.apache.rocketmq.logging.InnerLoggerFactory;
 
-    public static String buildPopRetryTopic(String topic, String cid) {
-        return MixAll.RETRY_GROUP_TOPIC_PREFIX + cid + "_" + topic;
-    }
+public abstract class AbstractBrokerRunnable implements Runnable {
+    protected final BrokerIdentity brokerIdentity;
 
-    public static String parseNormalTopic(String topic, String cid) {
-        if (topic.startsWith(MixAll.RETRY_GROUP_TOPIC_PREFIX)) {
-            return topic.substring((MixAll.RETRY_GROUP_TOPIC_PREFIX + cid + "_").length());
-        } else {
-            return topic;
-        }
+    public AbstractBrokerRunnable(BrokerIdentity brokerIdentity) {
+        this.brokerIdentity = brokerIdentity;
     }
 
-    public static String buildPollingKey(String topic, String cid, int queueId) {
-        return topic + PopAckConstants.SPLIT + cid + PopAckConstants.SPLIT + queueId;
+    /**
+     * real logic for running
+     */
+    public abstract void run2();
+
+    @Override
+    public void run() {
+        if (brokerIdentity.isInBrokerContainer()) {
+            // set threadlocal broker identity to forward logging to corresponding broker
+            InnerLoggerFactory.brokerIdentity.set(brokerIdentity.getCanonicalName());
+        }
+        run2();
     }
 }
diff --git a/common/src/main/java/org/apache/rocketmq/common/BrokerConfig.java b/common/src/main/java/org/apache/rocketmq/common/BrokerConfig.java
index 66fd449..efd4ec9 100644
--- a/common/src/main/java/org/apache/rocketmq/common/BrokerConfig.java
+++ b/common/src/main/java/org/apache/rocketmq/common/BrokerConfig.java
@@ -16,8 +16,6 @@
  */
 package org.apache.rocketmq.common;
 
-import java.net.InetAddress;
-import java.net.UnknownHostException;
 import org.apache.rocketmq.common.annotation.ImportantField;
 import org.apache.rocketmq.common.constant.LoggerName;
 import org.apache.rocketmq.common.constant.PermName;
@@ -27,21 +25,25 @@ import org.apache.rocketmq.logging.InternalLogger;
 import org.apache.rocketmq.logging.InternalLoggerFactory;
 import org.apache.rocketmq.remoting.common.RemotingUtil;
 
-public class BrokerConfig {
+public class BrokerConfig extends BrokerIdentity {
     private static final InternalLogger log = InternalLoggerFactory.getLogger(LoggerName.COMMON_LOGGER_NAME);
 
+    private String brokerConfigPath = null;
+
     private String rocketmqHome = System.getProperty(MixAll.ROCKETMQ_HOME_PROPERTY, System.getenv(MixAll.ROCKETMQ_HOME_ENV));
     @ImportantField
     private String namesrvAddr = System.getProperty(MixAll.NAMESRV_ADDR_PROPERTY, System.getenv(MixAll.NAMESRV_ADDR_ENV));
+
+    /**
+     * Listen port for single broker
+     */
+    @ImportantField
+    private int listenPort = 6888;
+
     @ImportantField
     private String brokerIP1 = RemotingUtil.getLocalAddress();
     private String brokerIP2 = RemotingUtil.getLocalAddress();
-    @ImportantField
-    private String brokerName = localHostName();
-    @ImportantField
-    private String brokerClusterName = "DefaultCluster";
-    @ImportantField
-    private long brokerId = MixAll.MASTER_ID;
+
     private int brokerPermission = PermName.PERM_READ | PermName.PERM_WRITE;
     private int defaultTopicQueueNums = 8;
     @ImportantField
@@ -53,6 +55,8 @@ public class BrokerConfig {
     @ImportantField
     private boolean autoCreateSubscriptionGroup = true;
     private String messageStorePlugIn = "";
+
+    private static final int PROCESSOR_NUMBER = Runtime.getRuntime().availableProcessors();
     @ImportantField
     private String msgTraceTopicName = TopicValidator.RMQ_SYS_TRACE_TOPIC;
     @ImportantField
@@ -60,24 +64,25 @@ public class BrokerConfig {
     /**
      * thread numbers for send message thread pool.
      */
-    private int sendMessageThreadPoolNums = Math.min(Runtime.getRuntime().availableProcessors(), 4);
-    private int putMessageFutureThreadPoolNums = Math.min(Runtime.getRuntime().availableProcessors(), 4);
-    private int pullMessageThreadPoolNums = 16 + Runtime.getRuntime().availableProcessors() * 2;
+    private int sendMessageThreadPoolNums = Math.min(PROCESSOR_NUMBER, 4);
+    private int putMessageFutureThreadPoolNums = Math.min(PROCESSOR_NUMBER, 4);
+    private int pullMessageThreadPoolNums = 16 + PROCESSOR_NUMBER * 2;
+    private int litePullMessageThreadPoolNums = 16 + PROCESSOR_NUMBER * 2;
     private int ackMessageThreadPoolNums = 3;
-    private int processReplyMessageThreadPoolNums = 16 + Runtime.getRuntime().availableProcessors() * 2;
-    private int queryMessageThreadPoolNums = 8 + Runtime.getRuntime().availableProcessors();
+    private int processReplyMessageThreadPoolNums = 16 + PROCESSOR_NUMBER * 2;
+    private int queryMessageThreadPoolNums = 8 + PROCESSOR_NUMBER;
 
     private int adminBrokerThreadPoolNums = 16;
     private int clientManageThreadPoolNums = 32;
     private int consumerManageThreadPoolNums = 32;
     private int loadBalanceProcessorThreadPoolNums = 32;
-    private int heartbeatThreadPoolNums = Math.min(32, Runtime.getRuntime().availableProcessors());
+    private int heartbeatThreadPoolNums = Math.min(32, PROCESSOR_NUMBER);
 
     /**
      * Thread numbers for EndTransactionProcessor
      */
-    private int endTransactionThreadPoolNums = Math.max(8 + Runtime.getRuntime().availableProcessors() * 2,
-        sendMessageThreadPoolNums * 4);
+    private int endTransactionThreadPoolNums = Math.max(8 + PROCESSOR_NUMBER * 2,
+            sendMessageThreadPoolNums * 4);
 
     private int flushConsumerOffsetInterval = 1000 * 5;
 
@@ -90,6 +95,7 @@ public class BrokerConfig {
     private int sendThreadPoolQueueCapacity = 10000;
     private int putThreadPoolQueueCapacity = 10000;
     private int pullThreadPoolQueueCapacity = 100000;
+    private int litePullThreadPoolQueueCapacity = 100000;
     private int ackThreadPoolQueueCapacity = 100000;
     private int replyThreadPoolQueueCapacity = 10000;
     private int queryThreadPoolQueueCapacity = 20000;
@@ -97,6 +103,8 @@ public class BrokerConfig {
     private int consumerManagerThreadPoolQueueCapacity = 1000000;
     private int heartbeatThreadPoolQueueCapacity = 50000;
     private int endTransactionPoolQueueCapacity = 100000;
+    private int adminBrokerThreadPoolQueueCapacity = 10000;
+    private int loadBalanceThreadPoolQueueCapacity = 100000;
 
     private int filterServerNums = 0;
 
@@ -108,17 +116,20 @@ public class BrokerConfig {
 
     private boolean highSpeedMode = false;
 
-    private boolean commercialEnable = true;
-    private int commercialTimerCount = 1;
-    private int commercialTransCount = 1;
-    private int commercialBigCount = 1;
     private int commercialBaseCount = 1;
 
+    private int commercialSizePerMsg = 4 * 1024;
+
+    private boolean accountStatsEnable = true;
+    private boolean accountStatsPrintZeroValues = true;
+
     private boolean transferMsgByHeap = true;
     private int maxDelayTime = 40;
 
     private String regionId = MixAll.DEFAULT_TRACE_REGION_ID;
-    private int registerBrokerTimeoutMills = 6000;
+    private int registerBrokerTimeoutMills = 24000;
+
+    private int sendHeartbeatTimeoutMillis = 1000;
 
     private boolean slaveReadEnable = false;
 
@@ -128,8 +139,10 @@ public class BrokerConfig {
     private boolean brokerFastFailureEnable = true;
     private long waitTimeMillsInSendQueue = 200;
     private long waitTimeMillsInPullQueue = 5 * 1000;
+    private long waitTimeMillsInLitePullQueue = 5 * 1000;
     private long waitTimeMillsInHeartbeatQueue = 31 * 1000;
     private long waitTimeMillsInTransactionQueue = 3 * 1000;
+    private long waitTimeMillsInAckQueue = 3000;
 
     private long startAcceptSendRequestTimeStamp = 0L;
 
@@ -141,6 +154,9 @@ public class BrokerConfig {
     // 2. Filter bit map will be saved to consume queue extend file if allowed.
     private boolean enableCalcFilterBitMap = false;
 
+    //Reject the pull consumer instance to pull messages from broker.
+    private boolean rejectPullConsumerEnable = false;
+
     // Expect num of consumers will use filter.
     private int expectConsumerNumUseFilter = 32;
 
@@ -160,10 +176,23 @@ public class BrokerConfig {
 
     /**
      * This configurable item defines interval of topics registration of broker to name server. Allowing values are
-     * between 10, 000 and 60, 000 milliseconds.
+     * between 10,000 and 60,000 milliseconds.
      */
     private int registerNameServerPeriod = 1000 * 30;
 
+    /**
+     * the interval to send heartbeat to name server for liveness detection.
+     */
+    private int brokerHeartbeatInterval = 1000;
+
+    /**
+     * How long the broker will be considered as inactive by nameserver since last heartbeat. Effective only if
+     * enableSlaveActingMaster is true
+     */
+    private long brokerNotActiveTimeoutMillis = 10 * 1000;
+
+    private boolean netWorkFlowController = true;
+
     private int popPollingSize = 1024;
     private int popPollingMapSize = 100000;
     // 20w cost 200M heap memory.
@@ -172,13 +201,20 @@ public class BrokerConfig {
     private long reviveInterval = 1000;
     private long reviveMaxSlow = 3;
     private long reviveScanTime = 10000;
-    private boolean enablePopLog = true;
+    private boolean enablePopLog = false;
     private boolean enablePopBufferMerge = false;
     private int popCkStayBufferTime = 10 * 1000;
     private int popCkStayBufferTimeOut = 3 * 1000;
     private int popCkMaxBufferSize = 200000;
     private int popCkOffsetMaxQueueSize = 20000;
 
+    private boolean realTimeNotifyConsumerChange = true;
+
+    private boolean litePullMessageEnable = true;
+
+    // The period to sync broker member group from namesrv, default value is 1 second
+    private int syncBrokerMemberGroupPeriod = 1000;
+
     /**
      * the interval of pulling topic information from the named server
      */
@@ -233,15 +269,31 @@ public class BrokerConfig {
 
     private long forwardTimeout = 3 * 1000;
 
-    public static String localHostName() {
-        try {
-            return InetAddress.getLocalHost().getHostName();
-        } catch (UnknownHostException e) {
-            log.error("Failed to obtain the host name", e);
-        }
+    /**
+     * Slave will act master when failover. For example, if master down, timer or transaction message which is expire in slave will
+     * put to master (master of the same process in broker container mode or other masters in cluster when enableFailoverRemotingActing is true)
+     * when enableSlaveActingMaster is true
+     */
+    private boolean enableSlaveActingMaster = false;
+
+    private boolean enableRemoteEscape = false;
 
-        return "DEFAULT_BROKER";
-    }
+    private boolean skipPreOnline = false;
+
+    private boolean asyncSendEnable = true;
+
+    private long consumerOffsetUpdateVersionStep = 500;
+
+    private long delayOffsetUpdateVersionStep = 200;
+
+    /**
+     * Whether to lock quorum replicas.
+     *
+     * True: need to lock quorum replicas succeed. False: only need to lock one replica succeed.
+     */
+    private boolean lockInStrictMode = false;
+
+    private String metaDataHosts = "";
 
     public long getMaxPopPollingSize() {
         return maxPopPollingSize;
@@ -451,22 +503,6 @@ public class BrokerConfig {
         this.highSpeedMode = highSpeedMode;
     }
 
-    public String getRocketmqHome() {
-        return rocketmqHome;
-    }
-
-    public void setRocketmqHome(String rocketmqHome) {
-        this.rocketmqHome = rocketmqHome;
-    }
-
-    public String getBrokerName() {
-        return brokerName;
-    }
-
-    public void setBrokerName(String brokerName) {
-        this.brokerName = brokerName;
-    }
-
     public int getBrokerPermission() {
         return brokerPermission;
     }
@@ -491,14 +527,6 @@ public class BrokerConfig {
         this.autoCreateTopicEnable = autoCreateTopic;
     }
 
-    public String getBrokerClusterName() {
-        return brokerClusterName;
-    }
-
-    public void setBrokerClusterName(String brokerClusterName) {
-        this.brokerClusterName = brokerClusterName;
-    }
-
     public String getBrokerIP1() {
         return brokerIP1;
     }
@@ -603,14 +631,6 @@ public class BrokerConfig {
         this.namesrvAddr = namesrvAddr;
     }
 
-    public long getBrokerId() {
-        return brokerId;
-    }
-
-    public void setBrokerId(long brokerId) {
-        this.brokerId = brokerId;
-    }
-
     public boolean isAutoCreateSubscriptionGroup() {
         return autoCreateSubscriptionGroup;
     }
@@ -619,6 +639,94 @@ public class BrokerConfig {
         this.autoCreateSubscriptionGroup = autoCreateSubscriptionGroup;
     }
 
+    public String getBrokerConfigPath() {
+        return brokerConfigPath;
+    }
+
+    public void setBrokerConfigPath(String brokerConfigPath) {
+        this.brokerConfigPath = brokerConfigPath;
+    }
+
+    public String getRocketmqHome() {
+        return rocketmqHome;
+    }
+
+    public void setRocketmqHome(String rocketmqHome) {
+        this.rocketmqHome = rocketmqHome;
+    }
+
+    public int getListenPort() {
+        return listenPort;
+    }
+
+    public void setListenPort(int listenPort) {
+        this.listenPort = listenPort;
+    }
+
+    public int getLitePullMessageThreadPoolNums() {
+        return litePullMessageThreadPoolNums;
+    }
+
+    public void setLitePullMessageThreadPoolNums(int litePullMessageThreadPoolNums) {
+        this.litePullMessageThreadPoolNums = litePullMessageThreadPoolNums;
+    }
+
+    public int getLitePullThreadPoolQueueCapacity() {
+        return litePullThreadPoolQueueCapacity;
+    }
+
+    public void setLitePullThreadPoolQueueCapacity(int litePullThreadPoolQueueCapacity) {
+        this.litePullThreadPoolQueueCapacity = litePullThreadPoolQueueCapacity;
+    }
+
+    public int getAdminBrokerThreadPoolQueueCapacity() {
+        return adminBrokerThreadPoolQueueCapacity;
+    }
+
+    public void setAdminBrokerThreadPoolQueueCapacity(int adminBrokerThreadPoolQueueCapacity) {
+        this.adminBrokerThreadPoolQueueCapacity = adminBrokerThreadPoolQueueCapacity;
+    }
+
+    public int getLoadBalanceThreadPoolQueueCapacity() {
+        return loadBalanceThreadPoolQueueCapacity;
+    }
+
+    public void setLoadBalanceThreadPoolQueueCapacity(int loadBalanceThreadPoolQueueCapacity) {
+        this.loadBalanceThreadPoolQueueCapacity = loadBalanceThreadPoolQueueCapacity;
+    }
+
+    public int getSendHeartbeatTimeoutMillis() {
+        return sendHeartbeatTimeoutMillis;
+    }
+
+    public void setSendHeartbeatTimeoutMillis(int sendHeartbeatTimeoutMillis) {
+        this.sendHeartbeatTimeoutMillis = sendHeartbeatTimeoutMillis;
+    }
+
+    public long getWaitTimeMillsInLitePullQueue() {
+        return waitTimeMillsInLitePullQueue;
+    }
+
+    public void setWaitTimeMillsInLitePullQueue(long waitTimeMillsInLitePullQueue) {
+        this.waitTimeMillsInLitePullQueue = waitTimeMillsInLitePullQueue;
+    }
+
+    public boolean isLitePullMessageEnable() {
+        return litePullMessageEnable;
+    }
+
+    public void setLitePullMessageEnable(boolean litePullMessageEnable) {
+        this.litePullMessageEnable = litePullMessageEnable;
+    }
+
+    public int getSyncBrokerMemberGroupPeriod() {
+        return syncBrokerMemberGroupPeriod;
+    }
+
+    public void setSyncBrokerMemberGroupPeriod(int syncBrokerMemberGroupPeriod) {
+        this.syncBrokerMemberGroupPeriod = syncBrokerMemberGroupPeriod;
+    }
+
     public boolean isRejectTransactionMessage() {
         return rejectTransactionMessage;
     }
@@ -731,38 +839,6 @@ public class BrokerConfig {
         this.clientManageThreadPoolNums = clientManageThreadPoolNums;
     }
 
-    public boolean isCommercialEnable() {
-        return commercialEnable;
-    }
-
-    public void setCommercialEnable(final boolean commercialEnable) {
-        this.commercialEnable = commercialEnable;
-    }
-
-    public int getCommercialTimerCount() {
-        return commercialTimerCount;
-    }
-
-    public void setCommercialTimerCount(final int commercialTimerCount) {
-        this.commercialTimerCount = commercialTimerCount;
-    }
-
-    public int getCommercialTransCount() {
-        return commercialTransCount;
-    }
-
-    public void setCommercialTransCount(final int commercialTransCount) {
-        this.commercialTransCount = commercialTransCount;
-    }
-
-    public int getCommercialBigCount() {
-        return commercialBigCount;
-    }
-
-    public void setCommercialBigCount(final int commercialBigCount) {
-        this.commercialBigCount = commercialBigCount;
-    }
-
     public int getMaxDelayTime() {
         return maxDelayTime;
     }
@@ -1051,6 +1127,142 @@ public class BrokerConfig {
         this.forwardTimeout = timeout;
     }
 
+    public int getBrokerHeartbeatInterval() {
+        return brokerHeartbeatInterval;
+    }
+
+    public void setBrokerHeartbeatInterval(int brokerHeartbeatInterval) {
+        this.brokerHeartbeatInterval = brokerHeartbeatInterval;
+    }
+
+    public long getBrokerNotActiveTimeoutMillis() {
+        return brokerNotActiveTimeoutMillis;
+    }
+
+    public void setBrokerNotActiveTimeoutMillis(long brokerNotActiveTimeoutMillis) {
+        this.brokerNotActiveTimeoutMillis = brokerNotActiveTimeoutMillis;
+    }
+
+    public boolean isNetWorkFlowController() {
+        return netWorkFlowController;
+    }
+
+    public void setNetWorkFlowController(boolean netWorkFlowController) {
+        this.netWorkFlowController = netWorkFlowController;
+    }
+
+    public boolean isRealTimeNotifyConsumerChange() {
+        return realTimeNotifyConsumerChange;
+    }
+
+    public void setRealTimeNotifyConsumerChange(boolean realTimeNotifyConsumerChange) {
+        this.realTimeNotifyConsumerChange = realTimeNotifyConsumerChange;
+    }
+
+    public boolean isEnableSlaveActingMaster() {
+        return enableSlaveActingMaster;
+    }
+
+    public void setEnableSlaveActingMaster(boolean enableSlaveActingMaster) {
+        this.enableSlaveActingMaster = enableSlaveActingMaster;
+    }
+
+    public boolean isEnableRemoteEscape() {
+        return enableRemoteEscape;
+    }
+
+    public void setEnableRemoteEscape(boolean enableRemoteEscape) {
+        this.enableRemoteEscape = enableRemoteEscape;
+    }
+
+    public boolean isSkipPreOnline() {
+        return skipPreOnline;
+    }
+
+    public void setSkipPreOnline(boolean skipPreOnline) {
+        this.skipPreOnline = skipPreOnline;
+    }
+
+    public boolean isAsyncSendEnable() {
+        return asyncSendEnable;
+    }
+
+    public void setAsyncSendEnable(boolean asyncSendEnable) {
+        this.asyncSendEnable = asyncSendEnable;
+    }
+
+    public long getConsumerOffsetUpdateVersionStep() {
+        return consumerOffsetUpdateVersionStep;
+    }
+
+    public void setConsumerOffsetUpdateVersionStep(long consumerOffsetUpdateVersionStep) {
+        this.consumerOffsetUpdateVersionStep = consumerOffsetUpdateVersionStep;
+    }
+
+    public long getDelayOffsetUpdateVersionStep() {
+        return delayOffsetUpdateVersionStep;
+    }
+
+    public void setDelayOffsetUpdateVersionStep(long delayOffsetUpdateVersionStep) {
+        this.delayOffsetUpdateVersionStep = delayOffsetUpdateVersionStep;
+    }
+
+    public int getCommercialSizePerMsg() {
+        return commercialSizePerMsg;
+    }
+
+    public void setCommercialSizePerMsg(int commercialSizePerMsg) {
+        this.commercialSizePerMsg = commercialSizePerMsg;
+    }
+
+    public long getWaitTimeMillsInAckQueue() {
+        return waitTimeMillsInAckQueue;
+    }
+
+    public void setWaitTimeMillsInAckQueue(long waitTimeMillsInAckQueue) {
+        this.waitTimeMillsInAckQueue = waitTimeMillsInAckQueue;
+    }
+
+    public boolean isRejectPullConsumerEnable() {
+        return rejectPullConsumerEnable;
+    }
+
+    public void setRejectPullConsumerEnable(boolean rejectPullConsumerEnable) {
+        this.rejectPullConsumerEnable = rejectPullConsumerEnable;
+    }
+
+    public boolean isAccountStatsEnable() {
+        return accountStatsEnable;
+    }
+
+    public void setAccountStatsEnable(boolean accountStatsEnable) {
+        this.accountStatsEnable = accountStatsEnable;
+    }
+
+    public boolean isAccountStatsPrintZeroValues() {
+        return accountStatsPrintZeroValues;
+    }
+
+    public void setAccountStatsPrintZeroValues(boolean accountStatsPrintZeroValues) {
+        this.accountStatsPrintZeroValues = accountStatsPrintZeroValues;
+    }
+
+    public boolean isLockInStrictMode() {
+        return lockInStrictMode;
+    }
+
+    public void setLockInStrictMode(boolean lockInStrictMode) {
+        this.lockInStrictMode = lockInStrictMode;
+    }
+
+    public String getMetaDataHosts() {
+        return metaDataHosts;
+    }
+
+    public void setMetaDataHosts(String metaDataHosts) {
+        this.metaDataHosts = metaDataHosts;
+    }
+
     public boolean isIsolateLogEnable() {
         return isolateLogEnable;
     }
diff --git a/common/src/main/java/org/apache/rocketmq/common/BrokerIdentity.java b/common/src/main/java/org/apache/rocketmq/common/BrokerIdentity.java
new file mode 100644
index 0000000..e5ef3d7
--- /dev/null
+++ b/common/src/main/java/org/apache/rocketmq/common/BrokerIdentity.java
@@ -0,0 +1,149 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.rocketmq.common;
+
+import java.net.InetAddress;
+import java.net.UnknownHostException;
+import org.apache.commons.lang3.builder.EqualsBuilder;
+import org.apache.commons.lang3.builder.HashCodeBuilder;
+import org.apache.rocketmq.common.annotation.ImportantField;
+import org.apache.rocketmq.common.constant.LoggerName;
+import org.apache.rocketmq.logging.InnerLoggerFactory;
+import org.apache.rocketmq.logging.InternalLogger;
+import org.apache.rocketmq.logging.InternalLoggerFactory;
+
+public class BrokerIdentity {
+    private static final String DEFAULT_CLUSTER_NAME = "DefaultCluster";
+    protected static final InternalLogger LOGGER = InternalLoggerFactory.getLogger(LoggerName.COMMON_LOGGER_NAME);
+
+    public static final BrokerIdentity BROKER_CONTAINER_IDENTITY = new BrokerIdentity(true);
+
+    @ImportantField
+    private String brokerName = localHostName();
+    @ImportantField
+    private String brokerClusterName = DEFAULT_CLUSTER_NAME;
+    @ImportantField
+    private volatile long brokerId = MixAll.MASTER_ID;
+
+    private boolean isBrokerContainer = false;
+
+    // Do not set it manually, it depends on the startup mode
+    // Broker start by BrokerStartup is false, start or add by BrokerContainer is true
+    private boolean isInBrokerContainer = false;
+
+    public BrokerIdentity() {
+    }
+
+    public BrokerIdentity(boolean isBrokerContainer) {
+        this.isBrokerContainer = isBrokerContainer;
+    }
+
+    public BrokerIdentity(String brokerClusterName, String brokerName, long brokerId) {
+        this.brokerName = brokerName;
+        this.brokerClusterName = brokerClusterName;
+        this.brokerId = brokerId;
+    }
+
+    public BrokerIdentity(String brokerClusterName, String brokerName, long brokerId, boolean isInBrokerContainer) {
+        this.brokerName = brokerName;
+        this.brokerClusterName = brokerClusterName;
+        this.brokerId = brokerId;
+        this.isInBrokerContainer = isInBrokerContainer;
+    }
+
+    public String getBrokerName() {
+        return brokerName;
+    }
+
+    public void setBrokerName(final String brokerName) {
+        this.brokerName = brokerName;
+    }
+
+    public String getBrokerClusterName() {
+        return brokerClusterName;
+    }
+
+    public void setBrokerClusterName(final String brokerClusterName) {
+        this.brokerClusterName = brokerClusterName;
+    }
+
+    public long getBrokerId() {
+        return brokerId;
+    }
+
+    public void setBrokerId(final long brokerId) {
+        this.brokerId = brokerId;
+    }
+
+    public boolean isInBrokerContainer() {
+        return isInBrokerContainer;
+    }
+
+    public void setInBrokerContainer(boolean inBrokerContainer) {
+        isInBrokerContainer = inBrokerContainer;
+    }
+
+    protected static String localHostName() {
+        try {
+            return InetAddress.getLocalHost().getHostName();
+        } catch (UnknownHostException e) {
+            LOGGER.error("Failed to obtain the host name", e);
+        }
+
+        return "DEFAULT_BROKER";
+    }
+
+    public String getCanonicalName() {
+        if (isBrokerContainer) {
+            return InnerLoggerFactory.BROKER_CONTAINER_NAME;
+        }
+        return this.getBrokerClusterName() + "_" + this.getBrokerName() + "_" + this.getBrokerId();
+    }
+
+    public String getLoggerIdentifier() {
+        return "#" + getCanonicalName() + "#";
+    }
+
+    @Override
+    public boolean equals(final Object o) {
+        if (this == o) {
+            return true;
+        }
+
+        if (o == null || getClass() != o.getClass()) {
+            return false;
+        }
+
+        final BrokerIdentity identity = (BrokerIdentity) o;
+
+        return new EqualsBuilder()
+            .append(brokerId, identity.brokerId)
+            .append(brokerName, identity.brokerName)
+            .append(brokerClusterName, identity.brokerClusterName)
+            .isEquals();
+    }
+
+    @Override
+    public int hashCode() {
+        return new HashCodeBuilder(17, 37)
+            .append(brokerName)
+            .append(brokerClusterName)
+            .append(brokerId)
+            .toHashCode();
+    }
+}
diff --git a/common/src/main/java/org/apache/rocketmq/common/BrokerSyncInfo.java b/common/src/main/java/org/apache/rocketmq/common/BrokerSyncInfo.java
new file mode 100644
index 0000000..1ec4bc5
--- /dev/null
+++ b/common/src/main/java/org/apache/rocketmq/common/BrokerSyncInfo.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.common;
+
+import org.apache.rocketmq.remoting.protocol.RemotingSerializable;
+
+public class BrokerSyncInfo extends RemotingSerializable {
+    /**
+     * For slave online sync, retrieve HA address before register
+     */
+    private String masterHaAddress;
+
+    private long masterFlushOffset;
+
+    private String masterAddress;
+
+    public BrokerSyncInfo(String masterHaAddress, long masterFlushOffset, String masterAddress) {
+        this.masterHaAddress = masterHaAddress;
+        this.masterFlushOffset = masterFlushOffset;
+        this.masterAddress = masterAddress;
+    }
+
+    public String getMasterHaAddress() {
+        return masterHaAddress;
+    }
+
+    public void setMasterHaAddress(String masterHaAddress) {
+        this.masterHaAddress = masterHaAddress;
+    }
+
+    public long getMasterFlushOffset() {
+        return masterFlushOffset;
+    }
+
+    public void setMasterFlushOffset(long masterFlushOffset) {
+        this.masterFlushOffset = masterFlushOffset;
+    }
+
+    public String getMasterAddress() {
+        return masterAddress;
+    }
+
+    public void setMasterAddress(String masterAddress) {
+        this.masterAddress = masterAddress;
+    }
+
+    @Override
+    public String toString() {
+        return "BrokerSyncInfo{" +
+            "masterHaAddress='" + masterHaAddress + '\'' +
+            ", masterFlushOffset=" + masterFlushOffset +
+            ", masterAddress=" + masterAddress +
+            '}';
+    }
+}
diff --git a/common/src/main/java/org/apache/rocketmq/common/Configuration.java b/common/src/main/java/org/apache/rocketmq/common/Configuration.java
index 064b5e6..29dafa1 100644
--- a/common/src/main/java/org/apache/rocketmq/common/Configuration.java
+++ b/common/src/main/java/org/apache/rocketmq/common/Configuration.java
@@ -55,6 +55,9 @@ public class Configuration {
             return;
         }
         for (Object configObject : configObjects) {
+            if (configObject == null) {
+                continue;
+            }
             registerConfig(configObject);
         }
     }
@@ -234,6 +237,24 @@ public class Configuration {
         return null;
     }
 
+    public String getClientConfigsFormatString(List<String> clientKeys) {
+        try {
+            readWriteLock.readLock().lockInterruptibly();
+
+            try {
+
+                return getClientConfigsInternal(clientKeys);
+
+            } finally {
+                readWriteLock.readLock().unlock();
+            }
+        } catch (InterruptedException e) {
+            log.error("getAllConfigsFormatString lock error");
+        }
+
+        return null;
+    }
+
     public String getDataVersionJson() {
         return this.dataVersion.toJson();
     }
@@ -276,6 +297,26 @@ public class Configuration {
         return stringBuilder.toString();
     }
 
+    private String getClientConfigsInternal(List<String> clientConigKeys) {
+        StringBuilder stringBuilder = new StringBuilder();
+        Properties clientProperties = new Properties();
+
+        // reload from config object ?
+        for (Object configObject : this.configObjectList) {
+            Properties properties = MixAll.object2Properties(configObject);
+
+            for (String nameNow : clientConigKeys) {
+                if (properties.containsKey(nameNow)) {
+                    clientProperties.put(nameNow, properties.get(nameNow));
+                }
+            }
+
+        }
+        stringBuilder.append(MixAll.properties2String(clientProperties));
+
+        return stringBuilder.toString();
+    }
+
     private void merge(Properties from, Properties to) {
         for (Entry<Object, Object> next : from.entrySet()) {
             Object fromObj = next.getValue(), toObj = to.get(next.getKey());
diff --git a/common/src/main/java/org/apache/rocketmq/common/DataVersion.java b/common/src/main/java/org/apache/rocketmq/common/DataVersion.java
index e54000d..6d53bd8 100644
--- a/common/src/main/java/org/apache/rocketmq/common/DataVersion.java
+++ b/common/src/main/java/org/apache/rocketmq/common/DataVersion.java
@@ -20,19 +20,34 @@ import java.util.concurrent.atomic.AtomicLong;
 import org.apache.rocketmq.remoting.protocol.RemotingSerializable;
 
 public class DataVersion extends RemotingSerializable {
+    private long stateVersion = 0L;
     private long timestamp = System.currentTimeMillis();
     private AtomicLong counter = new AtomicLong(0);
 
     public void assignNewOne(final DataVersion dataVersion) {
         this.timestamp = dataVersion.timestamp;
+        this.stateVersion = dataVersion.stateVersion;
         this.counter.set(dataVersion.counter.get());
     }
 
     public void nextVersion() {
+        this.nextVersion(0L);
+    }
+
+    public void nextVersion(long stateVersion) {
         this.timestamp = System.currentTimeMillis();
+        this.stateVersion = stateVersion;
         this.counter.incrementAndGet();
     }
 
+    public long getStateVersion() {
+        return stateVersion;
+    }
+
+    public void setStateVersion(long stateVersion) {
+        this.stateVersion = stateVersion;
+    }
+
     public long getTimestamp() {
         return timestamp;
     }
@@ -56,22 +71,25 @@ public class DataVersion extends RemotingSerializable {
         if (o == null || getClass() != o.getClass())
             return false;
 
-        final DataVersion that = (DataVersion) o;
+        DataVersion version = (DataVersion) o;
 
-        if (timestamp != that.timestamp) {
+        if (getStateVersion() != version.getStateVersion())
+            return false;
+        if (getTimestamp() != version.getTimestamp())
             return false;
-        }
 
-        if (counter != null && that.counter != null) {
-            return counter.longValue() == that.counter.longValue();
+        if (counter != null && version.counter != null) {
+            return counter.longValue() == version.counter.longValue();
         }
 
-        return (null == counter) && (null == that.counter);
+        return (null == counter) && (null == version.counter);
+
     }
 
     @Override
     public int hashCode() {
-        int result = (int) (timestamp ^ (timestamp >>> 32));
+        int result = (int) (getStateVersion() ^ (getStateVersion() >>> 32));
+        result = 31 * result + (int) (getTimestamp() ^ (getTimestamp() >>> 32));
         if (null != counter) {
             long l = counter.get();
             result = 31 * result + (int) (l ^ (l >>> 32));
@@ -87,4 +105,21 @@ public class DataVersion extends RemotingSerializable {
         sb.append(']');
         return sb.toString();
     }
+
+    public int compare(DataVersion dataVersion) {
+        if (this.getStateVersion() > dataVersion.getStateVersion()) {
+            return 1;
+        } else if (this.getStateVersion() < dataVersion.getStateVersion()) {
+            return -1;
+        } else if (this.getCounter().get() > dataVersion.getCounter().get()) {
+            return 1;
+        } else if (this.getCounter().get() < dataVersion.getCounter().get()) {
+            return -1;
+        } else if (this.getTimestamp() > dataVersion.getTimestamp()) {
+            return 1;
+        } else if (this.getTimestamp() < dataVersion.getTimestamp()) {
+            return -1;
+        }
+        return 0;
+    }
 }
diff --git a/common/src/main/java/org/apache/rocketmq/common/KeyBuilder.java b/common/src/main/java/org/apache/rocketmq/common/KeyBuilder.java
index d30789f..e1532d9 100644
--- a/common/src/main/java/org/apache/rocketmq/common/KeyBuilder.java
+++ b/common/src/main/java/org/apache/rocketmq/common/KeyBuilder.java
@@ -34,4 +34,8 @@ public class KeyBuilder {
     public static String buildPollingKey(String topic, String cid, int queueId) {
         return topic + PopAckConstants.SPLIT + cid + PopAckConstants.SPLIT + queueId;
     }
+
+    public static String buildPollingNotificationKey(String topic, int queueId) {
+        return topic + PopAckConstants.SPLIT + queueId;
+    }
 }
diff --git a/common/src/main/java/org/apache/rocketmq/common/message/MessageType.java b/common/src/main/java/org/apache/rocketmq/common/LockCallback.java
similarity index 76%
copy from common/src/main/java/org/apache/rocketmq/common/message/MessageType.java
copy to common/src/main/java/org/apache/rocketmq/common/LockCallback.java
index 08091f4..9abf367 100644
--- a/common/src/main/java/org/apache/rocketmq/common/message/MessageType.java
+++ b/common/src/main/java/org/apache/rocketmq/common/LockCallback.java
@@ -14,12 +14,13 @@
  * See the License for the specific language governing permissions and
  * limitations under the License.
  */
+package org.apache.rocketmq.common;
 
-package org.apache.rocketmq.common.message;
+import java.util.Set;
+import org.apache.rocketmq.common.message.MessageQueue;
 
-public enum MessageType {
-    Normal_Msg,
-    Trans_Msg_Half,
-    Trans_msg_Commit,
-    Delay_Msg,
+public interface LockCallback {
+    void onSuccess(final Set<MessageQueue> lockOKMQSet);
+
+    void onException(final Throwable e);
 }
diff --git a/common/src/main/java/org/apache/rocketmq/common/MixAll.java b/common/src/main/java/org/apache/rocketmq/common/MixAll.java
index a15294f..638d038 100644
--- a/common/src/main/java/org/apache/rocketmq/common/MixAll.java
+++ b/common/src/main/java/org/apache/rocketmq/common/MixAll.java
@@ -74,7 +74,10 @@ public class MixAll {
     public static final String LOCALHOST = localhost();
     public static final String DEFAULT_CHARSET = "UTF-8";
     public static final long MASTER_ID = 0L;
+    public static final long FIRST_SLAVE_ID = 1L;
     public static final long CURRENT_JVM_PID = getPID();
+    public final static int UNIT_PRE_SIZE_FOR_MSG = 28;
+
     public static final String RETRY_GROUP_TOPIC_PREFIX = "%RETRY%";
     public static final String DLQ_GROUP_TOPIC_PREFIX = "%DLQ%";
     public static final String REPLY_TOPIC_POSTFIX = "REPLY_TOPIC";
@@ -90,6 +93,7 @@ public class MixAll {
     public static final String LOGICAL_QUEUE_MOCK_BROKER_PREFIX = "__syslo__";
     public static final String METADATA_SCOPE_GLOBAL = "__global__";
     public static final String LOGICAL_QUEUE_MOCK_BROKER_NAME_NOT_EXIST = "__syslo__none__";
+    public static final String MULTI_PATH_SPLITTER = System.getProperty("rocketmq.broker.multiPathSplitter", ",");
 
     public static String getWSAddr() {
         String wsDomainName = System.getProperty("rocketmq.namesrv.domain", DEFAULT_NAMESRV_ADDR_LOOKUP);
diff --git a/common/src/main/java/org/apache/rocketmq/common/PopAckConstants.java b/common/src/main/java/org/apache/rocketmq/common/PopAckConstants.java
index 839f947..ac5a1a1 100644
--- a/common/src/main/java/org/apache/rocketmq/common/PopAckConstants.java
+++ b/common/src/main/java/org/apache/rocketmq/common/PopAckConstants.java
@@ -32,4 +32,13 @@ public class PopAckConstants {
     public static final String ACK_TAG = "ack";
     public static final String SPLIT = "@";
 
+    /**
+     * Build cluster revive topic
+     *
+     * @param clusterName cluster name
+     * @return revive topic
+     */
+    public static String buildClusterReviveTopic(String clusterName) {
+        return PopAckConstants.REVIVE_TOPIC + clusterName;
+    }
 }
diff --git a/common/src/main/java/org/apache/rocketmq/common/ServiceThread.java b/common/src/main/java/org/apache/rocketmq/common/ServiceThread.java
index ead8117..a84bcc8 100644
--- a/common/src/main/java/org/apache/rocketmq/common/ServiceThread.java
+++ b/common/src/main/java/org/apache/rocketmq/common/ServiceThread.java
@@ -27,7 +27,7 @@ public abstract class ServiceThread implements Runnable {
 
     private static final long JOIN_TIME = 90 * 1000;
 
-    private Thread thread;
+    protected Thread thread;
     protected final CountDownLatch2 waitPoint = new CountDownLatch2(1);
     protected volatile AtomicBoolean hasNotified = new AtomicBoolean(false);
     protected volatile boolean stopped = false;
@@ -76,17 +76,17 @@ public abstract class ServiceThread implements Runnable {
 
             long beginTime = System.currentTimeMillis();
             if (!this.thread.isDaemon()) {
-                this.thread.join(this.getJointime());
+                this.thread.join(this.getJoinTime());
             }
             long elapsedTime = System.currentTimeMillis() - beginTime;
             log.info("join thread " + this.getServiceName() + " elapsed time(ms) " + elapsedTime + " "
-                + this.getJointime());
+                + this.getJoinTime());
         } catch (InterruptedException e) {
             log.error("Interrupted", e);
         }
     }
 
-    public long getJointime() {
+    public long getJoinTime() {
         return JOIN_TIME;
     }
 
diff --git a/common/src/main/java/org/apache/rocketmq/common/ThreadFactoryImpl.java b/common/src/main/java/org/apache/rocketmq/common/ThreadFactoryImpl.java
index 564d60c..a76f605 100644
--- a/common/src/main/java/org/apache/rocketmq/common/ThreadFactoryImpl.java
+++ b/common/src/main/java/org/apache/rocketmq/common/ThreadFactoryImpl.java
@@ -34,6 +34,19 @@ public class ThreadFactoryImpl implements ThreadFactory {
         this.daemon = daemon;
     }
 
+    public ThreadFactoryImpl(final String threadNamePrefix, BrokerIdentity brokerIdentity) {
+        this(threadNamePrefix, false, brokerIdentity);
+    }
+
+    public ThreadFactoryImpl(final String threadNamePrefix, boolean daemon, BrokerIdentity brokerIdentity) {
+        this.daemon = daemon;
+        if (brokerIdentity != null && brokerIdentity.isInBrokerContainer()) {
+            this.threadNamePrefix = brokerIdentity.getLoggerIdentifier() + threadNamePrefix;
+        } else {
+            this.threadNamePrefix = threadNamePrefix;
+        }
+    }
+
     @Override
     public Thread newThread(Runnable r) {
         Thread thread = new Thread(r, threadNamePrefix + this.threadIndex.incrementAndGet());
diff --git a/common/src/main/java/org/apache/rocketmq/common/TopicConfig.java b/common/src/main/java/org/apache/rocketmq/common/TopicConfig.java
index 610c3e2..f11b3b3 100644
--- a/common/src/main/java/org/apache/rocketmq/common/TopicConfig.java
+++ b/common/src/main/java/org/apache/rocketmq/common/TopicConfig.java
@@ -40,16 +40,6 @@ public class TopicConfig {
         this.topicName = topicName;
     }
 
-    public TopicConfig(TopicConfig other) {
-        this.topicName = other.topicName;
-        this.readQueueNums = other.readQueueNums;
-        this.writeQueueNums = other.writeQueueNums;
-        this.perm = other.perm;
-        this.topicFilterType = other.topicFilterType;
-        this.topicSysFlag = other.topicSysFlag;
-        this.order = other.order;
-    }
-
     public TopicConfig(String topicName, int readQueueNums, int writeQueueNums) {
         this.topicName = topicName;
         this.readQueueNums = readQueueNums;
@@ -63,16 +53,38 @@ public class TopicConfig {
         this.perm = perm;
     }
 
+    public TopicConfig(String topicName, int readQueueNums, int writeQueueNums, int perm, int topicSysFlag) {
+        this.topicName = topicName;
+        this.readQueueNums = readQueueNums;
+        this.writeQueueNums = writeQueueNums;
+        this.perm = perm;
+        this.topicSysFlag = topicSysFlag;
+    }
+
+    public TopicConfig(TopicConfig other) {
+        this.topicName = other.topicName;
+        this.readQueueNums = other.readQueueNums;
+        this.writeQueueNums = other.writeQueueNums;
+        this.perm = other.perm;
+        this.topicFilterType = other.topicFilterType;
+        this.topicSysFlag = other.topicSysFlag;
+        this.order = other.order;
+    }
     public String encode() {
         StringBuilder sb = new StringBuilder();
+        //[0]
         sb.append(this.topicName);
         sb.append(SEPARATOR);
+        //[1]
         sb.append(this.readQueueNums);
         sb.append(SEPARATOR);
+        //[2]
         sb.append(this.writeQueueNums);
         sb.append(SEPARATOR);
+        //[3]
         sb.append(this.perm);
         sb.append(SEPARATOR);
+        //[4]
         sb.append(this.topicFilterType);
 
         // Leave the encode/decode [attributes] out for now
@@ -82,7 +94,7 @@ public class TopicConfig {
 
     public boolean decode(final String in) {
         String[] strs = in.split(SEPARATOR);
-        if (strs != null && strs.length == 5) {
+        if (strs != null && strs.length >= 5) {
             this.topicName = strs[0];
 
             this.readQueueNums = Integer.parseInt(strs[1]);
@@ -165,18 +177,27 @@ public class TopicConfig {
 
     @Override
     public boolean equals(Object o) {
-        if (this == o) return true;
-        if (o == null || getClass() != o.getClass()) return false;
+        if (this == o)
+            return true;
+        if (o == null || getClass() != o.getClass())
+            return false;
 
         TopicConfig that = (TopicConfig) o;
 
-        if (readQueueNums != that.readQueueNums) return false;
-        if (writeQueueNums != that.writeQueueNums) return false;
-        if (perm != that.perm) return false;
-        if (topicSysFlag != that.topicSysFlag) return false;
-        if (order != that.order) return false;
-        if (topicName != null ? !topicName.equals(that.topicName) : that.topicName != null) return false;
-        if (topicFilterType != that.topicFilterType) return false;
+        if (readQueueNums != that.readQueueNums)
+            return false;
+        if (writeQueueNums != that.writeQueueNums)
+            return false;
+        if (perm != that.perm)
+            return false;
+        if (topicSysFlag != that.topicSysFlag)
+            return false;
+        if (order != that.order)
+            return false;
+        if (topicName != null ? !topicName.equals(that.topicName) : that.topicName != null)
+            return false;
+        if (topicFilterType != that.topicFilterType)
+            return false;
         return attributes != null ? attributes.equals(that.attributes) : that.attributes == null;
     }
 
diff --git a/common/src/main/java/org/apache/rocketmq/common/message/MessageType.java b/common/src/main/java/org/apache/rocketmq/common/UnlockCallback.java
similarity index 84%
copy from common/src/main/java/org/apache/rocketmq/common/message/MessageType.java
copy to common/src/main/java/org/apache/rocketmq/common/UnlockCallback.java
index 08091f4..e86ec8b 100644
--- a/common/src/main/java/org/apache/rocketmq/common/message/MessageType.java
+++ b/common/src/main/java/org/apache/rocketmq/common/UnlockCallback.java
@@ -14,12 +14,10 @@
  * See the License for the specific language governing permissions and
  * limitations under the License.
  */
+package org.apache.rocketmq.common;
 
-package org.apache.rocketmq.common.message;
+public interface UnlockCallback {
+    void onSuccess();
 
-public enum MessageType {
-    Normal_Msg,
-    Trans_Msg_Half,
-    Trans_msg_Commit,
-    Delay_Msg,
+    void onException(final Throwable e);
 }
diff --git a/common/src/main/java/org/apache/rocketmq/common/UtilAll.java b/common/src/main/java/org/apache/rocketmq/common/UtilAll.java
index a0d35ab..7727544 100644
--- a/common/src/main/java/org/apache/rocketmq/common/UtilAll.java
+++ b/common/src/main/java/org/apache/rocketmq/common/UtilAll.java
@@ -21,10 +21,14 @@ import java.io.ByteArrayOutputStream;
 import java.io.File;
 import java.io.IOException;
 import java.lang.management.ManagementFactory;
+import java.lang.reflect.Method;
 import java.net.Inet4Address;
 import java.net.Inet6Address;
 import java.net.InetAddress;
 import java.net.NetworkInterface;
+import java.nio.ByteBuffer;
+import java.security.AccessController;
+import java.security.PrivilegedAction;
 import java.text.NumberFormat;
 import java.text.ParseException;
 import java.text.SimpleDateFormat;
@@ -229,7 +233,6 @@ public class UtilAll {
                 return -1;
             }
 
-
             long totalSpace = file.getTotalSpace();
 
             if (totalSpace > 0) {
@@ -251,6 +254,25 @@ public class UtilAll {
         return -1;
     }
 
+    public static long getDiskPartitionTotalSpace(final String path) {
+        if (null == path || path.isEmpty()) {
+            return -1;
+        }
+
+        try {
+            File file = new File(path);
+
+
+            if (!file.exists()) {
+                return -1;
+            }
+
+            return file.getTotalSpace() -  file.getFreeSpace() + file.getUsableSpace();
+        } catch (Exception e) {
+            return -1;
+        }
+    }
+
     public static int crc32(byte[] array) {
         if (array != null) {
             return crc32(array, 0, array.length);
@@ -619,4 +641,88 @@ public class UtilAll {
         String[] addrArray = str.split(splitter);
         return Arrays.asList(addrArray);
     }
+
+    public static void deleteEmptyDirectory(File file) {
+        if (file == null || !file.exists()) {
+            return;
+        }
+        if (!file.isDirectory()) {
+            return;
+        }
+        File[] files = file.listFiles();
+        if (files == null || files.length <= 0) {
+            file.delete();
+            log.info("delete empty direct, {}", file.getPath());
+        }
+    }
+
+    public static void cleanBuffer(final ByteBuffer buffer) {
+        if (buffer == null || !buffer.isDirect() || buffer.capacity() == 0) {
+            return;
+        }
+        invoke(invoke(viewed(buffer), "cleaner"), "clean");
+    }
+
+    public static Object invoke(final Object target, final String methodName, final Class<?>... args) {
+        return AccessController.doPrivileged(new PrivilegedAction<Object>() {
+            @Override
+            public Object run() {
+                try {
+                    Method method = method(target, methodName, args);
+                    method.setAccessible(true);
+                    return method.invoke(target);
+                } catch (Exception e) {
+                    throw new IllegalStateException(e);
+                }
+            }
+        });
+    }
+
+    public static Method method(Object target, String methodName, Class<?>[] args) throws NoSuchMethodException {
+        try {
+            return target.getClass().getMethod(methodName, args);
+        } catch (NoSuchMethodException e) {
+            return target.getClass().getDeclaredMethod(methodName, args);
+        }
+    }
+
+    private static ByteBuffer viewed(ByteBuffer buffer) {
+        String methodName = "viewedBuffer";
+
+        Method[] methods = buffer.getClass().getMethods();
+        for (Method method : methods) {
+            if (method.getName().equals("attachment")) {
+                methodName = "attachment";
+                break;
+            }
+        }
+
+        ByteBuffer viewedBuffer = (ByteBuffer) invoke(buffer, methodName);
+        if (viewedBuffer == null) {
+            return buffer;
+        } else {
+            return viewed(viewedBuffer);
+        }
+    }
+
+    public static void ensureDirOK(final String dirName) {
+        if (dirName != null) {
+            if (dirName.contains(MixAll.MULTI_PATH_SPLITTER)) {
+                String[] dirs = dirName.trim().split(MixAll.MULTI_PATH_SPLITTER);
+                for (String dir : dirs) {
+                    createDirIfNotExist(dir);
+                }
+            } else {
+                createDirIfNotExist(dirName);
+            }
+        }
+    }
+
+    private static void  createDirIfNotExist(String dirName) {
+        File f = new File(dirName);
+        if (!f.exists()) {
+            boolean result = f.mkdirs();
+            log.info(dirName + " mkdir " + (result ? "OK" : "Failed"));
+        }
+    }
 }
diff --git a/common/src/main/java/org/apache/rocketmq/common/constant/LoggerName.java b/common/src/main/java/org/apache/rocketmq/common/constant/LoggerName.java
index c29eccd..a77d5c2 100644
--- a/common/src/main/java/org/apache/rocketmq/common/constant/LoggerName.java
+++ b/common/src/main/java/org/apache/rocketmq/common/constant/LoggerName.java
@@ -19,7 +19,7 @@ package org.apache.rocketmq.common.constant;
 public class LoggerName {
     public static final String FILTERSRV_LOGGER_NAME = "RocketmqFiltersrv";
     public static final String NAMESRV_LOGGER_NAME = "RocketmqNamesrv";
-    public static final String NAMESRV_CONSOLE_NAME = "RocketmqNamesrvConsole";
+    public static final String NAMESRV_WATER_MARK_LOGGER_NAME = "RocketmqNamesrvWaterMark";
     public static final String BROKER_LOGGER_NAME = "RocketmqBroker";
     public static final String BROKER_CONSOLE_NAME = "RocketmqConsole";
     public static final String CLIENT_LOGGER_NAME = "RocketmqClient";
@@ -30,7 +30,11 @@ public class LoggerName {
     public static final String TRANSACTION_LOGGER_NAME = "RocketmqTransaction";
     public static final String REBALANCE_LOCK_LOGGER_NAME = "RocketmqRebalanceLock";
     public static final String ROCKETMQ_STATS_LOGGER_NAME = "RocketmqStats";
+    public static final String DLQ_STATS_LOGGER_NAME = "RocketmqDLQStats";
+    public static final String DLQ_LOGGER_NAME = "RocketmqDLQ";
+    public static final String CONSUMER_STATS_LOGGER_NAME = "RocketmqConsumerStats";
     public static final String COMMERCIAL_LOGGER_NAME = "RocketmqCommercial";
+    public static final String ACCOUNT_LOGGER_NAME = "RocketmqAccount";
     public static final String FLOW_CONTROL_LOGGER_NAME = "RocketmqFlowControl";
     public static final String ROCKETMQ_AUTHORIZE_LOGGER_NAME = "RocketmqAuthorize";
     public static final String DUPLICATION_LOGGER_NAME = "RocketmqDuplication";
@@ -38,5 +42,6 @@ public class LoggerName {
     public static final String WATER_MARK_LOGGER_NAME = "RocketmqWaterMark";
     public static final String FILTER_LOGGER_NAME = "RocketmqFilter";
     public static final String ROCKETMQ_POP_LOGGER_NAME = "RocketmqPop";
+    public static final String FAILOVER_LOGGER_NAME = "RocketmqFailover";
     public static final String STDOUT_LOGGER_NAME = "STDOUT";
 }
diff --git a/common/src/main/java/org/apache/rocketmq/common/constant/PermName.java b/common/src/main/java/org/apache/rocketmq/common/constant/PermName.java
index 1fe9aa1..434870c 100644
--- a/common/src/main/java/org/apache/rocketmq/common/constant/PermName.java
+++ b/common/src/main/java/org/apache/rocketmq/common/constant/PermName.java
@@ -17,10 +17,16 @@
 package org.apache.rocketmq.common.constant;
 
 public class PermName {
-    public static final int PERM_PRIORITY = 0x1 << 3;
-    public static final int PERM_READ = 0x1 << 2;
-    public static final int PERM_WRITE = 0x1 << 1;
-    public static final int PERM_INHERIT = 0x1;
+    public static final int INDEX_PERM_PRIORITY = 3;
+    public static final int INDEX_PERM_READ = 2;
+    public static final int INDEX_PERM_WRITE = 1;
+    public static final int INDEX_PERM_INHERIT = 0;
+
+
+    public static final int PERM_PRIORITY = 0x1 << INDEX_PERM_PRIORITY;
+    public static final int PERM_READ = 0x1 << INDEX_PERM_READ;
+    public static final int PERM_WRITE = 0x1 << INDEX_PERM_WRITE;
+    public static final int PERM_INHERIT = 0x1 << INDEX_PERM_INHERIT;
 
     public static String perm2String(final int perm) {
         final StringBuilder sb = new StringBuilder("---");
@@ -50,4 +56,8 @@ public class PermName {
     public static boolean isInherited(final int perm) {
         return (perm & PERM_INHERIT) == PERM_INHERIT;
     }
+
+    public static boolean isPriority(final int perm) {
+        return (perm & PERM_PRIORITY) == PERM_PRIORITY;
+    }
 }
diff --git a/common/src/main/java/org/apache/rocketmq/common/message/MessageType.java b/common/src/main/java/org/apache/rocketmq/common/future/FutureTaskExt.java
similarity index 59%
copy from common/src/main/java/org/apache/rocketmq/common/message/MessageType.java
copy to common/src/main/java/org/apache/rocketmq/common/future/FutureTaskExt.java
index 08091f4..1bd46ea 100644
--- a/common/src/main/java/org/apache/rocketmq/common/message/MessageType.java
+++ b/common/src/main/java/org/apache/rocketmq/common/future/FutureTaskExt.java
@@ -15,11 +15,25 @@
  * limitations under the License.
  */
 
-package org.apache.rocketmq.common.message;
+package org.apache.rocketmq.common.future;
 
-public enum MessageType {
-    Normal_Msg,
-    Trans_Msg_Half,
-    Trans_msg_Commit,
-    Delay_Msg,
+import java.util.concurrent.Callable;
+import java.util.concurrent.FutureTask;
+
+public class FutureTaskExt<V> extends FutureTask<V> {
+    private final Runnable runnable;
+
+    public FutureTaskExt(final Callable<V> callable) {
+        super(callable);
+        this.runnable = null;
+    }
+
+    public FutureTaskExt(final Runnable runnable, final V result) {
+        super(runnable, result);
+        this.runnable = runnable;
+    }
+
+    public Runnable getRunnable() {
+        return runnable;
+    }
 }
diff --git a/common/src/main/java/org/apache/rocketmq/common/message/MessageConst.java b/common/src/main/java/org/apache/rocketmq/common/message/MessageConst.java
index 105ef5b..a823466 100644
--- a/common/src/main/java/org/apache/rocketmq/common/message/MessageConst.java
+++ b/common/src/main/java/org/apache/rocketmq/common/message/MessageConst.java
@@ -39,13 +39,16 @@ public class MessageConst {
     public static final String PROPERTY_MSG_REGION = "MSG_REGION";
     public static final String PROPERTY_TRACE_SWITCH = "TRACE_ON";
     public static final String PROPERTY_UNIQ_CLIENT_MESSAGE_ID_KEYIDX = "UNIQ_KEY";
+    public static final String PROPERTY_EXTEND_UNIQ_INFO = "EXTEND_UNIQ_INFO";
     public static final String PROPERTY_MAX_RECONSUME_TIMES = "MAX_RECONSUME_TIMES";
     public static final String PROPERTY_CONSUME_START_TIMESTAMP = "CONSUME_START_TIME";
     public static final String PROPERTY_INNER_NUM = "INNER_NUM";
     public static final String PROPERTY_INNER_BASE = "INNER_BASE";
+    public static final String DUP_INFO = "DUP_INFO";
+    public static final String PROPERTY_CHECK_IMMUNITY_TIME_IN_SECONDS = "CHECK_IMMUNITY_TIME_IN_SECONDS";
     public static final String PROPERTY_TRANSACTION_PREPARED_QUEUE_OFFSET = "TRAN_PREPARED_QUEUE_OFFSET";
+    public static final String PROPERTY_TRANSACTION_ID = "__transactionId__";
     public static final String PROPERTY_TRANSACTION_CHECK_TIMES = "TRANSACTION_CHECK_TIMES";
-    public static final String PROPERTY_CHECK_IMMUNITY_TIME_IN_SECONDS = "CHECK_IMMUNITY_TIME_IN_SECONDS";
     public static final String PROPERTY_INSTANCE_ID = "INSTANCE_ID";
     public static final String PROPERTY_CORRELATION_ID = "CORRELATION_ID";
     public static final String PROPERTY_MESSAGE_REPLY_TO_CLIENT = "REPLY_TO_CLIENT";
@@ -55,12 +58,29 @@ public class MessageConst {
     public static final String PROPERTY_CLUSTER = "CLUSTER";
     public static final String PROPERTY_MESSAGE_TYPE = "MSG_TYPE";
     public static final String PROPERTY_POP_CK = "POP_CK";
+    public static final String PROPERTY_POP_CK_OFFSET = "POP_CK_OFFSET";
     public static final String PROPERTY_FIRST_POP_TIME = "1ST_POP_TIME";
+    public static final String PROPERTY_SHARDING_KEY = "__SHARDINGKEY";
     public static final String PROPERTY_FORWARD_QUEUE_ID = "PROPERTY_FORWARD_QUEUE_ID";
     public static final String PROPERTY_REDIRECT = "REDIRECT";
     public static final String PROPERTY_INNER_MULTI_DISPATCH = "INNER_MULTI_DISPATCH";
     public static final String PROPERTY_INNER_MULTI_QUEUE_OFFSET = "INNER_MULTI_QUEUE_OFFSET";
 
+    /**
+     * property which name starts with "__RMQ.TRANSIENT." is called transient one that will not stored in broker disks.
+     */
+    public static final String PROPERTY_TRANSIENT_PREFIX = "__RMQ.TRANSIENT.";
+
+    /**
+     * the transient property key of topicSysFlag (set by client when pulling messages)
+     */
+    public static final String PROPERTY_TRANSIENT_TOPIC_CONFIG = PROPERTY_TRANSIENT_PREFIX + "TOPIC_SYS_FLAG";
+
+    /**
+     * the transient property key of groupSysFlag (set by client when pulling messages)
+     */
+    public static final String PROPERTY_TRANSIENT_GROUP_CONFIG = PROPERTY_TRANSIENT_PREFIX + "GROUP_SYS_FLAG";
+
     public static final String KEY_SEPARATOR = " ";
 
     public static final HashSet<String> STRING_HASH_SET = new HashSet<>(64);
@@ -89,7 +109,11 @@ public class MessageConst {
         STRING_HASH_SET.add(PROPERTY_MAX_RECONSUME_TIMES);
         STRING_HASH_SET.add(PROPERTY_CONSUME_START_TIMESTAMP);
         STRING_HASH_SET.add(PROPERTY_POP_CK);
+        STRING_HASH_SET.add(PROPERTY_POP_CK_OFFSET);
         STRING_HASH_SET.add(PROPERTY_FIRST_POP_TIME);
+        STRING_HASH_SET.add(PROPERTY_TRANSACTION_PREPARED_QUEUE_OFFSET);
+        STRING_HASH_SET.add(DUP_INFO);
+        STRING_HASH_SET.add(PROPERTY_EXTEND_UNIQ_INFO);
         STRING_HASH_SET.add(PROPERTY_INSTANCE_ID);
         STRING_HASH_SET.add(PROPERTY_CORRELATION_ID);
         STRING_HASH_SET.add(PROPERTY_MESSAGE_REPLY_TO_CLIENT);
diff --git a/common/src/main/java/org/apache/rocketmq/common/message/MessageDecoder.java b/common/src/main/java/org/apache/rocketmq/common/message/MessageDecoder.java
index 813ad89..d6d4a8f 100644
--- a/common/src/main/java/org/apache/rocketmq/common/message/MessageDecoder.java
+++ b/common/src/main/java/org/apache/rocketmq/common/message/MessageDecoder.java
@@ -16,6 +16,7 @@
  */
 package org.apache.rocketmq.common.message;
 
+import java.io.IOException;
 import java.net.Inet4Address;
 import java.net.InetAddress;
 import java.net.InetSocketAddress;
@@ -263,13 +264,132 @@ public class MessageDecoder {
         return byteBuffer.array();
     }
 
+    /**
+     * Encode without store timestamp and store host, skip blank msg.
+     *
+     * @param messageExt msg
+     * @param needCompress need compress or not
+     * @return byte array
+     * @throws IOException when compress failed
+     */
+    public static byte[] encodeUniquely(MessageExt messageExt, boolean needCompress) throws IOException {
+        byte[] body = messageExt.getBody();
+        byte[] topics = messageExt.getTopic().getBytes(CHARSET_UTF8);
+        byte topicLen = (byte) topics.length;
+        String properties = messageProperties2String(messageExt.getProperties());
+        byte[] propertiesBytes = properties.getBytes(CHARSET_UTF8);
+        short propertiesLength = (short) propertiesBytes.length;
+        int sysFlag = messageExt.getSysFlag();
+        int bornhostLength = (sysFlag & MessageSysFlag.BORNHOST_V6_FLAG) == 0 ? 8 : 20;
+        byte[] newBody = messageExt.getBody();
+        if (needCompress && (sysFlag & MessageSysFlag.COMPRESSED_FLAG) == MessageSysFlag.COMPRESSED_FLAG) {
+            newBody = UtilAll.compress(body, 5);
+        }
+        int bodyLength = newBody.length;
+        int storeSize = messageExt.getStoreSize();
+        ByteBuffer byteBuffer;
+        if (storeSize > 0) {
+            byteBuffer = ByteBuffer.allocate(storeSize - 8); // except size for store timestamp
+        } else {
+            storeSize = 4 +  // 1 TOTALSIZE
+                4 +  // 2 MAGICCODE
+                4 +  // 3 BODYCRC
+                4 +  // 4 QUEUEID
+                4 +  // 5 FLAG
+                8 +  // 6 QUEUEOFFSET
+                8 +  // 7 PHYSICALOFFSET
+                4 +  // 8 SYSFLAG
+                8 +  // 9 BORNTIMESTAMP
+                bornhostLength + // 10 BORNHOST
+                4 +  // 11 RECONSUMETIMES
+                8 +  // 12 Prepared Transaction Offset
+                4 + bodyLength +  // 13 BODY
+                +1 + topicLen +  // 14 TOPIC
+                2 + propertiesLength // 15 propertiesLength
+            ;
+            byteBuffer = ByteBuffer.allocate(storeSize);
+        }
+
+        // 1 TOTALSIZE
+        byteBuffer.putInt(storeSize);
+
+        // 2 MAGICCODE
+        byteBuffer.putInt(MESSAGE_MAGIC_CODE);
+
+        // 3 BODYCRC
+        int bodyCRC = messageExt.getBodyCRC();
+        byteBuffer.putInt(bodyCRC);
+
+        // 4 QUEUEID
+        int queueId = messageExt.getQueueId();
+        byteBuffer.putInt(queueId);
+
+        // 5 FLAG
+        int flag = messageExt.getFlag();
+        byteBuffer.putInt(flag);
+
+        // 6 QUEUEOFFSET
+        long queueOffset = messageExt.getQueueOffset();
+        byteBuffer.putLong(queueOffset);
+
+        // 7 PHYSICALOFFSET
+        long physicOffset = messageExt.getCommitLogOffset();
+        byteBuffer.putLong(physicOffset);
+
+        // 8 SYSFLAG
+        byteBuffer.putInt(sysFlag);
+
+        // 9 BORNTIMESTAMP
+        long bornTimeStamp = messageExt.getBornTimestamp();
+        byteBuffer.putLong(bornTimeStamp);
+
+        // 10 BORNHOST
+        InetSocketAddress bornHost = (InetSocketAddress) messageExt.getBornHost();
+        byteBuffer.put(bornHost.getAddress().getAddress());
+        byteBuffer.putInt(bornHost.getPort());
+
+        // 11 RECONSUMETIMES
+        int reconsumeTimes = messageExt.getReconsumeTimes();
+        byteBuffer.putInt(reconsumeTimes);
+
+        // 12 Prepared Transaction Offset
+        long preparedTransactionOffset = messageExt.getPreparedTransactionOffset();
+        byteBuffer.putLong(preparedTransactionOffset);
+
+        // 13 BODY
+        byteBuffer.putInt(bodyLength);
+        byteBuffer.put(newBody);
+
+        // 14 TOPIC
+        byteBuffer.put(topicLen);
+        byteBuffer.put(topics);
+
+        // 15 properties
+        byteBuffer.putShort(propertiesLength);
+        byteBuffer.put(propertiesBytes);
+
+        return byteBuffer.array();
+    }
+
     public static MessageExt decode(
         ByteBuffer byteBuffer, final boolean readBody, final boolean deCompressBody) {
         return decode(byteBuffer, readBody, deCompressBody, false);
     }
 
     public static MessageExt decode(
-        ByteBuffer byteBuffer, final boolean readBody, final boolean deCompressBody, final boolean isClient) {
+        java.nio.ByteBuffer byteBuffer, final boolean readBody, final boolean deCompressBody, final boolean isClient) {
+        return decode(byteBuffer, readBody, deCompressBody, isClient, false, false);
+    }
+
+    public static MessageExt decode(
+        java.nio.ByteBuffer byteBuffer, final boolean readBody, final boolean deCompressBody, final boolean isClient,
+        final boolean isSetPropertiesString) {
+        return decode(byteBuffer, readBody, deCompressBody, isClient, isSetPropertiesString, false);
+    }
+
+    public static MessageExt decode(
+        java.nio.ByteBuffer byteBuffer, final boolean readBody, final boolean deCompressBody, final boolean isClient,
+        final boolean isSetPropertiesString, final boolean checkCRC) {
         try {
 
             MessageExt msgExt;
@@ -284,7 +404,10 @@ public class MessageDecoder {
             msgExt.setStoreSize(storeSize);
 
             // 2 MAGICCODE
-            byteBuffer.getInt();
+            int magicCode = byteBuffer.getInt();
+            if (magicCode != MESSAGE_MAGIC_CODE) {
+                throw new RuntimeException("Unknown magicCode: " + magicCode);
+            }
 
             // 3 BODYCRC
             int bodyCRC = byteBuffer.getInt();
@@ -347,6 +470,14 @@ public class MessageDecoder {
                     byte[] body = new byte[bodyLen];
                     byteBuffer.get(body);
 
+                    if (checkCRC) {
+                        //crc body
+                        int crc = UtilAll.crc32(body, 0, bodyLen);
+                        if (crc != bodyCRC) {
+                            throw new Exception("Msg crc is error!");
+                        }
+                    }
+
                     // uncompress body
                     if (deCompressBody && (sysFlag & MessageSysFlag.COMPRESSED_FLAG) == MessageSysFlag.COMPRESSED_FLAG) {
                         body = UtilAll.uncompress(body);
@@ -370,8 +501,14 @@ public class MessageDecoder {
                 byte[] properties = new byte[propertiesLength];
                 byteBuffer.get(properties);
                 String propertiesString = new String(properties, CHARSET_UTF8);
-                Map<String, String> map = string2messageProperties(propertiesString);
-                msgExt.setProperties(map);
+                if (!isSetPropertiesString) {
+                    Map<String, String> map = string2messageProperties(propertiesString);
+                    msgExt.setProperties(map);
+                } else {
+                    Map<String, String> map = string2messageProperties(propertiesString);
+                    map.put("propertiesString", propertiesString);
+                    msgExt.setProperties(map);
+                }
             }
 
             int msgIDLength = storehostIPLength + 4 + 8;
@@ -396,9 +533,9 @@ public class MessageDecoder {
     }
 
     public static List<MessageExt> decodesBatch(ByteBuffer byteBuffer,
-                                                final boolean readBody,
-                                                final boolean decompressBody,
-                                                final boolean isClient) {
+        final boolean readBody,
+        final boolean decompressBody,
+        final boolean isClient) {
         List<MessageExt> msgExts = new ArrayList<MessageExt>();
         while (byteBuffer.hasRemaining()) {
             MessageExt msgExt = decode(byteBuffer, readBody, decompressBody, isClient);
@@ -494,10 +631,7 @@ public class MessageDecoder {
         String properties = messageProperties2String(message.getProperties());
         byte[] propertiesBytes = properties.getBytes(CHARSET_UTF8);
         //note properties length must not more than Short.MAX
-        int propsLen = propertiesBytes.length;
-        if (propsLen > Short.MAX_VALUE)
-            throw new RuntimeException(String.format("Properties size of message exceeded, properties size: {}, maxSize: {}.", propsLen, Short.MAX_VALUE));
-        short propertiesLength = (short) propsLen;
+        short propertiesLength = (short) propertiesBytes.length;
         int sysFlag = message.getFlag();
         int storeSize = 4 // 1 TOTALSIZE
             + 4 // 2 MAGICCOD
diff --git a/common/src/main/java/org/apache/rocketmq/common/message/MessageExt.java b/common/src/main/java/org/apache/rocketmq/common/message/MessageExt.java
index 133cb93..5905d28 100644
--- a/common/src/main/java/org/apache/rocketmq/common/message/MessageExt.java
+++ b/common/src/main/java/org/apache/rocketmq/common/message/MessageExt.java
@@ -249,6 +249,61 @@ public class MessageExt extends Message {
         this.preparedTransactionOffset = preparedTransactionOffset;
     }
 
+    /**
+     *
+     * achieves topicSysFlag value from transient properties
+     *
+     * @return
+     */
+    public Integer getTopicSysFlag() {
+        String topicSysFlagString = getProperty(MessageConst.PROPERTY_TRANSIENT_TOPIC_CONFIG);
+        if (topicSysFlagString != null && topicSysFlagString.length() > 0) {
+            return Integer.valueOf(topicSysFlagString);
+        }
+        return null;
+    }
+
+    /**
+     * set topicSysFlag to transient properties, or clear it
+     *
+     * @param topicSysFlag
+     */
+    public void setTopicSysFlag(Integer topicSysFlag) {
+        if (topicSysFlag == null) {
+            clearProperty(MessageConst.PROPERTY_TRANSIENT_TOPIC_CONFIG);
+        } else {
+            putProperty(MessageConst.PROPERTY_TRANSIENT_TOPIC_CONFIG, String.valueOf(topicSysFlag));
+        }
+    }
+
+    /**
+     *
+     * achieves groupSysFlag value from transient properties
+     *
+     * @return
+     */
+    public Integer getGroupSysFlag() {
+        String groupSysFlagString = getProperty(MessageConst.PROPERTY_TRANSIENT_GROUP_CONFIG);
+        if (groupSysFlagString != null && groupSysFlagString.length() > 0) {
+            return Integer.valueOf(groupSysFlagString);
+        }
+        return null;
+    }
+
+    /**
+     *
+     * set groupSysFlag to transient properties, or clear it
+     *
+     * @param groupSysFlag
+     */
+    public void setGroupSysFlag(Integer groupSysFlag) {
+        if (groupSysFlag == null) {
+            clearProperty(MessageConst.PROPERTY_TRANSIENT_GROUP_CONFIG);
+        } else {
+            putProperty(MessageConst.PROPERTY_TRANSIENT_GROUP_CONFIG, String.valueOf(groupSysFlag));
+        }
+    }
+
     @Override
     public String toString() {
         return "MessageExt [brokerName=" + brokerName + ", queueId=" + queueId + ", storeSize=" + storeSize + ", queueOffset=" + queueOffset
diff --git a/common/src/main/java/org/apache/rocketmq/common/message/MessageExtBatch.java b/common/src/main/java/org/apache/rocketmq/common/message/MessageExtBatch.java
new file mode 100644
index 0000000..7cdd84d
--- /dev/null
+++ b/common/src/main/java/org/apache/rocketmq/common/message/MessageExtBatch.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.common.message;
+
+import java.nio.ByteBuffer;
+
+public class MessageExtBatch extends MessageExtBrokerInner {
+
+    private static final long serialVersionUID = -2353110995348498537L;
+    /**
+     * Inner batch means the batch dose not need to be unwrapped
+     */
+    private boolean isInnerBatch = false;
+    public ByteBuffer wrap() {
+        assert getBody() != null;
+        return ByteBuffer.wrap(getBody(), 0, getBody().length);
+    }
+
+    public boolean isInnerBatch() {
+        return isInnerBatch;
+    }
+
+    public void setInnerBatch(boolean innerBatch) {
+        isInnerBatch = innerBatch;
+    }
+
+    private ByteBuffer encodedBuff;
+
+    public ByteBuffer getEncodedBuff() {
+        return encodedBuff;
+    }
+
+    public void setEncodedBuff(ByteBuffer encodedBuff) {
+        this.encodedBuff = encodedBuff;
+    }
+}
diff --git a/common/src/main/java/org/apache/rocketmq/common/message/MessageExtBrokerInner.java b/common/src/main/java/org/apache/rocketmq/common/message/MessageExtBrokerInner.java
new file mode 100644
index 0000000..7af7014
--- /dev/null
+++ b/common/src/main/java/org/apache/rocketmq/common/message/MessageExtBrokerInner.java
@@ -0,0 +1,63 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.rocketmq.common.message;
+
+import java.nio.ByteBuffer;
+
+import org.apache.rocketmq.common.TopicFilterType;
+
+public class MessageExtBrokerInner extends MessageExt {
+    private static final long serialVersionUID = 7256001576878700634L;
+    private String propertiesString;
+    private long tagsCode;
+
+    private ByteBuffer encodedBuff;
+
+    public ByteBuffer getEncodedBuff() {
+        return encodedBuff;
+    }
+
+    public void setEncodedBuff(ByteBuffer encodedBuff) {
+        this.encodedBuff = encodedBuff;
+    }
+
+    public static long tagsString2tagsCode(final TopicFilterType filter, final String tags) {
+        if (null == tags || tags.length() == 0) { return 0; }
+
+        return tags.hashCode();
+    }
+
+    public static long tagsString2tagsCode(final String tags) {
+        return tagsString2tagsCode(null, tags);
+    }
+
+    public String getPropertiesString() {
+        return propertiesString;
+    }
+
+    public void setPropertiesString(String propertiesString) {
+        this.propertiesString = propertiesString;
+    }
+
+    public long getTagsCode() {
+        return tagsCode;
+    }
+
+    public void setTagsCode(long tagsCode) {
+        this.tagsCode = tagsCode;
+    }
+}
diff --git a/common/src/main/java/org/apache/rocketmq/common/message/MessageType.java b/common/src/main/java/org/apache/rocketmq/common/message/MessageType.java
index 08091f4..3afbb1f 100644
--- a/common/src/main/java/org/apache/rocketmq/common/message/MessageType.java
+++ b/common/src/main/java/org/apache/rocketmq/common/message/MessageType.java
@@ -18,8 +18,28 @@
 package org.apache.rocketmq.common.message;
 
 public enum MessageType {
-    Normal_Msg,
-    Trans_Msg_Half,
-    Trans_msg_Commit,
-    Delay_Msg,
+    Normal_Msg("Normal"),
+    Order_Msg("Order"),
+    Trans_Msg_Half("Trans"),
+    Trans_msg_Commit("TransCommit"),
+    Delay_Msg("Delay");
+
+    private final String shortName;
+
+    MessageType(String shortName) {
+        this.shortName = shortName;
+    }
+
+    public String getShortName() {
+        return shortName;
+    }
+
+    public static MessageType getByShortName(String shortName) {
+        for (MessageType msgType : MessageType.values()) {
+            if (msgType.getShortName().equals(shortName)) {
+                return msgType;
+            }
+        }
+        return Normal_Msg;
+    }
 }
diff --git a/common/src/main/java/org/apache/rocketmq/common/message/MessageVersion.java b/common/src/main/java/org/apache/rocketmq/common/message/MessageVersion.java
new file mode 100644
index 0000000..c81275a
--- /dev/null
+++ b/common/src/main/java/org/apache/rocketmq/common/message/MessageVersion.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.common.message;
+
+import java.nio.ByteBuffer;
+
+public enum MessageVersion {
+
+    MESSAGE_VERSION_V1(MessageDecoder.MESSAGE_MAGIC_CODE) {
+        @Override
+        public int getTopicLengthSize() {
+            return 1;
+        }
+
+        @Override
+        public int getTopicLength(ByteBuffer buffer) {
+            return buffer.get();
+        }
+
+        @Override
+        public int getTopicLength(ByteBuffer buffer, int index) {
+            return buffer.get(index);
+        }
+
+        @Override
+        public void putTopicLength(ByteBuffer buffer, int topicLength) {
+            buffer.put((byte) topicLength);
+        }
+    };
+
+    private final int magicCode;
+
+    MessageVersion(int magicCode) {
+        this.magicCode = magicCode;
+    }
+
+    public static MessageVersion valueOfMagicCode(int magicCode) {
+        for (MessageVersion version : MessageVersion.values()) {
+            if (version.getMagicCode() == magicCode) {
+                return version;
+            }
+        }
+
+        throw new IllegalArgumentException("Invalid magicCode " + magicCode);
+    }
+
+    public int getMagicCode() {
+        return magicCode;
+    }
+
+    public abstract int getTopicLengthSize();
+
+    public abstract int getTopicLength(java.nio.ByteBuffer buffer);
+    public abstract int getTopicLength(java.nio.ByteBuffer buffer, int index);
+    public abstract void putTopicLength(java.nio.ByteBuffer buffer, int topicLength);
+}
diff --git a/common/src/main/java/org/apache/rocketmq/common/namesrv/DefaultTopAddressing.java b/common/src/main/java/org/apache/rocketmq/common/namesrv/DefaultTopAddressing.java
new file mode 100644
index 0000000..6e7a97d
--- /dev/null
+++ b/common/src/main/java/org/apache/rocketmq/common/namesrv/DefaultTopAddressing.java
@@ -0,0 +1,165 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.rocketmq.common.namesrv;
+
+import com.google.common.base.Strings;
+import java.io.IOException;
+import java.util.ArrayList;
+import java.util.Iterator;
+import java.util.List;
+import java.util.ServiceLoader;
+import java.util.Map;
+import org.apache.rocketmq.common.MixAll;
+import org.apache.rocketmq.common.UtilAll;
+import org.apache.rocketmq.common.constant.LoggerName;
+import org.apache.rocketmq.common.help.FAQUrl;
+import org.apache.rocketmq.common.utils.HttpTinyClient;
+import org.apache.rocketmq.logging.InternalLogger;
+import org.apache.rocketmq.logging.InternalLoggerFactory;
+
+public class DefaultTopAddressing implements TopAddressing {
+
+    private static final InternalLogger LOGGER = InternalLoggerFactory.getLogger(LoggerName.COMMON_LOGGER_NAME);
+
+    private String nsAddr;
+    private String wsAddr;
+    private String unitName;
+    private Map<String, String> para;
+    private List<TopAddressing> topAddressingList;
+
+    public DefaultTopAddressing(final String wsAddr) {
+        this(wsAddr, null);
+    }
+
+    public DefaultTopAddressing(final String wsAddr, final String unitName) {
+        this.wsAddr = wsAddr;
+        this.unitName = unitName;
+        this.topAddressingList = loadCustomTopAddressing();
+    }
+
+
+    public DefaultTopAddressing(final String unitName, final Map<String, String> para, final String wsAddr) {
+        this.wsAddr = wsAddr;
+        this.unitName = unitName;
+        this.para = para;
+        this.topAddressingList = loadCustomTopAddressing();
+    }
+
+    private static String clearNewLine(final String str) {
+        String newString = str.trim();
+        int index = newString.indexOf("\r");
+        if (index != -1) {
+            return newString.substring(0, index);
+        }
+
+        index = newString.indexOf("\n");
+        if (index != -1) {
+            return newString.substring(0, index);
+        }
+
+        return newString;
+    }
+
+    private List<TopAddressing> loadCustomTopAddressing() {
+        ServiceLoader<TopAddressing> serviceLoader = ServiceLoader.load(TopAddressing.class);
+        Iterator<TopAddressing> iterator = serviceLoader.iterator();
+        List<TopAddressing> topAddressingList = new ArrayList<TopAddressing>();
+        if (iterator.hasNext()) {
+            topAddressingList.add(iterator.next());
+        }
+        return topAddressingList;
+    }
+
+    @Override
+    public final String fetchNSAddr() {
+        if (!topAddressingList.isEmpty()) {
+            for (TopAddressing topAddressing : topAddressingList) {
+                String nsAddress = topAddressing.fetchNSAddr();
+                if (!Strings.isNullOrEmpty(nsAddress)) {
+                    return nsAddress;
+                }
+            }
+        }
+        // Return result of default implementation
+        return fetchNSAddr(true, 3000);
+    }
+
+    @Override
+    public void registerChangeCallBack(NameServerUpdateCallback changeCallBack) {
+        if (!topAddressingList.isEmpty()) {
+            for (TopAddressing topAddressing : topAddressingList) {
+                topAddressing.registerChangeCallBack(changeCallBack);
+            }
+        }
+    }
+
+    public final String fetchNSAddr(boolean verbose, long timeoutMills) {
+        String url = this.wsAddr;
+        try {
+            if (null != para && para.size() > 0) {
+                if (!UtilAll.isBlank(this.unitName)) {
+                    url = url + "-" + this.unitName + "?nofix=1&";
+                }
+                else {
+                    url = url + "?";
+                }
+                for (Map.Entry<String, String> entry : this.para.entrySet()) {
+                    url += entry.getKey() + "=" + entry.getValue() + "&";
+                }
+                url = url.substring(0, url.length() - 1);
+            }
+            else {
+                if (!UtilAll.isBlank(this.unitName)) {
+                    url = url + "-" + this.unitName + "?nofix=1";
+                }
+            }
+
+            HttpTinyClient.HttpResult result = HttpTinyClient.httpGet(url, null, null, "UTF-8", timeoutMills);
+            if (200 == result.code) {
+                String responseStr = result.content;
+                if (responseStr != null) {
+                    return clearNewLine(responseStr);
+                } else {
+                    LOGGER.error("fetch nameserver address is null");
+                }
+            } else {
+                LOGGER.error("fetch nameserver address failed. statusCode=" + result.code);
+            }
+        } catch (IOException e) {
+            if (verbose) {
+                LOGGER.error("fetch name server address exception", e);
+            }
+        }
+
+        if (verbose) {
+            String errorMsg =
+                "connect to " + url + " failed, maybe the domain name " + MixAll.getWSAddr() + " not bind in /etc/hosts";
+            errorMsg += FAQUrl.suggestTodo(FAQUrl.NAME_SERVER_ADDR_NOT_EXIST_URL);
+
+            LOGGER.warn(errorMsg);
+        }
+        return null;
+    }
+
+    public String getNsAddr() {
+        return nsAddr;
+    }
+
+    public void setNsAddr(String nsAddr) {
+        this.nsAddr = nsAddr;
+    }
+}
diff --git a/common/src/main/java/org/apache/rocketmq/common/message/MessageType.java b/common/src/main/java/org/apache/rocketmq/common/namesrv/NameServerUpdateCallback.java
similarity index 84%
copy from common/src/main/java/org/apache/rocketmq/common/message/MessageType.java
copy to common/src/main/java/org/apache/rocketmq/common/namesrv/NameServerUpdateCallback.java
index 08091f4..67ce2b8 100644
--- a/common/src/main/java/org/apache/rocketmq/common/message/MessageType.java
+++ b/common/src/main/java/org/apache/rocketmq/common/namesrv/NameServerUpdateCallback.java
@@ -14,12 +14,8 @@
  * See the License for the specific language governing permissions and
  * limitations under the License.
  */
+package org.apache.rocketmq.common.namesrv;
 
-package org.apache.rocketmq.common.message;
-
-public enum MessageType {
-    Normal_Msg,
-    Trans_Msg_Half,
-    Trans_msg_Commit,
-    Delay_Msg,
+public interface NameServerUpdateCallback {
+    String onNameServerAddressChange(String namesrvAddress);
 }
diff --git a/common/src/main/java/org/apache/rocketmq/common/namesrv/NamesrvConfig.java b/common/src/main/java/org/apache/rocketmq/common/namesrv/NamesrvConfig.java
index f687d2c..002c956 100644
--- a/common/src/main/java/org/apache/rocketmq/common/namesrv/NamesrvConfig.java
+++ b/common/src/main/java/org/apache/rocketmq/common/namesrv/NamesrvConfig.java
@@ -35,6 +35,51 @@ public class NamesrvConfig {
     private String productEnvName = "center";
     private boolean clusterTest = false;
     private boolean orderMessageEnable = false;
+    private boolean returnOrderTopicConfigToBroker = true;
+    private volatile boolean remoteFaultTolerance = true;
+
+    /**
+     * Indicates the nums of thread to handle client requests, like GET_ROUTEINTO_BY_TOPIC.
+     */
+    private int clientRequestThreadPoolNums = 8;
+    /**
+     * Indicates the nums of thread to handle broker or operation requests, like REGISTER_BROKER.
+     */
+    private int defaultThreadPoolNums = 16;
+    /**
+     * Indicates the capacity of queue to hold client requests.
+     */
+    private int clientRequestThreadPoolQueueCapacity = 50000;
+    /**
+     * Indicates the capacity of queue to hold broker or operation requests.
+     */
+    private int defaultThreadPoolQueueCapacity = 10000;
+    /**
+     * Interval of periodic scanning for non-active broker;
+     */
+    private long scanNotActiveBrokerInterval = 5 * 1000;
+
+    private int unRegisterBrokerQueueCapacity = 3000;
+
+    /**
+     * Support acting master or not.
+     *
+     * The slave can be an acting master when master node is down to support following operations:
+     * 1. support lock/unlock message queue operation.
+     * 2. support searchOffset, query maxOffset/minOffset operation.
+     * 3. support query earliest msg store time.
+     */
+    private boolean supportActingMaster = false;
+
+    private volatile boolean enableAllTopicList = false;
+
+    public void setRemoteFaultTolerance(boolean remoteFaultTolerance) {
+        this.remoteFaultTolerance = remoteFaultTolerance;
+    }
+
+    public boolean isRemoteFaultTolerance() {
+        return remoteFaultTolerance;
+    }
 
     public boolean isOrderMessageEnable() {
         return orderMessageEnable;
@@ -83,4 +128,76 @@ public class NamesrvConfig {
     public void setConfigStorePath(final String configStorePath) {
         this.configStorePath = configStorePath;
     }
+
+    public boolean isReturnOrderTopicConfigToBroker() {
+        return returnOrderTopicConfigToBroker;
+    }
+
+    public void setReturnOrderTopicConfigToBroker(boolean returnOrderTopicConfigToBroker) {
+        this.returnOrderTopicConfigToBroker = returnOrderTopicConfigToBroker;
+    }
+
+    public int getClientRequestThreadPoolNums() {
+        return clientRequestThreadPoolNums;
+    }
+
+    public void setClientRequestThreadPoolNums(final int clientRequestThreadPoolNums) {
+        this.clientRequestThreadPoolNums = clientRequestThreadPoolNums;
+    }
+
+    public int getDefaultThreadPoolNums() {
+        return defaultThreadPoolNums;
+    }
+
+    public void setDefaultThreadPoolNums(final int defaultThreadPoolNums) {
+        this.defaultThreadPoolNums = defaultThreadPoolNums;
+    }
+
+    public int getClientRequestThreadPoolQueueCapacity() {
+        return clientRequestThreadPoolQueueCapacity;
+    }
+
+    public void setClientRequestThreadPoolQueueCapacity(final int clientRequestThreadPoolQueueCapacity) {
+        this.clientRequestThreadPoolQueueCapacity = clientRequestThreadPoolQueueCapacity;
+    }
+
+    public int getDefaultThreadPoolQueueCapacity() {
+        return defaultThreadPoolQueueCapacity;
+    }
+
+    public void setDefaultThreadPoolQueueCapacity(final int defaultThreadPoolQueueCapacity) {
+        this.defaultThreadPoolQueueCapacity = defaultThreadPoolQueueCapacity;
+    }
+
+    public long getScanNotActiveBrokerInterval() {
+        return scanNotActiveBrokerInterval;
+    }
+
+    public void setScanNotActiveBrokerInterval(long scanNotActiveBrokerInterval) {
+        this.scanNotActiveBrokerInterval = scanNotActiveBrokerInterval;
+    }
+
+    public int getUnRegisterBrokerQueueCapacity() {
+        return unRegisterBrokerQueueCapacity;
+    }
+
+    public void setUnRegisterBrokerQueueCapacity(final int unRegisterBrokerQueueCapacity) {
+        this.unRegisterBrokerQueueCapacity = unRegisterBrokerQueueCapacity;
+    }
+
+    public boolean isSupportActingMaster() {
+        return supportActingMaster;
+    }
+
+    public void setSupportActingMaster(final boolean supportActingMaster) {
+        this.supportActingMaster = supportActingMaster;
+    }
+
+    public boolean isEnableAllTopicList() {
+        return enableAllTopicList;
+    }
+
+    public void setEnableAllTopicList(boolean enableAllTopicList) {
+        this.enableAllTopicList = enableAllTopicList;
+    }
 }
diff --git a/common/src/main/java/org/apache/rocketmq/common/namesrv/TopAddressing.java b/common/src/main/java/org/apache/rocketmq/common/namesrv/TopAddressing.java
index 88a4122..3ca1828 100644
--- a/common/src/main/java/org/apache/rocketmq/common/namesrv/TopAddressing.java
+++ b/common/src/main/java/org/apache/rocketmq/common/namesrv/TopAddressing.java
@@ -14,94 +14,12 @@
  * See the License for the specific language governing permissions and
  * limitations under the License.
  */
-
-/**
- * $Id: TopAddressing.java 1831 2013-05-16 01:39:51Z vintagewang@apache.org $
- */
 package org.apache.rocketmq.common.namesrv;
 
-import java.io.IOException;
-import org.apache.rocketmq.common.MixAll;
-import org.apache.rocketmq.common.UtilAll;
-import org.apache.rocketmq.common.constant.LoggerName;
-import org.apache.rocketmq.common.help.FAQUrl;
-import org.apache.rocketmq.logging.InternalLogger;
-import org.apache.rocketmq.logging.InternalLoggerFactory;
-import org.apache.rocketmq.common.utils.HttpTinyClient;
-
-public class TopAddressing {
-    private static final InternalLogger log = InternalLoggerFactory.getLogger(LoggerName.COMMON_LOGGER_NAME);
-
-    private String nsAddr;
-    private String wsAddr;
-    private String unitName;
-
-    public TopAddressing(final String wsAddr) {
-        this(wsAddr, null);
-    }
-
-    public TopAddressing(final String wsAddr, final String unitName) {
-        this.wsAddr = wsAddr;
-        this.unitName = unitName;
-    }
-
-    private static String clearNewLine(final String str) {
-        String newString = str.trim();
-        int index = newString.indexOf("\r");
-        if (index != -1) {
-            return newString.substring(0, index);
-        }
-
-        index = newString.indexOf("\n");
-        if (index != -1) {
-            return newString.substring(0, index);
-        }
-
-        return newString;
-    }
-
-    public final String fetchNSAddr() {
-        return fetchNSAddr(true, 3000);
-    }
-
-    public final String fetchNSAddr(boolean verbose, long timeoutMills) {
-        String url = this.wsAddr;
-        try {
-            if (!UtilAll.isBlank(this.unitName)) {
-                url = url + "-" + this.unitName + "?nofix=1";
-            }
-            HttpTinyClient.HttpResult result = HttpTinyClient.httpGet(url, null, null, "UTF-8", timeoutMills);
-            if (200 == result.code) {
-                String responseStr = result.content;
-                if (responseStr != null) {
-                    return clearNewLine(responseStr);
-                } else {
-                    log.error("fetch nameserver address is null");
-                }
-            } else {
-                log.error("fetch nameserver address failed. statusCode=" + result.code);
-            }
-        } catch (IOException e) {
-            if (verbose) {
-                log.error("fetch name server address exception", e);
-            }
-        }
-
-        if (verbose) {
-            String errorMsg =
-                "connect to " + url + " failed, maybe the domain name " + MixAll.getWSAddr() + " not bind in /etc/hosts";
-            errorMsg += FAQUrl.suggestTodo(FAQUrl.NAME_SERVER_ADDR_NOT_EXIST_URL);
 
-            log.warn(errorMsg);
-        }
-        return null;
-    }
+public interface TopAddressing {
 
-    public String getNsAddr() {
-        return nsAddr;
-    }
+    String fetchNSAddr();
 
-    public void setNsAddr(String nsAddr) {
-        this.nsAddr = nsAddr;
-    }
+    void registerChangeCallBack(NameServerUpdateCallback changeCallBack);
 }
diff --git a/common/src/main/java/org/apache/rocketmq/common/protocol/header/namesrv/DeleteTopicFromNamesrvRequestHeader.java b/common/src/main/java/org/apache/rocketmq/common/protocol/ForbiddenType.java
similarity index 55%
copy from common/src/main/java/org/apache/rocketmq/common/protocol/header/namesrv/DeleteTopicFromNamesrvRequestHeader.java
copy to common/src/main/java/org/apache/rocketmq/common/protocol/ForbiddenType.java
index ed8e7a3..1c661c2 100644
--- a/common/src/main/java/org/apache/rocketmq/common/protocol/header/namesrv/DeleteTopicFromNamesrvRequestHeader.java
+++ b/common/src/main/java/org/apache/rocketmq/common/protocol/ForbiddenType.java
@@ -14,25 +14,35 @@
  * See the License for the specific language governing permissions and
  * limitations under the License.
  */
-package org.apache.rocketmq.common.protocol.header.namesrv;
 
-import org.apache.rocketmq.remoting.CommandCustomHeader;
-import org.apache.rocketmq.remoting.annotation.CFNotNull;
-import org.apache.rocketmq.remoting.exception.RemotingCommandException;
+package org.apache.rocketmq.common.protocol;
 
-public class DeleteTopicFromNamesrvRequestHeader implements CommandCustomHeader {
-    @CFNotNull
-    private String topic;
-
-    @Override
-    public void checkFields() throws RemotingCommandException {
-    }
+/**
+ * 
+ * gives the reason for a no permission messaging pulling.
+ *
+ */
+public interface ForbiddenType {
 
-    public String getTopic() {
-        return topic;
-    }
+    /**
+     * 1=forbidden by broker
+     */
+    int BROKER_FORBIDDEN               = 1;
+    /**
+     * 2=forbidden by groupId
+     */
+    int GROUP_FORBIDDEN                = 2;
+    /**
+     * 3=forbidden by topic
+     */
+    int TOPIC_FORBIDDEN                = 3;
+    /**
+     * 4=forbidden by brocasting mode
+     */
+    int BROADCASTING_DISABLE_FORBIDDEN = 4;
+    /**
+     * 5=forbidden for a substription(group with a topic)
+     */
+    int SUBSCRIPTION_FORBIDDEN         = 5;
 
-    public void setTopic(String topic) {
-        this.topic = topic;
-    }
 }
diff --git a/common/src/main/java/org/apache/rocketmq/common/protocol/RequestCode.java b/common/src/main/java/org/apache/rocketmq/common/protocol/RequestCode.java
index 534a2d2..541227f 100644
--- a/common/src/main/java/org/apache/rocketmq/common/protocol/RequestCode.java
+++ b/common/src/main/java/org/apache/rocketmq/common/protocol/RequestCode.java
@@ -70,6 +70,8 @@ public class RequestCode {
 
     public static final int CHECK_CLIENT_CONFIG = 46;
 
+    public static final int GET_CLIENT_CONFIG = 47;
+
     public static final int UPDATE_AND_CREATE_ACL_CONFIG = 50;
 
     public static final int DELETE_ACL_CONFIG = 51;
@@ -82,7 +84,10 @@ public class RequestCode {
 
     public static final int POP_MESSAGE = 200050;
     public static final int ACK_MESSAGE = 200051;
+    public static final int PEEK_MESSAGE = 200052;
     public static final int CHANGE_MESSAGE_INVISIBLETIME = 200053;
+    public static final int NOTIFICATION = 200054;
+    public static final int POLLING_INFO = 200055;
 
     public static final int PUT_KV_CONFIG = 100;
 
@@ -121,6 +126,7 @@ public class RequestCode {
     public static final int DELETE_TOPIC_IN_BROKER = 215;
 
     public static final int DELETE_TOPIC_IN_NAMESRV = 216;
+    public static final int REGISTER_TOPIC_IN_NAMESRV = 217;
     public static final int GET_KVLIST_BY_NAMESPACE = 219;
 
     public static final int RESET_CONSUMER_CLIENT_OFFSET = 220;
@@ -135,6 +141,9 @@ public class RequestCode {
 
     public static final int GET_TOPICS_BY_CLUSTER = 224;
 
+    public static final int QUERY_TOPICS_BY_CONSUMER = 343;
+    public static final int QUERY_SUBSCRIPTION_BY_CONSUMER = 345;
+
     public static final int REGISTER_FILTER_SERVER = 301;
     public static final int REGISTER_MESSAGE_FILTER_CLASS = 302;
 
@@ -195,11 +204,36 @@ public class RequestCode {
 
     public static final int ADD_WRITE_PERM_OF_BROKER = 327;
 
+
     public static final int GET_TOPIC_CONFIG = 351;
 
+    public static final int GET_SUBSCRIPTIONGROUP_CONFIG = 352;
+    public static final int UPDATE_AND_GET_GROUP_FORBIDDEN = 353;
+
+    public static final int LITE_PULL_MESSAGE = 361;
+
     public static final int QUERY_ASSIGNMENT = 400;
     public static final int SET_MESSAGE_REQUEST_MODE = 401;
 
     public static final int UPDATE_AND_CREATE_STATIC_TOPIC = 513;
 
+    /**
+     * Below request codes are used by broker container,
+     * these request codes are started with '9'.
+     */
+    public static final int GET_BROKER_MEMBER_GROUP = 901;
+
+    public static final int ADD_BROKER = 902;
+
+    public static final int REMOVE_BROKER = 903;
+
+    public static final int BROKER_HEARTBEAT = 904;
+
+    public static final int NOTIFY_MIN_BROKER_ID_CHANGE = 905;
+
+    public static final int EXCHANGE_BROKER_HA_INFO = 906;
+
+    public static final int GET_BROKER_HA_STATUS = 907;
+
+    public static final int RESET_MASTER_FLUSH_OFFSET = 908;
 }
diff --git a/common/src/main/java/org/apache/rocketmq/common/protocol/ResponseCode.java b/common/src/main/java/org/apache/rocketmq/common/protocol/ResponseCode.java
index 3f691b5..db2a275 100644
--- a/common/src/main/java/org/apache/rocketmq/common/protocol/ResponseCode.java
+++ b/common/src/main/java/org/apache/rocketmq/common/protocol/ResponseCode.java
@@ -20,7 +20,6 @@ package org.apache.rocketmq.common.protocol;
 import org.apache.rocketmq.remoting.protocol.RemotingSysResponseCode;
 
 public class ResponseCode extends RemotingSysResponseCode {
-    public static final int ASYNC_AND_RETURN_NULL = -2;
 
     public static final int FLUSH_DISK_TIMEOUT = 10;
 
@@ -85,6 +84,12 @@ public class ResponseCode extends RemotingSysResponseCode {
 
     public static final int POLLING_TIMEOUT = 210;
 
+    public static final int BROKER_NOT_EXIST = 211;
+
+    public static final int BROKER_DISPATCH_NOT_COMPLETE = 212;
+
+    public static final int FLOW_CONTROL = 215;
+
     public static final int NOT_LEADER_FOR_QUEUE = 501;
 
     public static final int RPC_UNKNOWN = -1000;
diff --git a/common/src/main/java/org/apache/rocketmq/common/protocol/body/BrokerMemberGroup.java b/common/src/main/java/org/apache/rocketmq/common/protocol/body/BrokerMemberGroup.java
new file mode 100644
index 0000000..eb02f88
--- /dev/null
+++ b/common/src/main/java/org/apache/rocketmq/common/protocol/body/BrokerMemberGroup.java
@@ -0,0 +1,90 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.rocketmq.common.protocol.body;
+
+import com.google.common.base.Objects;
+import java.util.Collections;
+import java.util.HashMap;
+import java.util.Map;
+
+public class BrokerMemberGroup {
+    private String cluster;
+    private String brokerName;
+    private Map<Long/* brokerId */, String/* broker address */> brokerAddrs;
+
+    // Provide default constructor for serializer
+    public BrokerMemberGroup() {
+        this.brokerAddrs = new HashMap<Long, String>();
+    }
+
+    public BrokerMemberGroup(final String cluster, final String brokerName) {
+        this.cluster = cluster;
+        this.brokerName = brokerName;
+        this.brokerAddrs = new HashMap<Long, String>();
+    }
+
+    public long minimumBrokerId() {
+        if (this.brokerAddrs.isEmpty()) {
+            return 0;
+        }
+        return Collections.min(brokerAddrs.keySet());
+    }
+
+    public String getCluster() {
+        return cluster;
+    }
+
+    public void setCluster(final String cluster) {
+        this.cluster = cluster;
+    }
+
+    public String getBrokerName() {
+        return brokerName;
+    }
+
+    public void setBrokerName(final String brokerName) {
+        this.brokerName = brokerName;
+    }
+
+    public Map<Long, String> getBrokerAddrs() {
+        return brokerAddrs;
+    }
+
+    public void setBrokerAddrs(final Map<Long, String> brokerAddrs) {
+        this.brokerAddrs = brokerAddrs;
+    }
+
+    @Override
+    public boolean equals(Object o) {
+        if (this == o) {
+            return true;
+        }
+        if (o == null || getClass() != o.getClass()) {
+            return false;
+        }
+        BrokerMemberGroup that = (BrokerMemberGroup) o;
+        return Objects.equal(cluster, that.cluster) &&
+            Objects.equal(brokerName, that.brokerName) &&
+            Objects.equal(brokerAddrs, that.brokerAddrs);
+    }
+
+    @Override
+    public int hashCode() {
+        return Objects.hashCode(cluster, brokerName, brokerAddrs);
+    }
+}
diff --git a/common/src/main/java/org/apache/rocketmq/common/protocol/body/CheckClientRequestBody.java b/common/src/main/java/org/apache/rocketmq/common/protocol/body/CheckClientRequestBody.java
index a78ce55..70d6011 100644
--- a/common/src/main/java/org/apache/rocketmq/common/protocol/body/CheckClientRequestBody.java
+++ b/common/src/main/java/org/apache/rocketmq/common/protocol/body/CheckClientRequestBody.java
@@ -25,6 +25,7 @@ public class CheckClientRequestBody extends RemotingSerializable {
     private String clientId;
     private String group;
     private SubscriptionData subscriptionData;
+    private String namespace;
 
     public String getClientId() {
         return clientId;
@@ -49,4 +50,12 @@ public class CheckClientRequestBody extends RemotingSerializable {
     public void setSubscriptionData(SubscriptionData subscriptionData) {
         this.subscriptionData = subscriptionData;
     }
+
+    public String getNamespace() {
+        return namespace;
+    }
+
+    public void setNamespace(String namespace) {
+        this.namespace = namespace;
+    }
 }
diff --git a/common/src/main/java/org/apache/rocketmq/common/protocol/body/ClusterInfo.java b/common/src/main/java/org/apache/rocketmq/common/protocol/body/ClusterInfo.java
index 222e51a..700cff5 100644
--- a/common/src/main/java/org/apache/rocketmq/common/protocol/body/ClusterInfo.java
+++ b/common/src/main/java/org/apache/rocketmq/common/protocol/body/ClusterInfo.java
@@ -19,29 +19,29 @@ package org.apache.rocketmq.common.protocol.body;
 
 import com.google.common.base.Objects;
 import java.util.ArrayList;
-import java.util.HashMap;
+import java.util.Map;
 import java.util.List;
 import java.util.Set;
 import org.apache.rocketmq.common.protocol.route.BrokerData;
 import org.apache.rocketmq.remoting.protocol.RemotingSerializable;
 
 public class ClusterInfo extends RemotingSerializable {
-    private HashMap<String/* brokerName */, BrokerData> brokerAddrTable;
-    private HashMap<String/* clusterName */, Set<String/* brokerName */>> clusterAddrTable;
+    private Map<String/* brokerName */, BrokerData> brokerAddrTable;
+    private Map<String/* clusterName */, Set<String/* brokerName */>> clusterAddrTable;
 
-    public HashMap<String, BrokerData> getBrokerAddrTable() {
+    public Map<String, BrokerData> getBrokerAddrTable() {
         return brokerAddrTable;
     }
 
-    public void setBrokerAddrTable(HashMap<String, BrokerData> brokerAddrTable) {
+    public void setBrokerAddrTable(Map<String, BrokerData> brokerAddrTable) {
         this.brokerAddrTable = brokerAddrTable;
     }
 
-    public HashMap<String, Set<String>> getClusterAddrTable() {
+    public Map<String, Set<String>> getClusterAddrTable() {
         return clusterAddrTable;
     }
 
-    public void setClusterAddrTable(HashMap<String, Set<String>> clusterAddrTable) {
+    public void setClusterAddrTable(Map<String, Set<String>> clusterAddrTable) {
         this.clusterAddrTable = clusterAddrTable;
     }
 
diff --git a/common/src/main/java/org/apache/rocketmq/common/protocol/body/ConsumerOffsetSerializeWrapper.java b/common/src/main/java/org/apache/rocketmq/common/protocol/body/ConsumerOffsetSerializeWrapper.java
index 5b08d78..1317d54 100644
--- a/common/src/main/java/org/apache/rocketmq/common/protocol/body/ConsumerOffsetSerializeWrapper.java
+++ b/common/src/main/java/org/apache/rocketmq/common/protocol/body/ConsumerOffsetSerializeWrapper.java
@@ -19,11 +19,13 @@ package org.apache.rocketmq.common.protocol.body;
 
 import java.util.concurrent.ConcurrentHashMap;
 import java.util.concurrent.ConcurrentMap;
+import org.apache.rocketmq.common.DataVersion;
 import org.apache.rocketmq.remoting.protocol.RemotingSerializable;
 
 public class ConsumerOffsetSerializeWrapper extends RemotingSerializable {
     private ConcurrentMap<String/* topic@group */, ConcurrentMap<Integer, Long>> offsetTable =
         new ConcurrentHashMap<String, ConcurrentMap<Integer, Long>>(512);
+    private DataVersion dataVersion;
 
     public ConcurrentMap<String, ConcurrentMap<Integer, Long>> getOffsetTable() {
         return offsetTable;
@@ -32,4 +34,12 @@ public class ConsumerOffsetSerializeWrapper extends RemotingSerializable {
     public void setOffsetTable(ConcurrentMap<String, ConcurrentMap<Integer, Long>> offsetTable) {
         this.offsetTable = offsetTable;
     }
+
+    public DataVersion getDataVersion() {
+        return dataVersion;
+    }
+
+    public void setDataVersion(DataVersion dataVersion) {
+        this.dataVersion = dataVersion;
+    }
 }
diff --git a/common/src/main/java/org/apache/rocketmq/common/protocol/body/ConsumerRunningInfo.java b/common/src/main/java/org/apache/rocketmq/common/protocol/body/ConsumerRunningInfo.java
index 0543add..6b0355f 100644
--- a/common/src/main/java/org/apache/rocketmq/common/protocol/body/ConsumerRunningInfo.java
+++ b/common/src/main/java/org/apache/rocketmq/common/protocol/body/ConsumerRunningInfo.java
@@ -45,6 +45,8 @@ public class ConsumerRunningInfo extends RemotingSerializable {
 
     private TreeMap<String/* Topic */, ConsumeStatus> statusTable = new TreeMap<String, ConsumeStatus>();
 
+    private TreeMap<String, String> userConsumerInfo = new TreeMap<String, String>();
+
     private String jstack;
 
     public static boolean analyzeSubscription(final TreeMap<String/* clientId */, ConsumerRunningInfo> criTable) {
@@ -87,13 +89,14 @@ public class ConsumerRunningInfo extends RemotingSerializable {
                     prev = next.getValue();
                 }
 
-                if (prev != null) {
-
-                    if (prev.getSubscriptionSet().isEmpty()) {
-                        // Subscription empty!
-                        return false;
-                    }
-                }
+                // after consumer.unsubscribe , SubscriptionSet is Empty
+                //if (prev != null) {
+                //
+                //    if (prev.getSubscriptionSet().isEmpty()) {
+                //        // Subscription empty!
+                //        return false;
+                //    }
+                //}
             }
         }
 
@@ -193,6 +196,10 @@ public class ConsumerRunningInfo extends RemotingSerializable {
         this.statusTable = statusTable;
     }
 
+    public TreeMap<String, String> getUserConsumerInfo() {
+        return userConsumerInfo;
+    }
+
     public String formatString() {
         StringBuilder sb = new StringBuilder();
 
@@ -318,6 +325,16 @@ public class ConsumerRunningInfo extends RemotingSerializable {
             }
         }
 
+        if (this.userConsumerInfo != null) {
+            sb.append("\n\n#User Consume Info#\n");
+            Iterator<Entry<String, String>> it = this.userConsumerInfo.entrySet().iterator();
+            while (it.hasNext()) {
+                Entry<String, String> next = it.next();
+                String item = String.format("%-40s: %s%n", next.getKey(), next.getValue());
+                sb.append(item);
+            }
+        }
+
         if (this.jstack != null) {
             sb.append("\n\n#Consumer jstack#\n");
             sb.append(this.jstack);
diff --git a/common/src/main/java/org/apache/rocketmq/common/protocol/body/ConsumerOffsetSerializeWrapper.java b/common/src/main/java/org/apache/rocketmq/common/protocol/body/GetBrokerMemberGroupResponseBody.java
similarity index 61%
copy from common/src/main/java/org/apache/rocketmq/common/protocol/body/ConsumerOffsetSerializeWrapper.java
copy to common/src/main/java/org/apache/rocketmq/common/protocol/body/GetBrokerMemberGroupResponseBody.java
index 5b08d78..2762587 100644
--- a/common/src/main/java/org/apache/rocketmq/common/protocol/body/ConsumerOffsetSerializeWrapper.java
+++ b/common/src/main/java/org/apache/rocketmq/common/protocol/body/GetBrokerMemberGroupResponseBody.java
@@ -17,19 +17,17 @@
 
 package org.apache.rocketmq.common.protocol.body;
 
-import java.util.concurrent.ConcurrentHashMap;
-import java.util.concurrent.ConcurrentMap;
 import org.apache.rocketmq.remoting.protocol.RemotingSerializable;
 
-public class ConsumerOffsetSerializeWrapper extends RemotingSerializable {
-    private ConcurrentMap<String/* topic@group */, ConcurrentMap<Integer, Long>> offsetTable =
-        new ConcurrentHashMap<String, ConcurrentMap<Integer, Long>>(512);
+public class GetBrokerMemberGroupResponseBody extends RemotingSerializable {
+    // Contains the broker member info of the same broker group
+    private BrokerMemberGroup brokerMemberGroup;
 
-    public ConcurrentMap<String, ConcurrentMap<Integer, Long>> getOffsetTable() {
-        return offsetTable;
+    public BrokerMemberGroup getBrokerMemberGroup() {
+        return brokerMemberGroup;
     }
 
-    public void setOffsetTable(ConcurrentMap<String, ConcurrentMap<Integer, Long>> offsetTable) {
-        this.offsetTable = offsetTable;
+    public void setBrokerMemberGroup(final BrokerMemberGroup brokerMemberGroup) {
+        this.brokerMemberGroup = brokerMemberGroup;
     }
 }
diff --git a/common/src/main/java/org/apache/rocketmq/common/message/MessageType.java b/common/src/main/java/org/apache/rocketmq/common/protocol/body/GetRemoteClientConfigBody.java
similarity index 64%
copy from common/src/main/java/org/apache/rocketmq/common/message/MessageType.java
copy to common/src/main/java/org/apache/rocketmq/common/protocol/body/GetRemoteClientConfigBody.java
index 08091f4..38b3b48 100644
--- a/common/src/main/java/org/apache/rocketmq/common/message/MessageType.java
+++ b/common/src/main/java/org/apache/rocketmq/common/protocol/body/GetRemoteClientConfigBody.java
@@ -14,12 +14,21 @@
  * See the License for the specific language governing permissions and
  * limitations under the License.
  */
+package org.apache.rocketmq.common.protocol.body;
 
-package org.apache.rocketmq.common.message;
+import org.apache.rocketmq.remoting.protocol.RemotingSerializable;
 
-public enum MessageType {
-    Normal_Msg,
-    Trans_Msg_Half,
-    Trans_msg_Commit,
-    Delay_Msg,
+import java.util.ArrayList;
+import java.util.List;
+
+public class GetRemoteClientConfigBody extends RemotingSerializable {
+    private List<String> keys = new ArrayList<String>();
+
+    public List<String> getKeys() {
+        return keys;
+    }
+
+    public void setKeys(List<String> keys) {
+        this.keys = keys;
+    }
 }
diff --git a/common/src/main/java/org/apache/rocketmq/common/protocol/body/HARuntimeInfo.java b/common/src/main/java/org/apache/rocketmq/common/protocol/body/HARuntimeInfo.java
new file mode 100644
index 0000000..8994b51
--- /dev/null
+++ b/common/src/main/java/org/apache/rocketmq/common/protocol/body/HARuntimeInfo.java
@@ -0,0 +1,188 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.rocketmq.common.protocol.body;
+
+import java.util.ArrayList;
+import java.util.List;
+
+import org.apache.rocketmq.remoting.protocol.RemotingSerializable;
+
+public class HARuntimeInfo extends RemotingSerializable {
+
+    private boolean master;
+    private long masterCommitLogMaxOffset;
+    private int inSyncSlaveNums;
+    private List<HAConnectionRuntimeInfo> haConnectionInfo = new ArrayList<HARuntimeInfo.HAConnectionRuntimeInfo>();
+    private HAClientRuntimeInfo haClientRuntimeInfo = new HAClientRuntimeInfo();
+
+    public boolean isMaster() {
+        return this.master;
+    }
+
+    public void setMaster(boolean master) {
+        this.master = master;
+    }
+
+    public long getMasterCommitLogMaxOffset() {
+        return this.masterCommitLogMaxOffset;
+    }
+
+    public void setMasterCommitLogMaxOffset(long masterCommitLogMaxOffset) {
+        this.masterCommitLogMaxOffset = masterCommitLogMaxOffset;
+    }
+
+    public int getInSyncSlaveNums() {
+        return this.inSyncSlaveNums;
+    }
+
+    public void setInSyncSlaveNums(int inSyncSlaveNums) {
+        this.inSyncSlaveNums = inSyncSlaveNums;
+    }
+
+    public List<HAConnectionRuntimeInfo> getHaConnectionInfo() {
+        return this.haConnectionInfo;
+    }
+
+    public void setHaConnectionInfo(List<HAConnectionRuntimeInfo> haConnectionInfo) {
+        this.haConnectionInfo = haConnectionInfo;
+    }
+
+    public HAClientRuntimeInfo getHaClientRuntimeInfo() {
+        return this.haClientRuntimeInfo;
+    }
+
+    public void setHaClientRuntimeInfo(HAClientRuntimeInfo haClientRuntimeInfo) {
+        this.haClientRuntimeInfo = haClientRuntimeInfo;
+    }
+
+    public static class HAConnectionRuntimeInfo extends RemotingSerializable {
+        private String addr;
+        private long slaveAckOffset;
+        private long diff;
+        private boolean inSync;
+        private long transferredByteInSecond;
+        private long transferFromWhere;
+
+        public String getAddr() {
+            return this.addr;
+        }
+
+        public void setAddr(String addr) {
+            this.addr = addr;
+        }
+
+        public long getSlaveAckOffset() {
+            return this.slaveAckOffset;
+        }
+
+        public void setSlaveAckOffset(long slaveAckOffset) {
+            this.slaveAckOffset = slaveAckOffset;
+        }
+
+        public long getDiff() {
+            return this.diff;
+        }
+
+        public void setDiff(long diff) {
+            this.diff = diff;
+        }
+
+        public boolean isInSync() {
+            return this.inSync;
+        }
+
+        public void setInSync(boolean inSync) {
+            this.inSync = inSync;
+        }
+
+        public long getTransferredByteInSecond() {
+            return this.transferredByteInSecond;
+        }
+
+        public void setTransferredByteInSecond(long transferredByteInSecond) {
+            this.transferredByteInSecond = transferredByteInSecond;
+        }
+
+        public long getTransferFromWhere() {
+            return transferFromWhere;
+        }
+
+        public void setTransferFromWhere(long transferFromWhere) {
+            this.transferFromWhere = transferFromWhere;
+        }
+    }
+
+    public static class HAClientRuntimeInfo extends RemotingSerializable {
+        private String masterAddr;
+        private long transferredByteInSecond;
+        private long maxOffset;
+        private long lastReadTimestamp;
+        private long lastWriteTimestamp;
+        private long masterFlushOffset;
+        private boolean isActivated = false;
+
+        public String getMasterAddr() {
+            return this.masterAddr;
+        }
+
+        public void setMasterAddr(String masterAddr) {
+            this.masterAddr = masterAddr;
+        }
+
+        public long getTransferredByteInSecond() {
+            return this.transferredByteInSecond;
+        }
+
+        public void setTransferredByteInSecond(long transferredByteInSecond) {
+            this.transferredByteInSecond = transferredByteInSecond;
+        }
+
+        public long getMaxOffset() {
+            return this.maxOffset;
+        }
+
+        public void setMaxOffset(long maxOffset) {
+            this.maxOffset = maxOffset;
+        }
+
+        public long getLastReadTimestamp() {
+            return this.lastReadTimestamp;
+        }
+
+        public void setLastReadTimestamp(long lastReadTimestamp) {
+            this.lastReadTimestamp = lastReadTimestamp;
+        }
+
+        public long getLastWriteTimestamp() {
+            return this.lastWriteTimestamp;
+        }
+
+        public void setLastWriteTimestamp(long lastWriteTimestamp) {
+            this.lastWriteTimestamp = lastWriteTimestamp;
+        }
+
+        public long getMasterFlushOffset() {
+            return masterFlushOffset;
+        }
+
+        public void setMasterFlushOffset(long masterFlushOffset) {
+            this.masterFlushOffset = masterFlushOffset;
+        }
+    }
+
+}
diff --git a/common/src/main/java/org/apache/rocketmq/common/protocol/body/LockBatchRequestBody.java b/common/src/main/java/org/apache/rocketmq/common/protocol/body/LockBatchRequestBody.java
index 480862b..92e4d4e 100644
--- a/common/src/main/java/org/apache/rocketmq/common/protocol/body/LockBatchRequestBody.java
+++ b/common/src/main/java/org/apache/rocketmq/common/protocol/body/LockBatchRequestBody.java
@@ -25,6 +25,7 @@ import org.apache.rocketmq.remoting.protocol.RemotingSerializable;
 public class LockBatchRequestBody extends RemotingSerializable {
     private String consumerGroup;
     private String clientId;
+    private boolean onlyThisBroker = false;
     private Set<MessageQueue> mqSet = new HashSet<MessageQueue>();
 
     public String getConsumerGroup() {
@@ -43,6 +44,14 @@ public class LockBatchRequestBody extends RemotingSerializable {
         this.clientId = clientId;
     }
 
+    public boolean isOnlyThisBroker() {
+        return onlyThisBroker;
+    }
+
+    public void setOnlyThisBroker(boolean onlyThisBroker) {
+        this.onlyThisBroker = onlyThisBroker;
+    }
+
     public Set<MessageQueue> getMqSet() {
         return mqSet;
     }
diff --git a/common/src/main/java/org/apache/rocketmq/common/protocol/body/CheckClientRequestBody.java b/common/src/main/java/org/apache/rocketmq/common/protocol/body/QuerySubscriptionResponseBody.java
similarity index 85%
copy from common/src/main/java/org/apache/rocketmq/common/protocol/body/CheckClientRequestBody.java
copy to common/src/main/java/org/apache/rocketmq/common/protocol/body/QuerySubscriptionResponseBody.java
index a78ce55..413f8c4 100644
--- a/common/src/main/java/org/apache/rocketmq/common/protocol/body/CheckClientRequestBody.java
+++ b/common/src/main/java/org/apache/rocketmq/common/protocol/body/QuerySubscriptionResponseBody.java
@@ -20,18 +20,18 @@ package org.apache.rocketmq.common.protocol.body;
 import org.apache.rocketmq.common.protocol.heartbeat.SubscriptionData;
 import org.apache.rocketmq.remoting.protocol.RemotingSerializable;
 
-public class CheckClientRequestBody extends RemotingSerializable {
+public class QuerySubscriptionResponseBody extends RemotingSerializable {
 
-    private String clientId;
-    private String group;
     private SubscriptionData subscriptionData;
+    private String group;
+    private String topic;
 
-    public String getClientId() {
-        return clientId;
+    public SubscriptionData getSubscriptionData() {
+        return subscriptionData;
     }
 
-    public void setClientId(String clientId) {
-        this.clientId = clientId;
+    public void setSubscriptionData(SubscriptionData subscriptionData) {
+        this.subscriptionData = subscriptionData;
     }
 
     public String getGroup() {
@@ -42,11 +42,11 @@ public class CheckClientRequestBody extends RemotingSerializable {
         this.group = group;
     }
 
-    public SubscriptionData getSubscriptionData() {
-        return subscriptionData;
+    public String getTopic() {
+        return topic;
     }
 
-    public void setSubscriptionData(SubscriptionData subscriptionData) {
-        this.subscriptionData = subscriptionData;
+    public void setTopic(String topic) {
+        this.topic = topic;
     }
 }
diff --git a/common/src/main/java/org/apache/rocketmq/common/protocol/body/UnlockBatchRequestBody.java b/common/src/main/java/org/apache/rocketmq/common/protocol/body/UnlockBatchRequestBody.java
index baf4071..f71dafb 100644
--- a/common/src/main/java/org/apache/rocketmq/common/protocol/body/UnlockBatchRequestBody.java
+++ b/common/src/main/java/org/apache/rocketmq/common/protocol/body/UnlockBatchRequestBody.java
@@ -25,6 +25,7 @@ import org.apache.rocketmq.remoting.protocol.RemotingSerializable;
 public class UnlockBatchRequestBody extends RemotingSerializable {
     private String consumerGroup;
     private String clientId;
+    private boolean onlyThisBroker = false;
     private Set<MessageQueue> mqSet = new HashSet<MessageQueue>();
 
     public String getConsumerGroup() {
@@ -43,6 +44,14 @@ public class UnlockBatchRequestBody extends RemotingSerializable {
         this.clientId = clientId;
     }
 
+    public boolean isOnlyThisBroker() {
+        return onlyThisBroker;
+    }
+
+    public void setOnlyThisBroker(boolean onlyThisBroker) {
+        this.onlyThisBroker = onlyThisBroker;
+    }
+
     public Set<MessageQueue> getMqSet() {
         return mqSet;
     }
diff --git a/common/src/main/java/org/apache/rocketmq/common/protocol/header/namesrv/DeleteTopicFromNamesrvRequestHeader.java b/common/src/main/java/org/apache/rocketmq/common/protocol/header/AddBrokerRequestHeader.java
similarity index 67%
copy from common/src/main/java/org/apache/rocketmq/common/protocol/header/namesrv/DeleteTopicFromNamesrvRequestHeader.java
copy to common/src/main/java/org/apache/rocketmq/common/protocol/header/AddBrokerRequestHeader.java
index ed8e7a3..e38e350 100644
--- a/common/src/main/java/org/apache/rocketmq/common/protocol/header/namesrv/DeleteTopicFromNamesrvRequestHeader.java
+++ b/common/src/main/java/org/apache/rocketmq/common/protocol/header/AddBrokerRequestHeader.java
@@ -14,25 +14,27 @@
  * See the License for the specific language governing permissions and
  * limitations under the License.
  */
-package org.apache.rocketmq.common.protocol.header.namesrv;
+
+package org.apache.rocketmq.common.protocol.header;
 
 import org.apache.rocketmq.remoting.CommandCustomHeader;
-import org.apache.rocketmq.remoting.annotation.CFNotNull;
+import org.apache.rocketmq.remoting.annotation.CFNullable;
 import org.apache.rocketmq.remoting.exception.RemotingCommandException;
 
-public class DeleteTopicFromNamesrvRequestHeader implements CommandCustomHeader {
-    @CFNotNull
-    private String topic;
+public class AddBrokerRequestHeader implements CommandCustomHeader {
+    @CFNullable
+    private String configPath;
+
+
+    @Override public void checkFields() throws RemotingCommandException {
 
-    @Override
-    public void checkFields() throws RemotingCommandException {
     }
 
-    public String getTopic() {
-        return topic;
+    public String getConfigPath() {
+        return configPath;
     }
 
-    public void setTopic(String topic) {
-        this.topic = topic;
+    public void setConfigPath(String configPath) {
+        this.configPath = configPath;
     }
 }
diff --git a/common/src/main/java/org/apache/rocketmq/common/protocol/header/ConsumeMessageDirectlyResultRequestHeader.java b/common/src/main/java/org/apache/rocketmq/common/protocol/header/ConsumeMessageDirectlyResultRequestHeader.java
index 1bd089d..7bad639 100644
--- a/common/src/main/java/org/apache/rocketmq/common/protocol/header/ConsumeMessageDirectlyResultRequestHeader.java
+++ b/common/src/main/java/org/apache/rocketmq/common/protocol/header/ConsumeMessageDirectlyResultRequestHeader.java
@@ -31,6 +31,12 @@ public class ConsumeMessageDirectlyResultRequestHeader implements CommandCustomH
     private String msgId;
     @CFNullable
     private String brokerName;
+    @CFNullable
+    private String topic;
+    @CFNullable
+    private Integer topicSysFlag;
+    @CFNullable
+    private Integer groupSysFlag;
 
     @Override
     public void checkFields() throws RemotingCommandException {
@@ -67,4 +73,28 @@ public class ConsumeMessageDirectlyResultRequestHeader implements CommandCustomH
     public void setMsgId(String msgId) {
         this.msgId = msgId;
     }
+
+    public String getTopic() {
+        return topic;
+    }
+
+    public void setTopic(String topic) {
+        this.topic = topic;
+    }
+
+    public Integer getTopicSysFlag() {
+        return topicSysFlag;
+    }
+
+    public void setTopicSysFlag(Integer topicSysFlag) {
+        this.topicSysFlag = topicSysFlag;
+    }
+
+    public Integer getGroupSysFlag() {
+        return groupSysFlag;
+    }
+
+    public void setGroupSysFlag(Integer groupSysFlag) {
+        this.groupSysFlag = groupSysFlag;
+    }
 }
diff --git a/common/src/main/java/org/apache/rocketmq/common/protocol/header/DeleteSubscriptionGroupRequestHeader.java b/common/src/main/java/org/apache/rocketmq/common/protocol/header/DeleteSubscriptionGroupRequestHeader.java
index 6591d77..8a33564 100644
--- a/common/src/main/java/org/apache/rocketmq/common/protocol/header/DeleteSubscriptionGroupRequestHeader.java
+++ b/common/src/main/java/org/apache/rocketmq/common/protocol/header/DeleteSubscriptionGroupRequestHeader.java
@@ -25,7 +25,7 @@ public class DeleteSubscriptionGroupRequestHeader implements CommandCustomHeader
     @CFNotNull
     private String groupName;
 
-    private boolean removeOffset;
+    private boolean cleanOffset = false;
 
     @Override
     public void checkFields() throws RemotingCommandException {
@@ -39,11 +39,11 @@ public class DeleteSubscriptionGroupRequestHeader implements CommandCustomHeader
         this.groupName = groupName;
     }
 
-    public boolean isRemoveOffset() {
-        return removeOffset;
+    public boolean isCleanOffset() {
+        return cleanOffset;
     }
 
-    public void setRemoveOffset(boolean removeOffset) {
-        this.removeOffset = removeOffset;
+    public void setCleanOffset(boolean cleanOffset) {
+        this.cleanOffset = cleanOffset;
     }
 }
diff --git a/common/src/main/java/org/apache/rocketmq/common/protocol/header/ConsumeMessageDirectlyResultRequestHeader.java b/common/src/main/java/org/apache/rocketmq/common/protocol/header/ExchangeHAInfoRequestHeader.java
similarity index 58%
copy from common/src/main/java/org/apache/rocketmq/common/protocol/header/ConsumeMessageDirectlyResultRequestHeader.java
copy to common/src/main/java/org/apache/rocketmq/common/protocol/header/ExchangeHAInfoRequestHeader.java
index 1bd089d..4a59320 100644
--- a/common/src/main/java/org/apache/rocketmq/common/protocol/header/ConsumeMessageDirectlyResultRequestHeader.java
+++ b/common/src/main/java/org/apache/rocketmq/common/protocol/header/ExchangeHAInfoRequestHeader.java
@@ -18,53 +18,45 @@
 package org.apache.rocketmq.common.protocol.header;
 
 import org.apache.rocketmq.remoting.CommandCustomHeader;
-import org.apache.rocketmq.remoting.annotation.CFNotNull;
 import org.apache.rocketmq.remoting.annotation.CFNullable;
 import org.apache.rocketmq.remoting.exception.RemotingCommandException;
 
-public class ConsumeMessageDirectlyResultRequestHeader implements CommandCustomHeader {
-    @CFNotNull
-    private String consumerGroup;
+public class ExchangeHAInfoRequestHeader implements CommandCustomHeader {
     @CFNullable
-    private String clientId;
+    public String masterHaAddress;
+
     @CFNullable
-    private String msgId;
+    public Long masterFlushOffset;
+
     @CFNullable
-    private String brokerName;
+    public String masterAddress;
 
     @Override
     public void checkFields() throws RemotingCommandException {
-    }
-
-    public String getConsumerGroup() {
-        return consumerGroup;
-    }
 
-    public void setConsumerGroup(String consumerGroup) {
-        this.consumerGroup = consumerGroup;
     }
 
-    public String getBrokerName() {
-        return brokerName;
+    public String getMasterHaAddress() {
+        return masterHaAddress;
     }
 
-    public void setBrokerName(String brokerName) {
-        this.brokerName = brokerName;
+    public void setMasterHaAddress(String masterHaAddress) {
+        this.masterHaAddress = masterHaAddress;
     }
 
-    public String getClientId() {
-        return clientId;
+    public Long getMasterFlushOffset() {
+        return masterFlushOffset;
     }
 
-    public void setClientId(String clientId) {
-        this.clientId = clientId;
+    public void setMasterFlushOffset(Long masterFlushOffset) {
+        this.masterFlushOffset = masterFlushOffset;
     }
 
-    public String getMsgId() {
-        return msgId;
+    public String getMasterAddress() {
+        return masterAddress;
     }
 
-    public void setMsgId(String msgId) {
-        this.msgId = msgId;
+    public void setMasterAddress(String masterAddress) {
+        this.masterAddress = masterAddress;
     }
 }
diff --git a/common/src/main/java/org/apache/rocketmq/common/protocol/header/ConsumeMessageDirectlyResultRequestHeader.java b/common/src/main/java/org/apache/rocketmq/common/protocol/header/ExchangeHAInfoResponseHeader.java
similarity index 58%
copy from common/src/main/java/org/apache/rocketmq/common/protocol/header/ConsumeMessageDirectlyResultRequestHeader.java
copy to common/src/main/java/org/apache/rocketmq/common/protocol/header/ExchangeHAInfoResponseHeader.java
index 1bd089d..7b6ea77 100644
--- a/common/src/main/java/org/apache/rocketmq/common/protocol/header/ConsumeMessageDirectlyResultRequestHeader.java
+++ b/common/src/main/java/org/apache/rocketmq/common/protocol/header/ExchangeHAInfoResponseHeader.java
@@ -18,53 +18,45 @@
 package org.apache.rocketmq.common.protocol.header;
 
 import org.apache.rocketmq.remoting.CommandCustomHeader;
-import org.apache.rocketmq.remoting.annotation.CFNotNull;
 import org.apache.rocketmq.remoting.annotation.CFNullable;
 import org.apache.rocketmq.remoting.exception.RemotingCommandException;
 
-public class ConsumeMessageDirectlyResultRequestHeader implements CommandCustomHeader {
-    @CFNotNull
-    private String consumerGroup;
+public class ExchangeHAInfoResponseHeader implements CommandCustomHeader {
     @CFNullable
-    private String clientId;
+    public String masterHaAddress;
+
     @CFNullable
-    private String msgId;
+    public Long masterFlushOffset;
+
     @CFNullable
-    private String brokerName;
+    public String masterAddress;
 
     @Override
     public void checkFields() throws RemotingCommandException {
-    }
-
-    public String getConsumerGroup() {
-        return consumerGroup;
-    }
 
-    public void setConsumerGroup(String consumerGroup) {
-        this.consumerGroup = consumerGroup;
     }
 
-    public String getBrokerName() {
-        return brokerName;
+    public String getMasterHaAddress() {
+        return masterHaAddress;
     }
 
-    public void setBrokerName(String brokerName) {
-        this.brokerName = brokerName;
+    public void setMasterHaAddress(String masterHaAddress) {
+        this.masterHaAddress = masterHaAddress;
     }
 
-    public String getClientId() {
-        return clientId;
+    public Long getMasterFlushOffset() {
+        return masterFlushOffset;
     }
 
-    public void setClientId(String clientId) {
-        this.clientId = clientId;
+    public void setMasterFlushOffset(Long masterFlushOffset) {
+        this.masterFlushOffset = masterFlushOffset;
     }
 
-    public String getMsgId() {
-        return msgId;
+    public String getMasterAddress() {
+        return masterAddress;
     }
 
-    public void setMsgId(String msgId) {
-        this.msgId = msgId;
+    public void setMasterAddress(String masterAddress) {
+        this.masterAddress = masterAddress;
     }
 }
diff --git a/common/src/main/java/org/apache/rocketmq/common/protocol/header/DeleteSubscriptionGroupRequestHeader.java b/common/src/main/java/org/apache/rocketmq/common/protocol/header/GetBrokerMemberGroupRequestHeader.java
similarity index 70%
copy from common/src/main/java/org/apache/rocketmq/common/protocol/header/DeleteSubscriptionGroupRequestHeader.java
copy to common/src/main/java/org/apache/rocketmq/common/protocol/header/GetBrokerMemberGroupRequestHeader.java
index 6591d77..c111463 100644
--- a/common/src/main/java/org/apache/rocketmq/common/protocol/header/DeleteSubscriptionGroupRequestHeader.java
+++ b/common/src/main/java/org/apache/rocketmq/common/protocol/header/GetBrokerMemberGroupRequestHeader.java
@@ -21,29 +21,31 @@ import org.apache.rocketmq.remoting.CommandCustomHeader;
 import org.apache.rocketmq.remoting.annotation.CFNotNull;
 import org.apache.rocketmq.remoting.exception.RemotingCommandException;
 
-public class DeleteSubscriptionGroupRequestHeader implements CommandCustomHeader {
+public class GetBrokerMemberGroupRequestHeader implements CommandCustomHeader {
     @CFNotNull
-    private String groupName;
+    private String clusterName;
 
-    private boolean removeOffset;
+    @CFNotNull
+    private String brokerName;
 
-    @Override
-    public void checkFields() throws RemotingCommandException {
+    public String getClusterName() {
+        return clusterName;
     }
 
-    public String getGroupName() {
-        return groupName;
+    public void setClusterName(final String clusterName) {
+        this.clusterName = clusterName;
     }
 
-    public void setGroupName(String groupName) {
-        this.groupName = groupName;
+    public String getBrokerName() {
+        return brokerName;
     }
 
-    public boolean isRemoveOffset() {
-        return removeOffset;
+    public void setBrokerName(final String brokerName) {
+        this.brokerName = brokerName;
     }
 
-    public void setRemoveOffset(boolean removeOffset) {
-        this.removeOffset = removeOffset;
+    @Override
+    public void checkFields() throws RemotingCommandException {
+
     }
 }
diff --git a/common/src/main/java/org/apache/rocketmq/common/protocol/header/GetMaxOffsetRequestHeader.java b/common/src/main/java/org/apache/rocketmq/common/protocol/header/GetMaxOffsetRequestHeader.java
index e961af9..f58e050 100644
--- a/common/src/main/java/org/apache/rocketmq/common/protocol/header/GetMaxOffsetRequestHeader.java
+++ b/common/src/main/java/org/apache/rocketmq/common/protocol/header/GetMaxOffsetRequestHeader.java
@@ -22,6 +22,7 @@ package org.apache.rocketmq.common.protocol.header;
 
 import org.apache.rocketmq.common.rpc.TopicQueueRequestHeader;
 import org.apache.rocketmq.remoting.annotation.CFNotNull;
+import org.apache.rocketmq.remoting.annotation.CFNullable;
 import org.apache.rocketmq.remoting.exception.RemotingCommandException;
 
 public class GetMaxOffsetRequestHeader extends TopicQueueRequestHeader {
@@ -30,6 +31,15 @@ public class GetMaxOffsetRequestHeader extends TopicQueueRequestHeader {
     @CFNotNull
     private Integer queueId;
 
+    /**
+     * A message at committed offset has been dispatched from Topic to MessageQueue, so it can be consumed immediately,
+     * while a message at inflight offset is not visible for a consumer temporarily.
+     * Set this flag true if the max committed offset is needed, or false if the max inflight offset is preferred.
+     * The default value is true.
+     */
+    @CFNullable
+    private boolean committed = true;
+
     @Override
     public void checkFields() throws RemotingCommandException {
     }
@@ -54,4 +64,11 @@ public class GetMaxOffsetRequestHeader extends TopicQueueRequestHeader {
         this.queueId = queueId;
     }
 
+    public boolean isCommitted() {
+        return committed;
+    }
+
+    public void setCommitted(final boolean committed) {
+        this.committed = committed;
+    }
 }
diff --git a/common/src/main/java/org/apache/rocketmq/common/protocol/header/DeleteSubscriptionGroupRequestHeader.java b/common/src/main/java/org/apache/rocketmq/common/protocol/header/GetSubscriptionGroupConfigRequestHeader.java
similarity index 71%
copy from common/src/main/java/org/apache/rocketmq/common/protocol/header/DeleteSubscriptionGroupRequestHeader.java
copy to common/src/main/java/org/apache/rocketmq/common/protocol/header/GetSubscriptionGroupConfigRequestHeader.java
index 6591d77..8b0cfbf 100644
--- a/common/src/main/java/org/apache/rocketmq/common/protocol/header/DeleteSubscriptionGroupRequestHeader.java
+++ b/common/src/main/java/org/apache/rocketmq/common/protocol/header/GetSubscriptionGroupConfigRequestHeader.java
@@ -15,35 +15,35 @@
  * limitations under the License.
  */
 
+/**
+ * $Id: GetAllTopicConfigResponseHeader.java 1835 2013-05-16 02:00:50Z vintagewang@apache.org $
+ */
 package org.apache.rocketmq.common.protocol.header;
 
 import org.apache.rocketmq.remoting.CommandCustomHeader;
 import org.apache.rocketmq.remoting.annotation.CFNotNull;
 import org.apache.rocketmq.remoting.exception.RemotingCommandException;
 
-public class DeleteSubscriptionGroupRequestHeader implements CommandCustomHeader {
-    @CFNotNull
-    private String groupName;
-
-    private boolean removeOffset;
+public class GetSubscriptionGroupConfigRequestHeader implements CommandCustomHeader {
 
     @Override
     public void checkFields() throws RemotingCommandException {
     }
 
-    public String getGroupName() {
-        return groupName;
-    }
-
-    public void setGroupName(String groupName) {
-        this.groupName = groupName;
-    }
+    @CFNotNull
+    private String group;
 
-    public boolean isRemoveOffset() {
-        return removeOffset;
+    /**
+     * @return the group
+     */
+    public String getGroup() {
+        return group;
     }
 
-    public void setRemoveOffset(boolean removeOffset) {
-        this.removeOffset = removeOffset;
+    /**
+     * @param group the group to set
+     */
+    public void setGroup(String group) {
+        this.group = group;
     }
 }
diff --git a/common/src/main/java/org/apache/rocketmq/common/protocol/header/namesrv/GetRouteInfoRequestHeader.java b/common/src/main/java/org/apache/rocketmq/common/protocol/header/InitConsumerOffsetRequestHeader.java
similarity index 78%
copy from common/src/main/java/org/apache/rocketmq/common/protocol/header/namesrv/GetRouteInfoRequestHeader.java
copy to common/src/main/java/org/apache/rocketmq/common/protocol/header/InitConsumerOffsetRequestHeader.java
index a2806e6..27a64ed 100644
--- a/common/src/main/java/org/apache/rocketmq/common/protocol/header/namesrv/GetRouteInfoRequestHeader.java
+++ b/common/src/main/java/org/apache/rocketmq/common/protocol/header/InitConsumerOffsetRequestHeader.java
@@ -14,22 +14,20 @@
  * See the License for the specific language governing permissions and
  * limitations under the License.
  */
-
-/**
- * $Id: GetRouteInfoRequestHeader.java 1835 2013-05-16 02:00:50Z vintagewang@apache.org $
- */
-package org.apache.rocketmq.common.protocol.header.namesrv;
+package org.apache.rocketmq.common.protocol.header;
 
 import org.apache.rocketmq.remoting.CommandCustomHeader;
-import org.apache.rocketmq.remoting.annotation.CFNotNull;
 import org.apache.rocketmq.remoting.exception.RemotingCommandException;
 
-public class GetRouteInfoRequestHeader implements CommandCustomHeader {
-    @CFNotNull
+public class InitConsumerOffsetRequestHeader implements CommandCustomHeader {
+
     private String topic;
+    // @see ConsumeInitMode
+    private int initMode;
 
     @Override
     public void checkFields() throws RemotingCommandException {
+
     }
 
     public String getTopic() {
@@ -39,4 +37,12 @@ public class GetRouteInfoRequestHeader implements CommandCustomHeader {
     public void setTopic(String topic) {
         this.topic = topic;
     }
+
+    public int getInitMode() {
+        return initMode;
+    }
+
+    public void setInitMode(int initMode) {
+        this.initMode = initMode;
+    }
 }
diff --git a/common/src/main/java/org/apache/rocketmq/common/protocol/header/ConsumeMessageDirectlyResultRequestHeader.java b/common/src/main/java/org/apache/rocketmq/common/protocol/header/NotificationRequestHeader.java
similarity index 62%
copy from common/src/main/java/org/apache/rocketmq/common/protocol/header/ConsumeMessageDirectlyResultRequestHeader.java
copy to common/src/main/java/org/apache/rocketmq/common/protocol/header/NotificationRequestHeader.java
index 1bd089d..79db24e 100644
--- a/common/src/main/java/org/apache/rocketmq/common/protocol/header/ConsumeMessageDirectlyResultRequestHeader.java
+++ b/common/src/main/java/org/apache/rocketmq/common/protocol/header/NotificationRequestHeader.java
@@ -14,28 +14,38 @@
  * See the License for the specific language governing permissions and
  * limitations under the License.
  */
-
 package org.apache.rocketmq.common.protocol.header;
 
 import org.apache.rocketmq.remoting.CommandCustomHeader;
 import org.apache.rocketmq.remoting.annotation.CFNotNull;
-import org.apache.rocketmq.remoting.annotation.CFNullable;
 import org.apache.rocketmq.remoting.exception.RemotingCommandException;
 
-public class ConsumeMessageDirectlyResultRequestHeader implements CommandCustomHeader {
+
+public class NotificationRequestHeader implements CommandCustomHeader {
     @CFNotNull
     private String consumerGroup;
-    @CFNullable
-    private String clientId;
-    @CFNullable
-    private String msgId;
-    @CFNullable
-    private String brokerName;
+    @CFNotNull
+    private String topic;
+    @CFNotNull
+    private int queueId;
+    @CFNotNull
+    private long pollTime;
+    @CFNotNull
+    private long bornTime;
 
+    @CFNotNull
     @Override
     public void checkFields() throws RemotingCommandException {
     }
 
+    public long getPollTime() {
+        return pollTime;
+    }
+
+    public void setPollTime(long pollTime) {
+        this.pollTime = pollTime;
+    }
+
     public String getConsumerGroup() {
         return consumerGroup;
     }
@@ -44,27 +54,31 @@ public class ConsumeMessageDirectlyResultRequestHeader implements CommandCustomH
         this.consumerGroup = consumerGroup;
     }
 
-    public String getBrokerName() {
-        return brokerName;
+    public long getBornTime() {
+        return bornTime;
     }
 
-    public void setBrokerName(String brokerName) {
-        this.brokerName = brokerName;
+    public void setBornTime(long bornTime) {
+        this.bornTime = bornTime;
     }
 
-    public String getClientId() {
-        return clientId;
+    public String getTopic() {
+        return topic;
     }
 
-    public void setClientId(String clientId) {
-        this.clientId = clientId;
+    public void setTopic(String topic) {
+        this.topic = topic;
     }
 
-    public String getMsgId() {
-        return msgId;
+    public int getQueueId() {
+        if (queueId < 0) {
+            return -1;
+        }
+        return queueId;
     }
 
-    public void setMsgId(String msgId) {
-        this.msgId = msgId;
+    public void setQueueId(int queueId) {
+        this.queueId = queueId;
     }
+
 }
diff --git a/common/src/main/java/org/apache/rocketmq/common/protocol/header/namesrv/DeleteTopicFromNamesrvRequestHeader.java b/common/src/main/java/org/apache/rocketmq/common/protocol/header/NotificationResponseHeader.java
similarity index 79%
copy from common/src/main/java/org/apache/rocketmq/common/protocol/header/namesrv/DeleteTopicFromNamesrvRequestHeader.java
copy to common/src/main/java/org/apache/rocketmq/common/protocol/header/NotificationResponseHeader.java
index ed8e7a3..20c19d6 100644
--- a/common/src/main/java/org/apache/rocketmq/common/protocol/header/namesrv/DeleteTopicFromNamesrvRequestHeader.java
+++ b/common/src/main/java/org/apache/rocketmq/common/protocol/header/NotificationResponseHeader.java
@@ -14,25 +14,27 @@
  * See the License for the specific language governing permissions and
  * limitations under the License.
  */
-package org.apache.rocketmq.common.protocol.header.namesrv;
+package org.apache.rocketmq.common.protocol.header;
 
 import org.apache.rocketmq.remoting.CommandCustomHeader;
 import org.apache.rocketmq.remoting.annotation.CFNotNull;
 import org.apache.rocketmq.remoting.exception.RemotingCommandException;
 
-public class DeleteTopicFromNamesrvRequestHeader implements CommandCustomHeader {
+public class NotificationResponseHeader implements CommandCustomHeader {
+
+
     @CFNotNull
-    private String topic;
+    private boolean hasMsg = false;
 
-    @Override
-    public void checkFields() throws RemotingCommandException {
+    public boolean isHasMsg() {
+        return hasMsg;
     }
 
-    public String getTopic() {
-        return topic;
+    public void setHasMsg(boolean hasMsg) {
+        this.hasMsg = hasMsg;
     }
 
-    public void setTopic(String topic) {
-        this.topic = topic;
+    @Override
+    public void checkFields() throws RemotingCommandException {
     }
 }
diff --git a/common/src/main/java/org/apache/rocketmq/common/protocol/header/ConsumeMessageDirectlyResultRequestHeader.java b/common/src/main/java/org/apache/rocketmq/common/protocol/header/NotifyMinBrokerIdChangeRequestHeader.java
similarity index 59%
copy from common/src/main/java/org/apache/rocketmq/common/protocol/header/ConsumeMessageDirectlyResultRequestHeader.java
copy to common/src/main/java/org/apache/rocketmq/common/protocol/header/NotifyMinBrokerIdChangeRequestHeader.java
index 1bd089d..c5ca4ef 100644
--- a/common/src/main/java/org/apache/rocketmq/common/protocol/header/ConsumeMessageDirectlyResultRequestHeader.java
+++ b/common/src/main/java/org/apache/rocketmq/common/protocol/header/NotifyMinBrokerIdChangeRequestHeader.java
@@ -18,30 +18,36 @@
 package org.apache.rocketmq.common.protocol.header;
 
 import org.apache.rocketmq.remoting.CommandCustomHeader;
-import org.apache.rocketmq.remoting.annotation.CFNotNull;
 import org.apache.rocketmq.remoting.annotation.CFNullable;
 import org.apache.rocketmq.remoting.exception.RemotingCommandException;
 
-public class ConsumeMessageDirectlyResultRequestHeader implements CommandCustomHeader {
-    @CFNotNull
-    private String consumerGroup;
+public class NotifyMinBrokerIdChangeRequestHeader implements CommandCustomHeader {
     @CFNullable
-    private String clientId;
-    @CFNullable
-    private String msgId;
+    private Long minBrokerId;
+
     @CFNullable
     private String brokerName;
 
+    @CFNullable
+    private String minBrokerAddr;
+
+    @CFNullable
+    private String offlineBrokerAddr;
+
+    @CFNullable
+    private String haBrokerAddr;
+
     @Override
     public void checkFields() throws RemotingCommandException {
+
     }
 
-    public String getConsumerGroup() {
-        return consumerGroup;
+    public Long getMinBrokerId() {
+        return minBrokerId;
     }
 
-    public void setConsumerGroup(String consumerGroup) {
-        this.consumerGroup = consumerGroup;
+    public void setMinBrokerId(Long minBrokerId) {
+        this.minBrokerId = minBrokerId;
     }
 
     public String getBrokerName() {
@@ -52,19 +58,27 @@ public class ConsumeMessageDirectlyResultRequestHeader implements CommandCustomH
         this.brokerName = brokerName;
     }
 
-    public String getClientId() {
-        return clientId;
+    public String getMinBrokerAddr() {
+        return minBrokerAddr;
+    }
+
+    public void setMinBrokerAddr(String minBrokerAddr) {
+        this.minBrokerAddr = minBrokerAddr;
+    }
+
+    public String getOfflineBrokerAddr() {
+        return offlineBrokerAddr;
     }
 
-    public void setClientId(String clientId) {
-        this.clientId = clientId;
+    public void setOfflineBrokerAddr(String offlineBrokerAddr) {
+        this.offlineBrokerAddr = offlineBrokerAddr;
     }
 
-    public String getMsgId() {
-        return msgId;
+    public String getHaBrokerAddr() {
+        return haBrokerAddr;
     }
 
-    public void setMsgId(String msgId) {
-        this.msgId = msgId;
+    public void setHaBrokerAddr(String haBrokerAddr) {
+        this.haBrokerAddr = haBrokerAddr;
     }
 }
diff --git a/common/src/main/java/org/apache/rocketmq/common/protocol/header/ConsumeMessageDirectlyResultRequestHeader.java b/common/src/main/java/org/apache/rocketmq/common/protocol/header/PeekMessageRequestHeader.java
similarity index 66%
copy from common/src/main/java/org/apache/rocketmq/common/protocol/header/ConsumeMessageDirectlyResultRequestHeader.java
copy to common/src/main/java/org/apache/rocketmq/common/protocol/header/PeekMessageRequestHeader.java
index 1bd089d..ba172f5 100644
--- a/common/src/main/java/org/apache/rocketmq/common/protocol/header/ConsumeMessageDirectlyResultRequestHeader.java
+++ b/common/src/main/java/org/apache/rocketmq/common/protocol/header/PeekMessageRequestHeader.java
@@ -14,23 +14,21 @@
  * See the License for the specific language governing permissions and
  * limitations under the License.
  */
-
 package org.apache.rocketmq.common.protocol.header;
 
 import org.apache.rocketmq.remoting.CommandCustomHeader;
 import org.apache.rocketmq.remoting.annotation.CFNotNull;
-import org.apache.rocketmq.remoting.annotation.CFNullable;
 import org.apache.rocketmq.remoting.exception.RemotingCommandException;
 
-public class ConsumeMessageDirectlyResultRequestHeader implements CommandCustomHeader {
+public class PeekMessageRequestHeader implements CommandCustomHeader {
+    @CFNotNull
+    private String topic;
+    @CFNotNull
+    private int queueId;
+    @CFNotNull
+    private int maxMsgNums;
     @CFNotNull
     private String consumerGroup;
-    @CFNullable
-    private String clientId;
-    @CFNullable
-    private String msgId;
-    @CFNullable
-    private String brokerName;
 
     @Override
     public void checkFields() throws RemotingCommandException {
@@ -44,27 +42,29 @@ public class ConsumeMessageDirectlyResultRequestHeader implements CommandCustomH
         this.consumerGroup = consumerGroup;
     }
 
-    public String getBrokerName() {
-        return brokerName;
+    public String getTopic() {
+        return topic;
     }
 
-    public void setBrokerName(String brokerName) {
-        this.brokerName = brokerName;
+    public void setTopic(String topic) {
+        this.topic = topic;
     }
 
-    public String getClientId() {
-        return clientId;
+    public int getQueueId() {
+        return queueId;
     }
 
-    public void setClientId(String clientId) {
-        this.clientId = clientId;
+    public void setQueueId(int queueId) {
+        this.queueId = queueId;
     }
 
-    public String getMsgId() {
-        return msgId;
+
+    public int getMaxMsgNums() {
+        return maxMsgNums;
     }
 
-    public void setMsgId(String msgId) {
-        this.msgId = msgId;
+    public void setMaxMsgNums(int maxMsgNums) {
+        this.maxMsgNums = maxMsgNums;
     }
+
 }
diff --git a/common/src/main/java/org/apache/rocketmq/common/protocol/header/GetMaxOffsetRequestHeader.java b/common/src/main/java/org/apache/rocketmq/common/protocol/header/PollingInfoRequestHeader.java
similarity index 72%
copy from common/src/main/java/org/apache/rocketmq/common/protocol/header/GetMaxOffsetRequestHeader.java
copy to common/src/main/java/org/apache/rocketmq/common/protocol/header/PollingInfoRequestHeader.java
index e961af9..a57fa6e 100644
--- a/common/src/main/java/org/apache/rocketmq/common/protocol/header/GetMaxOffsetRequestHeader.java
+++ b/common/src/main/java/org/apache/rocketmq/common/protocol/header/PollingInfoRequestHeader.java
@@ -15,42 +15,49 @@
  * limitations under the License.
  */
 
-/**
- * $Id: GetMaxOffsetRequestHeader.java 1835 2013-05-16 02:00:50Z vintagewang@apache.org $
- */
 package org.apache.rocketmq.common.protocol.header;
 
-import org.apache.rocketmq.common.rpc.TopicQueueRequestHeader;
+import org.apache.rocketmq.remoting.CommandCustomHeader;
 import org.apache.rocketmq.remoting.annotation.CFNotNull;
 import org.apache.rocketmq.remoting.exception.RemotingCommandException;
 
-public class GetMaxOffsetRequestHeader extends TopicQueueRequestHeader {
+
+public class PollingInfoRequestHeader implements CommandCustomHeader {
+    @CFNotNull
+    private String consumerGroup;
     @CFNotNull
     private String topic;
     @CFNotNull
-    private Integer queueId;
+    private int queueId;
 
     @Override
     public void checkFields() throws RemotingCommandException {
     }
 
-    @Override
+    public String getConsumerGroup() {
+        return consumerGroup;
+    }
+
+    public void setConsumerGroup(String consumerGroup) {
+        this.consumerGroup = consumerGroup;
+    }
+
     public String getTopic() {
         return topic;
     }
 
-    @Override
     public void setTopic(String topic) {
         this.topic = topic;
     }
 
-    @Override
-    public Integer getQueueId() {
+    public int getQueueId() {
+        if (queueId < 0) {
+            return -1;
+        }
         return queueId;
     }
 
-    @Override
-    public void setQueueId(Integer queueId) {
+    public void setQueueId(int queueId) {
         this.queueId = queueId;
     }
 
diff --git a/common/src/main/java/org/apache/rocketmq/common/protocol/header/namesrv/DeleteTopicFromNamesrvRequestHeader.java b/common/src/main/java/org/apache/rocketmq/common/protocol/header/PollingInfoResponseHeader.java
similarity index 78%
copy from common/src/main/java/org/apache/rocketmq/common/protocol/header/namesrv/DeleteTopicFromNamesrvRequestHeader.java
copy to common/src/main/java/org/apache/rocketmq/common/protocol/header/PollingInfoResponseHeader.java
index ed8e7a3..37962f9 100644
--- a/common/src/main/java/org/apache/rocketmq/common/protocol/header/namesrv/DeleteTopicFromNamesrvRequestHeader.java
+++ b/common/src/main/java/org/apache/rocketmq/common/protocol/header/PollingInfoResponseHeader.java
@@ -14,25 +14,28 @@
  * See the License for the specific language governing permissions and
  * limitations under the License.
  */
-package org.apache.rocketmq.common.protocol.header.namesrv;
+
+package org.apache.rocketmq.common.protocol.header;
 
 import org.apache.rocketmq.remoting.CommandCustomHeader;
 import org.apache.rocketmq.remoting.annotation.CFNotNull;
 import org.apache.rocketmq.remoting.exception.RemotingCommandException;
 
-public class DeleteTopicFromNamesrvRequestHeader implements CommandCustomHeader {
+public class PollingInfoResponseHeader implements CommandCustomHeader {
+
+
     @CFNotNull
-    private String topic;
+    private int pollingNum;
 
-    @Override
-    public void checkFields() throws RemotingCommandException {
+    public int getPollingNum() {
+        return pollingNum;
     }
 
-    public String getTopic() {
-        return topic;
+    public void setPollingNum(int pollingNum) {
+        this.pollingNum = pollingNum;
     }
 
-    public void setTopic(String topic) {
-        this.topic = topic;
+    @Override
+    public void checkFields() throws RemotingCommandException {
     }
 }
diff --git a/common/src/main/java/org/apache/rocketmq/common/protocol/header/PullMessageResponseHeader.java b/common/src/main/java/org/apache/rocketmq/common/protocol/header/PullMessageResponseHeader.java
index 88af984..84e6daf 100644
--- a/common/src/main/java/org/apache/rocketmq/common/protocol/header/PullMessageResponseHeader.java
+++ b/common/src/main/java/org/apache/rocketmq/common/protocol/header/PullMessageResponseHeader.java
@@ -36,6 +36,12 @@ public class PullMessageResponseHeader implements CommandCustomHeader {
     private Long maxOffset;
     @CFNullable
     private Long offsetDelta;
+    @CFNullable
+    private Integer topicSysFlag;
+    @CFNullable
+    private Integer groupSysFlag;
+    @CFNullable
+    private Integer forbiddenType;
 
     @Override
     public void checkFields() throws RemotingCommandException {
@@ -73,6 +79,33 @@ public class PullMessageResponseHeader implements CommandCustomHeader {
         this.suggestWhichBrokerId = suggestWhichBrokerId;
     }
 
+    public Integer getTopicSysFlag() {
+        return topicSysFlag;
+    }
+
+    public void setTopicSysFlag(Integer topicSysFlag) {
+        this.topicSysFlag = topicSysFlag;
+    }
+
+    public Integer getGroupSysFlag() {
+        return groupSysFlag;
+    }
+
+    public void setGroupSysFlag(Integer groupSysFlag) {
+        this.groupSysFlag = groupSysFlag;
+    }
+    /**
+     * @return the forbiddenType
+     */
+    public Integer getForbiddenType() {
+        return forbiddenType;
+    }
+    /**
+     */
+    public void setForbiddenType(Integer forbiddenType) {
+        this.forbiddenType = forbiddenType;
+    }
+
     public Long getOffsetDelta() {
         return offsetDelta;
     }
diff --git a/common/src/main/java/org/apache/rocketmq/common/protocol/header/namesrv/GetRouteInfoRequestHeader.java b/common/src/main/java/org/apache/rocketmq/common/protocol/header/QuerySubscriptionByConsumerRequestHeader.java
similarity index 77%
copy from common/src/main/java/org/apache/rocketmq/common/protocol/header/namesrv/GetRouteInfoRequestHeader.java
copy to common/src/main/java/org/apache/rocketmq/common/protocol/header/QuerySubscriptionByConsumerRequestHeader.java
index a2806e6..94a5ae5 100644
--- a/common/src/main/java/org/apache/rocketmq/common/protocol/header/namesrv/GetRouteInfoRequestHeader.java
+++ b/common/src/main/java/org/apache/rocketmq/common/protocol/header/QuerySubscriptionByConsumerRequestHeader.java
@@ -16,20 +16,30 @@
  */
 
 /**
- * $Id: GetRouteInfoRequestHeader.java 1835 2013-05-16 02:00:50Z vintagewang@apache.org $
+ * $Id: QueryMessageRequestHeader.java 1835 2013-05-16 02:00:50Z vintagewang@apache.org $
  */
-package org.apache.rocketmq.common.protocol.header.namesrv;
+package org.apache.rocketmq.common.protocol.header;
 
 import org.apache.rocketmq.remoting.CommandCustomHeader;
 import org.apache.rocketmq.remoting.annotation.CFNotNull;
 import org.apache.rocketmq.remoting.exception.RemotingCommandException;
 
-public class GetRouteInfoRequestHeader implements CommandCustomHeader {
+public class QuerySubscriptionByConsumerRequestHeader implements CommandCustomHeader {
     @CFNotNull
+    private String group;
     private String topic;
 
     @Override
     public void checkFields() throws RemotingCommandException {
+
+    }
+
+    public String getGroup() {
+        return group;
+    }
+
+    public void setGroup(String group) {
+        this.group = group;
     }
 
     public String getTopic() {
diff --git a/common/src/main/java/org/apache/rocketmq/common/protocol/header/namesrv/GetRouteInfoRequestHeader.java b/common/src/main/java/org/apache/rocketmq/common/protocol/header/QueryTopicsByConsumerRequestHeader.java
similarity index 76%
copy from common/src/main/java/org/apache/rocketmq/common/protocol/header/namesrv/GetRouteInfoRequestHeader.java
copy to common/src/main/java/org/apache/rocketmq/common/protocol/header/QueryTopicsByConsumerRequestHeader.java
index a2806e6..6a382d7 100644
--- a/common/src/main/java/org/apache/rocketmq/common/protocol/header/namesrv/GetRouteInfoRequestHeader.java
+++ b/common/src/main/java/org/apache/rocketmq/common/protocol/header/QueryTopicsByConsumerRequestHeader.java
@@ -16,27 +16,28 @@
  */
 
 /**
- * $Id: GetRouteInfoRequestHeader.java 1835 2013-05-16 02:00:50Z vintagewang@apache.org $
+ * $Id: QueryMessageRequestHeader.java 1835 2013-05-16 02:00:50Z vintagewang@apache.org $
  */
-package org.apache.rocketmq.common.protocol.header.namesrv;
+package org.apache.rocketmq.common.protocol.header;
 
 import org.apache.rocketmq.remoting.CommandCustomHeader;
 import org.apache.rocketmq.remoting.annotation.CFNotNull;
 import org.apache.rocketmq.remoting.exception.RemotingCommandException;
 
-public class GetRouteInfoRequestHeader implements CommandCustomHeader {
+public class QueryTopicsByConsumerRequestHeader implements CommandCustomHeader {
     @CFNotNull
-    private String topic;
+    private String group;
 
     @Override
     public void checkFields() throws RemotingCommandException {
+
     }
 
-    public String getTopic() {
-        return topic;
+    public String getGroup() {
+        return group;
     }
 
-    public void setTopic(String topic) {
-        this.topic = topic;
+    public void setGroup(String group) {
+        this.group = group;
     }
 }
diff --git a/common/src/main/java/org/apache/rocketmq/common/protocol/header/ConsumeMessageDirectlyResultRequestHeader.java b/common/src/main/java/org/apache/rocketmq/common/protocol/header/RemoveBrokerRequestHeader.java
similarity index 61%
copy from common/src/main/java/org/apache/rocketmq/common/protocol/header/ConsumeMessageDirectlyResultRequestHeader.java
copy to common/src/main/java/org/apache/rocketmq/common/protocol/header/RemoveBrokerRequestHeader.java
index 1bd089d..05a8f11 100644
--- a/common/src/main/java/org/apache/rocketmq/common/protocol/header/ConsumeMessageDirectlyResultRequestHeader.java
+++ b/common/src/main/java/org/apache/rocketmq/common/protocol/header/RemoveBrokerRequestHeader.java
@@ -19,29 +19,18 @@ package org.apache.rocketmq.common.protocol.header;
 
 import org.apache.rocketmq.remoting.CommandCustomHeader;
 import org.apache.rocketmq.remoting.annotation.CFNotNull;
-import org.apache.rocketmq.remoting.annotation.CFNullable;
 import org.apache.rocketmq.remoting.exception.RemotingCommandException;
 
-public class ConsumeMessageDirectlyResultRequestHeader implements CommandCustomHeader {
+public class RemoveBrokerRequestHeader implements CommandCustomHeader {
     @CFNotNull
-    private String consumerGroup;
-    @CFNullable
-    private String clientId;
-    @CFNullable
-    private String msgId;
-    @CFNullable
     private String brokerName;
+    @CFNotNull
+    private String brokerClusterName;
+    @CFNotNull
+    private Long brokerId;
 
-    @Override
-    public void checkFields() throws RemotingCommandException {
-    }
-
-    public String getConsumerGroup() {
-        return consumerGroup;
-    }
+    @Override public void checkFields() throws RemotingCommandException {
 
-    public void setConsumerGroup(String consumerGroup) {
-        this.consumerGroup = consumerGroup;
     }
 
     public String getBrokerName() {
@@ -52,19 +41,19 @@ public class ConsumeMessageDirectlyResultRequestHeader implements CommandCustomH
         this.brokerName = brokerName;
     }
 
-    public String getClientId() {
-        return clientId;
+    public String getBrokerClusterName() {
+        return brokerClusterName;
     }
 
-    public void setClientId(String clientId) {
-        this.clientId = clientId;
+    public void setBrokerClusterName(String brokerClusterName) {
+        this.brokerClusterName = brokerClusterName;
     }
 
-    public String getMsgId() {
-        return msgId;
+    public Long getBrokerId() {
+        return brokerId;
     }
 
-    public void setMsgId(String msgId) {
-        this.msgId = msgId;
+    public void setBrokerId(Long brokerId) {
+        this.brokerId = brokerId;
     }
 }
diff --git a/common/src/main/java/org/apache/rocketmq/common/protocol/header/namesrv/DeleteTopicFromNamesrvRequestHeader.java b/common/src/main/java/org/apache/rocketmq/common/protocol/header/ResetMasterFlushOffsetHeader.java
similarity index 75%
copy from common/src/main/java/org/apache/rocketmq/common/protocol/header/namesrv/DeleteTopicFromNamesrvRequestHeader.java
copy to common/src/main/java/org/apache/rocketmq/common/protocol/header/ResetMasterFlushOffsetHeader.java
index ed8e7a3..b511919 100644
--- a/common/src/main/java/org/apache/rocketmq/common/protocol/header/namesrv/DeleteTopicFromNamesrvRequestHeader.java
+++ b/common/src/main/java/org/apache/rocketmq/common/protocol/header/ResetMasterFlushOffsetHeader.java
@@ -14,25 +14,27 @@
  * See the License for the specific language governing permissions and
  * limitations under the License.
  */
-package org.apache.rocketmq.common.protocol.header.namesrv;
+
+package org.apache.rocketmq.common.protocol.header;
 
 import org.apache.rocketmq.remoting.CommandCustomHeader;
 import org.apache.rocketmq.remoting.annotation.CFNotNull;
 import org.apache.rocketmq.remoting.exception.RemotingCommandException;
 
-public class DeleteTopicFromNamesrvRequestHeader implements CommandCustomHeader {
+public class ResetMasterFlushOffsetHeader implements CommandCustomHeader {
     @CFNotNull
-    private String topic;
+    private Long masterFlushOffset;
 
     @Override
     public void checkFields() throws RemotingCommandException {
+
     }
 
-    public String getTopic() {
-        return topic;
+    public Long getMasterFlushOffset() {
+        return masterFlushOffset;
     }
 
-    public void setTopic(String topic) {
-        this.topic = topic;
+    public void setMasterFlushOffset(Long masterFlushOffset) {
+        this.masterFlushOffset = masterFlushOffset;
     }
 }
diff --git a/common/src/main/java/org/apache/rocketmq/common/protocol/header/ConsumeMessageDirectlyResultRequestHeader.java b/common/src/main/java/org/apache/rocketmq/common/protocol/header/StatisticsMessagesRequestHeader.java
similarity index 63%
copy from common/src/main/java/org/apache/rocketmq/common/protocol/header/ConsumeMessageDirectlyResultRequestHeader.java
copy to common/src/main/java/org/apache/rocketmq/common/protocol/header/StatisticsMessagesRequestHeader.java
index 1bd089d..e25e21a 100644
--- a/common/src/main/java/org/apache/rocketmq/common/protocol/header/ConsumeMessageDirectlyResultRequestHeader.java
+++ b/common/src/main/java/org/apache/rocketmq/common/protocol/header/StatisticsMessagesRequestHeader.java
@@ -19,18 +19,18 @@ package org.apache.rocketmq.common.protocol.header;
 
 import org.apache.rocketmq.remoting.CommandCustomHeader;
 import org.apache.rocketmq.remoting.annotation.CFNotNull;
-import org.apache.rocketmq.remoting.annotation.CFNullable;
 import org.apache.rocketmq.remoting.exception.RemotingCommandException;
 
-public class ConsumeMessageDirectlyResultRequestHeader implements CommandCustomHeader {
+public class StatisticsMessagesRequestHeader implements CommandCustomHeader {
     @CFNotNull
     private String consumerGroup;
-    @CFNullable
-    private String clientId;
-    @CFNullable
-    private String msgId;
-    @CFNullable
-    private String brokerName;
+    @CFNotNull
+    private String topic;
+    @CFNotNull
+    private int queueId;
+
+    private long fromTime;
+    private long toTime;
 
     @Override
     public void checkFields() throws RemotingCommandException {
@@ -44,27 +44,38 @@ public class ConsumeMessageDirectlyResultRequestHeader implements CommandCustomH
         this.consumerGroup = consumerGroup;
     }
 
-    public String getBrokerName() {
-        return brokerName;
+    public String getTopic() {
+        return topic;
+    }
+
+    public void setTopic(String topic) {
+        this.topic = topic;
+    }
+
+    public int getQueueId() {
+        if (queueId < 0) {
+            return -1;
+        }
+        return queueId;
     }
 
-    public void setBrokerName(String brokerName) {
-        this.brokerName = brokerName;
+    public void setQueueId(int queueId) {
+        this.queueId = queueId;
     }
 
-    public String getClientId() {
-        return clientId;
+    public long getFromTime() {
+        return fromTime;
     }
 
-    public void setClientId(String clientId) {
-        this.clientId = clientId;
+    public void setFromTime(long fromTime) {
+        this.fromTime = fromTime;
     }
 
-    public String getMsgId() {
-        return msgId;
+    public long getToTime() {
+        return toTime;
     }
 
-    public void setMsgId(String msgId) {
-        this.msgId = msgId;
+    public void setToTime(long toTime) {
+        this.toTime = toTime;
     }
-}
+}
\ No newline at end of file
diff --git a/common/src/main/java/org/apache/rocketmq/common/protocol/header/GetMaxOffsetRequestHeader.java b/common/src/main/java/org/apache/rocketmq/common/protocol/header/UpdateGroupForbiddenRequestHeader.java
similarity index 68%
copy from common/src/main/java/org/apache/rocketmq/common/protocol/header/GetMaxOffsetRequestHeader.java
copy to common/src/main/java/org/apache/rocketmq/common/protocol/header/UpdateGroupForbiddenRequestHeader.java
index e961af9..902b4ac 100644
--- a/common/src/main/java/org/apache/rocketmq/common/protocol/header/GetMaxOffsetRequestHeader.java
+++ b/common/src/main/java/org/apache/rocketmq/common/protocol/header/UpdateGroupForbiddenRequestHeader.java
@@ -16,42 +16,49 @@
  */
 
 /**
- * $Id: GetMaxOffsetRequestHeader.java 1835 2013-05-16 02:00:50Z vintagewang@apache.org $
+ * $Id: CreateTopicRequestHeader.java 1835 2013-05-16 02:00:50Z vintagewang@apache.org $
  */
 package org.apache.rocketmq.common.protocol.header;
 
-import org.apache.rocketmq.common.rpc.TopicQueueRequestHeader;
+import org.apache.rocketmq.remoting.CommandCustomHeader;
 import org.apache.rocketmq.remoting.annotation.CFNotNull;
 import org.apache.rocketmq.remoting.exception.RemotingCommandException;
 
-public class GetMaxOffsetRequestHeader extends TopicQueueRequestHeader {
+public class UpdateGroupForbiddenRequestHeader implements CommandCustomHeader {
     @CFNotNull
-    private String topic;
+    private String  group;
     @CFNotNull
-    private Integer queueId;
+    private String  topic;
+
+    private Boolean readable;
 
     @Override
     public void checkFields() throws RemotingCommandException {
+
     }
 
-    @Override
     public String getTopic() {
         return topic;
     }
 
-    @Override
     public void setTopic(String topic) {
         this.topic = topic;
     }
 
-    @Override
-    public Integer getQueueId() {
-        return queueId;
+    public String getGroup() {
+        return group;
     }
 
-    @Override
-    public void setQueueId(Integer queueId) {
-        this.queueId = queueId;
+    public void setGroup(String group) {
+        this.group = group;
+    }
+
+    public Boolean getReadable() {
+        return readable;
+    }
+
+    public void setReadable(Boolean readable) {
+        this.readable = readable;
     }
 
 }
diff --git a/common/src/main/java/org/apache/rocketmq/common/protocol/header/namesrv/RegisterBrokerRequestHeader.java b/common/src/main/java/org/apache/rocketmq/common/protocol/header/namesrv/BrokerHeartbeatRequestHeader.java
similarity index 61%
copy from common/src/main/java/org/apache/rocketmq/common/protocol/header/namesrv/RegisterBrokerRequestHeader.java
copy to common/src/main/java/org/apache/rocketmq/common/protocol/header/namesrv/BrokerHeartbeatRequestHeader.java
index 19175b0..e21ef90 100644
--- a/common/src/main/java/org/apache/rocketmq/common/protocol/header/namesrv/RegisterBrokerRequestHeader.java
+++ b/common/src/main/java/org/apache/rocketmq/common/protocol/header/namesrv/BrokerHeartbeatRequestHeader.java
@@ -15,40 +15,22 @@
  * limitations under the License.
  */
 
-/**
- * $Id: RegisterBrokerRequestHeader.java 1835 2013-05-16 02:00:50Z vintagewang@apache.org $
- */
 package org.apache.rocketmq.common.protocol.header.namesrv;
 
 import org.apache.rocketmq.remoting.CommandCustomHeader;
 import org.apache.rocketmq.remoting.annotation.CFNotNull;
 import org.apache.rocketmq.remoting.exception.RemotingCommandException;
 
-public class RegisterBrokerRequestHeader implements CommandCustomHeader {
-    @CFNotNull
-    private String brokerName;
-    @CFNotNull
-    private String brokerAddr;
+public class BrokerHeartbeatRequestHeader implements CommandCustomHeader {
     @CFNotNull
     private String clusterName;
     @CFNotNull
-    private String haServerAddr;
+    private String brokerAddr;
     @CFNotNull
-    private Long brokerId;
-
-    private boolean compressed;
-
-    private Integer bodyCrc32 = 0;
-
-    public void checkFields() throws RemotingCommandException {
-    }
+    private String brokerName;
 
-    public String getBrokerName() {
-        return brokerName;
-    }
+    @Override public void checkFields() throws RemotingCommandException {
 
-    public void setBrokerName(String brokerName) {
-        this.brokerName = brokerName;
     }
 
     public String getBrokerAddr() {
@@ -67,35 +49,11 @@ public class RegisterBrokerRequestHeader implements CommandCustomHeader {
         this.clusterName = clusterName;
     }
 
-    public String getHaServerAddr() {
-        return haServerAddr;
-    }
-
-    public void setHaServerAddr(String haServerAddr) {
-        this.haServerAddr = haServerAddr;
-    }
-
-    public Long getBrokerId() {
-        return brokerId;
-    }
-
-    public void setBrokerId(Long brokerId) {
-        this.brokerId = brokerId;
-    }
-
-    public boolean isCompressed() {
-        return compressed;
-    }
-
-    public void setCompressed(boolean compressed) {
-        this.compressed = compressed;
-    }
-
-    public Integer getBodyCrc32() {
-        return bodyCrc32;
+    public String getBrokerName() {
+        return brokerName;
     }
 
-    public void setBodyCrc32(Integer bodyCrc32) {
-        this.bodyCrc32 = bodyCrc32;
+    public void setBrokerName(String brokerName) {
+        this.brokerName = brokerName;
     }
 }
diff --git a/common/src/main/java/org/apache/rocketmq/common/protocol/header/namesrv/DeleteTopicFromNamesrvRequestHeader.java b/common/src/main/java/org/apache/rocketmq/common/protocol/header/namesrv/DeleteTopicFromNamesrvRequestHeader.java
index ed8e7a3..c9dd04b 100644
--- a/common/src/main/java/org/apache/rocketmq/common/protocol/header/namesrv/DeleteTopicFromNamesrvRequestHeader.java
+++ b/common/src/main/java/org/apache/rocketmq/common/protocol/header/namesrv/DeleteTopicFromNamesrvRequestHeader.java
@@ -24,6 +24,8 @@ public class DeleteTopicFromNamesrvRequestHeader implements CommandCustomHeader
     @CFNotNull
     private String topic;
 
+    private String clusterName;
+
     @Override
     public void checkFields() throws RemotingCommandException {
     }
@@ -35,4 +37,12 @@ public class DeleteTopicFromNamesrvRequestHeader implements CommandCustomHeader
     public void setTopic(String topic) {
         this.topic = topic;
     }
+
+    public String getClusterName() {
+        return clusterName;
+    }
+
+    public void setClusterName(String clusterName) {
+        this.clusterName = clusterName;
+    }
 }
diff --git a/common/src/main/java/org/apache/rocketmq/common/protocol/header/namesrv/GetRouteInfoRequestHeader.java b/common/src/main/java/org/apache/rocketmq/common/protocol/header/namesrv/GetRouteInfoRequestHeader.java
index a2806e6..d1f5ca8 100644
--- a/common/src/main/java/org/apache/rocketmq/common/protocol/header/namesrv/GetRouteInfoRequestHeader.java
+++ b/common/src/main/java/org/apache/rocketmq/common/protocol/header/namesrv/GetRouteInfoRequestHeader.java
@@ -25,6 +25,8 @@ import org.apache.rocketmq.remoting.annotation.CFNotNull;
 import org.apache.rocketmq.remoting.exception.RemotingCommandException;
 
 public class GetRouteInfoRequestHeader implements CommandCustomHeader {
+    public static char split = 1;
+
     @CFNotNull
     private String topic;
 
diff --git a/common/src/main/java/org/apache/rocketmq/common/protocol/header/namesrv/RegisterBrokerRequestHeader.java b/common/src/main/java/org/apache/rocketmq/common/protocol/header/namesrv/RegisterBrokerRequestHeader.java
index 19175b0..b2c7884 100644
--- a/common/src/main/java/org/apache/rocketmq/common/protocol/header/namesrv/RegisterBrokerRequestHeader.java
+++ b/common/src/main/java/org/apache/rocketmq/common/protocol/header/namesrv/RegisterBrokerRequestHeader.java
@@ -22,6 +22,7 @@ package org.apache.rocketmq.common.protocol.header.namesrv;
 
 import org.apache.rocketmq.remoting.CommandCustomHeader;
 import org.apache.rocketmq.remoting.annotation.CFNotNull;
+import org.apache.rocketmq.remoting.annotation.CFNullable;
 import org.apache.rocketmq.remoting.exception.RemotingCommandException;
 
 public class RegisterBrokerRequestHeader implements CommandCustomHeader {
@@ -35,11 +36,16 @@ public class RegisterBrokerRequestHeader implements CommandCustomHeader {
     private String haServerAddr;
     @CFNotNull
     private Long brokerId;
+    @CFNullable
+    private Long heartbeatTimeoutMillis;
+    @CFNullable
+    private Boolean enableActingMaster;
 
     private boolean compressed;
 
     private Integer bodyCrc32 = 0;
 
+    @Override
     public void checkFields() throws RemotingCommandException {
     }
 
@@ -83,6 +89,14 @@ public class RegisterBrokerRequestHeader implements CommandCustomHeader {
         this.brokerId = brokerId;
     }
 
+    public Long getHeartbeatTimeoutMillis() {
+        return heartbeatTimeoutMillis;
+    }
+
+    public void setHeartbeatTimeoutMillis(Long heartbeatTimeoutMillis) {
+        this.heartbeatTimeoutMillis = heartbeatTimeoutMillis;
+    }
+
     public boolean isCompressed() {
         return compressed;
     }
@@ -98,4 +112,12 @@ public class RegisterBrokerRequestHeader implements CommandCustomHeader {
     public void setBodyCrc32(Integer bodyCrc32) {
         this.bodyCrc32 = bodyCrc32;
     }
+
+    public Boolean getEnableActingMaster() {
+        return enableActingMaster;
+    }
+
+    public void setEnableActingMaster(Boolean enableActingMaster) {
+        this.enableActingMaster = enableActingMaster;
+    }
 }
diff --git a/common/src/main/java/org/apache/rocketmq/common/protocol/header/namesrv/DeleteTopicFromNamesrvRequestHeader.java b/common/src/main/java/org/apache/rocketmq/common/protocol/header/namesrv/RegisterTopicRequestHeader.java
similarity index 94%
copy from common/src/main/java/org/apache/rocketmq/common/protocol/header/namesrv/DeleteTopicFromNamesrvRequestHeader.java
copy to common/src/main/java/org/apache/rocketmq/common/protocol/header/namesrv/RegisterTopicRequestHeader.java
index ed8e7a3..ff19b28 100644
--- a/common/src/main/java/org/apache/rocketmq/common/protocol/header/namesrv/DeleteTopicFromNamesrvRequestHeader.java
+++ b/common/src/main/java/org/apache/rocketmq/common/protocol/header/namesrv/RegisterTopicRequestHeader.java
@@ -20,7 +20,7 @@ import org.apache.rocketmq.remoting.CommandCustomHeader;
 import org.apache.rocketmq.remoting.annotation.CFNotNull;
 import org.apache.rocketmq.remoting.exception.RemotingCommandException;
 
-public class DeleteTopicFromNamesrvRequestHeader implements CommandCustomHeader {
+public class RegisterTopicRequestHeader implements CommandCustomHeader {
     @CFNotNull
     private String topic;
 
diff --git a/common/src/main/java/org/apache/rocketmq/common/protocol/route/BrokerData.java b/common/src/main/java/org/apache/rocketmq/common/protocol/route/BrokerData.java
index 36599fb..73d725c 100644
--- a/common/src/main/java/org/apache/rocketmq/common/protocol/route/BrokerData.java
+++ b/common/src/main/java/org/apache/rocketmq/common/protocol/route/BrokerData.java
@@ -20,6 +20,7 @@ package org.apache.rocketmq.common.protocol.route;
 import java.util.ArrayList;
 import java.util.HashMap;
 import java.util.List;
+import java.util.Map;
 import java.util.Random;
 import org.apache.rocketmq.common.MixAll;
 
@@ -30,10 +31,27 @@ public class BrokerData implements Comparable<BrokerData> {
 
     private final Random random = new Random();
 
+    /**
+     * Enable acting master or not, used for old version HA adaption,
+     */
+    private boolean enableActingMaster = false;
+
     public BrokerData() {
 
     }
 
+    public BrokerData(BrokerData brokerData) {
+        this.cluster = brokerData.cluster;
+        this.brokerName = brokerData.brokerName;
+        if (brokerData.brokerAddrs != null) {
+            this.brokerAddrs = new HashMap<Long, String>();
+            for (final Map.Entry<Long, String> brokerEntry : brokerData.brokerAddrs.entrySet()) {
+                this.brokerAddrs.put(brokerEntry.getKey(), brokerEntry.getValue());
+            }
+        }
+        this.enableActingMaster = brokerData.enableActingMaster;
+    }
+
     public BrokerData(String cluster, String brokerName, HashMap<Long, String> brokerAddrs) {
         this.cluster = cluster;
         this.brokerName = brokerName;
@@ -73,6 +91,14 @@ public class BrokerData implements Comparable<BrokerData> {
         this.cluster = cluster;
     }
 
+    public boolean isEnableActingMaster() {
+        return enableActingMaster;
+    }
+
+    public void setEnableActingMaster(boolean enableActingMaster) {
+        this.enableActingMaster = enableActingMaster;
+    }
+
     @Override
     public int hashCode() {
         final int prime = 31;
@@ -84,29 +110,33 @@ public class BrokerData implements Comparable<BrokerData> {
 
     @Override
     public boolean equals(Object obj) {
-        if (this == obj)
+        if (this == obj) {
             return true;
-        if (obj == null)
+        }
+        if (obj == null) {
             return false;
-        if (getClass() != obj.getClass())
+        }
+        if (getClass() != obj.getClass()) {
             return false;
+        }
         BrokerData other = (BrokerData) obj;
         if (brokerAddrs == null) {
-            if (other.brokerAddrs != null)
+            if (other.brokerAddrs != null) {
                 return false;
-        } else if (!brokerAddrs.equals(other.brokerAddrs))
+            }
+        } else if (!brokerAddrs.equals(other.brokerAddrs)) {
             return false;
+        }
         if (brokerName == null) {
-            if (other.brokerName != null)
-                return false;
-        } else if (!brokerName.equals(other.brokerName))
-            return false;
-        return true;
+            return other.brokerName == null;
+        } else {
+            return brokerName.equals(other.brokerName);
+        }
     }
 
     @Override
     public String toString() {
-        return "BrokerData [brokerName=" + brokerName + ", brokerAddrs=" + brokerAddrs + "]";
+        return "BrokerData [brokerName=" + brokerName + ", brokerAddrs=" + brokerAddrs + ", enableActingMaster=" + enableActingMaster + "]";
     }
 
     @Override
diff --git a/common/src/main/java/org/apache/rocketmq/common/protocol/route/QueueData.java b/common/src/main/java/org/apache/rocketmq/common/protocol/route/QueueData.java
index 2dbb290..6e9e653 100644
--- a/common/src/main/java/org/apache/rocketmq/common/protocol/route/QueueData.java
+++ b/common/src/main/java/org/apache/rocketmq/common/protocol/route/QueueData.java
@@ -27,6 +27,19 @@ public class QueueData implements Comparable<QueueData> {
     private int perm;
     private int topicSysFlag;
 
+    public QueueData() {
+
+    }
+
+    // Deep copy QueueData
+    public QueueData(QueueData queueData) {
+        this.brokerName = queueData.brokerName;
+        this.readQueueNums = queueData.readQueueNums;
+        this.writeQueueNums = queueData.writeQueueNums;
+        this.perm = queueData.perm;
+        this.topicSysFlag = queueData.topicSysFlag;
+    }
+
     public int getReadQueueNums() {
         return readQueueNums;
     }
diff --git a/common/src/main/java/org/apache/rocketmq/common/protocol/route/TopicRouteData.java b/common/src/main/java/org/apache/rocketmq/common/protocol/route/TopicRouteData.java
index 3fe4630..0c02078 100644
--- a/common/src/main/java/org/apache/rocketmq/common/protocol/route/TopicRouteData.java
+++ b/common/src/main/java/org/apache/rocketmq/common/protocol/route/TopicRouteData.java
@@ -25,6 +25,8 @@ import java.util.HashMap;
 import java.util.List;
 import java.util.Map;
 
+import java.util.concurrent.ConcurrentHashMap;
+import java.util.concurrent.ConcurrentMap;
 import org.apache.rocketmq.common.statictopic.TopicQueueMappingInfo;
 import org.apache.rocketmq.remoting.protocol.RemotingSerializable;
 
@@ -37,6 +39,9 @@ public class TopicRouteData extends RemotingSerializable {
     private Map<String/*brokerName*/, TopicQueueMappingInfo> topicQueueMappingByBroker;
 
     public TopicRouteData() {
+        queueDatas = new ArrayList<QueueData>();
+        brokerDatas = new ArrayList<BrokerData>();
+        filterServerTable = new HashMap<String, List<String>>();
     }
 
     public TopicRouteData(TopicRouteData topicRouteData) {
@@ -62,6 +67,54 @@ public class TopicRouteData extends RemotingSerializable {
         }
     }
 
+    public TopicRouteData cloneTopicRouteData() {
+        TopicRouteData topicRouteData = new TopicRouteData();
+
+        topicRouteData.setOrderTopicConf(this.orderTopicConf);
+
+        topicRouteData.getQueueDatas().addAll(this.queueDatas);
+        topicRouteData.getBrokerDatas().addAll(this.brokerDatas);
+        topicRouteData.getFilterServerTable().putAll(this.filterServerTable);
+        if (this.topicQueueMappingByBroker != null) {
+            Map<String, TopicQueueMappingInfo> cloneMap = new HashMap<>(this.topicQueueMappingByBroker);
+            topicRouteData.setTopicQueueMappingByBroker(cloneMap);
+        }
+        return topicRouteData;
+    }
+
+    public TopicRouteData deepCloneTopicRouteData() {
+        TopicRouteData topicRouteData = new TopicRouteData();
+
+        topicRouteData.setOrderTopicConf(this.orderTopicConf);
+
+        for (final QueueData queueData : this.queueDatas) {
+            topicRouteData.getQueueDatas().add(new QueueData(queueData));
+        }
+
+        for (final BrokerData brokerData : this.brokerDatas) {
+            topicRouteData.getBrokerDatas().add(new BrokerData(brokerData));
+        }
+
+        for (final Map.Entry<String, List<String>> listEntry : this.filterServerTable.entrySet()) {
+            topicRouteData.getFilterServerTable().put(listEntry.getKey(),
+                new ArrayList<String>(listEntry.getValue()));
+        }
+        if (this.topicQueueMappingByBroker != null) {
+            Map<String, TopicQueueMappingInfo> cloneMap = new HashMap<>(this.topicQueueMappingByBroker.size());
+            for (final Map.Entry<String, TopicQueueMappingInfo> entry : this.getTopicQueueMappingByBroker().entrySet()) {
+                TopicQueueMappingInfo topicQueueMappingInfo = new TopicQueueMappingInfo(entry.getValue().getTopic(), entry.getValue().getTotalQueues(), entry.getValue().getBname(), entry.getValue().getEpoch());
+                topicQueueMappingInfo.setDirty(entry.getValue().isDirty());
+                topicQueueMappingInfo.setScope(entry.getValue().getScope());
+                ConcurrentMap<Integer, Integer> concurrentMap = new ConcurrentHashMap<Integer, Integer>(entry.getValue().getCurrIdMap());
+                topicQueueMappingInfo.setCurrIdMap(concurrentMap);
+                cloneMap.put(entry.getKey(), topicQueueMappingInfo);
+            }
+            topicRouteData.setTopicQueueMappingByBroker(cloneMap);
+        }
+
+        return topicRouteData;
+    }
+
     public List<QueueData> getQueueDatas() {
         return queueDatas;
     }
diff --git a/common/src/main/java/org/apache/rocketmq/common/message/MessageType.java b/common/src/main/java/org/apache/rocketmq/common/protocol/route/TopicRouteDatas.java
similarity index 61%
copy from common/src/main/java/org/apache/rocketmq/common/message/MessageType.java
copy to common/src/main/java/org/apache/rocketmq/common/protocol/route/TopicRouteDatas.java
index 08091f4..42a66b6 100644
--- a/common/src/main/java/org/apache/rocketmq/common/message/MessageType.java
+++ b/common/src/main/java/org/apache/rocketmq/common/protocol/route/TopicRouteDatas.java
@@ -15,11 +15,22 @@
  * limitations under the License.
  */
 
-package org.apache.rocketmq.common.message;
+package org.apache.rocketmq.common.protocol.route;
 
-public enum MessageType {
-    Normal_Msg,
-    Trans_Msg_Half,
-    Trans_msg_Commit,
-    Delay_Msg,
+import org.apache.rocketmq.remoting.protocol.RemotingSerializable;
+
+import java.util.HashMap;
+import java.util.Map;
+
+public class TopicRouteDatas extends RemotingSerializable {
+
+    private Map<String, TopicRouteData> topics = new HashMap<String, TopicRouteData>();
+
+    public Map<String, TopicRouteData> getTopics() {
+        return topics;
+    }
+
+    public void setTopics(Map<String, TopicRouteData> topics) {
+        this.topics = topics;
+    }
 }
diff --git a/common/src/main/java/org/apache/rocketmq/common/statistics/FutureHolder.java b/common/src/main/java/org/apache/rocketmq/common/statistics/FutureHolder.java
new file mode 100644
index 0000000..6356101
--- /dev/null
+++ b/common/src/main/java/org/apache/rocketmq/common/statistics/FutureHolder.java
@@ -0,0 +1,53 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.rocketmq.common.statistics;
+
+import java.util.concurrent.BlockingQueue;
+import java.util.concurrent.ConcurrentHashMap;
+import java.util.concurrent.ConcurrentMap;
+import java.util.concurrent.Future;
+import java.util.concurrent.LinkedBlockingQueue;
+
+public class FutureHolder<T> {
+    private ConcurrentMap<T, BlockingQueue<Future>> futureMap = new ConcurrentHashMap<T, BlockingQueue<Future>>(8);
+
+    public void addFuture(T t, Future future) {
+        BlockingQueue<Future> list = futureMap.get(t);
+        if (list == null) {
+            list = new LinkedBlockingQueue<Future>();
+            BlockingQueue<Future> old = futureMap.putIfAbsent(t, list);
+            if (old != null) {
+                list = old;
+            }
+        }
+        list.add(future);
+    }
+
+    public void removeAllFuture(T t) {
+        cancelAll(t, false);
+        futureMap.remove(t);
+    }
+
+    private void cancelAll(T t, boolean mayInterruptIfRunning) {
+        BlockingQueue<Future> list = futureMap.get(t);
+        if (list != null) {
+            for (Future future : list) {
+                future.cancel(mayInterruptIfRunning);
+            }
+        }
+    }
+}
diff --git a/common/src/main/java/org/apache/rocketmq/common/message/MessageType.java b/common/src/main/java/org/apache/rocketmq/common/statistics/Interceptor.java
similarity index 78%
copy from common/src/main/java/org/apache/rocketmq/common/message/MessageType.java
copy to common/src/main/java/org/apache/rocketmq/common/statistics/Interceptor.java
index 08091f4..0af55e0 100644
--- a/common/src/main/java/org/apache/rocketmq/common/message/MessageType.java
+++ b/common/src/main/java/org/apache/rocketmq/common/statistics/Interceptor.java
@@ -14,12 +14,18 @@
  * See the License for the specific language governing permissions and
  * limitations under the License.
  */
+package org.apache.rocketmq.common.statistics;
 
-package org.apache.rocketmq.common.message;
+/**
+ * interceptor
+ */
+public interface Interceptor {
+    /**
+     * increase multiple values
+     *
+     * @param deltas
+     */
+    void inc(long... deltas);
 
-public enum MessageType {
-    Normal_Msg,
-    Trans_Msg_Half,
-    Trans_msg_Commit,
-    Delay_Msg,
+    void reset();
 }
diff --git a/common/src/main/java/org/apache/rocketmq/common/statistics/StatisticsBrief.java b/common/src/main/java/org/apache/rocketmq/common/statistics/StatisticsBrief.java
new file mode 100644
index 0000000..970bff9
--- /dev/null
+++ b/common/src/main/java/org/apache/rocketmq/common/statistics/StatisticsBrief.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.common.statistics;
+
+import org.apache.commons.lang3.ArrayUtils;
+
+import java.util.concurrent.atomic.AtomicInteger;
+import java.util.concurrent.atomic.AtomicLong;
+
+public class StatisticsBrief {
+    public static final int META_RANGE_INDEX = 0;
+    public static final int META_SLOT_NUM_INDEX = 1;
+
+    // TopPercentile
+    private long[][] topPercentileMeta;
+    private AtomicInteger[] counts;
+    private AtomicLong totalCount;
+
+    // max min avg total
+    private long max;
+    private long min;
+    private long total;
+
+    public StatisticsBrief(long[][] topPercentileMeta) {
+        if (!isLegalMeta(topPercentileMeta)) {
+            throw new IllegalArgumentException("illegal topPercentileMeta");
+        }
+
+        this.topPercentileMeta = topPercentileMeta;
+        this.counts = new AtomicInteger[slotNum(topPercentileMeta)];
+        this.totalCount = new AtomicLong(0);
+        reset();
+    }
+
+    public void reset() {
+        for (int i = 0; i < counts.length; i++) {
+            if (counts[i] == null) {
+                counts[i] = new AtomicInteger(0);
+            } else {
+                counts[i].set(0);
+            }
+        }
+        totalCount.set(0);
+
+        synchronized (this) {
+            max = 0;
+            min = Long.MAX_VALUE;
+            total = 0;
+        }
+    }
+
+    private static boolean isLegalMeta(long[][] meta) {
+        if (ArrayUtils.isEmpty(meta)) {
+            return false;
+        }
+
+        for (long[] line : meta) {
+            if (ArrayUtils.isEmpty(line) || line.length != 2) {
+                return false;
+            }
+        }
+
+        return true;
+    }
+
+    private static int slotNum(long[][] meta) {
+        int ret = 1;
+        for (long[] line : meta) {
+            ret += line[META_SLOT_NUM_INDEX];
+        }
+        return ret;
+    }
+
+    public void sample(long value) {
+        int index = getSlotIndex(value);
+        counts[index].incrementAndGet();
+        totalCount.incrementAndGet();
+
+        synchronized (this) {
+            max = Math.max(max, value);
+            min = Math.min(min, value);
+            total += value;
+        }
+    }
+
+    public long tp999() {
+        return getTPValue(0.999f);
+    }
+
+    public long getTPValue(float ratio) {
+        if (ratio <= 0 || ratio >= 1) {
+            ratio = 0.99f;
+        }
+        long count = totalCount.get();
+        long excludes = (long)(count - count * ratio);
+        if (excludes == 0) {
+            return getMax();
+        }
+
+        int tmp = 0;
+        for (int i = counts.length - 1; i > 0; i--) {
+            tmp += counts[i].get();
+            if (tmp > excludes) {
+                return Math.min(getSlotTPValue(i), getMax());
+            }
+        }
+        return 0;
+    }
+
+    private long getSlotTPValue(int index) {
+        int slotNumLeft = index;
+        for (int i = 0; i < topPercentileMeta.length; i++) {
+            int slotNum = (int)topPercentileMeta[i][META_SLOT_NUM_INDEX];
+            if (slotNumLeft < slotNum) {
+                long metaRangeMax = topPercentileMeta[i][META_RANGE_INDEX];
+                long metaRangeMin = 0;
+                if (i > 0) {
+                    metaRangeMin = topPercentileMeta[i - 1][META_RANGE_INDEX];
+                }
+
+                return metaRangeMin + (metaRangeMax - metaRangeMin) / slotNum * (slotNumLeft + 1);
+            } else {
+                slotNumLeft -= slotNum;
+            }
+        }
+        // MAX_VALUE: the last slot
+        return Integer.MAX_VALUE;
+    }
+
+    private int getSlotIndex(long num) {
+        int index = 0;
+        for (int i = 0; i < topPercentileMeta.length; i++) {
+            long rangeMax = topPercentileMeta[i][META_RANGE_INDEX];
+            int slotNum = (int)topPercentileMeta[i][META_SLOT_NUM_INDEX];
+            long rangeMin = (i > 0) ? topPercentileMeta[i - 1][META_RANGE_INDEX] : 0;
+            if (rangeMin <= num && num < rangeMax) {
+                index += (num - rangeMin) / ((rangeMax - rangeMin) / slotNum);
+                break;
+            }
+
+            index += slotNum;
+        }
+        return index;
+    }
+
+    /**
+     * Getters
+     *
+     * @return
+     */
+    public long getMax() {
+        return max;
+    }
+
+    public long getMin() {
+        return totalCount.get() > 0 ? min : 0;
+    }
+
+    public long getTotal() {
+        return total;
+    }
+
+    public long getCnt() {
+        return totalCount.get();
+    }
+
+    public double getAvg() {
+        return totalCount.get() != 0 ? ((double)total) / totalCount.get() : 0;
+    }
+}
diff --git a/common/src/main/java/org/apache/rocketmq/common/statistics/StatisticsBriefInterceptor.java b/common/src/main/java/org/apache/rocketmq/common/statistics/StatisticsBriefInterceptor.java
new file mode 100644
index 0000000..b0b6937
--- /dev/null
+++ b/common/src/main/java/org/apache/rocketmq/common/statistics/StatisticsBriefInterceptor.java
@@ -0,0 +1,76 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.rocketmq.common.statistics;
+
+import org.apache.commons.lang3.ArrayUtils;
+import org.apache.commons.lang3.tuple.Pair;
+
+/**
+ * interceptor to generate statistics brief
+ */
+public class StatisticsBriefInterceptor implements Interceptor {
+    private int[] indexOfItems;
+
+    private StatisticsBrief[] statisticsBriefs;
+
+    public StatisticsBriefInterceptor(StatisticsItem item, Pair<String, long[][]>[] briefMetas) {
+        indexOfItems = new int[briefMetas.length];
+        statisticsBriefs = new StatisticsBrief[briefMetas.length];
+        for (int i = 0; i < briefMetas.length; i++) {
+            String name = briefMetas[i].getKey();
+            int index = ArrayUtils.indexOf(item.getItemNames(), name);
+            if (index < 0) {
+                throw new IllegalArgumentException("illegal breifItemName: " + name);
+            }
+            indexOfItems[i] = index;
+            statisticsBriefs[i] = new StatisticsBrief(briefMetas[i].getValue());
+        }
+    }
+
+    @Override
+    public void inc(long... itemValues) {
+        for (int i = 0; i < indexOfItems.length; i++) {
+            int indexOfItem = indexOfItems[i];
+            if (indexOfItem < itemValues.length) {
+                statisticsBriefs[i].sample(itemValues[indexOfItem]);
+            }
+        }
+    }
+
+    @Override
+    public void reset() {
+        for (StatisticsBrief brief : statisticsBriefs) {
+            brief.reset();
+        }
+    }
+
+    public int[] getIndexOfItems() {
+        return indexOfItems;
+    }
+
+    public void setIndexOfItems(int[] indexOfItems) {
+        this.indexOfItems = indexOfItems;
+    }
+
+    public StatisticsBrief[] getStatisticsBriefs() {
+        return statisticsBriefs;
+    }
+
+    public void setStatisticsBriefs(StatisticsBrief[] statisticsBriefs) {
+        this.statisticsBriefs = statisticsBriefs;
+    }
+}
diff --git a/common/src/main/java/org/apache/rocketmq/common/statistics/StatisticsItem.java b/common/src/main/java/org/apache/rocketmq/common/statistics/StatisticsItem.java
new file mode 100644
index 0000000..23633dc
--- /dev/null
+++ b/common/src/main/java/org/apache/rocketmq/common/statistics/StatisticsItem.java
@@ -0,0 +1,175 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.rocketmq.common.statistics;
+
+import java.security.InvalidParameterException;
+import java.util.Arrays;
+import java.util.concurrent.atomic.AtomicLong;
+
+import org.apache.commons.lang3.ArrayUtils;
+
+/**
+ * Statistics Item
+ */
+public class StatisticsItem {
+    private String statKind;
+    private String statObject;
+
+    private String[] itemNames;
+    private AtomicLong[] itemAccumulates;
+    private AtomicLong invokeTimes;
+
+    private Interceptor interceptor;
+
+    /**
+     * last timestamp when the item was updated
+     */
+    private AtomicLong lastTimeStamp;
+
+    public StatisticsItem(String statKind, String statObject, String... itemNames) {
+        if (itemNames == null || itemNames.length <= 0) {
+            throw new InvalidParameterException("StatisticsItem \"itemNames\" is empty");
+        }
+
+        this.statKind = statKind;
+        this.statObject = statObject;
+        this.itemNames = itemNames;
+
+        AtomicLong[] accs = new AtomicLong[itemNames.length];
+        for (int i = 0; i < itemNames.length; i++) {
+            accs[i] = new AtomicLong(0);
+        }
+
+        this.itemAccumulates = accs;
+        this.invokeTimes = new AtomicLong();
+        this.lastTimeStamp = new AtomicLong(System.currentTimeMillis());
+    }
+
+    public void incItems(long... itemIncs) {
+        int len = Math.min(itemIncs.length, itemAccumulates.length);
+        for (int i = 0; i < len; i++) {
+            itemAccumulates[i].addAndGet(itemIncs[i]);
+        }
+
+        invokeTimes.addAndGet(1);
+        lastTimeStamp.set(System.currentTimeMillis());
+
+        if (interceptor != null) {
+            interceptor.inc(itemIncs);
+        }
+    }
+
+    public boolean allZeros() {
+        if (invokeTimes.get() == 0) {
+            return true;
+        }
+
+        for (AtomicLong acc : itemAccumulates) {
+            if (acc.get() != 0) {
+                return false;
+            }
+        }
+        return true;
+    }
+
+    public String getStatKind() {
+        return statKind;
+    }
+
+    public String getStatObject() {
+        return statObject;
+    }
+
+    public String[] getItemNames() {
+        return itemNames;
+    }
+
+    public AtomicLong[] getItemAccumulates() {
+        return itemAccumulates;
+    }
+
+    public AtomicLong getInvokeTimes() {
+        return invokeTimes;
+    }
+
+    public AtomicLong getLastTimeStamp() {
+        return lastTimeStamp;
+    }
+
+    public AtomicLong getItemAccumulate(String itemName) {
+        int index = ArrayUtils.indexOf(itemNames, itemName);
+        if (index < 0) {
+            return new AtomicLong(0);
+        }
+        return itemAccumulates[index];
+    }
+
+    /**
+     * get snapshot
+     * <p>
+     * Warning: no guarantee of itemAccumulates consistency
+     *
+     * @return
+     */
+    public StatisticsItem snapshot() {
+        StatisticsItem ret = new StatisticsItem(statKind, statObject, itemNames);
+
+        ret.itemAccumulates = new AtomicLong[itemAccumulates.length];
+        for (int i = 0; i < itemAccumulates.length; i++) {
+            ret.itemAccumulates[i] = new AtomicLong(itemAccumulates[i].get());
+        }
+
+        ret.invokeTimes = new AtomicLong(invokeTimes.longValue());
+        ret.lastTimeStamp = new AtomicLong(lastTimeStamp.longValue());
+
+        return ret;
+    }
+
+    /**
+     * subtract another StatisticsItem
+     *
+     * @param item
+     * @return
+     */
+    public StatisticsItem subtract(StatisticsItem item) {
+        if (item == null) {
+            return snapshot();
+        }
+
+        if (!statKind.equals(item.statKind) || !statObject.equals(item.statObject) || !Arrays.equals(itemNames,
+            item.itemNames)) {
+            throw new IllegalArgumentException("StatisticsItem's kind, key and itemNames must be exactly the same");
+        }
+
+        StatisticsItem ret = new StatisticsItem(statKind, statObject, itemNames);
+        ret.invokeTimes = new AtomicLong(invokeTimes.get() - item.invokeTimes.get());
+        ret.itemAccumulates = new AtomicLong[itemAccumulates.length];
+        for (int i = 0; i < itemAccumulates.length; i++) {
+            ret.itemAccumulates[i] = new AtomicLong(itemAccumulates[i].get() - item.itemAccumulates[i].get());
+        }
+        return ret;
+    }
+
+    public Interceptor getInterceptor() {
+        return interceptor;
+    }
+
+    public void setInterceptor(Interceptor interceptor) {
+        this.interceptor = interceptor;
+    }
+
+}
diff --git a/common/src/main/java/org/apache/rocketmq/common/protocol/header/namesrv/DeleteTopicFromNamesrvRequestHeader.java b/common/src/main/java/org/apache/rocketmq/common/statistics/StatisticsItemFormatter.java
similarity index 57%
copy from common/src/main/java/org/apache/rocketmq/common/protocol/header/namesrv/DeleteTopicFromNamesrvRequestHeader.java
copy to common/src/main/java/org/apache/rocketmq/common/statistics/StatisticsItemFormatter.java
index ed8e7a3..87e301a 100644
--- a/common/src/main/java/org/apache/rocketmq/common/protocol/header/namesrv/DeleteTopicFromNamesrvRequestHeader.java
+++ b/common/src/main/java/org/apache/rocketmq/common/statistics/StatisticsItemFormatter.java
@@ -14,25 +14,20 @@
  * See the License for the specific language governing permissions and
  * limitations under the License.
  */
-package org.apache.rocketmq.common.protocol.header.namesrv;
+package org.apache.rocketmq.common.statistics;
 
-import org.apache.rocketmq.remoting.CommandCustomHeader;
-import org.apache.rocketmq.remoting.annotation.CFNotNull;
-import org.apache.rocketmq.remoting.exception.RemotingCommandException;
+import java.util.concurrent.atomic.AtomicLong;
 
-public class DeleteTopicFromNamesrvRequestHeader implements CommandCustomHeader {
-    @CFNotNull
-    private String topic;
-
-    @Override
-    public void checkFields() throws RemotingCommandException {
-    }
-
-    public String getTopic() {
-        return topic;
-    }
-
-    public void setTopic(String topic) {
-        this.topic = topic;
+public class StatisticsItemFormatter {
+    public String format(StatisticsItem statItem) {
+        final String seperator = "|";
+        StringBuilder sb = new StringBuilder();
+        sb.append(statItem.getStatKind()).append(seperator);
+        sb.append(statItem.getStatObject()).append(seperator);
+        for (AtomicLong acc : statItem.getItemAccumulates()) {
+            sb.append(acc.get()).append(seperator);
+        }
+        sb.append(statItem.getInvokeTimes());
+        return sb.toString();
     }
 }
diff --git a/common/src/main/java/org/apache/rocketmq/common/statistics/StatisticsItemPrinter.java b/common/src/main/java/org/apache/rocketmq/common/statistics/StatisticsItemPrinter.java
new file mode 100644
index 0000000..26c1df8
--- /dev/null
+++ b/common/src/main/java/org/apache/rocketmq/common/statistics/StatisticsItemPrinter.java
@@ -0,0 +1,50 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.rocketmq.common.statistics;
+
+import org.apache.rocketmq.logging.InternalLogger;
+
+public class StatisticsItemPrinter {
+    private InternalLogger log;
+
+    private StatisticsItemFormatter formatter;
+
+    public StatisticsItemPrinter(StatisticsItemFormatter formatter, InternalLogger log) {
+        this.formatter = formatter;
+        this.log = log;
+    }
+
+    public void log(InternalLogger log) {
+        this.log = log;
+    }
+
+    public void formatter(StatisticsItemFormatter formatter) {
+        this.formatter = formatter;
+    }
+
+    public void print(String prefix, StatisticsItem statItem, String... suffixs) {
+        StringBuilder suffix = new StringBuilder();
+        for (String str : suffixs) {
+            suffix.append(str);
+        }
+
+        if (log != null) {
+            log.info("{}{}{}", prefix, formatter.format(statItem), suffix.toString());
+        }
+        // System.out.printf("%s %s%s%s\n", new Date().toString(), prefix, formatter.format(statItem), suffix.toString());
+    }
+}
diff --git a/common/src/main/java/org/apache/rocketmq/common/statistics/StatisticsItemScheduledIncrementPrinter.java b/common/src/main/java/org/apache/rocketmq/common/statistics/StatisticsItemScheduledIncrementPrinter.java
new file mode 100644
index 0000000..d949b91
--- /dev/null
+++ b/common/src/main/java/org/apache/rocketmq/common/statistics/StatisticsItemScheduledIncrementPrinter.java
@@ -0,0 +1,290 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.rocketmq.common.statistics;
+
+import java.util.concurrent.ConcurrentHashMap;
+import java.util.concurrent.ScheduledExecutorService;
+import java.util.concurrent.ScheduledFuture;
+import java.util.concurrent.TimeUnit;
+
+public class StatisticsItemScheduledIncrementPrinter extends StatisticsItemScheduledPrinter {
+
+    private String[] tpsItemNames;
+
+    public static final int TPS_INITIAL_DELAY = 0;
+    public static final int TPS_INTREVAL = 1000;
+    public static final String SEPERATOR = "|";
+
+    /**
+     * last snapshots of all scheduled items
+     */
+    private final ConcurrentHashMap<String, ConcurrentHashMap<String, StatisticsItem>> lastItemSnapshots
+        = new ConcurrentHashMap<String, ConcurrentHashMap<String, StatisticsItem>>();
+
+    private final ConcurrentHashMap<String, ConcurrentHashMap<String, StatisticsItemSampleBrief>> sampleBriefs
+        = new ConcurrentHashMap<String, ConcurrentHashMap<String, StatisticsItemSampleBrief>>();
+
+    public StatisticsItemScheduledIncrementPrinter(String name, StatisticsItemPrinter printer,
+                                                   ScheduledExecutorService executor, InitialDelay initialDelay,
+                                                   long interval, String[] tpsItemNames, Valve valve) {
+        super(name, printer, executor, initialDelay, interval, valve);
+        this.tpsItemNames = tpsItemNames;
+    }
+
+    /**
+     * schedule a StatisticsItem to print the Increments periodically
+     */
+    @Override
+    public void schedule(final StatisticsItem item) {
+        setItemSampleBrief(item.getStatKind(), item.getStatObject(), new StatisticsItemSampleBrief(item, tpsItemNames));
+
+        // print log every ${interval} miliseconds
+        ScheduledFuture future = executor.scheduleAtFixedRate(new Runnable() {
+            @Override
+            public void run() {
+                if (!enabled()) {
+                    return;
+                }
+
+                StatisticsItem snapshot = item.snapshot();
+                StatisticsItem lastSnapshot = getItemSnapshot(lastItemSnapshots, item.getStatKind(),
+                    item.getStatObject());
+                StatisticsItem increment = snapshot.subtract(lastSnapshot);
+
+                Interceptor inteceptor = item.getInterceptor();
+                String inteceptorStr = formatInterceptor(inteceptor);
+                if (inteceptor != null) {
+                    inteceptor.reset();
+                }
+
+                StatisticsItemSampleBrief brief = getSampleBrief(item.getStatKind(), item.getStatObject());
+                if (brief != null && (!increment.allZeros() || printZeroLine())) {
+                    printer.print(name, increment, inteceptorStr, brief.toString());
+                }
+
+                setItemSnapshot(lastItemSnapshots, snapshot);
+
+                if (brief != null) {
+                    brief.reset();
+                }
+            }
+        }, getInitialDelay(), interval, TimeUnit.MILLISECONDS);
+        addFuture(item, future);
+
+        // sample every TPS_INTREVAL
+        ScheduledFuture futureSample = executor.scheduleAtFixedRate(new Runnable() {
+            @Override
+            public void run() {
+                if (!enabled()) {
+                    return;
+                }
+
+                StatisticsItem snapshot = item.snapshot();
+                StatisticsItemSampleBrief brief = getSampleBrief(item.getStatKind(), item.getStatObject());
+                if (brief != null) {
+                    brief.sample(snapshot);
+                }
+            }
+        }, TPS_INTREVAL, TPS_INTREVAL, TimeUnit.MILLISECONDS);
+        addFuture(item, futureSample);
+    }
+
+    @Override
+    public void remove(StatisticsItem item) {
+        // remove task
+        removeAllFuture(item);
+
+        String kind = item.getStatKind();
+        String key = item.getStatObject();
+
+        ConcurrentHashMap<String, StatisticsItem> lastItemMap = lastItemSnapshots.get(kind);
+        if (lastItemMap != null) {
+            lastItemMap.remove(key);
+        }
+
+        ConcurrentHashMap<String, StatisticsItemSampleBrief> briefMap = sampleBriefs.get(kind);
+        if (briefMap != null) {
+            briefMap.remove(key);
+        }
+    }
+
+    private StatisticsItem getItemSnapshot(
+        ConcurrentHashMap<String, ConcurrentHashMap<String, StatisticsItem>> snapshots,
+        String kind, String key) {
+        ConcurrentHashMap<String, StatisticsItem> itemMap = snapshots.get(kind);
+        return (itemMap != null) ? itemMap.get(key) : null;
+    }
+
+    private StatisticsItemSampleBrief getSampleBrief(String kind, String key) {
+        ConcurrentHashMap<String, StatisticsItemSampleBrief> itemMap = sampleBriefs.get(kind);
+        return (itemMap != null) ? itemMap.get(key) : null;
+    }
+
+    private void setItemSnapshot(ConcurrentHashMap<String, ConcurrentHashMap<String, StatisticsItem>> snapshots,
+                                 StatisticsItem item) {
+        String kind = item.getStatKind();
+        String key = item.getStatObject();
+        ConcurrentHashMap<String, StatisticsItem> itemMap = snapshots.get(kind);
+        if (itemMap == null) {
+            itemMap = new ConcurrentHashMap<String, StatisticsItem>();
+            ConcurrentHashMap<String, StatisticsItem> oldItemMap = snapshots.putIfAbsent(kind, itemMap);
+            if (oldItemMap != null) {
+                itemMap = oldItemMap;
+            }
+        }
+
+        itemMap.put(key, item);
+    }
+
+    private void setItemSampleBrief(String kind, String key,
+                                    StatisticsItemSampleBrief brief) {
+        ConcurrentHashMap<String, StatisticsItemSampleBrief> itemMap = sampleBriefs.get(kind);
+        if (itemMap == null) {
+            itemMap = new ConcurrentHashMap<String, StatisticsItemSampleBrief>();
+            ConcurrentHashMap<String, StatisticsItemSampleBrief> oldItemMap = sampleBriefs.putIfAbsent(kind, itemMap);
+            if (oldItemMap != null) {
+                itemMap = oldItemMap;
+            }
+        }
+
+        itemMap.put(key, brief);
+    }
+
+    private String formatInterceptor(Interceptor interceptor) {
+        if (interceptor == null) {
+            return "";
+        }
+
+        if (interceptor instanceof StatisticsBriefInterceptor) {
+            StringBuilder sb = new StringBuilder();
+            StatisticsBriefInterceptor briefInterceptor = (StatisticsBriefInterceptor)interceptor;
+            for (StatisticsBrief brief : briefInterceptor.getStatisticsBriefs()) {
+                long max = brief.getMax();
+                long tp999 = Math.min(brief.tp999(), max);
+                //sb.append(SEPERATOR).append(brief.getTotal());
+                sb.append(SEPERATOR).append(max);
+                //sb.append(SEPERATOR).append(brief.getMin());
+                sb.append(SEPERATOR).append(String.format("%.2f", brief.getAvg()));
+                sb.append(SEPERATOR).append(tp999);
+            }
+            return sb.toString();
+        }
+        return "";
+    }
+
+    public static class StatisticsItemSampleBrief {
+        private StatisticsItem lastSnapshot;
+
+        public String[] itemNames;
+        public ItemSampleBrief[] briefs;
+
+        public StatisticsItemSampleBrief(StatisticsItem statItem, String[] itemNames) {
+            this.lastSnapshot = statItem.snapshot();
+            this.itemNames = itemNames;
+            this.briefs = new ItemSampleBrief[itemNames.length];
+            for (int i = 0; i < itemNames.length; i++) {
+                this.briefs[i] = new ItemSampleBrief();
+            }
+        }
+
+        public synchronized void reset() {
+            for (ItemSampleBrief brief : briefs) {
+                brief.reset();
+            }
+        }
+
+        public synchronized void sample(StatisticsItem snapshot) {
+            if (snapshot == null) {
+                return;
+            }
+
+            for (int i = 0; i < itemNames.length; i++) {
+                String name = itemNames[i];
+
+                long lastValue = lastSnapshot != null ? lastSnapshot.getItemAccumulate(name).get() : 0;
+                long increment = snapshot.getItemAccumulate(name).get() - lastValue;
+                briefs[i].sample(increment);
+            }
+            lastSnapshot = snapshot;
+        }
+
+        @Override
+        public String toString() {
+            StringBuilder sb = new StringBuilder();
+            for (int i = 0; i < briefs.length; i++) {
+                ItemSampleBrief brief = briefs[i];
+                sb.append(SEPERATOR).append(brief.getMax());
+                //sb.append(SEPERATOR).append(brief.getMin());
+                sb.append(SEPERATOR).append(String.format("%.2f", brief.getAvg()));
+            }
+            return sb.toString();
+        }
+    }
+
+    /**
+     * sample brief of a item for a period of time
+     */
+    public static class ItemSampleBrief {
+        private long max;
+        private long min;
+        private long total;
+        private long cnt;
+
+        public ItemSampleBrief() {
+            reset();
+        }
+
+        public void sample(long value) {
+            max = Math.max(max, value);
+            min = Math.min(min, value);
+            total += value;
+            cnt++;
+        }
+
+        public void reset() {
+            max = 0;
+            min = Long.MAX_VALUE;
+            total = 0;
+            cnt = 0;
+        }
+
+        /**
+         * Getters
+         *
+         * @return
+         */
+        public long getMax() {
+            return max;
+        }
+
+        public long getMin() {
+            return cnt > 0 ? min : 0;
+        }
+
+        public long getTotal() {
+            return total;
+        }
+
+        public long getCnt() {
+            return cnt;
+        }
+
+        public double getAvg() {
+            return cnt != 0 ? ((double)total) / cnt : 0;
+        }
+    }
+}
diff --git a/common/src/main/java/org/apache/rocketmq/common/statistics/StatisticsItemScheduledPrinter.java b/common/src/main/java/org/apache/rocketmq/common/statistics/StatisticsItemScheduledPrinter.java
new file mode 100644
index 0000000..2f46ac5
--- /dev/null
+++ b/common/src/main/java/org/apache/rocketmq/common/statistics/StatisticsItemScheduledPrinter.java
@@ -0,0 +1,97 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.rocketmq.common.statistics;
+
+import java.util.concurrent.ScheduledExecutorService;
+import java.util.concurrent.ScheduledFuture;
+import java.util.concurrent.TimeUnit;
+
+public class StatisticsItemScheduledPrinter extends FutureHolder {
+    protected String name;
+
+    protected StatisticsItemPrinter printer;
+    protected ScheduledExecutorService executor;
+    protected long interval;
+    protected InitialDelay initialDelay;
+    protected Valve valve;
+
+    public StatisticsItemScheduledPrinter(String name, StatisticsItemPrinter printer,
+                                          ScheduledExecutorService executor, InitialDelay initialDelay,
+                                          long interval, Valve valve) {
+        this.name = name;
+        this.printer = printer;
+        this.executor = executor;
+        this.initialDelay = initialDelay;
+        this.interval = interval;
+        this.valve = valve;
+    }
+
+    /**
+     * schedule a StatisticsItem to print all the values periodically
+     */
+    public void schedule(final StatisticsItem statisticsItem) {
+        ScheduledFuture future = executor.scheduleAtFixedRate(new Runnable() {
+            @Override
+            public void run() {
+                if (enabled()) {
+                    printer.print(name, statisticsItem);
+                }
+            }
+        }, getInitialDelay(), interval, TimeUnit.MILLISECONDS);
+
+        addFuture(statisticsItem, future);
+    }
+
+    public void remove(final StatisticsItem statisticsItem) {
+        removeAllFuture(statisticsItem);
+    }
+
+    public interface InitialDelay {
+        /**
+         * Get inital delay value
+         * @return
+         */
+        long get();
+    }
+
+    public interface Valve {
+        /**
+         * whether enabled
+         * @return
+         */
+        boolean enabled();
+
+        /**
+         * whether print zero lines
+         * @return
+         */
+        boolean printZeroLine();
+    }
+
+    protected long getInitialDelay() {
+        return initialDelay != null ? initialDelay.get() : 0;
+    }
+
+    protected boolean enabled() {
+        return valve != null ? valve.enabled() : false;
+    }
+
+    protected boolean printZeroLine() {
+        return valve != null ? valve.printZeroLine() : false;
+    }
+
+}
diff --git a/common/src/main/java/org/apache/rocketmq/common/message/MessageType.java b/common/src/main/java/org/apache/rocketmq/common/statistics/StatisticsItemStateGetter.java
similarity index 84%
copy from common/src/main/java/org/apache/rocketmq/common/message/MessageType.java
copy to common/src/main/java/org/apache/rocketmq/common/statistics/StatisticsItemStateGetter.java
index 08091f4..3b16d00 100644
--- a/common/src/main/java/org/apache/rocketmq/common/message/MessageType.java
+++ b/common/src/main/java/org/apache/rocketmq/common/statistics/StatisticsItemStateGetter.java
@@ -14,12 +14,8 @@
  * See the License for the specific language governing permissions and
  * limitations under the License.
  */
+package org.apache.rocketmq.common.statistics;
 
-package org.apache.rocketmq.common.message;
-
-public enum MessageType {
-    Normal_Msg,
-    Trans_Msg_Half,
-    Trans_msg_Commit,
-    Delay_Msg,
+public interface StatisticsItemStateGetter {
+    boolean online(StatisticsItem item);
 }
diff --git a/common/src/main/java/org/apache/rocketmq/common/protocol/header/namesrv/GetRouteInfoRequestHeader.java b/common/src/main/java/org/apache/rocketmq/common/statistics/StatisticsKindMeta.java
similarity index 52%
copy from common/src/main/java/org/apache/rocketmq/common/protocol/header/namesrv/GetRouteInfoRequestHeader.java
copy to common/src/main/java/org/apache/rocketmq/common/statistics/StatisticsKindMeta.java
index a2806e6..27bee19 100644
--- a/common/src/main/java/org/apache/rocketmq/common/protocol/header/namesrv/GetRouteInfoRequestHeader.java
+++ b/common/src/main/java/org/apache/rocketmq/common/statistics/StatisticsKindMeta.java
@@ -14,29 +14,37 @@
  * See the License for the specific language governing permissions and
  * limitations under the License.
  */
+package org.apache.rocketmq.common.statistics;
 
 /**
- * $Id: GetRouteInfoRequestHeader.java 1835 2013-05-16 02:00:50Z vintagewang@apache.org $
+ * Statistics Kind Metadata
  */
-package org.apache.rocketmq.common.protocol.header.namesrv;
+public class StatisticsKindMeta {
+    private String name;
+    private String[] itemNames;
+    private StatisticsItemScheduledPrinter scheduledPrinter;
 
-import org.apache.rocketmq.remoting.CommandCustomHeader;
-import org.apache.rocketmq.remoting.annotation.CFNotNull;
-import org.apache.rocketmq.remoting.exception.RemotingCommandException;
+    public String getName() {
+        return name;
+    }
+
+    public void setName(String name) {
+        this.name = name;
+    }
 
-public class GetRouteInfoRequestHeader implements CommandCustomHeader {
-    @CFNotNull
-    private String topic;
+    public String[] getItemNames() {
+        return itemNames;
+    }
 
-    @Override
-    public void checkFields() throws RemotingCommandException {
+    public void setItemNames(String[] itemNames) {
+        this.itemNames = itemNames;
     }
 
-    public String getTopic() {
-        return topic;
+    public StatisticsItemScheduledPrinter getScheduledPrinter() {
+        return scheduledPrinter;
     }
 
-    public void setTopic(String topic) {
-        this.topic = topic;
+    public void setScheduledPrinter(StatisticsItemScheduledPrinter scheduledPrinter) {
+        this.scheduledPrinter = scheduledPrinter;
     }
-}
+}
\ No newline at end of file
diff --git a/common/src/main/java/org/apache/rocketmq/common/statistics/StatisticsManager.java b/common/src/main/java/org/apache/rocketmq/common/statistics/StatisticsManager.java
new file mode 100644
index 0000000..b517b63
--- /dev/null
+++ b/common/src/main/java/org/apache/rocketmq/common/statistics/StatisticsManager.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.common.statistics;
+
+import java.util.HashMap;
+import java.util.Iterator;
+import java.util.Map;
+import java.util.concurrent.ConcurrentHashMap;
+import java.util.concurrent.ScheduledExecutorService;
+import java.util.concurrent.TimeUnit;
+
+import org.apache.commons.lang3.tuple.Pair;
+import org.apache.rocketmq.common.utils.ThreadUtils;
+
+public class StatisticsManager {
+
+    /**
+     * Set of Statistics Kind Metadata
+     */
+    private Map<String, StatisticsKindMeta> kindMetaMap;
+
+    /**
+     * item names to calculate statistics brief
+     */
+    private Pair<String, long[][]>[] briefMetas;
+
+    /**
+     * Statistics
+     */
+    private final ConcurrentHashMap<String, ConcurrentHashMap<String, StatisticsItem>> statsTable
+        = new ConcurrentHashMap<String, ConcurrentHashMap<String, StatisticsItem>>();
+
+    private static final int MAX_IDLE_TIME = 10 * 60 * 1000;
+    private final ScheduledExecutorService executor = ThreadUtils.newSingleThreadScheduledExecutor(
+        "StatisticsManagerCleaner", true);
+
+    private StatisticsItemStateGetter statisticsItemStateGetter;
+
+    public StatisticsManager() {
+        kindMetaMap = new HashMap<String, StatisticsKindMeta>();
+        start();
+    }
+
+    public StatisticsManager(Map<String, StatisticsKindMeta> kindMeta) {
+        this.kindMetaMap = kindMeta;
+        start();
+    }
+
+    public void addStatisticsKindMeta(StatisticsKindMeta kindMeta) {
+        kindMetaMap.put(kindMeta.getName(), kindMeta);
+        statsTable.putIfAbsent(kindMeta.getName(), new ConcurrentHashMap<String, StatisticsItem>(16));
+    }
+
+    public void setBriefMeta(Pair<String, long[][]>[] briefMetas) {
+        this.briefMetas = briefMetas;
+    }
+
+    private void start() {
+        int maxIdleTime = MAX_IDLE_TIME;
+        executor.scheduleAtFixedRate(new Runnable() {
+            @Override
+            public void run() {
+                Iterator<Map.Entry<String, ConcurrentHashMap<String, StatisticsItem>>> iter
+                    = statsTable.entrySet().iterator();
+                while (iter.hasNext()) {
+                    Map.Entry<String, ConcurrentHashMap<String, StatisticsItem>> entry = iter.next();
+                    String kind = entry.getKey();
+                    ConcurrentHashMap<String, StatisticsItem> itemMap = entry.getValue();
+
+                    if (itemMap == null || itemMap.isEmpty()) {
+                        continue;
+                    }
+
+                    HashMap<String, StatisticsItem> tmpItemMap = new HashMap<String, StatisticsItem>(itemMap);
+                    for (StatisticsItem item : tmpItemMap.values()) {
+                        // remove when expired
+                        if (System.currentTimeMillis() - item.getLastTimeStamp().get() > MAX_IDLE_TIME
+                            && (statisticsItemStateGetter == null || !statisticsItemStateGetter.online(item))) {
+                            remove(item);
+                        }
+                    }
+                }
+            }
+        }, maxIdleTime, maxIdleTime / 3, TimeUnit.MILLISECONDS);
+    }
+
+    /**
+     * Increment a StatisticsItem
+     *
+     * @param kind
+     * @param key
+     * @param itemAccumulates
+     */
+    public boolean inc(String kind, String key, long... itemAccumulates) {
+        ConcurrentHashMap<String, StatisticsItem> itemMap = statsTable.get(kind);
+        if (itemMap != null) {
+            StatisticsItem item = itemMap.get(key);
+
+            // if not exist, create and schedule
+            if (item == null) {
+                item = new StatisticsItem(kind, key, kindMetaMap.get(kind).getItemNames());
+                item.setInterceptor(new StatisticsBriefInterceptor(item, briefMetas));
+                StatisticsItem oldItem = itemMap.putIfAbsent(key, item);
+                if (oldItem != null) {
+                    item = oldItem;
+                } else {
+                    scheduleStatisticsItem(item);
+                }
+            }
+
+            // do increment
+            item.incItems(itemAccumulates);
+
+            return true;
+        }
+
+        return false;
+    }
+
+    private void scheduleStatisticsItem(StatisticsItem item) {
+        kindMetaMap.get(item.getStatKind()).getScheduledPrinter().schedule(item);
+    }
+
+    public void remove(StatisticsItem item) {
+        ConcurrentHashMap<String, StatisticsItem> itemMap = statsTable.get(item.getStatKind());
+        if (itemMap != null) {
+            itemMap.remove(item.getStatObject(), item);
+        }
+
+        StatisticsKindMeta kindMeta = kindMetaMap.get(item.getStatKind());
+        if (kindMeta != null && kindMeta.getScheduledPrinter() != null) {
+            kindMeta.getScheduledPrinter().remove(item);
+        }
+    }
+
+    public StatisticsItemStateGetter getStatisticsItemStateGetter() {
+        return statisticsItemStateGetter;
+    }
+
+    public void setStatisticsItemStateGetter(StatisticsItemStateGetter statisticsItemStateGetter) {
+        this.statisticsItemStateGetter = statisticsItemStateGetter;
+    }
+}
diff --git a/common/src/main/java/org/apache/rocketmq/common/subscription/GroupForbidden.java b/common/src/main/java/org/apache/rocketmq/common/subscription/GroupForbidden.java
new file mode 100644
index 0000000..73d50c6
--- /dev/null
+++ b/common/src/main/java/org/apache/rocketmq/common/subscription/GroupForbidden.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.common.subscription;
+
+import org.apache.commons.lang3.builder.EqualsBuilder;
+import org.apache.rocketmq.remoting.protocol.RemotingSerializable;
+
+/**
+ *
+ */
+public class GroupForbidden extends RemotingSerializable {
+
+    private String  topic;
+    private String  group;
+    private Boolean readable;
+
+    public String getTopic() {
+        return topic;
+    }
+
+    public void setTopic(String topic) {
+        this.topic = topic;
+    }
+
+    public String getGroup() {
+        return group;
+    }
+
+    public void setGroup(String group) {
+        this.group = group;
+    }
+
+    public Boolean getReadable() {
+        return readable;
+    }
+
+    public void setReadable(Boolean readable) {
+        this.readable = readable;
+    }
+
+    @Override
+    public int hashCode() {
+        final int prime = 31;
+        int result = 1;
+        result = prime * result + ((group == null) ? 0 : group.hashCode());
+        result = prime * result + ((readable == null) ? 0 : readable.hashCode());
+        result = prime * result + ((topic == null) ? 0 : topic.hashCode());
+        return result;
+    }
+
+    @Override
+    public boolean equals(Object obj) {
+        if (this == obj)
+            return true;
+        if (obj == null)
+            return false;
+        if (getClass() != obj.getClass())
+            return false;
+        GroupForbidden other = (GroupForbidden) obj;
+        return new EqualsBuilder()
+                .append(topic, other.topic)
+                .append(group, other.group)
+                .append(readable, other.readable)
+                .isEquals();
+    }
+
+    @Override
+    public String toString() {
+        return "GroupForbidden [topic=" + topic + ", group=" + group + ", readable=" + readable + "]";
+    }
+
+}
diff --git a/common/src/main/java/org/apache/rocketmq/common/subscription/SubscriptionGroupConfig.java b/common/src/main/java/org/apache/rocketmq/common/subscription/SubscriptionGroupConfig.java
index 8f4703f..6981c23 100644
--- a/common/src/main/java/org/apache/rocketmq/common/subscription/SubscriptionGroupConfig.java
+++ b/common/src/main/java/org/apache/rocketmq/common/subscription/SubscriptionGroupConfig.java
@@ -17,6 +17,7 @@
 
 package org.apache.rocketmq.common.subscription;
 
+import org.apache.commons.lang3.builder.EqualsBuilder;
 import org.apache.rocketmq.common.MixAll;
 
 public class SubscriptionGroupConfig {
@@ -38,6 +39,8 @@ public class SubscriptionGroupConfig {
 
     private boolean notifyConsumerIdsChangedEnable = true;
 
+    private int groupSysFlag = 0;
+
     public String getGroupName() {
         return groupName;
     }
@@ -110,6 +113,14 @@ public class SubscriptionGroupConfig {
         this.notifyConsumerIdsChangedEnable = notifyConsumerIdsChangedEnable;
     }
 
+    public int getGroupSysFlag() {
+        return groupSysFlag;
+    }
+
+    public void setGroupSysFlag(int groupSysFlag) {
+        this.groupSysFlag = groupSysFlag;
+    }
+
     @Override
     public int hashCode() {
         final int prime = 31;
@@ -136,28 +147,18 @@ public class SubscriptionGroupConfig {
         if (getClass() != obj.getClass())
             return false;
         SubscriptionGroupConfig other = (SubscriptionGroupConfig) obj;
-        if (brokerId != other.brokerId)
-            return false;
-        if (consumeBroadcastEnable != other.consumeBroadcastEnable)
-            return false;
-        if (consumeEnable != other.consumeEnable)
-            return false;
-        if (consumeFromMinEnable != other.consumeFromMinEnable)
-            return false;
-        if (groupName == null) {
-            if (other.groupName != null)
-                return false;
-        } else if (!groupName.equals(other.groupName))
-            return false;
-        if (retryMaxTimes != other.retryMaxTimes)
-            return false;
-        if (retryQueueNums != other.retryQueueNums)
-            return false;
-        if (whichBrokerWhenConsumeSlowly != other.whichBrokerWhenConsumeSlowly)
-            return false;
-        if (notifyConsumerIdsChangedEnable != other.notifyConsumerIdsChangedEnable)
-            return false;
-        return true;
+        return new EqualsBuilder()
+            .append(groupName, other.groupName)
+            .append(consumeEnable, other.consumeEnable)
+            .append(consumeFromMinEnable, other.consumeFromMinEnable)
+            .append(consumeBroadcastEnable, other.consumeBroadcastEnable)
+            .append(retryQueueNums, other.retryQueueNums)
+            .append(retryMaxTimes, other.retryMaxTimes)
+            .append(brokerId, other.brokerId)
+            .append(whichBrokerWhenConsumeSlowly, other.whichBrokerWhenConsumeSlowly)
+            .append(notifyConsumerIdsChangedEnable, other.notifyConsumerIdsChangedEnable)
+            .append(groupSysFlag, other.groupSysFlag)
+            .isEquals();
     }
 
     @Override
@@ -167,6 +168,6 @@ public class SubscriptionGroupConfig {
             + consumeBroadcastEnable + ", retryQueueNums=" + retryQueueNums + ", retryMaxTimes="
             + retryMaxTimes + ", brokerId=" + brokerId + ", whichBrokerWhenConsumeSlowly="
             + whichBrokerWhenConsumeSlowly + ", notifyConsumerIdsChangedEnable="
-            + notifyConsumerIdsChangedEnable + "]";
+            + notifyConsumerIdsChangedEnable + ", groupSysFlag=" + groupSysFlag + "]";
     }
 }
diff --git a/common/src/main/java/org/apache/rocketmq/common/topic/TopicValidator.java b/common/src/main/java/org/apache/rocketmq/common/topic/TopicValidator.java
index d4a9d80..540f53a 100644
--- a/common/src/main/java/org/apache/rocketmq/common/topic/TopicValidator.java
+++ b/common/src/main/java/org/apache/rocketmq/common/topic/TopicValidator.java
@@ -36,6 +36,8 @@ public class TopicValidator {
     public static final String RMQ_SYS_OFFSET_MOVED_EVENT = "OFFSET_MOVED_EVENT";
 
     public static final String SYSTEM_TOPIC_PREFIX = "rmq_sys_";
+    public static final String SYNC_BROKER_MEMBER_GROUP_PREFIX = SYSTEM_TOPIC_PREFIX + "SYNC_BROKER_MEMBER_";
+
     public static final boolean[] VALID_CHAR_BIT_MAP = new boolean[128];
     private static final int TOPIC_MAX_LENGTH = 127;
 
diff --git a/common/src/main/java/org/apache/rocketmq/common/utils/MessageUtils.java b/common/src/main/java/org/apache/rocketmq/common/utils/MessageUtils.java
new file mode 100644
index 0000000..a2affc7
--- /dev/null
+++ b/common/src/main/java/org/apache/rocketmq/common/utils/MessageUtils.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.common.utils;
+
+import java.util.Collection;
+import java.util.HashSet;
+import java.util.Set;
+
+import com.google.common.hash.Hashing;
+import org.apache.rocketmq.common.message.MessageConst;
+import org.apache.rocketmq.common.message.MessageExt;
+
+public class MessageUtils {
+
+    public static int getShardingKeyIndex(String shardingKey, int indexSize) {
+        return Math.abs(Hashing.murmur3_32().hashBytes(shardingKey.getBytes()).asInt() % indexSize);
+    }
+
+    public static int getShardingKeyIndexByMsg(MessageExt msg, int indexSize) {
+        String shardingKey = msg.getProperty(MessageConst.PROPERTY_SHARDING_KEY);
+        if (shardingKey == null) {
+            shardingKey = "";
+        }
+
+        return getShardingKeyIndex(shardingKey, indexSize);
+    }
+
+    public static Set<Integer> getShardingKeyIndexes(Collection<MessageExt> msgs, int indexSize) {
+        Set<Integer> indexSet = new HashSet<Integer>(indexSize);
+        for (MessageExt msg : msgs) {
+            indexSet.add(getShardingKeyIndexByMsg(msg, indexSize));
+        }
+        return indexSet;
+    }
+}
diff --git a/common/src/main/java/org/apache/rocketmq/common/message/MessageType.java b/common/src/main/java/org/apache/rocketmq/common/utils/PositiveAtomicCounter.java
similarity index 62%
copy from common/src/main/java/org/apache/rocketmq/common/message/MessageType.java
copy to common/src/main/java/org/apache/rocketmq/common/utils/PositiveAtomicCounter.java
index 08091f4..105b88c 100644
--- a/common/src/main/java/org/apache/rocketmq/common/message/MessageType.java
+++ b/common/src/main/java/org/apache/rocketmq/common/utils/PositiveAtomicCounter.java
@@ -14,12 +14,27 @@
  * See the License for the specific language governing permissions and
  * limitations under the License.
  */
+package org.apache.rocketmq.common.utils;
 
-package org.apache.rocketmq.common.message;
+import java.util.concurrent.atomic.AtomicInteger;
 
-public enum MessageType {
-    Normal_Msg,
-    Trans_Msg_Half,
-    Trans_msg_Commit,
-    Delay_Msg,
+public class PositiveAtomicCounter {
+    private static final int MASK = 0x7FFFFFFF;
+    private final AtomicInteger atom;
+
+
+    public PositiveAtomicCounter() {
+        atom = new AtomicInteger(0);
+    }
+
+
+    public final int incrementAndGet() {
+        final int rt = atom.incrementAndGet();
+        return rt & MASK;
+    }
+
+
+    public int intValue() {
+        return atom.intValue();
+    }
 }
diff --git a/common/src/main/java/org/apache/rocketmq/common/utils/ServiceProvider.java b/common/src/main/java/org/apache/rocketmq/common/utils/ServiceProvider.java
new file mode 100644
index 0000000..72857bc
--- /dev/null
+++ b/common/src/main/java/org/apache/rocketmq/common/utils/ServiceProvider.java
@@ -0,0 +1,211 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.rocketmq.common.utils;
+
+import org.apache.rocketmq.common.constant.LoggerName;
+import org.apache.rocketmq.logging.InternalLogger;
+import org.apache.rocketmq.logging.InternalLoggerFactory;
+
+import java.io.BufferedReader;
+import java.io.InputStream;
+import java.io.InputStreamReader;
+import java.util.ArrayList;
+import java.util.List;
+
+public class ServiceProvider {
+
+    private static final InternalLogger LOG = InternalLoggerFactory.getLogger(LoggerName.COMMON_LOGGER_NAME);
+    /**
+     * A reference to the classloader that loaded this class. It's more efficient to compute it once and cache it here.
+     */
+    private static ClassLoader thisClassLoader;
+
+    /**
+     * JDK1.3+ <a href= "http://java.sun.com/j2se/1.3/docs/guide/jar/jar.html#Service%20Provider" > 'Service Provider' specification</a>.
+     */
+    public static final String TRANSACTION_SERVICE_ID = "META-INF/service/org.apache.rocketmq.broker.transaction.TransactionalMessageService";
+
+    public static final String TRANSACTION_LISTENER_ID = "META-INF/service/org.apache.rocketmq.broker.transaction.AbstractTransactionalMessageCheckListener";
+
+    public static final String HA_SERVICE_ID = "META-INF/service/org.apache.rocketmq.store.ha.HAService";
+
+
+    public static final String RPC_HOOK_ID = "META-INF/service/org.apache.rocketmq.remoting.RPCHook";
+
+
+    public static final String ACL_VALIDATOR_ID = "META-INF/service/org.apache.rocketmq.acl.AccessValidator";
+
+
+
+    static {
+        thisClassLoader = getClassLoader(ServiceProvider.class);
+    }
+
+    /**
+     * Returns a string that uniquely identifies the specified object, including its class.
+     * <p>
+     * The returned string is of form "classname@hashcode", ie is the same as the return value of the Object.toString() method, but works even when the specified object's class has overidden the toString method.
+     *
+     * @param o may be null.
+     * @return a string of form classname@hashcode, or "null" if param o is null.
+     */
+    protected static String objectId(Object o) {
+        if (o == null) {
+            return "null";
+        } else {
+            return o.getClass().getName() + "@" + System.identityHashCode(o);
+        }
+    }
+
+    protected static ClassLoader getClassLoader(Class<?> clazz) {
+        try {
+            return clazz.getClassLoader();
+        } catch (SecurityException e) {
+            LOG.error("Unable to get classloader for class {} due to security restrictions !",
+                clazz, e.getMessage());
+            throw e;
+        }
+    }
+
+    protected static ClassLoader getContextClassLoader() {
+        ClassLoader classLoader = null;
+        try {
+            classLoader = Thread.currentThread().getContextClassLoader();
+        } catch (SecurityException ex) {
+            /**
+             * The getContextClassLoader() method throws SecurityException when the context
+             * class loader isn't an ancestor of the calling class's class
+             * loader, or if security permissions are restricted.
+             */
+        }
+        return classLoader;
+    }
+
+    protected static InputStream getResourceAsStream(ClassLoader loader, String name) {
+        if (loader != null) {
+            return loader.getResourceAsStream(name);
+        } else {
+            return ClassLoader.getSystemResourceAsStream(name);
+        }
+    }
+
+    public static <T> List<T> load(String name, Class<?> clazz) {
+        LOG.info("Looking for a resource file of name [{}] ...", name);
+        List<T> services = new ArrayList<T>();
+        try {
+            ArrayList<String> names = new ArrayList<String>();
+            final InputStream is = getResourceAsStream(getContextClassLoader(), name);
+            if (is != null) {
+                BufferedReader reader;
+                try {
+                    reader = new BufferedReader(new InputStreamReader(is, "UTF-8"));
+                } catch (java.io.UnsupportedEncodingException e) {
+                    reader = new BufferedReader(new InputStreamReader(is));
+                }
+                String serviceName = reader.readLine();
+                while (serviceName != null && !"".equals(serviceName)) {
+                    LOG.info(
+                        "Creating an instance as specified by file {} which was present in the path of the context classloader.",
+                        name);
+                    if (!names.contains(serviceName)) {
+                        names.add(serviceName);
+                    }
+
+                    services.add((T)initService(getContextClassLoader(), serviceName, clazz));
+
+                    serviceName = reader.readLine();
+                }
+                reader.close();
+            } else {
+                // is == null
+                LOG.warn("No resource file with name [{}] found.", name);
+            }
+        } catch (Exception e) {
+            LOG.error("Error occured when looking for resource file " + name, e);
+        }
+        return services;
+    }
+
+    public static <T> T loadClass(String name, Class<?> clazz) {
+        final InputStream is = getResourceAsStream(getContextClassLoader(), name);
+        if (is != null) {
+            BufferedReader reader;
+            try {
+                try {
+                    reader = new BufferedReader(new InputStreamReader(is, "UTF-8"));
+                } catch (java.io.UnsupportedEncodingException e) {
+                    reader = new BufferedReader(new InputStreamReader(is));
+                }
+                String serviceName = reader.readLine();
+                reader.close();
+                if (serviceName != null && !"".equals(serviceName)) {
+                    return initService(getContextClassLoader(), serviceName, clazz);
+                } else {
+                    LOG.warn("ServiceName is empty!");
+                    return null;
+                }
+            } catch (Exception e) {
+                LOG.warn("Error occurred when looking for resource file " + name, e);
+            }
+        }
+        return null;
+    }
+
+    protected static <T> T initService(ClassLoader classLoader, String serviceName, Class<?> clazz) {
+        Class<?> serviceClazz = null;
+        try {
+            if (classLoader != null) {
+                try {
+                    // Warning: must typecast here & allow exception to be generated/caught & recast properly
+                    serviceClazz = classLoader.loadClass(serviceName);
+                    if (clazz.isAssignableFrom(serviceClazz)) {
+                        LOG.info("Loaded class {} from classloader {}", serviceClazz.getName(),
+                            objectId(classLoader));
+                    } else {
+                        // This indicates a problem with the ClassLoader tree. An incompatible ClassLoader was used to load the implementation.
+                        LOG.error(
+                            "Class {} loaded from classloader {} does not extend {} as loaded by this classloader.",
+                            new Object[] {serviceClazz.getName(),
+                                objectId(serviceClazz.getClassLoader()), clazz.getName()});
+                    }
+                    return (T)serviceClazz.newInstance();
+                } catch (ClassNotFoundException ex) {
+                    if (classLoader == thisClassLoader) {
+                        // Nothing more to try, onwards.
+                        LOG.warn("Unable to locate any class {} via classloader", serviceName,
+                            objectId(classLoader));
+                        throw ex;
+                    }
+                    // Ignore exception, continue
+                } catch (NoClassDefFoundError e) {
+                    if (classLoader == thisClassLoader) {
+                        // Nothing more to try, onwards.
+                        LOG.warn(
+                            "Class {} cannot be loaded via classloader {}.it depends on some other class that cannot be found.",
+                            serviceClazz, objectId(classLoader));
+                        throw e;
+                    }
+                    // Ignore exception, continue
+                }
+            }
+        } catch (Exception e) {
+            LOG.error("Unable to init service.", e);
+        }
+        return (T)serviceClazz;
+    }
+}
diff --git a/common/src/main/java/org/apache/rocketmq/common/utils/ThreadUtils.java b/common/src/main/java/org/apache/rocketmq/common/utils/ThreadUtils.java
index 13c0293..61ca4a8 100644
--- a/common/src/main/java/org/apache/rocketmq/common/utils/ThreadUtils.java
+++ b/common/src/main/java/org/apache/rocketmq/common/utils/ThreadUtils.java
@@ -30,7 +30,7 @@ import org.apache.rocketmq.logging.InternalLogger;
 import org.apache.rocketmq.logging.InternalLoggerFactory;
 
 public final class ThreadUtils {
-    private static final InternalLogger log = InternalLoggerFactory.getLogger(LoggerName.TOOLS_LOGGER_NAME);
+    private static final InternalLogger LOGGER = InternalLoggerFactory.getLogger(LoggerName.TOOLS_LOGGER_NAME);
 
     public static ExecutorService newThreadPoolExecutor(int corePoolSize, int maximumPoolSize, long keepAliveTime,
         TimeUnit unit, BlockingQueue<Runnable> workQueue, String processName, boolean isDaemon) {
@@ -102,7 +102,7 @@ public final class ThreadUtils {
         thread.setDaemon(daemon);
         thread.setUncaughtExceptionHandler(new Thread.UncaughtExceptionHandler() {
             public void uncaughtException(Thread t, Throwable e) {
-                log.error("Uncaught exception in thread '" + t.getName() + "':", e);
+                LOGGER.error("Uncaught exception in thread '" + t.getName() + "':", e);
             }
         });
         return thread;
@@ -153,7 +153,7 @@ public final class ThreadUtils {
                 executor.shutdownNow();
                 // Wait a while for tasks to respond to being cancelled.
                 if (!executor.awaitTermination(timeout, timeUnit)) {
-                    log.warn(String.format("%s didn't terminate!", executor));
+                    LOGGER.warn(String.format("%s didn't terminate!", executor));
                 }
             }
         } catch (InterruptedException ie) {
@@ -165,6 +165,17 @@ public final class ThreadUtils {
     }
 
     /**
+     * Shutdown the specific ExecutorService
+     *
+     * @param executorService the executor
+     */
+    public static void shutdown(ExecutorService executorService) {
+        if (executorService != null) {
+            executorService.shutdown();
+        }
+    }
+
+    /**
      * A constructor to stop this class being constructed.
      */
     private ThreadUtils() {
diff --git a/common/src/test/java/org/apache/rocketmq/common/DataVersionTest.java b/common/src/test/java/org/apache/rocketmq/common/DataVersionTest.java
index f4d14e5..d6cc4b0 100644
--- a/common/src/test/java/org/apache/rocketmq/common/DataVersionTest.java
+++ b/common/src/test/java/org/apache/rocketmq/common/DataVersionTest.java
@@ -67,4 +67,11 @@ public class DataVersionTest {
         other.setTimestamp(dataVersion.getTimestamp());
         Assert.assertTrue(dataVersion.equals(other));
     }
+
+    @Test
+    public void testEncode() {
+        DataVersion dataVersion = new DataVersion();
+        Assert.assertTrue(dataVersion.encode().length > 0);
+        Assert.assertNotNull(dataVersion.toJson());
+    }
 }
\ No newline at end of file
diff --git a/common/src/test/java/org/apache/rocketmq/common/RegisterBrokerBodyTest.java b/common/src/test/java/org/apache/rocketmq/common/RegisterBrokerBodyTest.java
index 428a928..73703c0 100644
--- a/common/src/test/java/org/apache/rocketmq/common/RegisterBrokerBodyTest.java
+++ b/common/src/test/java/org/apache/rocketmq/common/RegisterBrokerBodyTest.java
@@ -22,7 +22,7 @@ import java.util.concurrent.ConcurrentHashMap;
 import java.util.concurrent.ConcurrentMap;
 import org.apache.rocketmq.common.protocol.body.RegisterBrokerBody;
 import org.apache.rocketmq.common.protocol.body.TopicConfigAndMappingSerializeWrapper;
-import org.apache.rocketmq.common.protocol.body.TopicConfigSerializeWrapper;
+
 import static org.junit.Assert.assertEquals;
 import org.junit.Test;
 
diff --git a/common/src/test/java/org/apache/rocketmq/common/UtilAllTest.java b/common/src/test/java/org/apache/rocketmq/common/UtilAllTest.java
index 7ef7eac..d1ed4bb 100644
--- a/common/src/test/java/org/apache/rocketmq/common/UtilAllTest.java
+++ b/common/src/test/java/org/apache/rocketmq/common/UtilAllTest.java
@@ -19,13 +19,11 @@ package org.apache.rocketmq.common;
 
 import java.net.InetAddress;
 import java.net.UnknownHostException;
+import java.nio.ByteBuffer;
 import java.util.Arrays;
 import java.util.Collections;
 import java.util.List;
 import java.util.Properties;
-
-import com.alibaba.fastjson.JSON;
-import com.google.common.collect.ImmutableList;
 import org.junit.Test;
 
 import static org.assertj.core.api.Assertions.assertThat;
@@ -174,4 +172,21 @@ public class UtilAllTest {
                 '}';
         }
     }
+
+    @Test
+    public void testCleanBuffer() {
+        UtilAll.cleanBuffer(null);
+        UtilAll.cleanBuffer(ByteBuffer.allocate(10));
+        UtilAll.cleanBuffer(ByteBuffer.allocate(0));
+    }
+
+    @Test(expected = NoSuchMethodException.class)
+    public void testMethod() throws NoSuchMethodException {
+        UtilAll.method(new Object(), "noMethod", null);
+    }
+
+    @Test(expected = IllegalStateException.class)
+    public void testInvoke() throws Exception {
+        UtilAll.invoke(new Object(), "noMethod");
+    }
 }
diff --git a/common/src/test/java/org/apache/rocketmq/common/message/MessageDecoderTest.java b/common/src/test/java/org/apache/rocketmq/common/message/MessageDecoderTest.java
index 5af7345..c57c758 100644
--- a/common/src/test/java/org/apache/rocketmq/common/message/MessageDecoderTest.java
+++ b/common/src/test/java/org/apache/rocketmq/common/message/MessageDecoderTest.java
@@ -17,6 +17,7 @@
 
 package org.apache.rocketmq.common.message;
 
+import org.apache.rocketmq.common.UtilAll;
 import org.apache.rocketmq.common.sysflag.MessageSysFlag;
 import org.junit.Test;
 
@@ -163,6 +164,8 @@ public class MessageDecoderTest {
         messageExt.putUserProperty("b", "hello");
         messageExt.putUserProperty("c", "3.14");
 
+        messageExt.setBodyCRC(UtilAll.crc32(messageExt.getBody()));
+
         byte[] msgBytes = new byte[0];
         try {
             msgBytes = MessageDecoder.encode(messageExt, false);
@@ -174,7 +177,7 @@ public class MessageDecoderTest {
         ByteBuffer byteBuffer = ByteBuffer.allocate(msgBytes.length);
         byteBuffer.put(msgBytes);
 
-        byteBuffer.clear();
+        byteBuffer.flip();
         MessageExt decodedMsg = MessageDecoder.decode(byteBuffer);
 
         assertThat(decodedMsg).isNotNull();
@@ -222,6 +225,8 @@ public class MessageDecoderTest {
         messageExt.putUserProperty("b", "hello");
         messageExt.putUserProperty("c", "3.14");
 
+        messageExt.setBodyCRC(UtilAll.crc32(messageExt.getBody()));
+
         byte[] msgBytes = new byte[0];
         try {
             msgBytes = MessageDecoder.encode(messageExt, false);
@@ -233,7 +238,7 @@ public class MessageDecoderTest {
         ByteBuffer byteBuffer = ByteBuffer.allocate(msgBytes.length);
         byteBuffer.put(msgBytes);
 
-        byteBuffer.clear();
+        byteBuffer.flip();
         MessageExt decodedMsg = MessageDecoder.decode(byteBuffer);
 
         assertThat(decodedMsg).isNotNull();
diff --git a/common/src/test/java/org/apache/rocketmq/common/message/MessageTest.java b/common/src/test/java/org/apache/rocketmq/common/message/MessageTest.java
index c867360..c950970 100644
--- a/common/src/test/java/org/apache/rocketmq/common/message/MessageTest.java
+++ b/common/src/test/java/org/apache/rocketmq/common/message/MessageTest.java
@@ -20,7 +20,6 @@ import org.junit.Assert;
 import org.junit.Test;
 
 import static org.apache.rocketmq.common.message.MessageConst.PROPERTY_TRACE_SWITCH;
-import static org.junit.Assert.*;
 
 public class MessageTest {
     @Test(expected = RuntimeException.class)
diff --git a/common/src/test/java/org/apache/rocketmq/common/protocol/body/KVTableTest.java b/common/src/test/java/org/apache/rocketmq/common/protocol/body/KVTableTest.java
index 836733c..6a39cee 100644
--- a/common/src/test/java/org/apache/rocketmq/common/protocol/body/KVTableTest.java
+++ b/common/src/test/java/org/apache/rocketmq/common/protocol/body/KVTableTest.java
@@ -18,7 +18,6 @@
 package org.apache.rocketmq.common.protocol.body;
 
 import org.apache.rocketmq.remoting.protocol.RemotingSerializable;
-import org.junit.Assert;
 import org.junit.Test;
 
 import java.util.HashMap;
diff --git a/common/src/test/java/org/apache/rocketmq/common/statictopic/TopicQueueMappingTest.java b/common/src/test/java/org/apache/rocketmq/common/statictopic/TopicQueueMappingTest.java
index 4e93275..6296dd5 100644
--- a/common/src/test/java/org/apache/rocketmq/common/statictopic/TopicQueueMappingTest.java
+++ b/common/src/test/java/org/apache/rocketmq/common/statictopic/TopicQueueMappingTest.java
@@ -21,11 +21,9 @@ import com.alibaba.fastjson.JSON;
 import com.alibaba.fastjson.JSONArray;
 import com.alibaba.fastjson.JSONObject;
 import com.google.common.collect.ImmutableList;
-import org.apache.rocketmq.common.protocol.header.GetMinOffsetRequestHeader;
 import org.apache.rocketmq.remoting.protocol.RemotingSerializable;
 import org.junit.Assert;
 import org.junit.Test;
-import org.junit.experimental.theories.suppliers.TestedOn;
 
 import java.util.Map;
 
diff --git a/namesrv/pom.xml b/namesrv/pom.xml
index 11ddfb7..0b04d50 100644
--- a/namesrv/pom.xml
+++ b/namesrv/pom.xml
@@ -45,12 +45,23 @@
             <artifactId>logback-classic</artifactId>
         </dependency>
         <dependency>
+            <groupId>ch.qos.logback</groupId>
+            <artifactId>logback-core</artifactId>
+        </dependency>
+        <dependency>
             <groupId>org.slf4j</groupId>
             <artifactId>slf4j-api</artifactId>
         </dependency>
         <dependency>
-            <groupId>com.google.guava</groupId>
-            <artifactId>guava</artifactId>
+            <groupId>org.openjdk.jmh</groupId>
+            <artifactId>jmh-core</artifactId>
+            <version>1.19</version>
+            <scope>test</scope>
+        </dependency>
+        <dependency>
+            <groupId>org.openjdk.jmh</groupId>
+            <artifactId>jmh-generator-annprocess</artifactId>
+            <version>1.19</version>
             <scope>test</scope>
         </dependency>
         <dependency>
diff --git a/namesrv/src/main/java/org/apache/rocketmq/namesrv/NamesrvController.java b/namesrv/src/main/java/org/apache/rocketmq/namesrv/NamesrvController.java
index a6654f2..d02e6c0 100644
--- a/namesrv/src/main/java/org/apache/rocketmq/namesrv/NamesrvController.java
+++ b/namesrv/src/main/java/org/apache/rocketmq/namesrv/NamesrvController.java
@@ -16,58 +16,92 @@
  */
 package org.apache.rocketmq.namesrv;
 
+import java.util.Arrays;
+import java.util.concurrent.BlockingQueue;
 import java.util.concurrent.ExecutorService;
-import java.util.concurrent.Executors;
+import java.util.concurrent.LinkedBlockingQueue;
+import java.util.concurrent.RunnableFuture;
 import java.util.concurrent.ScheduledExecutorService;
+import java.util.concurrent.ScheduledThreadPoolExecutor;
+import java.util.concurrent.ThreadPoolExecutor;
 import java.util.concurrent.TimeUnit;
+import org.apache.commons.lang3.concurrent.BasicThreadFactory;
 import org.apache.rocketmq.common.Configuration;
 import org.apache.rocketmq.common.ThreadFactoryImpl;
 import org.apache.rocketmq.common.constant.LoggerName;
+import org.apache.rocketmq.common.future.FutureTaskExt;
+import org.apache.rocketmq.common.protocol.RequestCode;
 import org.apache.rocketmq.logging.InternalLogger;
 import org.apache.rocketmq.logging.InternalLoggerFactory;
 import org.apache.rocketmq.common.namesrv.NamesrvConfig;
 import org.apache.rocketmq.namesrv.kvconfig.KVConfigManager;
+import org.apache.rocketmq.namesrv.processor.ClientRequestProcessor;
 import org.apache.rocketmq.namesrv.processor.ClusterTestRequestProcessor;
 import org.apache.rocketmq.namesrv.processor.DefaultRequestProcessor;
 import org.apache.rocketmq.namesrv.routeinfo.BrokerHousekeepingService;
 import org.apache.rocketmq.namesrv.routeinfo.RouteInfoManager;
+import org.apache.rocketmq.remoting.RemotingClient;
 import org.apache.rocketmq.remoting.RemotingServer;
+import org.apache.rocketmq.remoting.common.RemotingUtil;
 import org.apache.rocketmq.remoting.common.TlsMode;
+import org.apache.rocketmq.remoting.netty.NettyClientConfig;
+import org.apache.rocketmq.remoting.netty.NettyRemotingClient;
 import org.apache.rocketmq.remoting.netty.NettyRemotingServer;
 import org.apache.rocketmq.remoting.netty.NettyServerConfig;
+import org.apache.rocketmq.remoting.netty.RequestTask;
 import org.apache.rocketmq.remoting.netty.TlsSystemConfig;
 import org.apache.rocketmq.srvutil.FileWatchService;
 
 
 public class NamesrvController {
-    private static final InternalLogger log = InternalLoggerFactory.getLogger(LoggerName.NAMESRV_LOGGER_NAME);
+    private static final InternalLogger LOGGER = InternalLoggerFactory.getLogger(LoggerName.NAMESRV_LOGGER_NAME);
+    private static final InternalLogger WATER_MARK_LOG = InternalLoggerFactory.getLogger(LoggerName.NAMESRV_WATER_MARK_LOGGER_NAME);
 
     private final NamesrvConfig namesrvConfig;
 
     private final NettyServerConfig nettyServerConfig;
-
-    private final ScheduledExecutorService scheduledExecutorService = Executors.newSingleThreadScheduledExecutor(new ThreadFactoryImpl(
-        "NSScheduledThread"));
+    private final NettyClientConfig nettyClientConfig;
+
+    private final ScheduledExecutorService scheduledExecutorService = new ScheduledThreadPoolExecutor(1,
+        new BasicThreadFactory.Builder()
+            .namingPattern("NSScheduledThread")
+            .daemon(true)
+            .build());
+    private final ScheduledExecutorService scanExecutorService = new ScheduledThreadPoolExecutor(1,
+        new BasicThreadFactory.Builder()
+            .namingPattern("NSScanScheduledThread")
+            .daemon(true)
+            .build());
     private final KVConfigManager kvConfigManager;
     private final RouteInfoManager routeInfoManager;
 
+    private RemotingClient remotingClient;
     private RemotingServer remotingServer;
 
     private BrokerHousekeepingService brokerHousekeepingService;
 
-    private ExecutorService remotingExecutor;
+    private ExecutorService defaultExecutor;
+    private ExecutorService clientRequestExecutor;
+
+    private BlockingQueue<Runnable> defaultThreadPoolQueue;
+    private BlockingQueue<Runnable> clientRequestThreadPoolQueue;
 
     private Configuration configuration;
     private FileWatchService fileWatchService;
 
     public NamesrvController(NamesrvConfig namesrvConfig, NettyServerConfig nettyServerConfig) {
+        this(namesrvConfig, nettyServerConfig, new NettyClientConfig());
+    }
+
+    public NamesrvController(NamesrvConfig namesrvConfig, NettyServerConfig nettyServerConfig, NettyClientConfig nettyClientConfig) {
         this.namesrvConfig = namesrvConfig;
         this.nettyServerConfig = nettyServerConfig;
+        this.nettyClientConfig = nettyClientConfig;
         this.kvConfigManager = new KVConfigManager(this);
-        this.routeInfoManager = new RouteInfoManager();
         this.brokerHousekeepingService = new BrokerHousekeepingService(this);
+        this.routeInfoManager = new RouteInfoManager(namesrvConfig, this);
         this.configuration = new Configuration(
-            log,
+            LOGGER,
             this.namesrvConfig, this.nettyServerConfig
         );
         this.configuration.setStorePathFromConfig(this.namesrvConfig, "configStorePath");
@@ -79,26 +113,51 @@ public class NamesrvController {
 
         this.remotingServer = new NettyRemotingServer(this.nettyServerConfig, this.brokerHousekeepingService);
 
-        this.remotingExecutor =
-            Executors.newFixedThreadPool(nettyServerConfig.getServerWorkerThreads(), new ThreadFactoryImpl("RemotingExecutorThread_"));
+        this.defaultThreadPoolQueue = new LinkedBlockingQueue<>(this.namesrvConfig.getDefaultThreadPoolQueueCapacity());
+        this.defaultExecutor = new ThreadPoolExecutor(
+            this.namesrvConfig.getDefaultThreadPoolNums(),
+            this.namesrvConfig.getDefaultThreadPoolNums(),
+            1000 * 60,
+            TimeUnit.MILLISECONDS,
+            this.defaultThreadPoolQueue,
+            new ThreadFactoryImpl("RemotingExecutorThread_")) {
+            @Override
+            protected <T> RunnableFuture<T> newTaskFor(final Runnable runnable, final T value) {
+                return new FutureTaskExt<T>(runnable, value);
+            }
+        };
+
+        this.clientRequestThreadPoolQueue = new LinkedBlockingQueue<>(this.namesrvConfig.getClientRequestThreadPoolQueueCapacity());
+        this.clientRequestExecutor = new ThreadPoolExecutor(
+            this.namesrvConfig.getClientRequestThreadPoolNums(),
+            this.namesrvConfig.getClientRequestThreadPoolNums(),
+            1000 * 60,
+            TimeUnit.MILLISECONDS,
+            this.clientRequestThreadPoolQueue,
+            new ThreadFactoryImpl("ClientRequestExecutorThread_")) {
+            @Override
+            protected <T> RunnableFuture<T> newTaskFor(final Runnable runnable, final T value) {
+                return new FutureTaskExt<T>(runnable, value);
+            }
+        };
+        this.remotingClient = new NettyRemotingClient(this.nettyClientConfig);
+        this.remotingClient.updateNameServerAddressList(Arrays.asList(RemotingUtil.getLocalAddress() + ":" + this.nettyServerConfig.getListenPort()));
 
         this.registerProcessor();
 
-        this.scheduledExecutorService.scheduleAtFixedRate(new Runnable() {
-
-            @Override
-            public void run() {
-                NamesrvController.this.routeInfoManager.scanNotActiveBroker();
-            }
-        }, 5, 10, TimeUnit.SECONDS);
+        this.scanExecutorService.scheduleAtFixedRate(NamesrvController.this.routeInfoManager::scanNotActiveBroker,
+            5, this.namesrvConfig.getScanNotActiveBrokerInterval(), TimeUnit.MILLISECONDS);
 
-        this.scheduledExecutorService.scheduleAtFixedRate(new Runnable() {
+        this.scheduledExecutorService.scheduleAtFixedRate(NamesrvController.this.kvConfigManager::printAllPeriodically,
+            1, 10, TimeUnit.MINUTES);
 
-            @Override
-            public void run() {
-                NamesrvController.this.kvConfigManager.printAllPeriodically();
+        this.scheduledExecutorService.scheduleAtFixedRate(() -> {
+            try {
+                NamesrvController.this.printWaterMark();
+            } catch (Throwable e) {
+                LOGGER.error("printWaterMark error.", e);
             }
-        }, 1, 10, TimeUnit.MINUTES);
+        }, 10, 1, TimeUnit.SECONDS);
 
         if (TlsSystemConfig.tlsMode != TlsMode.DISABLED) {
             // Register a listener to reload SslContext
@@ -111,10 +170,11 @@ public class NamesrvController {
                     },
                     new FileWatchService.Listener() {
                         boolean certChanged, keyChanged = false;
+
                         @Override
                         public void onChanged(String path) {
                             if (path.equals(TlsSystemConfig.tlsServerTrustCertPath)) {
-                                log.info("The trust certificate changed, reload the ssl context");
+                                LOGGER.info("The trust certificate changed, reload the ssl context");
                                 reloadServerSslContext();
                             }
                             if (path.equals(TlsSystemConfig.tlsServerCertPath)) {
@@ -124,46 +184,82 @@ public class NamesrvController {
                                 keyChanged = true;
                             }
                             if (certChanged && keyChanged) {
-                                log.info("The certificate and private key changed, reload the ssl context");
+                                LOGGER.info("The certificate and private key changed, reload the ssl context");
                                 certChanged = keyChanged = false;
                                 reloadServerSslContext();
                             }
                         }
+
                         private void reloadServerSslContext() {
                             ((NettyRemotingServer) remotingServer).loadSslContext();
                         }
                     });
             } catch (Exception e) {
-                log.warn("FileWatchService created error, can't load the certificate dynamically");
+                LOGGER.warn("FileWatchService created error, can't load the certificate dynamically");
             }
         }
 
         return true;
     }
 
+    private void printWaterMark() {
+        WATER_MARK_LOG.info("[WATERMARK] ClientQueueSize:{} ClientQueueSlowTime:{} " +
+                "DefaultQueueSize:{} DefaultQueueSlowTime:{}",
+            this.clientRequestThreadPoolQueue.size(), headSlowTimeMills(this.clientRequestThreadPoolQueue),
+            this.defaultThreadPoolQueue.size(), headSlowTimeMills(this.defaultThreadPoolQueue));
+    }
+
+    private long headSlowTimeMills(BlockingQueue<Runnable> q) {
+        long slowTimeMills = 0;
+        final Runnable firstRunnable = q.peek();
+
+        if (firstRunnable instanceof FutureTaskExt) {
+            final Runnable inner = ((FutureTaskExt) firstRunnable).getRunnable();
+            if (inner instanceof RequestTask) {
+                slowTimeMills = System.currentTimeMillis() - ((RequestTask) inner).getCreateTimestamp();
+            }
+        }
+
+        if (slowTimeMills < 0) {
+            slowTimeMills = 0;
+        }
+
+        return slowTimeMills;
+    }
+
     private void registerProcessor() {
         if (namesrvConfig.isClusterTest()) {
 
             this.remotingServer.registerDefaultProcessor(new ClusterTestRequestProcessor(this, namesrvConfig.getProductEnvName()),
-                this.remotingExecutor);
+                this.defaultExecutor);
         } else {
+            // Support get route info only temporarily
+            ClientRequestProcessor clientRequestProcessor = new ClientRequestProcessor(this);
+            this.remotingServer.registerProcessor(RequestCode.GET_ROUTEINFO_BY_TOPIC, clientRequestProcessor, this.clientRequestExecutor);
 
-            this.remotingServer.registerDefaultProcessor(new DefaultRequestProcessor(this), this.remotingExecutor);
+            this.remotingServer.registerDefaultProcessor(new DefaultRequestProcessor(this), this.defaultExecutor);
         }
     }
 
     public void start() throws Exception {
         this.remotingServer.start();
+        this.remotingClient.start();
 
         if (this.fileWatchService != null) {
             this.fileWatchService.start();
         }
+
+        this.routeInfoManager.start();
     }
 
     public void shutdown() {
+        this.remotingClient.shutdown();
         this.remotingServer.shutdown();
-        this.remotingExecutor.shutdown();
+        this.defaultExecutor.shutdown();
+        this.clientRequestExecutor.shutdown();
         this.scheduledExecutorService.shutdown();
+        this.scanExecutorService.shutdown();
+        this.routeInfoManager.shutdown();
 
         if (this.fileWatchService != null) {
             this.fileWatchService.shutdown();
@@ -190,6 +286,10 @@ public class NamesrvController {
         return remotingServer;
     }
 
+    public RemotingClient getRemotingClient() {
+        return remotingClient;
+    }
+
     public void setRemotingServer(RemotingServer remotingServer) {
         this.remotingServer = remotingServer;
     }
diff --git a/namesrv/src/main/java/org/apache/rocketmq/namesrv/NamesrvStartup.java b/namesrv/src/main/java/org/apache/rocketmq/namesrv/NamesrvStartup.java
index d006128..715f238 100644
--- a/namesrv/src/main/java/org/apache/rocketmq/namesrv/NamesrvStartup.java
+++ b/namesrv/src/main/java/org/apache/rocketmq/namesrv/NamesrvStartup.java
@@ -35,6 +35,7 @@ import org.apache.rocketmq.common.constant.LoggerName;
 import org.apache.rocketmq.logging.InternalLogger;
 import org.apache.rocketmq.logging.InternalLoggerFactory;
 import org.apache.rocketmq.common.namesrv.NamesrvConfig;
+import org.apache.rocketmq.remoting.netty.NettyClientConfig;
 import org.apache.rocketmq.remoting.netty.NettyServerConfig;
 import org.apache.rocketmq.remoting.protocol.RemotingCommand;
 import org.apache.rocketmq.srvutil.ServerUtil;
@@ -81,6 +82,7 @@ public class NamesrvStartup {
 
         final NamesrvConfig namesrvConfig = new NamesrvConfig();
         final NettyServerConfig nettyServerConfig = new NettyServerConfig();
+        final NettyClientConfig nettyClientConfig = new NettyClientConfig();
         nettyServerConfig.setListenPort(9876);
         if (commandLine.hasOption('c')) {
             String file = commandLine.getOptionValue('c');
@@ -90,6 +92,7 @@ public class NamesrvStartup {
                 properties.load(in);
                 MixAll.properties2Object(properties, namesrvConfig);
                 MixAll.properties2Object(properties, nettyServerConfig);
+                MixAll.properties2Object(properties, nettyClientConfig);
 
                 namesrvConfig.setConfigStorePath(file);
 
@@ -99,9 +102,9 @@ public class NamesrvStartup {
         }
 
         if (commandLine.hasOption('p')) {
-            InternalLogger console = InternalLoggerFactory.getLogger(LoggerName.NAMESRV_CONSOLE_NAME);
-            MixAll.printObjectProperties(console, namesrvConfig);
-            MixAll.printObjectProperties(console, nettyServerConfig);
+            MixAll.printObjectProperties(null, namesrvConfig);
+            MixAll.printObjectProperties(null, nettyServerConfig);
+            MixAll.printObjectProperties(null, nettyClientConfig);
             System.exit(0);
         }
 
@@ -123,7 +126,7 @@ public class NamesrvStartup {
         MixAll.printObjectProperties(log, namesrvConfig);
         MixAll.printObjectProperties(log, nettyServerConfig);
 
-        final NamesrvController controller = new NamesrvController(namesrvConfig, nettyServerConfig);
+        final NamesrvController controller = new NamesrvController(namesrvConfig, nettyServerConfig, nettyClientConfig);
 
         // remember all configs to prevent discard
         controller.getConfiguration().registerConfig(properties);
diff --git a/namesrv/src/main/java/org/apache/rocketmq/namesrv/processor/ClientRequestProcessor.java b/namesrv/src/main/java/org/apache/rocketmq/namesrv/processor/ClientRequestProcessor.java
new file mode 100644
index 0000000..6f624e3
--- /dev/null
+++ b/namesrv/src/main/java/org/apache/rocketmq/namesrv/processor/ClientRequestProcessor.java
@@ -0,0 +1,106 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.rocketmq.namesrv.processor;
+
+import io.netty.channel.ChannelHandlerContext;
+import org.apache.rocketmq.common.help.FAQUrl;
+import org.apache.rocketmq.common.namesrv.NamesrvUtil;
+import org.apache.rocketmq.common.protocol.ResponseCode;
+import org.apache.rocketmq.common.protocol.header.namesrv.GetRouteInfoRequestHeader;
+import org.apache.rocketmq.common.protocol.route.TopicRouteData;
+import org.apache.rocketmq.common.protocol.route.TopicRouteDatas;
+import org.apache.rocketmq.namesrv.NamesrvController;
+import org.apache.rocketmq.remoting.exception.RemotingCommandException;
+import org.apache.rocketmq.remoting.netty.NettyRequestProcessor;
+import org.apache.rocketmq.remoting.protocol.RemotingCommand;
+
+public class ClientRequestProcessor implements NettyRequestProcessor {
+    protected NamesrvController namesrvController;
+
+    public ClientRequestProcessor(final NamesrvController namesrvController) {
+        this.namesrvController = namesrvController;
+    }
+
+    @Override
+    public RemotingCommand processRequest(final ChannelHandlerContext ctx,
+        final RemotingCommand request) throws Exception {
+        return this.getRouteInfoByTopic(ctx, request);
+    }
+
+    public RemotingCommand getRouteInfoByTopic(ChannelHandlerContext ctx,
+        RemotingCommand request) throws RemotingCommandException {
+        final RemotingCommand response = RemotingCommand.createResponseCommand(null);
+        final GetRouteInfoRequestHeader requestHeader =
+            (GetRouteInfoRequestHeader) request.decodeCommandCustomHeader(GetRouteInfoRequestHeader.class);
+
+        if (requestHeader.getTopic().indexOf(GetRouteInfoRequestHeader.split) < 0) {
+            TopicRouteData topicRouteData = this.namesrvController.getRouteInfoManager().pickupTopicRouteData(requestHeader.getTopic());
+
+            if (topicRouteData != null) {
+                if (this.namesrvController.getNamesrvConfig().isOrderMessageEnable()) {
+                    String orderTopicConf =
+                        this.namesrvController.getKvConfigManager().getKVConfig(NamesrvUtil.NAMESPACE_ORDER_TOPIC_CONFIG,
+                            requestHeader.getTopic());
+
+                    // Get a shallow clone of route data to modify order topic conf
+                    topicRouteData = topicRouteData.cloneTopicRouteData();
+                    topicRouteData.setOrderTopicConf(orderTopicConf);
+                }
+
+                byte[] content = topicRouteData.encode();
+                response.setBody(content);
+                response.setCode(ResponseCode.SUCCESS);
+                response.setRemark(null);
+                return response;
+            }
+
+            response.setCode(ResponseCode.TOPIC_NOT_EXIST);
+            response.setRemark("No topic route info in name server for the topic: " + requestHeader.getTopic()
+                + FAQUrl.suggestTodo(FAQUrl.APPLY_TOPIC_URL));
+            return response;
+        }
+
+        String[] topics = requestHeader.getTopic().split(String.valueOf(GetRouteInfoRequestHeader.split));
+        TopicRouteDatas topicRouteDatas = new TopicRouteDatas();
+
+        for (String topic : topics) {
+            TopicRouteData topicRouteData = this.namesrvController.getRouteInfoManager().pickupTopicRouteData(topic);
+            if (topicRouteData == null) {
+                continue;
+            }
+
+            if (this.namesrvController.getNamesrvConfig().isOrderMessageEnable()) {
+                String orderTopicConf =
+                    this.namesrvController.getKvConfigManager().getKVConfig(NamesrvUtil.NAMESPACE_ORDER_TOPIC_CONFIG, topic);
+                // Get a shallow clone of route data to modify order topic conf
+                topicRouteData = topicRouteData.cloneTopicRouteData();
+                topicRouteData.setOrderTopicConf(orderTopicConf);
+            }
+
+            topicRouteDatas.getTopics().put(topic, topicRouteData);
+        }
+        response.setBody(topicRouteDatas.encode());
+        response.setCode(ResponseCode.SUCCESS);
+        response.setRemark(null);
+        return response;
+    }
+
+    @Override public boolean rejectRequest() {
+        return false;
+    }
+}
diff --git a/namesrv/src/main/java/org/apache/rocketmq/namesrv/processor/ClusterTestRequestProcessor.java b/namesrv/src/main/java/org/apache/rocketmq/namesrv/processor/ClusterTestRequestProcessor.java
index a58a3b9..e5ca2d3 100644
--- a/namesrv/src/main/java/org/apache/rocketmq/namesrv/processor/ClusterTestRequestProcessor.java
+++ b/namesrv/src/main/java/org/apache/rocketmq/namesrv/processor/ClusterTestRequestProcessor.java
@@ -31,7 +31,7 @@ import org.apache.rocketmq.remoting.exception.RemotingCommandException;
 import org.apache.rocketmq.remoting.protocol.RemotingCommand;
 import org.apache.rocketmq.tools.admin.DefaultMQAdminExt;
 
-public class ClusterTestRequestProcessor extends DefaultRequestProcessor {
+public class ClusterTestRequestProcessor extends ClientRequestProcessor {
     private static final InternalLogger log = InternalLoggerFactory.getLogger(LoggerName.NAMESRV_LOGGER_NAME);
     private final DefaultMQAdminExt adminExt;
     private final String productEnvName;
diff --git a/namesrv/src/main/java/org/apache/rocketmq/namesrv/processor/DefaultRequestProcessor.java b/namesrv/src/main/java/org/apache/rocketmq/namesrv/processor/DefaultRequestProcessor.java
index bde0348..a293683 100644
--- a/namesrv/src/main/java/org/apache/rocketmq/namesrv/processor/DefaultRequestProcessor.java
+++ b/namesrv/src/main/java/org/apache/rocketmq/namesrv/processor/DefaultRequestProcessor.java
@@ -19,6 +19,7 @@ package org.apache.rocketmq.namesrv.processor;
 import io.netty.channel.ChannelHandlerContext;
 import java.io.UnsupportedEncodingException;
 import java.util.Properties;
+import java.util.List;
 import java.util.concurrent.atomic.AtomicLong;
 import org.apache.rocketmq.common.DataVersion;
 import org.apache.rocketmq.common.MQVersion;
@@ -26,9 +27,14 @@ import org.apache.rocketmq.common.MQVersion.Version;
 import org.apache.rocketmq.common.MixAll;
 import org.apache.rocketmq.common.UtilAll;
 import org.apache.rocketmq.common.constant.LoggerName;
-import org.apache.rocketmq.common.help.FAQUrl;
+import org.apache.rocketmq.common.protocol.body.BrokerMemberGroup;
+import org.apache.rocketmq.common.protocol.body.GetBrokerMemberGroupResponseBody;
+import org.apache.rocketmq.common.protocol.body.GetRemoteClientConfigBody;
+import org.apache.rocketmq.common.protocol.header.GetBrokerMemberGroupRequestHeader;
 import org.apache.rocketmq.common.protocol.header.namesrv.AddWritePermOfBrokerRequestHeader;
 import org.apache.rocketmq.common.protocol.header.namesrv.AddWritePermOfBrokerResponseHeader;
+import org.apache.rocketmq.common.protocol.header.namesrv.BrokerHeartbeatRequestHeader;
+import org.apache.rocketmq.common.protocol.header.namesrv.RegisterTopicRequestHeader;
 import org.apache.rocketmq.logging.InternalLogger;
 import org.apache.rocketmq.logging.InternalLoggerFactory;
 import org.apache.rocketmq.common.namesrv.NamesrvUtil;
@@ -43,7 +49,6 @@ import org.apache.rocketmq.common.protocol.header.namesrv.DeleteTopicFromNamesrv
 import org.apache.rocketmq.common.protocol.header.namesrv.GetKVConfigRequestHeader;
 import org.apache.rocketmq.common.protocol.header.namesrv.GetKVConfigResponseHeader;
 import org.apache.rocketmq.common.protocol.header.namesrv.GetKVListByNamespaceRequestHeader;
-import org.apache.rocketmq.common.protocol.header.namesrv.GetRouteInfoRequestHeader;
 import org.apache.rocketmq.common.protocol.header.namesrv.PutKVConfigRequestHeader;
 import org.apache.rocketmq.common.protocol.header.namesrv.QueryDataVersionRequestHeader;
 import org.apache.rocketmq.common.protocol.header.namesrv.QueryDataVersionResponseHeader;
@@ -56,11 +61,11 @@ import org.apache.rocketmq.common.protocol.route.TopicRouteData;
 import org.apache.rocketmq.namesrv.NamesrvController;
 import org.apache.rocketmq.remoting.common.RemotingHelper;
 import org.apache.rocketmq.remoting.exception.RemotingCommandException;
-import org.apache.rocketmq.remoting.netty.AsyncNettyRequestProcessor;
 import org.apache.rocketmq.remoting.netty.NettyRequestProcessor;
 import org.apache.rocketmq.remoting.protocol.RemotingCommand;
+import org.apache.rocketmq.remoting.protocol.RemotingSysResponseCode;
 
-public class DefaultRequestProcessor extends AsyncNettyRequestProcessor implements NettyRequestProcessor {
+public class DefaultRequestProcessor implements NettyRequestProcessor {
     private static InternalLogger log = InternalLoggerFactory.getLogger(LoggerName.NAMESRV_LOGGER_NAME);
 
     protected final NamesrvController namesrvController;
@@ -80,7 +85,6 @@ public class DefaultRequestProcessor extends AsyncNettyRequestProcessor implemen
                 request);
         }
 
-
         switch (request.getCode()) {
             case RequestCode.PUT_KV_CONFIG:
                 return this.putKVConfig(ctx, request);
@@ -89,18 +93,15 @@ public class DefaultRequestProcessor extends AsyncNettyRequestProcessor implemen
             case RequestCode.DELETE_KV_CONFIG:
                 return this.deleteKVConfig(ctx, request);
             case RequestCode.QUERY_DATA_VERSION:
-                return queryBrokerTopicConfig(ctx, request);
+                return this.queryBrokerTopicConfig(ctx, request);
             case RequestCode.REGISTER_BROKER:
-                Version brokerVersion = MQVersion.value2Version(request.getVersion());
-                if (brokerVersion.ordinal() >= MQVersion.Version.V3_0_11.ordinal()) {
-                    return this.registerBrokerWithFilterServer(ctx, request);
-                } else {
-                    return this.registerBroker(ctx, request);
-                }
+                return this.registerBroker(ctx, request);
             case RequestCode.UNREGISTER_BROKER:
                 return this.unregisterBroker(ctx, request);
-            case RequestCode.GET_ROUTEINFO_BY_TOPIC:
-                return this.getRouteInfoByTopic(ctx, request);
+            case RequestCode.BROKER_HEARTBEAT:
+                return this.brokerHeartbeat(ctx, request);
+            case RequestCode.GET_BROKER_MEMBER_GROUP:
+                return this.getBrokerMemberGroup(ctx, request);
             case RequestCode.GET_BROKER_CLUSTER_INFO:
                 return this.getBrokerClusterInfo(ctx, request);
             case RequestCode.WIPE_WRITE_PERM_OF_BROKER:
@@ -108,9 +109,11 @@ public class DefaultRequestProcessor extends AsyncNettyRequestProcessor implemen
             case RequestCode.ADD_WRITE_PERM_OF_BROKER:
                 return this.addWritePermOfBroker(ctx, request);
             case RequestCode.GET_ALL_TOPIC_LIST_FROM_NAMESERVER:
-                return getAllTopicListFromNameserver(ctx, request);
+                return this.getAllTopicListFromNameserver(ctx, request);
             case RequestCode.DELETE_TOPIC_IN_NAMESRV:
-                return deleteTopicInNamesrv(ctx, request);
+                return this.deleteTopicInNamesrv(ctx, request);
+            case RequestCode.REGISTER_TOPIC_IN_NAMESRV:
+                return this.registerTopicToNamesrv(ctx, request);
             case RequestCode.GET_KVLIST_BY_NAMESPACE:
                 return this.getKVListByNamespace(ctx, request);
             case RequestCode.GET_TOPICS_BY_CLUSTER:
@@ -127,14 +130,15 @@ public class DefaultRequestProcessor extends AsyncNettyRequestProcessor implemen
                 return this.updateConfig(ctx, request);
             case RequestCode.GET_NAMESRV_CONFIG:
                 return this.getConfig(ctx, request);
+            case RequestCode.GET_CLIENT_CONFIG:
+                return this.getClientConfigs(ctx, request);
             default:
-                break;
+                String error = " request type " + request.getCode() + " not supported";
+                return RemotingCommand.createResponseCommand(RemotingSysResponseCode.REQUEST_CODE_NOT_SUPPORTED, error);
         }
-        return null;
     }
 
-    @Override
-    public boolean rejectRequest() {
+    @Override public boolean rejectRequest() {
         return false;
     }
 
@@ -200,8 +204,8 @@ public class DefaultRequestProcessor extends AsyncNettyRequestProcessor implemen
         return response;
     }
 
-    public RemotingCommand registerBrokerWithFilterServer(ChannelHandlerContext ctx, RemotingCommand request)
-        throws RemotingCommandException {
+    public RemotingCommand registerBroker(ChannelHandlerContext ctx,
+        RemotingCommand request) throws RemotingCommandException {
         final RemotingCommand response = RemotingCommand.createResponseCommand(RegisterBrokerResponseHeader.class);
         final RegisterBrokerResponseHeader responseHeader = (RegisterBrokerResponseHeader) response.readCustomHeader();
         final RegisterBrokerRequestHeader requestHeader =
@@ -213,17 +217,17 @@ public class DefaultRequestProcessor extends AsyncNettyRequestProcessor implemen
             return response;
         }
 
-        RegisterBrokerBody registerBrokerBody = new RegisterBrokerBody();
+        TopicConfigSerializeWrapper topicConfigWrapper = null;
+        List<String> filterServerList = null;
 
-        if (request.getBody() != null) {
-            try {
-                registerBrokerBody = RegisterBrokerBody.decode(request.getBody(), requestHeader.isCompressed());
-            } catch (Exception e) {
-                throw new RemotingCommandException("Failed to decode RegisterBrokerBody", e);
-            }
+        Version brokerVersion = MQVersion.value2Version(request.getVersion());
+        if (brokerVersion.ordinal() >= MQVersion.Version.V3_0_11.ordinal()) {
+            final RegisterBrokerBody registerBrokerBody = extractRegisterBrokerBodyFromRequest(request, requestHeader);
+            topicConfigWrapper = registerBrokerBody.getTopicConfigSerializeWrapper();
+            filterServerList = registerBrokerBody.getFilterServerList();
         } else {
-            registerBrokerBody.getTopicConfigSerializeWrapper().getDataVersion().setCounter(new AtomicLong(0));
-            registerBrokerBody.getTopicConfigSerializeWrapper().getDataVersion().setTimestamp(0);
+            // RegisterBrokerBody of old version only contains TopicConfig.
+            topicConfigWrapper = extractRegisterTopicConfigFromRequest(request);
         }
 
         RegisterBrokerResult result = this.namesrvController.getRouteInfoManager().registerBroker(
@@ -232,21 +236,73 @@ public class DefaultRequestProcessor extends AsyncNettyRequestProcessor implemen
             requestHeader.getBrokerName(),
             requestHeader.getBrokerId(),
             requestHeader.getHaServerAddr(),
-            registerBrokerBody.getTopicConfigSerializeWrapper(),
-            registerBrokerBody.getFilterServerList(),
-            ctx.channel());
+            requestHeader.getHeartbeatTimeoutMillis(),
+            requestHeader.getEnableActingMaster(),
+            topicConfigWrapper,
+            filterServerList,
+            ctx.channel()
+        );
 
         responseHeader.setHaServerAddr(result.getHaServerAddr());
         responseHeader.setMasterAddr(result.getMasterAddr());
 
-        byte[] jsonValue = this.namesrvController.getKvConfigManager().getKVListByNamespace(NamesrvUtil.NAMESPACE_ORDER_TOPIC_CONFIG);
-        response.setBody(jsonValue);
+        if (this.namesrvController.getNamesrvConfig().isReturnOrderTopicConfigToBroker()) {
+            byte[] jsonValue = this.namesrvController.getKvConfigManager().getKVListByNamespace(NamesrvUtil.NAMESPACE_ORDER_TOPIC_CONFIG);
+            response.setBody(jsonValue);
+        }
 
         response.setCode(ResponseCode.SUCCESS);
         response.setRemark(null);
         return response;
     }
 
+    private TopicConfigSerializeWrapper extractRegisterTopicConfigFromRequest(final RemotingCommand request) {
+        TopicConfigSerializeWrapper topicConfigWrapper;
+        if (request.getBody() != null) {
+            topicConfigWrapper = TopicConfigSerializeWrapper.decode(request.getBody(), TopicConfigSerializeWrapper.class);
+        } else {
+            topicConfigWrapper = new TopicConfigSerializeWrapper();
+            topicConfigWrapper.getDataVersion().setCounter(new AtomicLong(0));
+            topicConfigWrapper.getDataVersion().setTimestamp(0L);
+            topicConfigWrapper.getDataVersion().setStateVersion(0L);
+        }
+        return topicConfigWrapper;
+    }
+
+    private RegisterBrokerBody extractRegisterBrokerBodyFromRequest(RemotingCommand request,
+        RegisterBrokerRequestHeader requestHeader) throws RemotingCommandException {
+        RegisterBrokerBody registerBrokerBody = new RegisterBrokerBody();
+
+        if (request.getBody() != null) {
+            try {
+                registerBrokerBody = RegisterBrokerBody.decode(request.getBody(), requestHeader.isCompressed());
+            } catch (Exception e) {
+                throw new RemotingCommandException("Failed to decode RegisterBrokerBody", e);
+            }
+        } else {
+            registerBrokerBody.getTopicConfigSerializeWrapper().getDataVersion().setCounter(new AtomicLong(0));
+            registerBrokerBody.getTopicConfigSerializeWrapper().getDataVersion().setTimestamp(0L);
+            registerBrokerBody.getTopicConfigSerializeWrapper().getDataVersion().setStateVersion(0L);
+        }
+        return registerBrokerBody;
+    }
+
+    private RemotingCommand getBrokerMemberGroup(ChannelHandlerContext ctx,
+        RemotingCommand request) throws RemotingCommandException {
+        GetBrokerMemberGroupRequestHeader requestHeader = (GetBrokerMemberGroupRequestHeader) request.decodeCommandCustomHeader(GetBrokerMemberGroupRequestHeader.class);
+
+        BrokerMemberGroup memberGroup = this.namesrvController.getRouteInfoManager()
+            .getBrokerMemberGroup(requestHeader.getClusterName(), requestHeader.getBrokerName());
+
+        GetBrokerMemberGroupResponseBody responseBody = new GetBrokerMemberGroupResponseBody();
+        responseBody.setBrokerMemberGroup(memberGroup);
+
+        RemotingCommand response = RemotingCommand.createResponseCommand(null);
+        response.setCode(ResponseCode.SUCCESS);
+        response.setBody(responseBody.encode());
+        return response;
+    }
+
     private boolean checksum(ChannelHandlerContext ctx, RemotingCommand request,
         RegisterBrokerRequestHeader requestHeader) {
         if (requestHeader.getBodyCrc32() != 0) {
@@ -267,13 +323,13 @@ public class DefaultRequestProcessor extends AsyncNettyRequestProcessor implemen
         final QueryDataVersionRequestHeader requestHeader =
             (QueryDataVersionRequestHeader) request.decodeCommandCustomHeader(QueryDataVersionRequestHeader.class);
         DataVersion dataVersion = DataVersion.decode(request.getBody(), DataVersion.class);
+        String clusterName = requestHeader.getClusterName();
+        String brokerAddr = requestHeader.getBrokerAddr();
 
-        Boolean changed = this.namesrvController.getRouteInfoManager().isBrokerTopicConfigChanged(requestHeader.getBrokerAddr(), dataVersion);
-        if (!changed) {
-            this.namesrvController.getRouteInfoManager().updateBrokerInfoUpdateTimestamp(requestHeader.getBrokerAddr());
-        }
+        Boolean changed = this.namesrvController.getRouteInfoManager().isBrokerTopicConfigChanged(clusterName, brokerAddr, dataVersion);
+        this.namesrvController.getRouteInfoManager().updateBrokerInfoUpdateTimestamp(clusterName, brokerAddr);
 
-        DataVersion nameSeverDataVersion = this.namesrvController.getRouteInfoManager().queryBrokerTopicConfig(requestHeader.getBrokerAddr());
+        DataVersion nameSeverDataVersion = this.namesrvController.getRouteInfoManager().queryBrokerTopicConfig(clusterName, brokerAddr);
         response.setCode(ResponseCode.SUCCESS);
         response.setRemark(null);
 
@@ -284,92 +340,31 @@ public class DefaultRequestProcessor extends AsyncNettyRequestProcessor implemen
         return response;
     }
 
-    public RemotingCommand registerBroker(ChannelHandlerContext ctx,
-        RemotingCommand request) throws RemotingCommandException {
-        final RemotingCommand response = RemotingCommand.createResponseCommand(RegisterBrokerResponseHeader.class);
-        final RegisterBrokerResponseHeader responseHeader = (RegisterBrokerResponseHeader) response.readCustomHeader();
-        final RegisterBrokerRequestHeader requestHeader =
-            (RegisterBrokerRequestHeader) request.decodeCommandCustomHeader(RegisterBrokerRequestHeader.class);
-
-        if (!checksum(ctx, request, requestHeader)) {
-            response.setCode(ResponseCode.SYSTEM_ERROR);
-            response.setRemark("crc32 not match");
-            return response;
-        }
-
-        TopicConfigSerializeWrapper topicConfigWrapper;
-        if (request.getBody() != null) {
-            topicConfigWrapper = TopicConfigSerializeWrapper.decode(request.getBody(), TopicConfigSerializeWrapper.class);
-        } else {
-            topicConfigWrapper = new TopicConfigSerializeWrapper();
-            topicConfigWrapper.getDataVersion().setCounter(new AtomicLong(0));
-            topicConfigWrapper.getDataVersion().setTimestamp(0);
-        }
-
-        RegisterBrokerResult result = this.namesrvController.getRouteInfoManager().registerBroker(
-            requestHeader.getClusterName(),
-            requestHeader.getBrokerAddr(),
-            requestHeader.getBrokerName(),
-            requestHeader.getBrokerId(),
-            requestHeader.getHaServerAddr(),
-            topicConfigWrapper,
-            null,
-            ctx.channel()
-        );
-
-        responseHeader.setHaServerAddr(result.getHaServerAddr());
-        responseHeader.setMasterAddr(result.getMasterAddr());
-
-        byte[] jsonValue = this.namesrvController.getKvConfigManager().getKVListByNamespace(NamesrvUtil.NAMESPACE_ORDER_TOPIC_CONFIG);
-        response.setBody(jsonValue);
-        response.setCode(ResponseCode.SUCCESS);
-        response.setRemark(null);
-        return response;
-    }
-
     public RemotingCommand unregisterBroker(ChannelHandlerContext ctx,
         RemotingCommand request) throws RemotingCommandException {
         final RemotingCommand response = RemotingCommand.createResponseCommand(null);
         final UnRegisterBrokerRequestHeader requestHeader =
             (UnRegisterBrokerRequestHeader) request.decodeCommandCustomHeader(UnRegisterBrokerRequestHeader.class);
 
-        this.namesrvController.getRouteInfoManager().unregisterBroker(
-            requestHeader.getClusterName(),
-            requestHeader.getBrokerAddr(),
-            requestHeader.getBrokerName(),
-            requestHeader.getBrokerId());
+        if (!this.namesrvController.getRouteInfoManager().submitUnRegisterBrokerRequest(requestHeader)) {
+            log.warn("Couldn't submit the unregister broker request to handler, broker info: {}", requestHeader);
+        }
 
         response.setCode(ResponseCode.SUCCESS);
         response.setRemark(null);
         return response;
     }
 
-    public RemotingCommand getRouteInfoByTopic(ChannelHandlerContext ctx,
+    public RemotingCommand brokerHeartbeat(ChannelHandlerContext ctx,
         RemotingCommand request) throws RemotingCommandException {
         final RemotingCommand response = RemotingCommand.createResponseCommand(null);
-        final GetRouteInfoRequestHeader requestHeader =
-            (GetRouteInfoRequestHeader) request.decodeCommandCustomHeader(GetRouteInfoRequestHeader.class);
+        final BrokerHeartbeatRequestHeader requestHeader =
+            (BrokerHeartbeatRequestHeader) request.decodeCommandCustomHeader(BrokerHeartbeatRequestHeader.class);
 
-        TopicRouteData topicRouteData = this.namesrvController.getRouteInfoManager().pickupTopicRouteData(requestHeader.getTopic());
-
-        if (topicRouteData != null) {
-            if (this.namesrvController.getNamesrvConfig().isOrderMessageEnable()) {
-                String orderTopicConf =
-                    this.namesrvController.getKvConfigManager().getKVConfig(NamesrvUtil.NAMESPACE_ORDER_TOPIC_CONFIG,
-                        requestHeader.getTopic());
-                topicRouteData.setOrderTopicConf(orderTopicConf);
-            }
+        this.namesrvController.getRouteInfoManager().updateBrokerInfoUpdateTimestamp(requestHeader.getClusterName(), requestHeader.getBrokerAddr());
 
-            byte[] content = topicRouteData.encode();
-            response.setBody(content);
-            response.setCode(ResponseCode.SUCCESS);
-            response.setRemark(null);
-            return response;
-        }
-
-        response.setCode(ResponseCode.TOPIC_NOT_EXIST);
-        response.setRemark("No topic route info in name server for the topic: " + requestHeader.getTopic()
-            + FAQUrl.suggestTodo(FAQUrl.APPLY_TOPIC_URL));
+        response.setCode(ResponseCode.SUCCESS);
+        response.setRemark(null);
         return response;
     }
 
@@ -395,9 +390,9 @@ public class DefaultRequestProcessor extends AsyncNettyRequestProcessor implemen
 
         if (ctx != null) {
             log.info("wipe write perm of broker[{}], client: {}, {}",
-                    requestHeader.getBrokerName(),
-                    RemotingHelper.parseChannelRemoteAddr(ctx.channel()),
-                    wipeTopicCnt);
+                requestHeader.getBrokerName(),
+                RemotingHelper.parseChannelRemoteAddr(ctx.channel()),
+                wipeTopicCnt);
         }
 
         responseHeader.setWipeTopicCount(wipeTopicCnt);
@@ -406,7 +401,8 @@ public class DefaultRequestProcessor extends AsyncNettyRequestProcessor implemen
         return response;
     }
 
-    private RemotingCommand addWritePermOfBroker(ChannelHandlerContext ctx, RemotingCommand request) throws RemotingCommandException {
+    private RemotingCommand addWritePermOfBroker(ChannelHandlerContext ctx,
+        RemotingCommand request) throws RemotingCommandException {
         final RemotingCommand response = RemotingCommand.createResponseCommand(AddWritePermOfBrokerResponseHeader.class);
         final AddWritePermOfBrokerResponseHeader responseHeader = (AddWritePermOfBrokerResponseHeader) response.readCustomHeader();
         final AddWritePermOfBrokerRequestHeader requestHeader = (AddWritePermOfBrokerRequestHeader) request.decodeCommandCustomHeader(AddWritePermOfBrokerRequestHeader.class);
@@ -414,9 +410,9 @@ public class DefaultRequestProcessor extends AsyncNettyRequestProcessor implemen
         int addTopicCnt = this.namesrvController.getRouteInfoManager().addWritePermOfBrokerByLock(requestHeader.getBrokerName());
 
         log.info("add write perm of broker[{}], client: {}, {}",
-                requestHeader.getBrokerName(),
-                RemotingHelper.parseChannelRemoteAddr(ctx.channel()),
-                addTopicCnt);
+            requestHeader.getBrokerName(),
+            RemotingHelper.parseChannelRemoteAddr(ctx.channel()),
+            addTopicCnt);
 
         responseHeader.setAddTopicCount(addTopicCnt);
         response.setCode(ResponseCode.SUCCESS);
@@ -426,10 +422,33 @@ public class DefaultRequestProcessor extends AsyncNettyRequestProcessor implemen
 
     private RemotingCommand getAllTopicListFromNameserver(ChannelHandlerContext ctx, RemotingCommand request) {
         final RemotingCommand response = RemotingCommand.createResponseCommand(null);
+        boolean enableAllTopicList = namesrvController.getNamesrvConfig().isEnableAllTopicList();
+        log.warn("getAllTopicListFromNameserver {} enable {}", ctx.channel().remoteAddress(), enableAllTopicList);
+        if (enableAllTopicList) {
+            byte[] body = this.namesrvController.getRouteInfoManager().getAllTopicList();
+            response.setBody(body);
+            response.setCode(ResponseCode.SUCCESS);
+            response.setRemark(null);
+        } else {
+            response.setCode(ResponseCode.SYSTEM_ERROR);
+            response.setRemark("disable");
+        }
 
-        byte[] body = this.namesrvController.getRouteInfoManager().getAllTopicList();
+        return response;
+    }
+
+    private RemotingCommand registerTopicToNamesrv(ChannelHandlerContext ctx,
+        RemotingCommand request) throws RemotingCommandException {
+        final RemotingCommand response = RemotingCommand.createResponseCommand(null);
+
+        final RegisterTopicRequestHeader requestHeader =
+            (RegisterTopicRequestHeader) request.decodeCommandCustomHeader(RegisterTopicRequestHeader.class);
+
+        TopicRouteData topicRouteData = TopicRouteData.decode(request.getBody(), TopicRouteData.class);
+        if (topicRouteData != null && topicRouteData.getQueueDatas() != null && !topicRouteData.getQueueDatas().isEmpty()) {
+            this.namesrvController.getRouteInfoManager().registerTopic(requestHeader.getTopic(), topicRouteData.getQueueDatas());
+        }
 
-        response.setBody(body);
         response.setCode(ResponseCode.SUCCESS);
         response.setRemark(null);
         return response;
@@ -441,7 +460,12 @@ public class DefaultRequestProcessor extends AsyncNettyRequestProcessor implemen
         final DeleteTopicFromNamesrvRequestHeader requestHeader =
             (DeleteTopicFromNamesrvRequestHeader) request.decodeCommandCustomHeader(DeleteTopicFromNamesrvRequestHeader.class);
 
-        this.namesrvController.getRouteInfoManager().deleteTopic(requestHeader.getTopic());
+        if (requestHeader.getClusterName() != null
+            && !requestHeader.getClusterName().isEmpty()) {
+            this.namesrvController.getRouteInfoManager().deleteTopic(requestHeader.getTopic(), requestHeader.getClusterName());
+        } else {
+            this.namesrvController.getRouteInfoManager().deleteTopic(requestHeader.getTopic());
+        }
 
         response.setCode(ResponseCode.SUCCESS);
         response.setRemark(null);
@@ -585,4 +609,25 @@ public class DefaultRequestProcessor extends AsyncNettyRequestProcessor implemen
         return response;
     }
 
+    private RemotingCommand getClientConfigs(ChannelHandlerContext ctx, RemotingCommand request) {
+        final RemotingCommand response = RemotingCommand.createResponseCommand(null);
+        final GetRemoteClientConfigBody body = GetRemoteClientConfigBody.decode(request.getBody(), GetRemoteClientConfigBody.class);
+
+        String content = this.namesrvController.getConfiguration().getClientConfigsFormatString(body.getKeys());
+        if (content != null && content.length() > 0) {
+            try {
+                response.setBody(content.getBytes(MixAll.DEFAULT_CHARSET));
+            } catch (UnsupportedEncodingException e) {
+                log.error("getConfig error, ", e);
+                response.setCode(ResponseCode.SYSTEM_ERROR);
+                response.setRemark("UnsupportedEncodingException " + e);
+                return response;
+            }
+        }
+
+        response.setCode(ResponseCode.SUCCESS);
+        response.setRemark(null);
+        return response;
+    }
+
 }
diff --git a/namesrv/src/main/java/org/apache/rocketmq/namesrv/routeinfo/BatchUnRegisterService.java b/namesrv/src/main/java/org/apache/rocketmq/namesrv/routeinfo/BatchUnRegisterService.java
new file mode 100644
index 0000000..81460e7
--- /dev/null
+++ b/namesrv/src/main/java/org/apache/rocketmq/namesrv/routeinfo/BatchUnRegisterService.java
@@ -0,0 +1,85 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.rocketmq.namesrv.routeinfo;
+
+import java.util.HashSet;
+import java.util.Set;
+import java.util.concurrent.BlockingQueue;
+import java.util.concurrent.LinkedBlockingQueue;
+import java.util.concurrent.TimeUnit;
+import org.apache.rocketmq.common.ServiceThread;
+import org.apache.rocketmq.common.constant.LoggerName;
+import org.apache.rocketmq.common.namesrv.NamesrvConfig;
+import org.apache.rocketmq.common.protocol.header.namesrv.UnRegisterBrokerRequestHeader;
+import org.apache.rocketmq.logging.InternalLogger;
+import org.apache.rocketmq.logging.InternalLoggerFactory;
+
+/**
+ * BatchUnRegisterProcessor provides a mechanism to unregister broker in batch manner, which can speed up broker offline
+ * process.
+ */
+public class BatchUnRegisterService extends ServiceThread {
+    private final RouteInfoManager routeInfoManager;
+    private BlockingQueue<UnRegisterBrokerRequestHeader> unRegisterQueue;
+    private static final InternalLogger log = InternalLoggerFactory.getLogger(LoggerName.NAMESRV_LOGGER_NAME);
+
+    public BatchUnRegisterService(RouteInfoManager routeInfoManager, NamesrvConfig namesrvConfig) {
+        this.routeInfoManager = routeInfoManager;
+        this.unRegisterQueue = new LinkedBlockingQueue<>(namesrvConfig.getUnRegisterBrokerQueueCapacity());
+    }
+
+    /**
+     * Submits an unregister request to this queue.
+     *
+     * @param unRegisterRequest the request to submit
+     * @return {@code true} if the request was added to this queue, else {@code false}
+     */
+    public boolean submit(UnRegisterBrokerRequestHeader unRegisterRequest) {
+        return unRegisterQueue.offer(unRegisterRequest);
+    }
+
+    @Override
+    public String getServiceName() {
+        return BatchUnRegisterService.class.getName();
+    }
+
+    @Override
+    public void run() {
+        while (!this.isStopped()) {
+            try {
+                final UnRegisterBrokerRequestHeader request = unRegisterQueue.poll(3, TimeUnit.SECONDS);
+                if (request != null) {
+                    Set<UnRegisterBrokerRequestHeader> unRegisterRequests = new HashSet<>();
+                    unRegisterQueue.drainTo(unRegisterRequests);
+
+                    // Add polled request
+                    unRegisterRequests.add(request);
+
+                    this.routeInfoManager.unRegisterBroker(unRegisterRequests);
+                }
+            } catch (Throwable e) {
+                log.error("Handle unregister broker request failed", e);
+            }
+        }
+    }
+
+    // For test only
+    int queueLength() {
+        return this.unRegisterQueue.size();
+    }
+}
diff --git a/namesrv/src/main/java/org/apache/rocketmq/namesrv/routeinfo/BrokerHousekeepingService.java b/namesrv/src/main/java/org/apache/rocketmq/namesrv/routeinfo/BrokerHousekeepingService.java
index fedb4ae..dbf4786 100644
--- a/namesrv/src/main/java/org/apache/rocketmq/namesrv/routeinfo/BrokerHousekeepingService.java
+++ b/namesrv/src/main/java/org/apache/rocketmq/namesrv/routeinfo/BrokerHousekeepingService.java
@@ -37,16 +37,16 @@ public class BrokerHousekeepingService implements ChannelEventListener {
 
     @Override
     public void onChannelClose(String remoteAddr, Channel channel) {
-        this.namesrvController.getRouteInfoManager().onChannelDestroy(remoteAddr, channel);
+        this.namesrvController.getRouteInfoManager().onChannelDestroy(channel);
     }
 
     @Override
     public void onChannelException(String remoteAddr, Channel channel) {
-        this.namesrvController.getRouteInfoManager().onChannelDestroy(remoteAddr, channel);
+        this.namesrvController.getRouteInfoManager().onChannelDestroy(channel);
     }
 
     @Override
     public void onChannelIdle(String remoteAddr, Channel channel) {
-        this.namesrvController.getRouteInfoManager().onChannelDestroy(remoteAddr, channel);
+        this.namesrvController.getRouteInfoManager().onChannelDestroy(channel);
     }
 }
diff --git a/namesrv/src/main/java/org/apache/rocketmq/namesrv/routeinfo/RouteInfoManager.java b/namesrv/src/main/java/org/apache/rocketmq/namesrv/routeinfo/RouteInfoManager.java
index a02d3f1..4becdd6 100644
--- a/namesrv/src/main/java/org/apache/rocketmq/namesrv/routeinfo/RouteInfoManager.java
+++ b/namesrv/src/main/java/org/apache/rocketmq/namesrv/routeinfo/RouteInfoManager.java
@@ -16,26 +16,34 @@
  */
 package org.apache.rocketmq.namesrv.routeinfo;
 
+import com.google.common.collect.Sets;
 import io.netty.channel.Channel;
+import java.util.ArrayList;
+import java.util.Collections;
 import java.util.HashMap;
 import java.util.HashSet;
 import java.util.Iterator;
-import java.util.LinkedList;
 import java.util.List;
 import java.util.Map;
 import java.util.Map.Entry;
 import java.util.Set;
+import java.util.concurrent.ConcurrentHashMap;
 import java.util.concurrent.ConcurrentMap;
 import java.util.concurrent.locks.ReadWriteLock;
 import java.util.concurrent.locks.ReentrantReadWriteLock;
 import org.apache.rocketmq.common.DataVersion;
 import org.apache.rocketmq.common.MixAll;
 import org.apache.rocketmq.common.TopicConfig;
+import org.apache.rocketmq.common.namesrv.NamesrvConfig;
+import org.apache.rocketmq.common.protocol.body.BrokerMemberGroup;
+import org.apache.rocketmq.common.protocol.header.NotifyMinBrokerIdChangeRequestHeader;
+import org.apache.rocketmq.common.protocol.header.namesrv.UnRegisterBrokerRequestHeader;
 import org.apache.rocketmq.common.statictopic.TopicQueueMappingInfo;
 import org.apache.rocketmq.common.constant.LoggerName;
 import org.apache.rocketmq.common.constant.PermName;
 import org.apache.rocketmq.common.protocol.RequestCode;
 import org.apache.rocketmq.common.protocol.body.TopicConfigAndMappingSerializeWrapper;
+import org.apache.rocketmq.common.topic.TopicValidator;
 import org.apache.rocketmq.logging.InternalLogger;
 import org.apache.rocketmq.logging.InternalLoggerFactory;
 import org.apache.rocketmq.common.namesrv.RegisterBrokerResult;
@@ -46,27 +54,61 @@ import org.apache.rocketmq.common.protocol.route.BrokerData;
 import org.apache.rocketmq.common.protocol.route.QueueData;
 import org.apache.rocketmq.common.protocol.route.TopicRouteData;
 import org.apache.rocketmq.common.sysflag.TopicSysFlag;
+import org.apache.rocketmq.namesrv.NamesrvController;
 import org.apache.rocketmq.remoting.common.RemotingUtil;
+import org.apache.rocketmq.remoting.exception.RemotingConnectException;
+import org.apache.rocketmq.remoting.exception.RemotingSendRequestException;
+import org.apache.rocketmq.remoting.exception.RemotingTimeoutException;
+import org.apache.rocketmq.remoting.exception.RemotingTooMuchRequestException;
+import org.apache.rocketmq.remoting.protocol.RemotingCommand;
 
 public class RouteInfoManager {
     private static final InternalLogger log = InternalLoggerFactory.getLogger(LoggerName.NAMESRV_LOGGER_NAME);
-    private final static long BROKER_CHANNEL_EXPIRED_TIME = 1000 * 60 * 2;
+    private final static long DEFAULT_BROKER_CHANNEL_EXPIRED_TIME = 1000 * 60 * 2;
     private final ReadWriteLock lock = new ReentrantReadWriteLock();
-    private final HashMap<String/* topic */, List<QueueData>> topicQueueTable;
-    private final HashMap<String/* brokerName */, BrokerData> brokerAddrTable;
-    private final HashMap<String/* clusterName */, Set<String/* brokerName */>> clusterAddrTable;
-    private final HashMap<String/* brokerAddr */, BrokerLiveInfo> brokerLiveTable;
-    private final HashMap<String/* brokerAddr */, List<String>/* Filter Server */> filterServerTable;
-    private final HashMap<String/* topic */, Map<String/*brokerName*/, TopicQueueMappingInfo>> topicQueueMappingInfoTable;
+    private final Map<String/* topic */, Map<String, QueueData>> topicQueueTable;
+    private final Map<String/* brokerName */, BrokerData> brokerAddrTable;
+    private final Map<String/* clusterName */, Set<String/* brokerName */>> clusterAddrTable;
+    private final Map<BrokerAddrInfo/* brokerAddr */, BrokerLiveInfo> brokerLiveTable;
+    private final Map<BrokerAddrInfo/* brokerAddr */, List<String>/* Filter Server */> filterServerTable;
+    private final Map<String/* topic */, Map<String/*brokerName*/, TopicQueueMappingInfo>> topicQueueMappingInfoTable;
+
+    // Contains topic route data needed by clients
+    private final Map<String /* topic */, TopicRouteData> topicRouteDataMap;
+
+    private final BatchUnRegisterService unRegisterService;
+
+    private final NamesrvController namesrvController;
+    private final NamesrvConfig namesrvConfig;
+
+    public RouteInfoManager(final NamesrvConfig namesrvConfig, NamesrvController namesrvController) {
+        this.topicQueueTable = new ConcurrentHashMap<String, Map<String, QueueData>>(1024);
+        this.brokerAddrTable = new ConcurrentHashMap<String, BrokerData>(128);
+        this.clusterAddrTable = new ConcurrentHashMap<String, Set<String>>(32);
+        this.brokerLiveTable = new ConcurrentHashMap<BrokerAddrInfo, BrokerLiveInfo>(256);
+        this.filterServerTable = new ConcurrentHashMap<BrokerAddrInfo, List<String>>(256);
+        this.topicQueueMappingInfoTable = new ConcurrentHashMap<String, Map<String, TopicQueueMappingInfo>>(1024);
+        this.topicRouteDataMap = new ConcurrentHashMap<>(1024);
+        this.unRegisterService = new BatchUnRegisterService(this, namesrvConfig);
+        this.namesrvConfig = namesrvConfig;
+        this.namesrvController = namesrvController;
+    }
+
+    public void start() {
+        this.unRegisterService.start();
+    }
+
+    public void shutdown() {
+        this.unRegisterService.shutdown(true);
+    }
 
+    public boolean submitUnRegisterBrokerRequest(UnRegisterBrokerRequestHeader unRegisterRequest) {
+        return this.unRegisterService.submit(unRegisterRequest);
+    }
 
-    public RouteInfoManager() {
-        this.topicQueueTable = new HashMap<String, List<QueueData>>(1024);
-        this.brokerAddrTable = new HashMap<String, BrokerData>(128);
-        this.clusterAddrTable = new HashMap<String, Set<String>>(32);
-        this.brokerLiveTable = new HashMap<String, BrokerLiveInfo>(256);
-        this.filterServerTable = new HashMap<String, List<String>>(256);
-        this.topicQueueMappingInfoTable = new HashMap<String, Map<String, TopicQueueMappingInfo>>(1024);
+    // For test only
+    int blockedUnRegisterRequests() {
+        return this.unRegisterService.queueLength();
     }
 
     public byte[] getAllClusterInfo() {
@@ -76,11 +118,76 @@ public class RouteInfoManager {
         return clusterInfoSerializeWrapper.encode();
     }
 
+    public void registerTopic(final String topic, List<QueueData> queueDatas) {
+        if (queueDatas == null || queueDatas.isEmpty()) {
+            return;
+        }
+        try {
+            try {
+                this.lock.writeLock().lockInterruptibly();
+                if (this.topicQueueTable.containsKey(topic)) {
+                    log.info("Topic route already exist.{}, {}", topic, this.topicQueueTable.get(topic));
+                } else {
+                    // check and construct queue data map
+                    Map<String, QueueData> queueDataMap = new HashMap<>();
+                    for (QueueData queueData : queueDatas) {
+                        if (!this.brokerAddrTable.containsKey(queueData.getBrokerName())) {
+                            log.warn("Register topic contains illegal broker, {}, {}", topic, queueData);
+                            return;
+                        }
+                        queueDataMap.put(queueData.getBrokerName(), queueData);
+                    }
+
+                    this.topicQueueTable.put(topic, queueDataMap);
+                    log.info("Register topic route:{}, {}", topic, queueDatas);
+
+                    updateTopicRouteData(Sets.newHashSet(topic));
+                }
+            } finally {
+                this.lock.writeLock().unlock();
+            }
+        } catch (Exception e) {
+            log.error("registerTopic Exception", e);
+        }
+    }
+
     public void deleteTopic(final String topic) {
         try {
             try {
                 this.lock.writeLock().lockInterruptibly();
                 this.topicQueueTable.remove(topic);
+                this.updateTopicRouteData(Sets.newHashSet(topic));
+            } finally {
+                this.lock.writeLock().unlock();
+            }
+        } catch (Exception e) {
+            log.error("deleteTopic Exception", e);
+        }
+    }
+
+    public void deleteTopic(final String topic, final String clusterName) {
+        try {
+            try {
+                this.lock.writeLock().lockInterruptibly();
+                Set<String> brokerNames = this.clusterAddrTable.get(clusterName);
+                if (brokerNames != null
+                    && !brokerNames.isEmpty()) {
+                    Map<String, QueueData> queueDataMap = this.topicQueueTable.get(topic);
+                    for (String brokerName : brokerNames) {
+                        final QueueData removedQD = queueDataMap.remove(brokerName);
+                        if (removedQD != null) {
+                            log.info("deleteTopic, remove one broker's topic {} {} {}", brokerName, topic,
+                                removedQD);
+                        }
+
+                        if (queueDataMap.isEmpty()) {
+                            log.info("deleteTopic, remove the topic all queue {} {}", brokerName, topic);
+                            this.topicQueueTable.remove(topic);
+                            break;
+                        }
+                    }
+                    this.updateTopicRouteData(Sets.newHashSet(topic));
+                }
             } finally {
                 this.lock.writeLock().unlock();
             }
@@ -111,6 +218,21 @@ public class RouteInfoManager {
         final String brokerName,
         final long brokerId,
         final String haServerAddr,
+        final Long timeoutMillis,
+        final TopicConfigSerializeWrapper topicConfigWrapper,
+        final List<String> filterServerList,
+        final Channel channel) {
+        return registerBroker(clusterName, brokerAddr, brokerName, brokerId, haServerAddr, timeoutMillis, false, topicConfigWrapper, filterServerList, channel);
+    }
+
+    public RegisterBrokerResult registerBroker(
+        final String clusterName,
+        final String brokerAddr,
+        final String brokerName,
+        final long brokerId,
+        final String haServerAddr,
+        final Long timeoutMillis,
+        final Boolean enableActingMaster,
         final TopicConfigSerializeWrapper topicConfigWrapper,
         final List<String> filterServerList,
         final Channel channel) {
@@ -119,11 +241,7 @@ public class RouteInfoManager {
             try {
                 this.lock.writeLock().lockInterruptibly();
 
-                Set<String> brokerNames = this.clusterAddrTable.get(clusterName);
-                if (null == brokerNames) {
-                    brokerNames = new HashSet<String>();
-                    this.clusterAddrTable.put(clusterName, brokerNames);
-                }
+                Set<String> brokerNames = this.clusterAddrTable.computeIfAbsent(clusterName, k -> new HashSet<>());
                 brokerNames.add(brokerName);
 
                 boolean registerFirst = false;
@@ -131,37 +249,81 @@ public class RouteInfoManager {
                 BrokerData brokerData = this.brokerAddrTable.get(brokerName);
                 if (null == brokerData) {
                     registerFirst = true;
-                    brokerData = new BrokerData(clusterName, brokerName, new HashMap<Long, String>());
+                    brokerData = new BrokerData(clusterName, brokerName, new HashMap<>());
                     this.brokerAddrTable.put(brokerName, brokerData);
                 }
+
+                boolean isOldVersionBroker = enableActingMaster == null;
+                brokerData.setEnableActingMaster(!isOldVersionBroker && enableActingMaster);
+
                 Map<Long, String> brokerAddrsMap = brokerData.getBrokerAddrs();
+
+                boolean isMinBrokerIdChanged = false;
+                long prevMinBrokerId = 0;
+                if (!brokerAddrsMap.isEmpty()) {
+                    prevMinBrokerId = Collections.min(brokerAddrsMap.keySet());
+                }
+
+                if (brokerId < prevMinBrokerId) {
+                    isMinBrokerIdChanged = true;
+                }
+
                 //Switch slave to master: first remove <1, IP:PORT> in namesrv, then add <0, IP:PORT>
                 //The same IP:PORT must only have one record in brokerAddrTable
-                Iterator<Entry<Long, String>> it = brokerAddrsMap.entrySet().iterator();
-                while (it.hasNext()) {
-                    Entry<Long, String> item = it.next();
-                    if (null != brokerAddr && brokerAddr.equals(item.getValue()) && brokerId != item.getKey()) {
-                        it.remove();
+                brokerAddrsMap.entrySet().removeIf(item -> null != brokerAddr && brokerAddr.equals(item.getValue()) && brokerId != item.getKey());
+
+                //If Local brokerId stateVersion bigger than the registering one,
+                String oldBrokerAddr = brokerAddrsMap.get(brokerId);
+                if (null != oldBrokerAddr && !oldBrokerAddr.equals(brokerAddr)) {
+                    BrokerLiveInfo oldBrokerInfo = brokerLiveTable.get(new BrokerAddrInfo(clusterName, oldBrokerAddr));
+
+                    if (null != oldBrokerInfo) {
+                        long oldStateVersion = oldBrokerInfo.getDataVersion().getStateVersion();
+                        long newStateVersion = topicConfigWrapper.getDataVersion().getStateVersion();
+                        if (oldStateVersion > newStateVersion) {
+                            log.warn("Registered Broker conflicts with the existed one, just ignore.: Cluster:{}, BrokerName:{}, BrokerId:{}, " +
+                                    "Old BrokerAddr:{}, Old Version:{}, New BrokerAddr:{}, New Version:{}.",
+                                clusterName, brokerName, brokerId, oldBrokerAddr, oldStateVersion, brokerAddr, newStateVersion);
+                            //Remove the rejected brokerAddr from brokerLiveTable.
+                            brokerLiveTable.remove(new BrokerAddrInfo(clusterName, brokerAddr));
+                            return result;
+                        }
                     }
                 }
 
-                String oldAddr = brokerData.getBrokerAddrs().put(brokerId, brokerAddr);
+                String oldAddr = brokerAddrsMap.put(brokerId, brokerAddr);
                 registerFirst = registerFirst || (null == oldAddr);
 
-                if (null != topicConfigWrapper
-                    && MixAll.MASTER_ID == brokerId) {
-                    if (this.isBrokerTopicConfigChanged(brokerAddr, topicConfigWrapper.getDataVersion())
-                        || registerFirst) {
-                        ConcurrentMap<String, TopicConfig> tcTable =
-                            topicConfigWrapper.getTopicConfigTable();
-                        if (tcTable != null) {
-                            for (Map.Entry<String, TopicConfig> entry : tcTable.entrySet()) {
-                                this.createAndUpdateQueueData(brokerName, entry.getValue());
+                Set<String> configChangedTopics = new HashSet<>();
+
+                boolean isMaster = MixAll.MASTER_ID == brokerId;
+                boolean isPrimeSlave = !isOldVersionBroker && !isMaster
+                    && brokerId == Collections.min(brokerAddrsMap.keySet());
+
+                if (null != topicConfigWrapper && (isMaster || isPrimeSlave)) {
+
+                    ConcurrentMap<String, TopicConfig> tcTable =
+                        topicConfigWrapper.getTopicConfigTable();
+                    if (tcTable != null) {
+                        for (Map.Entry<String, TopicConfig> entry : tcTable.entrySet()) {
+                            final TopicConfig topicConfig = entry.getValue();
+                            String topicName = topicConfig.getTopicName();
+                            if (registerFirst || this.isTopicConfigChanged(clusterName, brokerAddr,
+                                topicConfigWrapper.getDataVersion(), brokerName, topicName)) {
+                                configChangedTopics.add(topicName);
+
+                                if (isPrimeSlave) {
+                                    // Wipe the write perm for prime slave
+                                    topicConfig.setPerm(topicConfig.getPerm() & (~PermName.PERM_WRITE));
+                                }
+
+                                this.createAndUpdateQueueData(brokerName, topicConfig);
                             }
                         }
+                    }
 
+                    if (this.isBrokerTopicConfigChanged(clusterName, brokerAddr, topicConfigWrapper.getDataVersion()) || registerFirst) {
                         TopicConfigAndMappingSerializeWrapper mappingSerializeWrapper = TopicConfigAndMappingSerializeWrapper.from(topicConfigWrapper);
-
                         Map<String, TopicQueueMappingInfo> topicQueueMappingInfoMap = mappingSerializeWrapper.getTopicQueueMappingInfoMap();
                         //the topicQueueMappingInfoMap should never be null, but can be empty
                         for (Map.Entry<String, TopicQueueMappingInfo> entry : topicQueueMappingInfoMap.entrySet()) {
@@ -175,34 +337,49 @@ public class RouteInfoManager {
                     }
                 }
 
-                BrokerLiveInfo prevBrokerLiveInfo = this.brokerLiveTable.put(brokerAddr,
+                BrokerAddrInfo brokerAddrInfo = new BrokerAddrInfo(clusterName, brokerAddr);
+                BrokerLiveInfo prevBrokerLiveInfo = this.brokerLiveTable.put(brokerAddrInfo,
                     new BrokerLiveInfo(
                         System.currentTimeMillis(),
-                        topicConfigWrapper != null ? topicConfigWrapper.getDataVersion() : new DataVersion(),
+                        timeoutMillis == null ? DEFAULT_BROKER_CHANNEL_EXPIRED_TIME : timeoutMillis,
+                        topicConfigWrapper == null ? new DataVersion() : topicConfigWrapper.getDataVersion(),
                         channel,
                         haServerAddr));
                 if (null == prevBrokerLiveInfo) {
-                    log.info("new broker registered, {} HAServer: {}", brokerAddr, haServerAddr);
+                    log.info("new broker registered, {} HAService: {}", brokerAddrInfo, haServerAddr);
                 }
 
                 if (filterServerList != null) {
                     if (filterServerList.isEmpty()) {
-                        this.filterServerTable.remove(brokerAddr);
+                        this.filterServerTable.remove(brokerAddrInfo);
                     } else {
-                        this.filterServerTable.put(brokerAddr, filterServerList);
+                        this.filterServerTable.put(brokerAddrInfo, filterServerList);
                     }
                 }
 
                 if (MixAll.MASTER_ID != brokerId) {
                     String masterAddr = brokerData.getBrokerAddrs().get(MixAll.MASTER_ID);
                     if (masterAddr != null) {
-                        BrokerLiveInfo brokerLiveInfo = this.brokerLiveTable.get(masterAddr);
-                        if (brokerLiveInfo != null) {
-                            result.setHaServerAddr(brokerLiveInfo.getHaServerAddr());
+                        BrokerAddrInfo masterAddrInfo = new BrokerAddrInfo(clusterName, masterAddr);
+                        BrokerLiveInfo masterLiveInfo = this.brokerLiveTable.get(masterAddrInfo);
+                        if (masterLiveInfo != null) {
+                            result.setHaServerAddr(masterLiveInfo.getHaServerAddr());
                             result.setMasterAddr(masterAddr);
                         }
                     }
                 }
+
+                if (registerFirst && MixAll.MASTER_ID != brokerId) {
+                    configChangedTopics = this.topicSetOfBrokerName(brokerName);
+                }
+
+                // Update topicRouteDataMap
+                updateTopicRouteData(configChangedTopics);
+
+                if (isMinBrokerIdChanged && brokerData.isEnableActingMaster()) {
+                    notifyMinBrokerIdChanged(brokerAddrsMap, null,
+                        this.brokerLiveTable.get(brokerAddrInfo).getHaServerAddr());
+                }
             } finally {
                 this.lock.writeLock().unlock();
             }
@@ -213,21 +390,109 @@ public class RouteInfoManager {
         return result;
     }
 
-    public boolean isBrokerTopicConfigChanged(final String brokerAddr, final DataVersion dataVersion) {
-        DataVersion prev = queryBrokerTopicConfig(brokerAddr);
+    private void updateTopicRouteData(final Set<String> changedTopics) {
+        for (final String changedTopic : changedTopics) {
+            if (!this.topicQueueTable.containsKey(changedTopic)) {
+                // This topic doesn't have any queue data route info
+                // Just remove it from topicRouteDataMap
+                this.topicRouteDataMap.remove(changedTopic);
+                continue;
+            }
+
+            TopicRouteData topicRouteData = new TopicRouteData();
+            boolean foundBrokerData = false;
+            List<BrokerData> brokerDataList = topicRouteData.getBrokerDatas();
+            HashMap<String, List<String>> filterServerMap = topicRouteData.getFilterServerTable();
+
+            Map<String, QueueData> queueDataMap = this.topicQueueTable.get(changedTopic);
+            if (queueDataMap != null) {
+                // Deep copy the queue data list
+                List<QueueData> queueDataList = topicRouteData.getQueueDatas();
+                for (final QueueData queueData : queueDataMap.values()) {
+                    queueDataList.add(new QueueData(queueData));
+                }
+
+                Set<String> brokerNameSet = new HashSet<String>(queueDataMap.keySet());
+
+                for (String brokerName : brokerNameSet) {
+                    BrokerData brokerData = this.brokerAddrTable.get(brokerName);
+                    if (null != brokerData) {
+                        BrokerData brokerDataClone = new BrokerData(brokerData);
+                        brokerDataList.add(brokerDataClone);
+                        foundBrokerData = true;
+
+                        for (final String brokerAddr : brokerDataClone.getBrokerAddrs().values()) {
+                            BrokerAddrInfo brokerAddrInfo = new BrokerAddrInfo(brokerDataClone.getCluster(), brokerAddr);
+                            List<String> filterServerList = this.filterServerTable.get(brokerAddrInfo);
+                            if (filterServerList != null) {
+                                filterServerMap.put(brokerAddr, new ArrayList<String>(filterServerList));
+                            }
+                        }
+                    }
+                }
+            }
+
+            if (foundBrokerData) {
+                this.topicRouteDataMap.put(changedTopic, topicRouteData);
+            }
+        }
+    }
+
+    public BrokerMemberGroup getBrokerMemberGroup(String clusterName, String brokerName) {
+        BrokerMemberGroup groupMember = new BrokerMemberGroup(clusterName, brokerName);
+        try {
+            try {
+                this.lock.readLock().lockInterruptibly();
+                final BrokerData brokerData = this.brokerAddrTable.get(brokerName);
+                if (brokerData != null) {
+                    groupMember.getBrokerAddrs().putAll(brokerData.getBrokerAddrs());
+                }
+            } finally {
+                this.lock.readLock().unlock();
+            }
+        } catch (Exception e) {
+            log.error("Get broker member group exception", e);
+        }
+        return groupMember;
+    }
+
+    public boolean isBrokerTopicConfigChanged(final String clusterName, final String brokerAddr,
+        final DataVersion dataVersion) {
+        DataVersion prev = queryBrokerTopicConfig(clusterName, brokerAddr);
         return null == prev || !prev.equals(dataVersion);
     }
 
-    public DataVersion queryBrokerTopicConfig(final String brokerAddr) {
-        BrokerLiveInfo prev = this.brokerLiveTable.get(brokerAddr);
+    public boolean isTopicConfigChanged(final String clusterName, final String brokerAddr,
+        final DataVersion dataVersion, String brokerName, String topic) {
+        boolean isChange = isBrokerTopicConfigChanged(clusterName, brokerAddr, dataVersion);
+        if (isChange) {
+            return true;
+        }
+        final Map<String, QueueData> queueDataMap = this.topicQueueTable.get(topic);
+        if (queueDataMap == null || queueDataMap.isEmpty()) {
+            return true;
+        }
+
+        if (queueDataMap.containsKey(brokerName)) {
+            // The topicQueueTable already contains the broker
+            return false;
+        }
+
+        return true;
+    }
+
+    public DataVersion queryBrokerTopicConfig(final String clusterName, final String brokerAddr) {
+        BrokerAddrInfo addrInfo = new BrokerAddrInfo(clusterName, brokerAddr);
+        BrokerLiveInfo prev = this.brokerLiveTable.get(addrInfo);
         if (prev != null) {
             return prev.getDataVersion();
         }
         return null;
     }
 
-    public void updateBrokerInfoUpdateTimestamp(final String brokerAddr) {
-        BrokerLiveInfo prev = this.brokerLiveTable.get(brokerAddr);
+    public void updateBrokerInfoUpdateTimestamp(final String clusterName, final String brokerAddr) {
+        BrokerAddrInfo addrInfo = new BrokerAddrInfo(clusterName, brokerAddr);
+        BrokerLiveInfo prev = this.brokerLiveTable.get(addrInfo);
         if (prev != null) {
             prev.setLastUpdateTimestamp(System.currentTimeMillis());
         }
@@ -241,81 +506,79 @@ public class RouteInfoManager {
         queueData.setPerm(topicConfig.getPerm());
         queueData.setTopicSysFlag(topicConfig.getTopicSysFlag());
 
-        List<QueueData> queueDataList = this.topicQueueTable.get(topicConfig.getTopicName());
-        if (null == queueDataList) {
-            queueDataList = new LinkedList<QueueData>();
-            queueDataList.add(queueData);
-            this.topicQueueTable.put(topicConfig.getTopicName(), queueDataList);
+        Map<String, QueueData> queueDataMap = this.topicQueueTable.get(topicConfig.getTopicName());
+        if (null == queueDataMap) {
+            queueDataMap = new HashMap<>();
+            queueDataMap.put(brokerName, queueData);
+            this.topicQueueTable.put(topicConfig.getTopicName(), queueDataMap);
             log.info("new topic registered, {} {}", topicConfig.getTopicName(), queueData);
         } else {
-            boolean addNewOne = true;
-
-            Iterator<QueueData> it = queueDataList.iterator();
-            while (it.hasNext()) {
-                QueueData qd = it.next();
-                if (qd.getBrokerName().equals(brokerName)) {
-                    if (qd.equals(queueData)) {
-                        addNewOne = false;
-                    } else {
-                        log.info("topic changed, {} OLD: {} NEW: {}", topicConfig.getTopicName(), qd,
-                            queueData);
-                        it.remove();
-                    }
-                }
-            }
-
-            if (addNewOne) {
-                queueDataList.add(queueData);
+            final QueueData existedQD = queueDataMap.get(brokerName);
+            if (existedQD == null) {
+                queueDataMap.put(brokerName, queueData);
+            } else if (!existedQD.equals(queueData)) {
+                log.info("topic changed, {} OLD: {} NEW: {}", topicConfig.getTopicName(), existedQD,
+                    queueData);
+                queueDataMap.put(brokerName, queueData);
             }
         }
     }
 
     public int wipeWritePermOfBrokerByLock(final String brokerName) {
-        return operateWritePermOfBrokerByLock(brokerName, RequestCode.WIPE_WRITE_PERM_OF_BROKER);
-    }
+        try {
+            try {
+                this.lock.writeLock().lockInterruptibly();
+                return operateWritePermOfBroker(brokerName, RequestCode.WIPE_WRITE_PERM_OF_BROKER);
+            } finally {
+                this.lock.writeLock().unlock();
+            }
+        } catch (Exception e) {
+            log.error("wipeWritePermOfBrokerByLock Exception", e);
+        }
 
-    public int addWritePermOfBrokerByLock(final String brokerName) {
-        return operateWritePermOfBrokerByLock(brokerName, RequestCode.ADD_WRITE_PERM_OF_BROKER);
+        return 0;
     }
 
-    private int operateWritePermOfBrokerByLock(final String brokerName, final int requestCode) {
+    public int addWritePermOfBrokerByLock(final String brokerName) {
         try {
             try {
                 this.lock.writeLock().lockInterruptibly();
-                return operateWritePermOfBroker(brokerName, requestCode);
+                return operateWritePermOfBroker(brokerName, RequestCode.ADD_WRITE_PERM_OF_BROKER);
             } finally {
                 this.lock.writeLock().unlock();
             }
         } catch (Exception e) {
-            log.error("operateWritePermOfBrokerByLock Exception", e);
+            log.error("wipeWritePermOfBrokerByLock Exception", e);
         }
-
         return 0;
     }
 
-
     private int operateWritePermOfBroker(final String brokerName, final int requestCode) {
+        Set<String> changedTopics = new HashSet<>();
         int topicCnt = 0;
-        for (Entry<String, List<QueueData>> entry : this.topicQueueTable.entrySet()) {
-            List<QueueData> qdList = entry.getValue();
-
-            for (QueueData qd : qdList) {
-                if (qd.getBrokerName().equals(brokerName)) {
-                    int perm = qd.getPerm();
-                    switch (requestCode) {
-                        case RequestCode.WIPE_WRITE_PERM_OF_BROKER:
-                            perm &= ~PermName.PERM_WRITE;
-                            break;
-                        case RequestCode.ADD_WRITE_PERM_OF_BROKER:
-                            perm = PermName.PERM_READ | PermName.PERM_WRITE;
-                            break;
-                    }
-                    qd.setPerm(perm);
-                    topicCnt++;
+
+        Iterator<Entry<String, Map<String, QueueData>>> itTopic = this.topicQueueTable.entrySet().iterator();
+        while (itTopic.hasNext()) {
+            Entry<String, Map<String, QueueData>> entry = itTopic.next();
+            Map<String, QueueData> qdMap = entry.getValue();
+
+            final QueueData qd = qdMap.get(brokerName);
+            if (qd != null) {
+                changedTopics.add(entry.getKey());
+                int perm = qd.getPerm();
+                switch (requestCode) {
+                    case RequestCode.WIPE_WRITE_PERM_OF_BROKER:
+                        perm &= ~PermName.PERM_WRITE;
+                        break;
+                    case RequestCode.ADD_WRITE_PERM_OF_BROKER:
+                        perm = PermName.PERM_READ | PermName.PERM_WRITE;
+                        break;
                 }
+                qd.setPerm(perm);
+                topicCnt++;
             }
         }
-
+        this.updateTopicRouteData(changedTopics);
         return topicCnt;
     }
 
@@ -324,52 +587,89 @@ public class RouteInfoManager {
         final String brokerAddr,
         final String brokerName,
         final long brokerId) {
+        UnRegisterBrokerRequestHeader unRegisterBrokerRequest = new UnRegisterBrokerRequestHeader();
+        unRegisterBrokerRequest.setClusterName(clusterName);
+        unRegisterBrokerRequest.setBrokerAddr(brokerAddr);
+        unRegisterBrokerRequest.setBrokerName(brokerName);
+        unRegisterBrokerRequest.setBrokerId(brokerId);
+
+        unRegisterBroker(Sets.newHashSet(unRegisterBrokerRequest));
+    }
+
+    public void unRegisterBroker(Set<UnRegisterBrokerRequestHeader> unRegisterRequests) {
         try {
             try {
+                Set<String> removedBroker = new HashSet<>();
+                Set<String> reducedBroker = new HashSet<>();
+                Map<String, BrokerStatusChangeInfo> needNotifyBrokerMap = new HashMap<>();
+
                 this.lock.writeLock().lockInterruptibly();
-                BrokerLiveInfo brokerLiveInfo = this.brokerLiveTable.remove(brokerAddr);
-                log.info("unregisterBroker, remove from brokerLiveTable {}, {}",
-                    brokerLiveInfo != null ? "OK" : "Failed",
-                    brokerAddr
-                );
+                for (final UnRegisterBrokerRequestHeader unRegisterRequest : unRegisterRequests) {
+                    final String brokerName = unRegisterRequest.getBrokerName();
+                    final String clusterName = unRegisterRequest.getClusterName();
 
-                this.filterServerTable.remove(brokerAddr);
+                    BrokerAddrInfo brokerAddrInfo = new BrokerAddrInfo(clusterName, unRegisterRequest.getBrokerAddr());
 
-                boolean removeBrokerName = false;
-                BrokerData brokerData = this.brokerAddrTable.get(brokerName);
-                if (null != brokerData) {
-                    String addr = brokerData.getBrokerAddrs().remove(brokerId);
-                    log.info("unregisterBroker, remove addr from brokerAddrTable {}, {}",
-                        addr != null ? "OK" : "Failed",
-                        brokerAddr
+                    BrokerLiveInfo brokerLiveInfo = this.brokerLiveTable.remove(brokerAddrInfo);
+                    log.info("unregisterBroker, remove from brokerLiveTable {}, {}",
+                        brokerLiveInfo != null ? "OK" : "Failed",
+                        brokerAddrInfo
                     );
 
-                    if (brokerData.getBrokerAddrs().isEmpty()) {
-                        this.brokerAddrTable.remove(brokerName);
-                        log.info("unregisterBroker, remove name from brokerAddrTable OK, {}",
-                            brokerName
+                    this.filterServerTable.remove(brokerAddrInfo);
+
+                    boolean removeBrokerName = false;
+                    boolean isMinBrokerIdChanged = false;
+                    BrokerData brokerData = this.brokerAddrTable.get(brokerName);
+                    if (null != brokerData) {
+                        if (!brokerData.getBrokerAddrs().isEmpty() &&
+                            unRegisterRequest.getBrokerId().equals(Collections.min(brokerData.getBrokerAddrs().keySet()))) {
+                            isMinBrokerIdChanged = true;
+                        }
+                        String addr = brokerData.getBrokerAddrs().remove(unRegisterRequest.getBrokerId());
+                        log.info("unregisterBroker, remove addr from brokerAddrTable {}, {}",
+                            addr != null ? "OK" : "Failed",
+                            brokerAddrInfo
                         );
+                        if (brokerData.getBrokerAddrs().isEmpty()) {
+                            this.brokerAddrTable.remove(brokerName);
+                            log.info("unregisterBroker, remove name from brokerAddrTable OK, {}",
+                                brokerName
+                            );
 
-                        removeBrokerName = true;
+                            removeBrokerName = true;
+                        } else if (isMinBrokerIdChanged) {
+                            needNotifyBrokerMap.put(brokerName, new BrokerStatusChangeInfo(
+                                brokerData.getBrokerAddrs(), addr, null));
+                        }
                     }
-                }
-
-                if (removeBrokerName) {
-                    Set<String> nameSet = this.clusterAddrTable.get(clusterName);
-                    if (nameSet != null) {
-                        boolean removed = nameSet.remove(brokerName);
-                        log.info("unregisterBroker, remove name from clusterAddrTable {}, {}",
-                            removed ? "OK" : "Failed",
-                            brokerName);
 
-                        if (nameSet.isEmpty()) {
-                            this.clusterAddrTable.remove(clusterName);
-                            log.info("unregisterBroker, remove cluster from clusterAddrTable {}",
-                                clusterName
-                            );
+                    if (removeBrokerName) {
+                        Set<String> nameSet = this.clusterAddrTable.get(clusterName);
+                        if (nameSet != null) {
+                            boolean removed = nameSet.remove(brokerName);
+                            log.info("unregisterBroker, remove name from clusterAddrTable {}, {}",
+                                removed ? "OK" : "Failed",
+                                brokerName);
+
+                            if (nameSet.isEmpty()) {
+                                this.clusterAddrTable.remove(clusterName);
+                                log.info("unregisterBroker, remove cluster from clusterAddrTable {}",
+                                    clusterName
+                                );
+                            }
                         }
+                        removedBroker.add(brokerName);
+                    } else {
+                        reducedBroker.add(brokerName);
                     }
-                    this.removeTopicByBrokerName(brokerName);
+                }
+
+                Set<String> changedTopics = cleanTopicByUnRegisterRequests(removedBroker, reducedBroker);
+                this.updateTopicRouteData(changedTopics);
+
+                if (!needNotifyBrokerMap.isEmpty()) {
+                    notifyMinBrokerIdChanged(needNotifyBrokerMap);
                 }
             } finally {
                 this.lock.writeLock().unlock();
@@ -379,114 +679,189 @@ public class RouteInfoManager {
         }
     }
 
-    private void removeTopicByBrokerName(final String brokerName) {
-        Iterator<Entry<String, List<QueueData>>> itMap = this.topicQueueTable.entrySet().iterator();
+    private Set<String> cleanTopicByUnRegisterRequests(Set<String> removedBroker, Set<String> reducedBroker) {
+        Set<String> changedTopics = new HashSet<>();
+        Iterator<Entry<String, Map<String, QueueData>>> itMap = this.topicQueueTable.entrySet().iterator();
         while (itMap.hasNext()) {
-            Entry<String, List<QueueData>> entry = itMap.next();
+            Entry<String, Map<String, QueueData>> entry = itMap.next();
 
             String topic = entry.getKey();
-            List<QueueData> queueDataList = entry.getValue();
-            Iterator<QueueData> it = queueDataList.iterator();
-            while (it.hasNext()) {
-                QueueData qd = it.next();
-                if (qd.getBrokerName().equals(brokerName)) {
-                    log.info("removeTopicByBrokerName, remove one broker's topic {} {}", topic, qd);
-                    it.remove();
+            Map<String, QueueData> queueDataMap = entry.getValue();
+
+            for (final String brokerName : removedBroker) {
+                final QueueData removedQD = queueDataMap.remove(brokerName);
+                if (removedQD != null) {
+                    changedTopics.add(topic);
+                    log.info("removeTopicByBrokerName, remove one broker's topic {} {}", topic, removedQD);
                 }
             }
 
-            if (queueDataList.isEmpty()) {
+            if (queueDataMap.isEmpty()) {
                 log.info("removeTopicByBrokerName, remove the topic all queue {}", topic);
                 itMap.remove();
             }
+
+            for (final String brokerName : reducedBroker) {
+                final QueueData queueData = queueDataMap.get(brokerName);
+
+                if (queueData != null) {
+                    changedTopics.add(topic);
+                    if (this.brokerAddrTable.get(brokerName).isEnableActingMaster()) {
+                        // Master has been unregistered, wipe the write perm
+                        if (isNoMasterExists(brokerName)) {
+                            queueData.setPerm(queueData.getPerm() & (~PermName.PERM_WRITE));
+                        }
+                    }
+                }
+            }
         }
+
+        return changedTopics;
+    }
+
+    private boolean isNoMasterExists(String brokerName) {
+        final BrokerData brokerData = this.brokerAddrTable.get(brokerName);
+        if (brokerData == null) {
+            return true;
+        }
+
+        if (brokerData.getBrokerAddrs().size() == 0) {
+            return true;
+        }
+
+        return Collections.min(brokerData.getBrokerAddrs().keySet()) > 0;
+    }
+
+    private Set<String> topicSetOfBrokerName(final String brokerName) {
+        Set<String> topicOfBroker = new HashSet<>();
+        for (final Entry<String, Map<String, QueueData>> entry : this.topicQueueTable.entrySet()) {
+            if (entry.getValue().containsKey(brokerName)) {
+                topicOfBroker.add(entry.getKey());
+            }
+        }
+        return topicOfBroker;
     }
 
     public TopicRouteData pickupTopicRouteData(final String topic) {
-        TopicRouteData topicRouteData = new TopicRouteData();
-        boolean foundQueueData = false;
-        boolean foundBrokerData = false;
-        Set<String> brokerNameSet = new HashSet<String>();
-        List<BrokerData> brokerDataList = new LinkedList<BrokerData>();
-        topicRouteData.setBrokerDatas(brokerDataList);
+        if (topic != null) {
+            final TopicRouteData routeData = this.topicRouteDataMap.get(topic);
+            if (routeData == null) {
+                return null;
+            }
+            routeData.setTopicQueueMappingByBroker(this.topicQueueMappingInfoTable.get(topic));
+            log.debug("pickupTopicRouteData {} {}", topic, routeData);
 
-        HashMap<String, List<String>> filterServerMap = new HashMap<String, List<String>>();
-        topicRouteData.setFilterServerTable(filterServerMap);
-        topicRouteData.setTopicQueueMappingByBroker(this.topicQueueMappingInfoTable.get(topic));
+            if (topic.startsWith(TopicValidator.SYNC_BROKER_MEMBER_GROUP_PREFIX)) {
+                return routeData;
+            }
 
-        try {
-            try {
-                this.lock.readLock().lockInterruptibly();
-                List<QueueData> queueDataList = this.topicQueueTable.get(topic);
-                if (queueDataList != null) {
-                    topicRouteData.setQueueDatas(queueDataList);
-                    foundQueueData = true;
+            if (!namesrvConfig.isSupportActingMaster()) {
+                return routeData;
+            }
 
-                    Iterator<QueueData> it = queueDataList.iterator();
-                    while (it.hasNext()) {
-                        QueueData qd = it.next();
-                        brokerNameSet.add(qd.getBrokerName());
-                    }
+            if (routeData.getBrokerDatas().size() == 0 || routeData.getQueueDatas().size() == 0) {
+                return routeData;
+            }
 
-                    for (String brokerName : brokerNameSet) {
-                        BrokerData brokerData = this.brokerAddrTable.get(brokerName);
-                        if (null != brokerData) {
-                            BrokerData brokerDataClone = new BrokerData(brokerData.getCluster(), brokerData.getBrokerName(), (HashMap<Long, String>) brokerData
-                                .getBrokerAddrs().clone());
-                            brokerDataList.add(brokerDataClone);
-                            foundBrokerData = true;
-                            for (final String brokerAddr : brokerDataClone.getBrokerAddrs().values()) {
-                                List<String> filterServerList = this.filterServerTable.get(brokerAddr);
-                                filterServerMap.put(brokerAddr, filterServerList);
-                            }
+            boolean needActingMaster = false;
+
+            for (final BrokerData brokerData : routeData.getBrokerDatas()) {
+                if (brokerData.getBrokerAddrs().size() != 0
+                    && !brokerData.getBrokerAddrs().containsKey(MixAll.MASTER_ID)) {
+                    needActingMaster = true;
+                    break;
+                }
+            }
+
+            if (!needActingMaster) {
+                return routeData;
+            }
+
+            final TopicRouteData cloneTopicRouteData = routeData.deepCloneTopicRouteData();
+
+            for (final BrokerData brokerData : cloneTopicRouteData.getBrokerDatas()) {
+                final HashMap<Long, String> brokerAddrs = brokerData.getBrokerAddrs();
+                if (brokerAddrs.size() == 0 || brokerAddrs.containsKey(MixAll.MASTER_ID) || !brokerData.isEnableActingMaster()) {
+                    continue;
+                }
+
+                // No master
+                for (final QueueData queueData : cloneTopicRouteData.getQueueDatas()) {
+                    if (queueData.getBrokerName().equals(brokerData.getBrokerName())) {
+                        if (!PermName.isWriteable(queueData.getPerm())) {
+                            final Long minBrokerId = Collections.min(brokerAddrs.keySet());
+                            final String actingMasterAddr = brokerAddrs.remove(minBrokerId);
+                            brokerAddrs.put(MixAll.MASTER_ID, actingMasterAddr);
                         }
+                        break;
                     }
                 }
-            } finally {
-                this.lock.readLock().unlock();
-            }
-        } catch (Exception e) {
-            log.error("pickupTopicRouteData Exception", e);
-        }
 
-        log.debug("pickupTopicRouteData {} {}", topic, topicRouteData);
+            }
 
-        if (foundBrokerData && foundQueueData) {
-            return topicRouteData;
+            return cloneTopicRouteData;
         }
-
         return null;
     }
 
     public void scanNotActiveBroker() {
-        Iterator<Entry<String, BrokerLiveInfo>> it = this.brokerLiveTable.entrySet().iterator();
-        while (it.hasNext()) {
-            Entry<String, BrokerLiveInfo> next = it.next();
-            long last = next.getValue().getLastUpdateTimestamp();
-            if ((last + BROKER_CHANNEL_EXPIRED_TIME) < System.currentTimeMillis()) {
-                RemotingUtil.closeChannel(next.getValue().getChannel());
-                it.remove();
-                log.warn("The broker channel expired, {} {}ms", next.getKey(), BROKER_CHANNEL_EXPIRED_TIME);
-                this.onChannelDestroy(next.getKey(), next.getValue().getChannel());
+        try {
+            log.info("start scanNotActiveBroker");
+            for (Entry<BrokerAddrInfo, BrokerLiveInfo> next : this.brokerLiveTable.entrySet()) {
+                long last = next.getValue().getLastUpdateTimestamp();
+                long timeoutMillis = next.getValue().getHeartbeatTimeoutMillis();
+                if ((last + timeoutMillis) < System.currentTimeMillis()) {
+                    RemotingUtil.closeChannel(next.getValue().getChannel());
+                    log.warn("The broker channel expired, {} {}ms", next.getKey(), timeoutMillis);
+                    this.onChannelDestroy(next.getKey());
+                }
+            }
+        } catch (Exception e) {
+            log.error("scanNotActiveBroker exception", e);
+        }
+    }
+
+    public void onChannelDestroy(BrokerAddrInfo brokerAddrInfo) {
+        UnRegisterBrokerRequestHeader unRegisterRequest = new UnRegisterBrokerRequestHeader();
+        boolean needUnRegister = false;
+        if (brokerAddrInfo != null) {
+            try {
+                try {
+                    this.lock.readLock().lockInterruptibly();
+                    needUnRegister = setupUnRegisterRequest(unRegisterRequest, brokerAddrInfo);
+                } finally {
+                    this.lock.readLock().unlock();
+                }
+            } catch (Exception e) {
+                log.error("onChannelDestroy Exception", e);
             }
         }
+
+        if (needUnRegister) {
+            boolean result = this.submitUnRegisterBrokerRequest(unRegisterRequest);
+            log.info("the broker's channel destroyed, submit the unregister request at once, " +
+                "broker info: {}, submit result: {}", unRegisterRequest, result);
+        }
     }
 
-    public void onChannelDestroy(String remoteAddr, Channel channel) {
-        String brokerAddrFound = null;
+    public void onChannelDestroy(Channel channel) {
+        UnRegisterBrokerRequestHeader unRegisterRequest = new UnRegisterBrokerRequestHeader();
+        BrokerAddrInfo brokerAddrFound = null;
+        boolean needUnRegister = false;
         if (channel != null) {
             try {
                 try {
                     this.lock.readLock().lockInterruptibly();
-                    Iterator<Entry<String, BrokerLiveInfo>> itBrokerLiveTable =
-                        this.brokerLiveTable.entrySet().iterator();
-                    while (itBrokerLiveTable.hasNext()) {
-                        Entry<String, BrokerLiveInfo> entry = itBrokerLiveTable.next();
+                    for (Entry<BrokerAddrInfo, BrokerLiveInfo> entry : this.brokerLiveTable.entrySet()) {
                         if (entry.getValue().getChannel() == channel) {
                             brokerAddrFound = entry.getKey();
                             break;
                         }
                     }
+
+                    if (brokerAddrFound != null) {
+                        needUnRegister = setupUnRegisterRequest(unRegisterRequest, brokerAddrFound);
+                    }
                 } finally {
                     this.lock.readLock().unlock();
                 }
@@ -495,104 +870,93 @@ public class RouteInfoManager {
             }
         }
 
-        if (null == brokerAddrFound) {
-            brokerAddrFound = remoteAddr;
-        } else {
-            log.info("the broker's channel destroyed, {}, clean it's data structure at once", brokerAddrFound);
+        if (needUnRegister) {
+            boolean result = this.submitUnRegisterBrokerRequest(unRegisterRequest);
+            log.info("the broker's channel destroyed, submit the unregister request at once, " +
+                "broker info: {}, submit result: {}", unRegisterRequest, result);
         }
+    }
 
-        if (brokerAddrFound != null && brokerAddrFound.length() > 0) {
+    private boolean setupUnRegisterRequest(UnRegisterBrokerRequestHeader unRegisterRequest,
+        BrokerAddrInfo brokerAddrInfo) {
+        unRegisterRequest.setClusterName(brokerAddrInfo.getClusterName());
+        unRegisterRequest.setBrokerAddr(brokerAddrInfo.getBrokerAddr());
 
-            try {
-                try {
-                    this.lock.writeLock().lockInterruptibly();
-                    this.brokerLiveTable.remove(brokerAddrFound);
-                    this.filterServerTable.remove(brokerAddrFound);
-                    String brokerNameFound = null;
-                    boolean removeBrokerName = false;
-                    Iterator<Entry<String, BrokerData>> itBrokerAddrTable =
-                        this.brokerAddrTable.entrySet().iterator();
-                    while (itBrokerAddrTable.hasNext() && (null == brokerNameFound)) {
-                        BrokerData brokerData = itBrokerAddrTable.next().getValue();
-
-                        Iterator<Entry<Long, String>> it = brokerData.getBrokerAddrs().entrySet().iterator();
-                        while (it.hasNext()) {
-                            Entry<Long, String> entry = it.next();
-                            Long brokerId = entry.getKey();
-                            String brokerAddr = entry.getValue();
-                            if (brokerAddr.equals(brokerAddrFound)) {
-                                brokerNameFound = brokerData.getBrokerName();
-                                it.remove();
-                                log.info("remove brokerAddr[{}, {}] from brokerAddrTable, because channel destroyed",
-                                    brokerId, brokerAddr);
-                                break;
-                            }
-                        }
+        for (Entry<String, BrokerData> stringBrokerDataEntry : this.brokerAddrTable.entrySet()) {
+            BrokerData brokerData = stringBrokerDataEntry.getValue();
+            if (!brokerAddrInfo.getClusterName().equals(brokerData.getCluster())) {
+                continue;
+            }
 
-                        if (brokerData.getBrokerAddrs().isEmpty()) {
-                            removeBrokerName = true;
-                            itBrokerAddrTable.remove();
-                            log.info("remove brokerName[{}] from brokerAddrTable, because channel destroyed",
-                                brokerData.getBrokerName());
-                        }
-                    }
+            for (Entry<Long, String> entry : brokerData.getBrokerAddrs().entrySet()) {
+                Long brokerId = entry.getKey();
+                String brokerAddr = entry.getValue();
+                if (brokerAddr.equals(brokerAddrInfo.getBrokerAddr())) {
+                    unRegisterRequest.setBrokerName(brokerData.getBrokerName());
+                    unRegisterRequest.setBrokerId(brokerId);
+                    return true;
+                }
+            }
+        }
 
-                    if (brokerNameFound != null && removeBrokerName) {
-                        Iterator<Entry<String, Set<String>>> it = this.clusterAddrTable.entrySet().iterator();
-                        while (it.hasNext()) {
-                            Entry<String, Set<String>> entry = it.next();
-                            String clusterName = entry.getKey();
-                            Set<String> brokerNames = entry.getValue();
-                            boolean removed = brokerNames.remove(brokerNameFound);
-                            if (removed) {
-                                log.info("remove brokerName[{}], clusterName[{}] from clusterAddrTable, because channel destroyed",
-                                    brokerNameFound, clusterName);
-
-                                if (brokerNames.isEmpty()) {
-                                    log.info("remove the clusterName[{}] from clusterAddrTable, because channel destroyed and no broker in this cluster",
-                                        clusterName);
-                                    it.remove();
-                                }
+        return false;
+    }
 
-                                break;
-                            }
-                        }
-                    }
+    private void notifyMinBrokerIdChanged(Map<String, BrokerStatusChangeInfo> needNotifyBrokerMap)
+        throws InterruptedException, RemotingConnectException, RemotingTimeoutException, RemotingSendRequestException,
+        RemotingTooMuchRequestException {
+        for (String brokerName : needNotifyBrokerMap.keySet()) {
+            BrokerStatusChangeInfo brokerStatusChangeInfo = needNotifyBrokerMap.get(brokerName);
+            BrokerData brokerData = brokerAddrTable.get(brokerName);
+            if (brokerData != null && brokerData.isEnableActingMaster()) {
+                notifyMinBrokerIdChanged(brokerStatusChangeInfo.getBrokerAddrs(),
+                    brokerStatusChangeInfo.getOfflineBrokerAddr(), brokerStatusChangeInfo.getHaBrokerAddr());
+            }
+        }
+    }
 
-                    if (removeBrokerName) {
-                        Iterator<Entry<String, List<QueueData>>> itTopicQueueTable =
-                            this.topicQueueTable.entrySet().iterator();
-                        while (itTopicQueueTable.hasNext()) {
-                            Entry<String, List<QueueData>> entry = itTopicQueueTable.next();
-                            String topic = entry.getKey();
-                            List<QueueData> queueDataList = entry.getValue();
-
-                            Iterator<QueueData> itQueueData = queueDataList.iterator();
-                            while (itQueueData.hasNext()) {
-                                QueueData queueData = itQueueData.next();
-                                if (queueData.getBrokerName().equals(brokerNameFound)) {
-                                    itQueueData.remove();
-                                    log.info("remove topic[{} {}], from topicQueueTable, because channel destroyed",
-                                        topic, queueData);
-                                }
-                            }
+    private void notifyMinBrokerIdChanged(Map<Long, String> brokerAddrMap, String offlineBrokerAddr,
+        String haBrokerAddr)
+        throws InterruptedException, RemotingSendRequestException, RemotingTimeoutException,
+        RemotingTooMuchRequestException, RemotingConnectException {
+        if (brokerAddrMap == null || brokerAddrMap.isEmpty() || this.namesrvController == null) {
+            return;
+        }
 
-                            if (queueDataList.isEmpty()) {
-                                itTopicQueueTable.remove();
-                                log.info("remove topic[{}] all queue, from topicQueueTable, because channel destroyed",
-                                    topic);
-                            }
-                        }
-                    }
-                } finally {
-                    this.lock.writeLock().unlock();
-                }
-            } catch (Exception e) {
-                log.error("onChannelDestroy Exception", e);
+        NotifyMinBrokerIdChangeRequestHeader requestHeader = new NotifyMinBrokerIdChangeRequestHeader();
+        long minBrokerId = Collections.min(brokerAddrMap.keySet());
+        requestHeader.setMinBrokerId(minBrokerId);
+        requestHeader.setMinBrokerAddr(brokerAddrMap.get(minBrokerId));
+        requestHeader.setOfflineBrokerAddr(offlineBrokerAddr);
+        requestHeader.setHaBrokerAddr(haBrokerAddr);
+
+        List<String> brokerAddrsNotify = chooseBrokerAddrsToNotify(brokerAddrMap, offlineBrokerAddr);
+        log.info("min broker id changed to {}, notify {}, offline broker addr {}", minBrokerId, brokerAddrsNotify, offlineBrokerAddr);
+        RemotingCommand request =
+            RemotingCommand.createRequestCommand(RequestCode.NOTIFY_MIN_BROKER_ID_CHANGE, requestHeader);
+        for (String brokerAddr : brokerAddrsNotify) {
+            this.namesrvController.getRemotingClient().invokeOneway(brokerAddr, request, 300);
+        }
+    }
+
+    private List<String> chooseBrokerAddrsToNotify(Map<Long, String> brokerAddrMap, String offlineBrokerAddr) {
+        if (offlineBrokerAddr != null || brokerAddrMap.size() == 1) {
+            // notify the reset brokers.
+            return new ArrayList<>(brokerAddrMap.values());
+        }
+
+        // new broker registered, notify previous brokers.
+        long minBrokerId = Collections.min(brokerAddrMap.keySet());
+        List<String> brokerAddrList = new ArrayList<>();
+        for (Long brokerId : brokerAddrMap.keySet()) {
+            if (brokerId != minBrokerId) {
+                brokerAddrList.add(brokerAddrMap.get(brokerId));
             }
         }
+        return brokerAddrList;
     }
 
+    // For test only
     public void printAllPeriodically() {
         try {
             try {
@@ -600,9 +964,9 @@ public class RouteInfoManager {
                 log.info("--------------------------------------------------------");
                 {
                     log.info("topicQueueTable SIZE: {}", this.topicQueueTable.size());
-                    Iterator<Entry<String, List<QueueData>>> it = this.topicQueueTable.entrySet().iterator();
+                    Iterator<Entry<String, Map<String, QueueData>>> it = this.topicQueueTable.entrySet().iterator();
                     while (it.hasNext()) {
-                        Entry<String, List<QueueData>> next = it.next();
+                        Entry<String, Map<String, QueueData>> next = it.next();
                         log.info("topicQueueTable Topic: {} {}", next.getKey(), next.getValue());
                     }
                 }
@@ -618,9 +982,9 @@ public class RouteInfoManager {
 
                 {
                     log.info("brokerLiveTable SIZE: {}", this.brokerLiveTable.size());
-                    Iterator<Entry<String, BrokerLiveInfo>> it = this.brokerLiveTable.entrySet().iterator();
+                    Iterator<Entry<BrokerAddrInfo, BrokerLiveInfo>> it = this.brokerLiveTable.entrySet().iterator();
                     while (it.hasNext()) {
-                        Entry<String, BrokerLiveInfo> next = it.next();
+                        Entry<BrokerAddrInfo, BrokerLiveInfo> next = it.next();
                         log.info("brokerLiveTable brokerAddr: {} {}", next.getKey(), next.getValue());
                     }
                 }
@@ -680,17 +1044,15 @@ public class RouteInfoManager {
                 this.lock.readLock().lockInterruptibly();
                 Set<String> brokerNameSet = this.clusterAddrTable.get(cluster);
                 for (String brokerName : brokerNameSet) {
-                    Iterator<Entry<String, List<QueueData>>> topicTableIt =
+                    Iterator<Entry<String, Map<String, QueueData>>> topicTableIt =
                         this.topicQueueTable.entrySet().iterator();
                     while (topicTableIt.hasNext()) {
-                        Entry<String, List<QueueData>> topicEntry = topicTableIt.next();
+                        Entry<String, Map<String, QueueData>> topicEntry = topicTableIt.next();
                         String topic = topicEntry.getKey();
-                        List<QueueData> queueDatas = topicEntry.getValue();
-                        for (QueueData queueData : queueDatas) {
-                            if (brokerName.equals(queueData.getBrokerName())) {
-                                topicList.getTopicList().add(topic);
-                                break;
-                            }
+                        Map<String, QueueData> queueDataMap = topicEntry.getValue();
+                        final QueueData qd = queueDataMap.get(brokerName);
+                        if (qd != null) {
+                            topicList.getTopicList().add(topic);
                         }
                     }
                 }
@@ -709,14 +1071,14 @@ public class RouteInfoManager {
         try {
             try {
                 this.lock.readLock().lockInterruptibly();
-                Iterator<Entry<String, List<QueueData>>> topicTableIt =
+                Iterator<Entry<String, Map<String, QueueData>>> topicTableIt =
                     this.topicQueueTable.entrySet().iterator();
                 while (topicTableIt.hasNext()) {
-                    Entry<String, List<QueueData>> topicEntry = topicTableIt.next();
+                    Entry<String, Map<String, QueueData>> topicEntry = topicTableIt.next();
                     String topic = topicEntry.getKey();
-                    List<QueueData> queueDatas = topicEntry.getValue();
+                    Map<String, QueueData> queueDatas = topicEntry.getValue();
                     if (queueDatas != null && queueDatas.size() > 0
-                        && TopicSysFlag.hasUnitFlag(queueDatas.get(0).getTopicSysFlag())) {
+                        && TopicSysFlag.hasUnitFlag(queueDatas.values().iterator().next().getTopicSysFlag())) {
                         topicList.getTopicList().add(topic);
                     }
                 }
@@ -735,14 +1097,14 @@ public class RouteInfoManager {
         try {
             try {
                 this.lock.readLock().lockInterruptibly();
-                Iterator<Entry<String, List<QueueData>>> topicTableIt =
+                Iterator<Entry<String, Map<String, QueueData>>> topicTableIt =
                     this.topicQueueTable.entrySet().iterator();
                 while (topicTableIt.hasNext()) {
-                    Entry<String, List<QueueData>> topicEntry = topicTableIt.next();
+                    Entry<String, Map<String, QueueData>> topicEntry = topicTableIt.next();
                     String topic = topicEntry.getKey();
-                    List<QueueData> queueDatas = topicEntry.getValue();
+                    Map<String, QueueData> queueDatas = topicEntry.getValue();
                     if (queueDatas != null && queueDatas.size() > 0
-                        && TopicSysFlag.hasUnitSubFlag(queueDatas.get(0).getTopicSysFlag())) {
+                        && TopicSysFlag.hasUnitSubFlag(queueDatas.values().iterator().next().getTopicSysFlag())) {
                         topicList.getTopicList().add(topic);
                     }
                 }
@@ -761,15 +1123,15 @@ public class RouteInfoManager {
         try {
             try {
                 this.lock.readLock().lockInterruptibly();
-                Iterator<Entry<String, List<QueueData>>> topicTableIt =
+                Iterator<Entry<String, Map<String, QueueData>>> topicTableIt =
                     this.topicQueueTable.entrySet().iterator();
                 while (topicTableIt.hasNext()) {
-                    Entry<String, List<QueueData>> topicEntry = topicTableIt.next();
+                    Entry<String, Map<String, QueueData>> topicEntry = topicTableIt.next();
                     String topic = topicEntry.getKey();
-                    List<QueueData> queueDatas = topicEntry.getValue();
+                    Map<String, QueueData> queueDatas = topicEntry.getValue();
                     if (queueDatas != null && queueDatas.size() > 0
-                        && !TopicSysFlag.hasUnitFlag(queueDatas.get(0).getTopicSysFlag())
-                        && TopicSysFlag.hasUnitSubFlag(queueDatas.get(0).getTopicSysFlag())) {
+                        && !TopicSysFlag.hasUnitFlag(queueDatas.values().iterator().next().getTopicSysFlag())
+                        && TopicSysFlag.hasUnitSubFlag(queueDatas.values().iterator().next().getTopicSysFlag())) {
                         topicList.getTopicList().add(topic);
                     }
                 }
@@ -784,15 +1146,82 @@ public class RouteInfoManager {
     }
 }
 
+/**
+ * broker address information
+ */
+class BrokerAddrInfo {
+    private String clusterName;
+    private String brokerAddr;
+
+    private int hash;
+
+    public BrokerAddrInfo(String clusterName, String brokerAddr) {
+        this.clusterName = clusterName;
+        this.brokerAddr = brokerAddr;
+    }
+
+    public String getClusterName() {
+        return clusterName;
+    }
+
+    public String getBrokerAddr() {
+        return brokerAddr;
+    }
+
+    public boolean isEmpty() {
+        return clusterName.isEmpty() && brokerAddr.isEmpty();
+    }
+
+    @Override
+    public boolean equals(Object obj) {
+        if (this == obj) {
+            return true;
+        }
+        if (obj == null) {
+            return false;
+        }
+
+        if (obj instanceof BrokerAddrInfo) {
+            BrokerAddrInfo addr = (BrokerAddrInfo) obj;
+            return clusterName.equals(addr.clusterName) && brokerAddr.equals(addr.brokerAddr);
+        }
+        return false;
+    }
+
+    @Override
+    public int hashCode() {
+        int h = hash;
+        if (h == 0 && clusterName.length() + brokerAddr.length() > 0) {
+            for (int i = 0; i < clusterName.length(); i++) {
+                h = 31 * h + clusterName.charAt(i);
+            }
+            h = 31 * h + '_';
+            for (int i = 0; i < brokerAddr.length(); i++) {
+                h = 31 * h + brokerAddr.charAt(i);
+            }
+            hash = h;
+        }
+        return h;
+    }
+
+    @Override
+    public String toString() {
+        return "BrokerAddrInfo [clusterName=" + clusterName + ", brokerAddr=" + brokerAddr + "]";
+    }
+}
+
 class BrokerLiveInfo {
     private long lastUpdateTimestamp;
+    private long heartbeatTimeoutMillis;
     private DataVersion dataVersion;
     private Channel channel;
     private String haServerAddr;
 
-    public BrokerLiveInfo(long lastUpdateTimestamp, DataVersion dataVersion, Channel channel,
+    public BrokerLiveInfo(long lastUpdateTimestamp, long heartbeatTimeoutMillis, DataVersion dataVersion,
+        Channel channel,
         String haServerAddr) {
         this.lastUpdateTimestamp = lastUpdateTimestamp;
+        this.heartbeatTimeoutMillis = heartbeatTimeoutMillis;
         this.dataVersion = dataVersion;
         this.channel = channel;
         this.haServerAddr = haServerAddr;
@@ -806,6 +1235,14 @@ class BrokerLiveInfo {
         this.lastUpdateTimestamp = lastUpdateTimestamp;
     }
 
+    public long getHeartbeatTimeoutMillis() {
+        return heartbeatTimeoutMillis;
+    }
+
+    public void setHeartbeatTimeoutMillis(long heartbeatTimeoutMillis) {
+        this.heartbeatTimeoutMillis = heartbeatTimeoutMillis;
+    }
+
     public DataVersion getDataVersion() {
         return dataVersion;
     }
@@ -836,3 +1273,39 @@ class BrokerLiveInfo {
             + ", channel=" + channel + ", haServerAddr=" + haServerAddr + "]";
     }
 }
+
+class BrokerStatusChangeInfo {
+    Map<Long, String> brokerAddrs;
+    String offlineBrokerAddr;
+    String haBrokerAddr;
+
+    public BrokerStatusChangeInfo(Map<Long, String> brokerAddrs, String offlineBrokerAddr, String haBrokerAddr) {
+        this.brokerAddrs = brokerAddrs;
+        this.offlineBrokerAddr = offlineBrokerAddr;
+        this.haBrokerAddr = haBrokerAddr;
+    }
+
+    public Map<Long, String> getBrokerAddrs() {
+        return brokerAddrs;
+    }
+
+    public void setBrokerAddrs(Map<Long, String> brokerAddrs) {
+        this.brokerAddrs = brokerAddrs;
+    }
+
+    public String getOfflineBrokerAddr() {
+        return offlineBrokerAddr;
+    }
+
+    public void setOfflineBrokerAddr(String offlineBrokerAddr) {
+        this.offlineBrokerAddr = offlineBrokerAddr;
+    }
+
+    public String getHaBrokerAddr() {
+        return haBrokerAddr;
+    }
+
+    public void setHaBrokerAddr(String haBrokerAddr) {
+        this.haBrokerAddr = haBrokerAddr;
+    }
+}
diff --git a/namesrv/src/test/java/org/apache/rocketmq/namesrv/processor/RequestProcessorTest.java b/namesrv/src/test/java/org/apache/rocketmq/namesrv/processor/RequestProcessorTest.java
new file mode 100644
index 0000000..8f91d5b
--- /dev/null
+++ b/namesrv/src/test/java/org/apache/rocketmq/namesrv/processor/RequestProcessorTest.java
@@ -0,0 +1,563 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.rocketmq.namesrv.processor;
+
+import io.netty.channel.Channel;
+import io.netty.channel.ChannelHandlerContext;
+import java.lang.reflect.Field;
+import java.lang.reflect.Modifier;
+import java.net.InetSocketAddress;
+import java.util.ArrayList;
+import java.util.HashMap;
+import java.util.Map;
+import java.util.concurrent.ConcurrentHashMap;
+
+import org.apache.rocketmq.common.TopicConfig;
+import org.apache.rocketmq.common.namesrv.NamesrvConfig;
+import org.apache.rocketmq.common.namesrv.RegisterBrokerResult;
+import org.apache.rocketmq.common.protocol.RequestCode;
+import org.apache.rocketmq.common.protocol.ResponseCode;
+import org.apache.rocketmq.common.protocol.body.TopicConfigSerializeWrapper;
+import org.apache.rocketmq.common.protocol.header.namesrv.DeleteKVConfigRequestHeader;
+import org.apache.rocketmq.common.protocol.header.namesrv.GetKVConfigRequestHeader;
+import org.apache.rocketmq.common.protocol.header.namesrv.GetKVConfigResponseHeader;
+import org.apache.rocketmq.common.protocol.header.namesrv.PutKVConfigRequestHeader;
+import org.apache.rocketmq.common.protocol.header.namesrv.RegisterBrokerRequestHeader;
+import org.apache.rocketmq.common.protocol.route.BrokerData;
+import org.apache.rocketmq.logging.InternalLogger;
+import org.apache.rocketmq.namesrv.NamesrvController;
+import org.apache.rocketmq.namesrv.routeinfo.RouteInfoManager;
+import org.apache.rocketmq.remoting.exception.RemotingCommandException;
+import org.apache.rocketmq.remoting.netty.NettyServerConfig;
+import org.apache.rocketmq.remoting.protocol.RemotingCommand;
+import org.assertj.core.util.Maps;
+import org.junit.Before;
+import org.junit.Test;
+
+import static org.assertj.core.api.Assertions.assertThat;
+import static org.mockito.Mockito.mock;
+import static org.mockito.Mockito.when;
+
+public class RequestProcessorTest {
+    private DefaultRequestProcessor defaultRequestProcessor;
+
+    private ClientRequestProcessor clientRequestProcessor;
+
+    private NamesrvController namesrvController;
+
+    private NamesrvConfig namesrvConfig;
+
+    private NettyServerConfig nettyServerConfig;
+
+    private RouteInfoManager routeInfoManager;
+
+    private InternalLogger logger;
+
+    @Before
+    public void init() throws Exception {
+        namesrvConfig = new NamesrvConfig();
+        namesrvConfig.setEnableAllTopicList(true);
+        nettyServerConfig = new NettyServerConfig();
+        routeInfoManager = new RouteInfoManager(new NamesrvConfig(), null);
+
+        namesrvController = new NamesrvController(namesrvConfig, nettyServerConfig);
+
+        Field field = NamesrvController.class.getDeclaredField("routeInfoManager");
+        field.setAccessible(true);
+        field.set(namesrvController, routeInfoManager);
+        defaultRequestProcessor = new DefaultRequestProcessor(namesrvController);
+
+        clientRequestProcessor = new ClientRequestProcessor(namesrvController);
+
+
+        registerRouteInfoManager();
+
+        logger = mock(InternalLogger.class);
+        setFinalStatic(DefaultRequestProcessor.class.getDeclaredField("log"), logger);
+    }
+
+    @Test
+    public void testProcessRequest_PutKVConfig() throws RemotingCommandException {
+        PutKVConfigRequestHeader header = new PutKVConfigRequestHeader();
+        RemotingCommand request = RemotingCommand.createRequestCommand(RequestCode.PUT_KV_CONFIG,
+            header);
+        request.addExtField("namespace", "namespace");
+        request.addExtField("key", "key");
+        request.addExtField("value", "value");
+
+        RemotingCommand response = defaultRequestProcessor.processRequest(null, request);
+
+        assertThat(response.getCode()).isEqualTo(ResponseCode.SUCCESS);
+        assertThat(response.getRemark()).isNull();
+
+        assertThat(namesrvController.getKvConfigManager().getKVConfig("namespace", "key"))
+            .isEqualTo("value");
+    }
+
+    @Test
+    public void testProcessRequest_GetKVConfigReturnNotNull() throws RemotingCommandException {
+        namesrvController.getKvConfigManager().putKVConfig("namespace", "key", "value");
+
+        GetKVConfigRequestHeader header = new GetKVConfigRequestHeader();
+        RemotingCommand request = RemotingCommand.createRequestCommand(RequestCode.GET_KV_CONFIG,
+            header);
+        request.addExtField("namespace", "namespace");
+        request.addExtField("key", "key");
+
+        RemotingCommand response = defaultRequestProcessor.processRequest(null, request);
+
+        assertThat(response.getCode()).isEqualTo(ResponseCode.SUCCESS);
+        assertThat(response.getRemark()).isNull();
+
+        GetKVConfigResponseHeader responseHeader = (GetKVConfigResponseHeader) response
+            .readCustomHeader();
+
+        assertThat(responseHeader.getValue()).isEqualTo("value");
+    }
+
+    @Test
+    public void testProcessRequest_GetKVConfigReturnNull() throws RemotingCommandException {
+        GetKVConfigRequestHeader header = new GetKVConfigRequestHeader();
+        RemotingCommand request = RemotingCommand.createRequestCommand(RequestCode.GET_KV_CONFIG,
+            header);
+        request.addExtField("namespace", "namespace");
+        request.addExtField("key", "key");
+
+        RemotingCommand response = defaultRequestProcessor.processRequest(null, request);
+
+        assertThat(response.getCode()).isEqualTo(ResponseCode.QUERY_NOT_FOUND);
+        assertThat(response.getRemark()).isEqualTo("No config item, Namespace: namespace Key: key");
+
+        GetKVConfigResponseHeader responseHeader = (GetKVConfigResponseHeader) response
+            .readCustomHeader();
+
+        assertThat(responseHeader.getValue()).isNull();
+    }
+
+    @Test
+    public void testProcessRequest_DeleteKVConfig() throws RemotingCommandException {
+        namesrvController.getKvConfigManager().putKVConfig("namespace", "key", "value");
+
+        DeleteKVConfigRequestHeader header = new DeleteKVConfigRequestHeader();
+        RemotingCommand request = RemotingCommand.createRequestCommand(RequestCode.DELETE_KV_CONFIG,
+            header);
+        request.addExtField("namespace", "namespace");
+        request.addExtField("key", "key");
+
+        RemotingCommand response = defaultRequestProcessor.processRequest(null, request);
+
+        assertThat(response.getCode()).isEqualTo(ResponseCode.SUCCESS);
+        assertThat(response.getRemark()).isNull();
+
+        assertThat(namesrvController.getKvConfigManager().getKVConfig("namespace", "key"))
+            .isNull();
+    }
+
+    @Test
+    public void testProcessRequest_UnSupportedRequest() throws RemotingCommandException {
+        final RemotingCommand unSupportedRequest = RemotingCommand.createRequestCommand(99999, null);
+        final RemotingCommand response = defaultRequestProcessor.processRequest(null, unSupportedRequest);
+
+        assertThat(response).isNotNull();
+        assertThat(response.getCode()).isEqualTo(ResponseCode.REQUEST_CODE_NOT_SUPPORTED);
+    }
+
+    @Test
+    public void testProcessRequest_RegisterBroker() throws RemotingCommandException,
+        NoSuchFieldException, IllegalAccessException {
+        RemotingCommand request = genSampleRegisterCmd(true);
+
+        ChannelHandlerContext ctx = mock(ChannelHandlerContext.class);
+        when(ctx.channel()).thenReturn(null);
+
+        RemotingCommand response = defaultRequestProcessor.processRequest(ctx, request);
+
+        assertThat(response.getCode()).isEqualTo(ResponseCode.SUCCESS);
+        assertThat(response.getRemark()).isNull();
+
+        RouteInfoManager routes = namesrvController.getRouteInfoManager();
+        Field brokerAddrTable = RouteInfoManager.class.getDeclaredField("brokerAddrTable");
+        brokerAddrTable.setAccessible(true);
+
+        BrokerData broker = new BrokerData();
+        broker.setBrokerName("broker");
+        broker.setBrokerAddrs((HashMap) Maps.newHashMap(new Long(2333), "10.10.1.1"));
+
+        assertThat((Map) brokerAddrTable.get(routes))
+            .contains(new HashMap.SimpleEntry("broker", broker));
+    }
+
+    /*@Test
+    public void testProcessRequest_RegisterBrokerLogicalQueue() throws Exception {
+        String cluster = "cluster";
+        String broker1Name = "broker1";
+        String broker1Addr = "10.10.1.1";
+        String broker2Name = "broker2";
+        String broker2Addr = "10.10.1.2";
+        String topic = "foobar";
+
+        LogicalQueueRouteData queueRouteData1 = new LogicalQueueRouteData(0, 0, new MessageQueue(topic, broker1Name, 0), MessageQueueRouteState.ReadOnly, 0, 10, 100, 100, broker1Addr);
+        {
+            RegisterBrokerRequestHeader header = new RegisterBrokerRequestHeader();
+            header.setBrokerName(broker1Name);
+            RemotingCommand request = RemotingCommand.createRequestCommand(
+                RequestCode.REGISTER_BROKER, header);
+            request.addExtField("brokerName", broker1Name);
+            request.addExtField("brokerAddr", broker1Addr);
+            request.addExtField("clusterName", cluster);
+            request.addExtField("haServerAddr", "10.10.2.1");
+            request.addExtField("brokerId", String.valueOf(MixAll.MASTER_ID));
+            request.setVersion(MQVersion.CURRENT_VERSION);
+            TopicConfigSerializeWrapper topicConfigSerializeWrapper = new TopicConfigSerializeWrapper();
+            topicConfigSerializeWrapper.setTopicConfigTable(new ConcurrentHashMap<>(Collections.singletonMap(topic, new TopicConfig(topic))));
+            topicConfigSerializeWrapper.setLogicalQueuesInfoMap(Maps.newHashMap(topic, new LogicalQueuesInfo(Collections.singletonMap(0, Lists.newArrayList(
+                queueRouteData1
+            )))));
+            topicConfigSerializeWrapper.setDataVersion(new DataVersion());
+            RegisterBrokerBody requestBody = new RegisterBrokerBody();
+            requestBody.setTopicConfigSerializeWrapper(topicConfigSerializeWrapper);
+            requestBody.setFilterServerList(Lists.<String>newArrayList());
+            request.setBody(requestBody.encode());
+
+            ChannelHandlerContext ctx = mock(ChannelHandlerContext.class);
+            when(ctx.channel()).thenReturn(null);
+
+            RemotingCommand response = defaultRequestProcessor.processRequest(ctx, request);
+
+            assertThat(response.getCode()).isEqualTo(ResponseCode.SUCCESS);
+            assertThat(response.getRemark()).isNull();
+        }
+        LogicalQueueRouteData queueRouteData2 = new LogicalQueueRouteData(0, 100, new MessageQueue(topic, broker2Name, 0), MessageQueueRouteState.Normal, 0, -1, -1, -1, broker2Addr);
+        LogicalQueueRouteData queueRouteData3 = new LogicalQueueRouteData(1, 100, new MessageQueue(topic, broker2Name, 0), MessageQueueRouteState.Normal, 0, -1, -1, -1, broker2Addr);
+        {
+            RegisterBrokerRequestHeader header = new RegisterBrokerRequestHeader();
+            header.setBrokerName(broker2Name);
+            RemotingCommand request = RemotingCommand.createRequestCommand(
+                RequestCode.REGISTER_BROKER, header);
+            request.addExtField("brokerName", broker2Name);
+            request.addExtField("brokerAddr", broker2Addr);
+            request.addExtField("clusterName", cluster);
+            request.addExtField("haServerAddr", "10.10.2.1");
+            request.addExtField("brokerId", String.valueOf(MixAll.MASTER_ID));
+            request.setVersion(MQVersion.CURRENT_VERSION);
+            TopicConfigSerializeWrapper topicConfigSerializeWrapper = new TopicConfigSerializeWrapper();
+            topicConfigSerializeWrapper.setTopicConfigTable(new ConcurrentHashMap<>(Collections.singletonMap(topic, new TopicConfig(topic))));
+            topicConfigSerializeWrapper.setLogicalQueuesInfoMap(Maps.newHashMap(topic, new LogicalQueuesInfo(ImmutableMap.of(
+                0, Collections.singletonList(queueRouteData2),
+                1, Collections.singletonList(queueRouteData3)
+            ))));
+            topicConfigSerializeWrapper.setDataVersion(new DataVersion());
+            RegisterBrokerBody requestBody = new RegisterBrokerBody();
+            requestBody.setTopicConfigSerializeWrapper(topicConfigSerializeWrapper);
+            requestBody.setFilterServerList(Lists.<String>newArrayList());
+            request.setBody(requestBody.encode());
+
+            ChannelHandlerContext ctx = mock(ChannelHandlerContext.class);
+            when(ctx.channel()).thenReturn(null);
+
+            RemotingCommand response = defaultRequestProcessor.processRequest(ctx, request);
+
+            assertThat(response.getCode()).isEqualTo(ResponseCode.SUCCESS);
+            assertThat(response.getRemark()).isNull();
+        }
+
+        {
+            GetRouteInfoRequestHeader header = new GetRouteInfoRequestHeader();
+            header.setTopic(topic);
+            header.setSysFlag(MessageSysFlag.LOGICAL_QUEUE_FLAG);
... 1569 lines suppressed ...

[rocketmq] 07/07: Delete useless code file

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

jinrongtong pushed a commit to branch 5.0.0-beta-tmp
in repository https://gitbox.apache.org/repos/asf/rocketmq.git

commit 190617e8b971d09b32e6a7b77cb77be9382fe496
Author: RongtongJin <ji...@mails.ucas.ac.cn>
AuthorDate: Mon Mar 14 14:46:19 2022 +0800

    Delete useless code file
---
 .../broker/transaction/TransactionRecord.java      |  43 --
 .../broker/transaction/TransactionStore.java       |  42 --
 .../transaction/jdbc/JDBCTransactionStore.java     | 242 ------
 .../jdbc/JDBCTransactionStoreConfig.java           |  57 --
 .../rocketmq/broker/util/ServiceProvider.java      | 201 -----
 .../processor/DefaultRequestProcessorTest.java     | 523 -------------
 .../remoting/netty/AsyncNettyRequestProcessor.java |  29 -
 .../org/apache/rocketmq/store/MessageExtBatch.java |  51 --
 .../rocketmq/store/MessageExtBrokerInner.java      |  64 --
 .../schedule/DelayOffsetSerializeWrapper.java      |  34 -
 .../store/schedule/ScheduleMessageService.java     | 819 ---------------------
 .../rocketmq/store/ScheduleMessageServiceTest.java | 194 -----
 .../store/schedule/ScheduleMessageServiceTest.java | 235 ------
 13 files changed, 2534 deletions(-)

diff --git a/broker/src/main/java/org/apache/rocketmq/broker/transaction/TransactionRecord.java b/broker/src/main/java/org/apache/rocketmq/broker/transaction/TransactionRecord.java
deleted file mode 100644
index 772f08e..0000000
--- a/broker/src/main/java/org/apache/rocketmq/broker/transaction/TransactionRecord.java
+++ /dev/null
@@ -1,43 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one or more
- * contributor license agreements.  See the NOTICE file distributed with
- * this work for additional information regarding copyright ownership.
- * The ASF licenses this file to You under the Apache License, Version 2.0
- * (the "License"); you may not use this file except in compliance with
- * the License.  You may obtain a copy of the License at
- *
- *     http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-package org.apache.rocketmq.broker.transaction;
-
-/**
- * This class will be removed in the version 4.4.0 and {@link OperationResult} class is recommended.
- */
-@Deprecated
-public class TransactionRecord {
-    // Commit Log Offset
-    private long offset;
-    private String producerGroup;
-
-    public long getOffset() {
-        return offset;
-    }
-
-    public void setOffset(long offset) {
-        this.offset = offset;
-    }
-
-    public String getProducerGroup() {
-        return producerGroup;
-    }
-
-    public void setProducerGroup(String producerGroup) {
-        this.producerGroup = producerGroup;
-    }
-}
diff --git a/broker/src/main/java/org/apache/rocketmq/broker/transaction/TransactionStore.java b/broker/src/main/java/org/apache/rocketmq/broker/transaction/TransactionStore.java
deleted file mode 100644
index 03e0227..0000000
--- a/broker/src/main/java/org/apache/rocketmq/broker/transaction/TransactionStore.java
+++ /dev/null
@@ -1,42 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one or more
- * contributor license agreements.  See the NOTICE file distributed with
- * this work for additional information regarding copyright ownership.
- * The ASF licenses this file to You under the Apache License, Version 2.0
- * (the "License"); you may not use this file except in compliance with
- * the License.  You may obtain a copy of the License at
- *
- *     http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-
-package org.apache.rocketmq.broker.transaction;
-
-import java.util.List;
-
-/**
- * This class will be removed in ther version 4.4.0, and {@link TransactionalMessageService} class is recommended.
- */
-@Deprecated
-public interface TransactionStore {
-    boolean open();
-
-    void close();
-
-    boolean put(final List<TransactionRecord> trs);
-
-    void remove(final List<Long> pks);
-
-    List<TransactionRecord> traverse(final long pk, final int nums);
-
-    long totalRecords();
-
-    long minPK();
-
-    long maxPK();
-}
diff --git a/broker/src/main/java/org/apache/rocketmq/broker/transaction/jdbc/JDBCTransactionStore.java b/broker/src/main/java/org/apache/rocketmq/broker/transaction/jdbc/JDBCTransactionStore.java
deleted file mode 100644
index da4958d..0000000
--- a/broker/src/main/java/org/apache/rocketmq/broker/transaction/jdbc/JDBCTransactionStore.java
+++ /dev/null
@@ -1,242 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one or more
- * contributor license agreements.  See the NOTICE file distributed with
- * this work for additional information regarding copyright ownership.
- * The ASF licenses this file to You under the Apache License, Version 2.0
- * (the "License"); you may not use this file except in compliance with
- * the License.  You may obtain a copy of the License at
- *
- *     http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-
-package org.apache.rocketmq.broker.transaction.jdbc;
-
-import java.net.URL;
-import java.sql.Connection;
-import java.sql.DriverManager;
-import java.sql.PreparedStatement;
-import java.sql.ResultSet;
-import java.sql.SQLException;
-import java.sql.Statement;
-import java.util.List;
-import java.util.Properties;
-import java.util.concurrent.atomic.AtomicLong;
-import org.apache.rocketmq.broker.transaction.TransactionRecord;
-import org.apache.rocketmq.broker.transaction.TransactionStore;
-import org.apache.rocketmq.common.MixAll;
-import org.apache.rocketmq.common.constant.LoggerName;
-import org.apache.rocketmq.logging.InternalLogger;
-import org.apache.rocketmq.logging.InternalLoggerFactory;
-
-public class JDBCTransactionStore implements TransactionStore {
-    private static final InternalLogger log = InternalLoggerFactory.getLogger(LoggerName.TRANSACTION_LOGGER_NAME);
-    private final JDBCTransactionStoreConfig jdbcTransactionStoreConfig;
-    private Connection connection;
-    private AtomicLong totalRecordsValue = new AtomicLong(0);
-
-    public JDBCTransactionStore(JDBCTransactionStoreConfig jdbcTransactionStoreConfig) {
-        this.jdbcTransactionStoreConfig = jdbcTransactionStoreConfig;
-    }
-
-    @Override
-    public boolean open() {
-        if (this.loadDriver()) {
-            Properties props = new Properties();
-            props.put("user", jdbcTransactionStoreConfig.getJdbcUser());
-            props.put("password", jdbcTransactionStoreConfig.getJdbcPassword());
-
-            try {
-                this.connection =
-                    DriverManager.getConnection(this.jdbcTransactionStoreConfig.getJdbcURL(), props);
-
-                this.connection.setAutoCommit(false);
-
-                if (!this.computeTotalRecords()) {
-                    return this.createDB();
-                }
-
-                return true;
-            } catch (SQLException e) {
-                log.info("Create JDBC Connection Exception", e);
-            }
-        }
-
-        return false;
-    }
-
-    private boolean loadDriver() {
-        try {
-            Class.forName(this.jdbcTransactionStoreConfig.getJdbcDriverClass()).newInstance();
-            log.info("Loaded the appropriate driver, {}",
-                this.jdbcTransactionStoreConfig.getJdbcDriverClass());
-            return true;
-        } catch (Exception e) {
-            log.info("Loaded the appropriate driver Exception", e);
-        }
-
-        return false;
-    }
-
-    private boolean computeTotalRecords() {
-        Statement statement = null;
-        ResultSet resultSet = null;
-        try {
-            statement = this.connection.createStatement();
-
-            resultSet = statement.executeQuery("select count(offset) as total from t_transaction");
-            if (!resultSet.next()) {
-                log.warn("computeTotalRecords ResultSet is empty");
-                return false;
-            }
-
-            this.totalRecordsValue.set(resultSet.getLong(1));
-        } catch (Exception e) {
-            log.warn("computeTotalRecords Exception", e);
-            return false;
-        } finally {
-            if (null != statement) {
-                try {
-                    statement.close();
-                } catch (SQLException e) {
-                }
-            }
-
-            if (null != resultSet) {
-                try {
-                    resultSet.close();
-                } catch (SQLException e) {
-                }
-            }
-        }
-
-        return true;
-    }
-
-    private boolean createDB() {
-        Statement statement = null;
-        try {
-            statement = this.connection.createStatement();
-
-            String sql = this.createTableSql();
-            log.info("createDB SQL:\n {}", sql);
-            statement.execute(sql);
-            this.connection.commit();
-            return true;
-        } catch (Exception e) {
-            log.warn("createDB Exception", e);
-            return false;
-        } finally {
-            if (null != statement) {
-                try {
-                    statement.close();
-                } catch (SQLException e) {
-                    log.warn("Close statement exception", e);
-                }
-            }
-        }
-    }
-
-    private String createTableSql() {
-        URL resource = JDBCTransactionStore.class.getClassLoader().getResource("transaction.sql");
-        String fileContent = MixAll.file2String(resource);
-        return fileContent;
-    }
-
-    @Override
-    public void close() {
-        try {
-            if (this.connection != null) {
-                this.connection.close();
-            }
-        } catch (SQLException e) {
-        }
-    }
-
-    @Override
-    public boolean put(List<TransactionRecord> trs) {
-        PreparedStatement statement = null;
-        try {
-            this.connection.setAutoCommit(false);
-            statement = this.connection.prepareStatement("insert into t_transaction values (?, ?)");
-            for (TransactionRecord tr : trs) {
-                statement.setLong(1, tr.getOffset());
-                statement.setString(2, tr.getProducerGroup());
-                statement.addBatch();
-            }
-            int[] executeBatch = statement.executeBatch();
-            this.connection.commit();
-            this.totalRecordsValue.addAndGet(updatedRows(executeBatch));
-            return true;
-        } catch (Exception e) {
-            log.warn("createDB Exception", e);
-            return false;
-        } finally {
-            if (null != statement) {
-                try {
-                    statement.close();
-                } catch (SQLException e) {
-                    log.warn("Close statement exception", e);
-                }
-            }
-        }
-    }
-
-    private long updatedRows(int[] rows) {
-        long res = 0;
-        for (int i : rows) {
-            res += i;
-        }
-
-        return res;
-    }
-
-    @Override
-    public void remove(List<Long> pks) {
-        PreparedStatement statement = null;
-        try {
-            this.connection.setAutoCommit(false);
-            statement = this.connection.prepareStatement("DELETE FROM t_transaction WHERE offset = ?");
-            for (long pk : pks) {
-                statement.setLong(1, pk);
-                statement.addBatch();
-            }
-            int[] executeBatch = statement.executeBatch();
-            this.connection.commit();
-        } catch (Exception e) {
-            log.warn("createDB Exception", e);
-        } finally {
-            if (null != statement) {
-                try {
-                    statement.close();
-                } catch (SQLException e) {
-                }
-            }
-        }
-    }
-
-    @Override
-    public List<TransactionRecord> traverse(long pk, int nums) {
-        return null;
-    }
-
-    @Override
-    public long totalRecords() {
-        return this.totalRecordsValue.get();
-    }
-
-    @Override
-    public long minPK() {
-        return 0;
-    }
-
-    @Override
-    public long maxPK() {
-        return 0;
-    }
-}
diff --git a/broker/src/main/java/org/apache/rocketmq/broker/transaction/jdbc/JDBCTransactionStoreConfig.java b/broker/src/main/java/org/apache/rocketmq/broker/transaction/jdbc/JDBCTransactionStoreConfig.java
deleted file mode 100644
index 4b07959..0000000
--- a/broker/src/main/java/org/apache/rocketmq/broker/transaction/jdbc/JDBCTransactionStoreConfig.java
+++ /dev/null
@@ -1,57 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one or more
- * contributor license agreements.  See the NOTICE file distributed with
- * this work for additional information regarding copyright ownership.
- * The ASF licenses this file to You under the Apache License, Version 2.0
- * (the "License"); you may not use this file except in compliance with
- * the License.  You may obtain a copy of the License at
- *
- *     http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-
-package org.apache.rocketmq.broker.transaction.jdbc;
-
-public class JDBCTransactionStoreConfig {
-    private String jdbcDriverClass = "com.mysql.jdbc.Driver";
-    private String jdbcURL = "jdbc:mysql://xxx.xxx.xxx.xxx:1000/xxx?useUnicode=true&characterEncoding=UTF-8";
-    private String jdbcUser = "xxx";
-    private String jdbcPassword = "xxx";
-
-    public String getJdbcDriverClass() {
-        return jdbcDriverClass;
-    }
-
-    public void setJdbcDriverClass(String jdbcDriverClass) {
-        this.jdbcDriverClass = jdbcDriverClass;
-    }
-
-    public String getJdbcURL() {
-        return jdbcURL;
-    }
-
-    public void setJdbcURL(String jdbcURL) {
-        this.jdbcURL = jdbcURL;
-    }
-
-    public String getJdbcUser() {
-        return jdbcUser;
-    }
-
-    public void setJdbcUser(String jdbcUser) {
-        this.jdbcUser = jdbcUser;
-    }
-
-    public String getJdbcPassword() {
-        return jdbcPassword;
-    }
-
-    public void setJdbcPassword(String jdbcPassword) {
-        this.jdbcPassword = jdbcPassword;
-    }
-}
diff --git a/broker/src/main/java/org/apache/rocketmq/broker/util/ServiceProvider.java b/broker/src/main/java/org/apache/rocketmq/broker/util/ServiceProvider.java
deleted file mode 100644
index e679660..0000000
--- a/broker/src/main/java/org/apache/rocketmq/broker/util/ServiceProvider.java
+++ /dev/null
@@ -1,201 +0,0 @@
-/**
- * Licensed to the Apache Software Foundation (ASF) under one or more contributor license agreements.  See the NOTICE file distributed with this work for additional information regarding copyright ownership. The ASF licenses this file to
- * You under the Apache License, Version 2.0 (the "License"); you may not use this file except in compliance with the License.  You may obtain a copy of the License at
- * <p>
- * http://www.apache.org/licenses/LICENSE-2.0
- * <p>
- * Unless required by applicable law or agreed to in writing, software distributed under the License is distributed on an "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. See the License for the
- * specific language governing permissions and limitations under the License.
- */
-package org.apache.rocketmq.broker.util;
-
-import org.slf4j.Logger;
-import org.slf4j.LoggerFactory;
-
-import java.io.BufferedReader;
-import java.io.InputStream;
-import java.io.InputStreamReader;
-import java.util.ArrayList;
-import java.util.List;
-
-public class ServiceProvider {
-
-    private final static Logger LOG = LoggerFactory
-        .getLogger(ServiceProvider.class);
-    /**
-     * A reference to the classloader that loaded this class. It's more efficient to compute it once and cache it here.
-     */
-    private static ClassLoader thisClassLoader;
-
-    /**
-     * JDK1.3+ <a href= "http://java.sun.com/j2se/1.3/docs/guide/jar/jar.html#Service%20Provider" > 'Service Provider' specification</a>.
-     */
-    public static final String TRANSACTION_SERVICE_ID = "META-INF/service/org.apache.rocketmq.broker.transaction.TransactionalMessageService";
-
-    public static final String TRANSACTION_LISTENER_ID = "META-INF/service/org.apache.rocketmq.broker.transaction.AbstractTransactionalMessageCheckListener";
-
-
-    public static final String RPC_HOOK_ID = "META-INF/service/org.apache.rocketmq.remoting.RPCHook";
-
-
-    public static final String ACL_VALIDATOR_ID = "META-INF/service/org.apache.rocketmq.acl.AccessValidator";
-
-
-
-    static {
-        thisClassLoader = getClassLoader(ServiceProvider.class);
-    }
-
-    /**
-     * Returns a string that uniquely identifies the specified object, including its class.
-     * <p>
-     * The returned string is of form "classname@hashcode", ie is the same as the return value of the Object.toString() method, but works even when the specified object's class has overidden the toString method.
-     *
-     * @param o may be null.
-     * @return a string of form classname@hashcode, or "null" if param o is null.
-     */
-    protected static String objectId(Object o) {
-        if (o == null) {
-            return "null";
-        } else {
-            return o.getClass().getName() + "@" + System.identityHashCode(o);
-        }
-    }
-
-    protected static ClassLoader getClassLoader(Class<?> clazz) {
-        try {
-            return clazz.getClassLoader();
-        } catch (SecurityException e) {
-            LOG.error("Unable to get classloader for class {} due to security restrictions !",
-                clazz, e.getMessage());
-            throw e;
-        }
-    }
-
-    protected static ClassLoader getContextClassLoader() {
-        ClassLoader classLoader = null;
-        try {
-            classLoader = Thread.currentThread().getContextClassLoader();
-        } catch (SecurityException ex) {
-            /**
-             * The getContextClassLoader() method throws SecurityException when the context
-             * class loader isn't an ancestor of the calling class's class
-             * loader, or if security permissions are restricted.
-             */
-        }
-        return classLoader;
-    }
-
-    protected static InputStream getResourceAsStream(ClassLoader loader, String name) {
-        if (loader != null) {
-            return loader.getResourceAsStream(name);
-        } else {
-            return ClassLoader.getSystemResourceAsStream(name);
-        }
-    }
-
-    public static <T> List<T> load(String name, Class<?> clazz) {
-        LOG.info("Looking for a resource file of name [{}] ...", name);
-        List<T> services = new ArrayList<T>();
-        try {
-            ArrayList<String> names = new ArrayList<String>();
-            final InputStream is = getResourceAsStream(getContextClassLoader(), name);
-            if (is != null) {
-                BufferedReader reader;
-                try {
-                    reader = new BufferedReader(new InputStreamReader(is, "UTF-8"));
-                } catch (java.io.UnsupportedEncodingException e) {
-                    reader = new BufferedReader(new InputStreamReader(is));
-                }
-                String serviceName = reader.readLine();
-                while (serviceName != null && !"".equals(serviceName)) {
-                    LOG.info(
-                        "Creating an instance as specified by file {} which was present in the path of the context classloader.",
-                        name);
-                    if (!names.contains(serviceName)) {
-                        names.add(serviceName);
-                    }
-
-                    services.add((T)initService(getContextClassLoader(), serviceName, clazz));
-
-                    serviceName = reader.readLine();
-                }
-                reader.close();
-            } else {
-                // is == null
-                LOG.warn("No resource file with name [{}] found.", name);
-            }
-        } catch (Exception e) {
-            LOG.error("Error occured when looking for resource file " + name, e);
-        }
-        return services;
-    }
-
-    public static <T> T loadClass(String name, Class<?> clazz) {
-        final InputStream is = getResourceAsStream(getContextClassLoader(), name);
-        if (is != null) {
-            BufferedReader reader;
-            try {
-                try {
-                    reader = new BufferedReader(new InputStreamReader(is, "UTF-8"));
-                } catch (java.io.UnsupportedEncodingException e) {
-                    reader = new BufferedReader(new InputStreamReader(is));
-                }
-                String serviceName = reader.readLine();
-                reader.close();
-                if (serviceName != null && !"".equals(serviceName)) {
-                    return initService(getContextClassLoader(), serviceName, clazz);
-                } else {
-                    LOG.warn("ServiceName is empty!");
-                    return null;
-                }
-            } catch (Exception e) {
-                LOG.warn("Error occurred when looking for resource file " + name, e);
-            }
-        }
-        return null;
-    }
-
-    protected static <T> T initService(ClassLoader classLoader, String serviceName, Class<?> clazz) {
-        Class<?> serviceClazz = null;
-        try {
-            if (classLoader != null) {
-                try {
-                    // Warning: must typecast here & allow exception to be generated/caught & recast properly
-                    serviceClazz = classLoader.loadClass(serviceName);
-                    if (clazz.isAssignableFrom(serviceClazz)) {
-                        LOG.info("Loaded class {} from classloader {}", serviceClazz.getName(),
-                            objectId(classLoader));
-                    } else {
-                        // This indicates a problem with the ClassLoader tree. An incompatible ClassLoader was used to load the implementation.
-                        LOG.error(
-                            "Class {} loaded from classloader {} does not extend {} as loaded by this classloader.",
-                            new Object[] {serviceClazz.getName(),
-                                objectId(serviceClazz.getClassLoader()), clazz.getName()});
-                    }
-                    return (T)serviceClazz.newInstance();
-                } catch (ClassNotFoundException ex) {
-                    if (classLoader == thisClassLoader) {
-                        // Nothing more to try, onwards.
-                        LOG.warn("Unable to locate any class {} via classloader", serviceName,
-                            objectId(classLoader));
-                        throw ex;
-                    }
-                    // Ignore exception, continue
-                } catch (NoClassDefFoundError e) {
-                    if (classLoader == thisClassLoader) {
-                        // Nothing more to try, onwards.
-                        LOG.warn(
-                            "Class {} cannot be loaded via classloader {}.it depends on some other class that cannot be found.",
-                            serviceClazz, objectId(classLoader));
-                        throw e;
-                    }
-                    // Ignore exception, continue
-                }
-            }
-        } catch (Exception e) {
-            LOG.error("Unable to init service.", e);
-        }
-        return (T)serviceClazz;
-    }
-}
diff --git a/namesrv/src/test/java/org/apache/rocketmq/namesrv/processor/DefaultRequestProcessorTest.java b/namesrv/src/test/java/org/apache/rocketmq/namesrv/processor/DefaultRequestProcessorTest.java
deleted file mode 100644
index 6e3e6ef..0000000
--- a/namesrv/src/test/java/org/apache/rocketmq/namesrv/processor/DefaultRequestProcessorTest.java
+++ /dev/null
@@ -1,523 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one or more
- * contributor license agreements.  See the NOTICE file distributed with
- * this work for additional information regarding copyright ownership.
- * The ASF licenses this file to You under the Apache License, Version 2.0
- * (the "License"); you may not use this file except in compliance with
- * the License.  You may obtain a copy of the License at
- *
- *     http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-package org.apache.rocketmq.namesrv.processor;
-
-import io.netty.channel.Channel;
-import io.netty.channel.ChannelHandlerContext;
-import java.lang.reflect.Field;
-import java.lang.reflect.Modifier;
-import java.util.ArrayList;
-import java.util.HashMap;
-import java.util.Map;
-import java.util.concurrent.ConcurrentHashMap;
-
-import org.apache.rocketmq.common.TopicConfig;
-import org.apache.rocketmq.common.namesrv.NamesrvConfig;
-import org.apache.rocketmq.common.namesrv.RegisterBrokerResult;
-import org.apache.rocketmq.common.protocol.RequestCode;
-import org.apache.rocketmq.common.protocol.ResponseCode;
-import org.apache.rocketmq.common.protocol.body.TopicConfigSerializeWrapper;
-import org.apache.rocketmq.common.protocol.header.namesrv.DeleteKVConfigRequestHeader;
-import org.apache.rocketmq.common.protocol.header.namesrv.GetKVConfigRequestHeader;
-import org.apache.rocketmq.common.protocol.header.namesrv.GetKVConfigResponseHeader;
-import org.apache.rocketmq.common.protocol.header.namesrv.PutKVConfigRequestHeader;
-import org.apache.rocketmq.common.protocol.header.namesrv.RegisterBrokerRequestHeader;
-import org.apache.rocketmq.common.protocol.route.BrokerData;
-import org.apache.rocketmq.logging.InternalLogger;
-import org.apache.rocketmq.namesrv.NamesrvController;
-import org.apache.rocketmq.namesrv.routeinfo.RouteInfoManager;
-import org.apache.rocketmq.remoting.exception.RemotingCommandException;
-import org.apache.rocketmq.remoting.netty.NettyServerConfig;
-import org.apache.rocketmq.remoting.protocol.RemotingCommand;
-import org.assertj.core.util.Maps;
-import org.junit.Before;
-import org.junit.Test;
-
-import static org.assertj.core.api.Assertions.assertThat;
-import static org.mockito.Mockito.mock;
-import static org.mockito.Mockito.when;
-
-public class DefaultRequestProcessorTest {
-    private DefaultRequestProcessor defaultRequestProcessor;
-
-    private NamesrvController namesrvController;
-
-    private NamesrvConfig namesrvConfig;
-
-    private NettyServerConfig nettyServerConfig;
-
-    private RouteInfoManager routeInfoManager;
-
-    private InternalLogger logger;
-
-    @Before
-    public void init() throws Exception {
-        namesrvConfig = new NamesrvConfig();
-        nettyServerConfig = new NettyServerConfig();
-        routeInfoManager = new RouteInfoManager();
-
-        namesrvController = new NamesrvController(namesrvConfig, nettyServerConfig);
-
-        Field field = NamesrvController.class.getDeclaredField("routeInfoManager");
-        field.setAccessible(true);
-        field.set(namesrvController, routeInfoManager);
-        defaultRequestProcessor = new DefaultRequestProcessor(namesrvController);
-
-        registerRouteInfoManager();
-
-        logger = mock(InternalLogger.class);
-        setFinalStatic(DefaultRequestProcessor.class.getDeclaredField("log"), logger);
-    }
-
-    @Test
-    public void testProcessRequest_PutKVConfig() throws RemotingCommandException {
-        PutKVConfigRequestHeader header = new PutKVConfigRequestHeader();
-        RemotingCommand request = RemotingCommand.createRequestCommand(RequestCode.PUT_KV_CONFIG,
-            header);
-        request.addExtField("namespace", "namespace");
-        request.addExtField("key", "key");
-        request.addExtField("value", "value");
-
-        RemotingCommand response = defaultRequestProcessor.processRequest(null, request);
-
-        assertThat(response.getCode()).isEqualTo(ResponseCode.SUCCESS);
-        assertThat(response.getRemark()).isNull();
-
-        assertThat(namesrvController.getKvConfigManager().getKVConfig("namespace", "key"))
-            .isEqualTo("value");
-    }
-
-    @Test
-    public void testProcessRequest_GetKVConfigReturnNotNull() throws RemotingCommandException {
-        namesrvController.getKvConfigManager().putKVConfig("namespace", "key", "value");
-
-        GetKVConfigRequestHeader header = new GetKVConfigRequestHeader();
-        RemotingCommand request = RemotingCommand.createRequestCommand(RequestCode.GET_KV_CONFIG,
-            header);
-        request.addExtField("namespace", "namespace");
-        request.addExtField("key", "key");
-
-        RemotingCommand response = defaultRequestProcessor.processRequest(null, request);
-
-        assertThat(response.getCode()).isEqualTo(ResponseCode.SUCCESS);
-        assertThat(response.getRemark()).isNull();
-
-        GetKVConfigResponseHeader responseHeader = (GetKVConfigResponseHeader) response
-            .readCustomHeader();
-
-        assertThat(responseHeader.getValue()).isEqualTo("value");
-    }
-
-    @Test
-    public void testProcessRequest_GetKVConfigReturnNull() throws RemotingCommandException {
-        GetKVConfigRequestHeader header = new GetKVConfigRequestHeader();
-        RemotingCommand request = RemotingCommand.createRequestCommand(RequestCode.GET_KV_CONFIG,
-            header);
-        request.addExtField("namespace", "namespace");
-        request.addExtField("key", "key");
-
-        RemotingCommand response = defaultRequestProcessor.processRequest(null, request);
-
-        assertThat(response.getCode()).isEqualTo(ResponseCode.QUERY_NOT_FOUND);
-        assertThat(response.getRemark()).isEqualTo("No config item, Namespace: namespace Key: key");
-
-        GetKVConfigResponseHeader responseHeader = (GetKVConfigResponseHeader) response
-            .readCustomHeader();
-
-        assertThat(responseHeader.getValue()).isNull();
-    }
-
-    @Test
-    public void testProcessRequest_DeleteKVConfig() throws RemotingCommandException {
-        namesrvController.getKvConfigManager().putKVConfig("namespace", "key", "value");
-
-        DeleteKVConfigRequestHeader header = new DeleteKVConfigRequestHeader();
-        RemotingCommand request = RemotingCommand.createRequestCommand(RequestCode.DELETE_KV_CONFIG,
-            header);
-        request.addExtField("namespace", "namespace");
-        request.addExtField("key", "key");
-
-        RemotingCommand response = defaultRequestProcessor.processRequest(null, request);
-
-        assertThat(response.getCode()).isEqualTo(ResponseCode.SUCCESS);
-        assertThat(response.getRemark()).isNull();
-
-        assertThat(namesrvController.getKvConfigManager().getKVConfig("namespace", "key"))
-            .isNull();
-    }
-
-    @Test
-    public void testProcessRequest_RegisterBroker() throws RemotingCommandException,
-        NoSuchFieldException, IllegalAccessException {
-        RemotingCommand request = genSampleRegisterCmd(true);
-
-        ChannelHandlerContext ctx = mock(ChannelHandlerContext.class);
-        when(ctx.channel()).thenReturn(null);
-
-        RemotingCommand response = defaultRequestProcessor.processRequest(ctx, request);
-
-        assertThat(response.getCode()).isEqualTo(ResponseCode.SUCCESS);
-        assertThat(response.getRemark()).isNull();
-
-        RouteInfoManager routes = namesrvController.getRouteInfoManager();
-        Field brokerAddrTable = RouteInfoManager.class.getDeclaredField("brokerAddrTable");
-        brokerAddrTable.setAccessible(true);
-
-        BrokerData broker = new BrokerData();
-        broker.setBrokerName("broker");
-        broker.setBrokerAddrs((HashMap) Maps.newHashMap(new Long(2333), "10.10.1.1"));
-
-        assertThat((Map) brokerAddrTable.get(routes))
-            .contains(new HashMap.SimpleEntry("broker", broker));
-    }
-
-    /*@Test
-    public void testProcessRequest_RegisterBrokerLogicalQueue() throws Exception {
-        String cluster = "cluster";
-        String broker1Name = "broker1";
-        String broker1Addr = "10.10.1.1";
-        String broker2Name = "broker2";
-        String broker2Addr = "10.10.1.2";
-        String topic = "foobar";
-
-        LogicalQueueRouteData queueRouteData1 = new LogicalQueueRouteData(0, 0, new MessageQueue(topic, broker1Name, 0), MessageQueueRouteState.ReadOnly, 0, 10, 100, 100, broker1Addr);
-        {
-            RegisterBrokerRequestHeader header = new RegisterBrokerRequestHeader();
-            header.setBrokerName(broker1Name);
-            RemotingCommand request = RemotingCommand.createRequestCommand(
-                RequestCode.REGISTER_BROKER, header);
-            request.addExtField("brokerName", broker1Name);
-            request.addExtField("brokerAddr", broker1Addr);
-            request.addExtField("clusterName", cluster);
-            request.addExtField("haServerAddr", "10.10.2.1");
-            request.addExtField("brokerId", String.valueOf(MixAll.MASTER_ID));
-            request.setVersion(MQVersion.CURRENT_VERSION);
-            TopicConfigSerializeWrapper topicConfigSerializeWrapper = new TopicConfigSerializeWrapper();
-            topicConfigSerializeWrapper.setTopicConfigTable(new ConcurrentHashMap<>(Collections.singletonMap(topic, new TopicConfig(topic))));
-            topicConfigSerializeWrapper.setLogicalQueuesInfoMap(Maps.newHashMap(topic, new LogicalQueuesInfo(Collections.singletonMap(0, Lists.newArrayList(
-                queueRouteData1
-            )))));
-            topicConfigSerializeWrapper.setDataVersion(new DataVersion());
-            RegisterBrokerBody requestBody = new RegisterBrokerBody();
-            requestBody.setTopicConfigSerializeWrapper(topicConfigSerializeWrapper);
-            requestBody.setFilterServerList(Lists.<String>newArrayList());
-            request.setBody(requestBody.encode());
-
-            ChannelHandlerContext ctx = mock(ChannelHandlerContext.class);
-            when(ctx.channel()).thenReturn(null);
-
-            RemotingCommand response = defaultRequestProcessor.processRequest(ctx, request);
-
-            assertThat(response.getCode()).isEqualTo(ResponseCode.SUCCESS);
-            assertThat(response.getRemark()).isNull();
-        }
-        LogicalQueueRouteData queueRouteData2 = new LogicalQueueRouteData(0, 100, new MessageQueue(topic, broker2Name, 0), MessageQueueRouteState.Normal, 0, -1, -1, -1, broker2Addr);
-        LogicalQueueRouteData queueRouteData3 = new LogicalQueueRouteData(1, 100, new MessageQueue(topic, broker2Name, 0), MessageQueueRouteState.Normal, 0, -1, -1, -1, broker2Addr);
-        {
-            RegisterBrokerRequestHeader header = new RegisterBrokerRequestHeader();
-            header.setBrokerName(broker2Name);
-            RemotingCommand request = RemotingCommand.createRequestCommand(
-                RequestCode.REGISTER_BROKER, header);
-            request.addExtField("brokerName", broker2Name);
-            request.addExtField("brokerAddr", broker2Addr);
-            request.addExtField("clusterName", cluster);
-            request.addExtField("haServerAddr", "10.10.2.1");
-            request.addExtField("brokerId", String.valueOf(MixAll.MASTER_ID));
-            request.setVersion(MQVersion.CURRENT_VERSION);
-            TopicConfigSerializeWrapper topicConfigSerializeWrapper = new TopicConfigSerializeWrapper();
-            topicConfigSerializeWrapper.setTopicConfigTable(new ConcurrentHashMap<>(Collections.singletonMap(topic, new TopicConfig(topic))));
-            topicConfigSerializeWrapper.setLogicalQueuesInfoMap(Maps.newHashMap(topic, new LogicalQueuesInfo(ImmutableMap.of(
-                0, Collections.singletonList(queueRouteData2),
-                1, Collections.singletonList(queueRouteData3)
-            ))));
-            topicConfigSerializeWrapper.setDataVersion(new DataVersion());
-            RegisterBrokerBody requestBody = new RegisterBrokerBody();
-            requestBody.setTopicConfigSerializeWrapper(topicConfigSerializeWrapper);
-            requestBody.setFilterServerList(Lists.<String>newArrayList());
-            request.setBody(requestBody.encode());
-
-            ChannelHandlerContext ctx = mock(ChannelHandlerContext.class);
-            when(ctx.channel()).thenReturn(null);
-
-            RemotingCommand response = defaultRequestProcessor.processRequest(ctx, request);
-
-            assertThat(response.getCode()).isEqualTo(ResponseCode.SUCCESS);
-            assertThat(response.getRemark()).isNull();
-        }
-
-        {
-            GetRouteInfoRequestHeader header = new GetRouteInfoRequestHeader();
-            header.setTopic(topic);
-            header.setSysFlag(MessageSysFlag.LOGICAL_QUEUE_FLAG);
-            RemotingCommand request = RemotingCommand.createRequestCommand(RequestCode.GET_ROUTEINFO_BY_TOPIC, header);
-            request.makeCustomHeaderToNet();
-
-            ChannelHandlerContext ctx = mock(ChannelHandlerContext.class);
-            when(ctx.channel()).thenReturn(null);
-
-            RemotingCommand response = defaultRequestProcessor.processRequest(ctx, request);
-
-            assertThat(response.getCode()).isEqualTo(ResponseCode.SUCCESS);
-
-            TopicRouteDataNameSrv topicRouteDataNameSrv = JSON.parseObject(response.getBody(), TopicRouteDataNameSrv.class);
-            assertThat(topicRouteDataNameSrv).isNotNull();
-            LogicalQueuesInfoUnordered logicalQueuesInfoUnordered = new LogicalQueuesInfoUnordered();
-            logicalQueuesInfoUnordered.put(0, ImmutableMap.of(
-                new LogicalQueuesInfoUnordered.Key(queueRouteData1.getBrokerName(), queueRouteData1.getQueueId(), queueRouteData1.getOffsetDelta()), queueRouteData1,
-                new LogicalQueuesInfoUnordered.Key(queueRouteData2.getBrokerName(), queueRouteData2.getQueueId(), queueRouteData2.getOffsetDelta()), queueRouteData2
-            ));
-            logicalQueuesInfoUnordered.put(1, ImmutableMap.of(new LogicalQueuesInfoUnordered.Key(queueRouteData3.getBrokerName(), queueRouteData3.getQueueId(), queueRouteData3.getOffsetDelta()), queueRouteData3));
-            assertThat(topicRouteDataNameSrv.getLogicalQueuesInfoUnordered()).isEqualTo(logicalQueuesInfoUnordered);
-        }
-    }
-*/
-    @Test
-    public void testProcessRequest_RegisterBrokerWithFilterServer() throws RemotingCommandException,
-        NoSuchFieldException, IllegalAccessException {
-        RemotingCommand request = genSampleRegisterCmd(true);
-
-        // version >= MQVersion.Version.V3_0_11.ordinal() to register with filter server
-        request.setVersion(100);
-
-        ChannelHandlerContext ctx = mock(ChannelHandlerContext.class);
-        when(ctx.channel()).thenReturn(null);
-
-        RemotingCommand response = defaultRequestProcessor.processRequest(ctx, request);
-
-        assertThat(response.getCode()).isEqualTo(ResponseCode.SUCCESS);
-        assertThat(response.getRemark()).isNull();
-
-        RouteInfoManager routes = namesrvController.getRouteInfoManager();
-        Field brokerAddrTable = RouteInfoManager.class.getDeclaredField("brokerAddrTable");
-        brokerAddrTable.setAccessible(true);
-
-        BrokerData broker = new BrokerData();
-        broker.setBrokerName("broker");
-        broker.setBrokerAddrs((HashMap) Maps.newHashMap(new Long(2333), "10.10.1.1"));
-
-        assertThat((Map) brokerAddrTable.get(routes))
-            .contains(new HashMap.SimpleEntry("broker", broker));
-    }
-
-    @Test
-    public void testProcessRequest_UnregisterBroker() throws RemotingCommandException, NoSuchFieldException, IllegalAccessException {
-        ChannelHandlerContext ctx = mock(ChannelHandlerContext.class);
-        when(ctx.channel()).thenReturn(null);
-
-        //Register broker
-        RemotingCommand regRequest = genSampleRegisterCmd(true);
-        defaultRequestProcessor.processRequest(ctx, regRequest);
-
-        //Unregister broker
-        RemotingCommand unregRequest = genSampleRegisterCmd(false);
-        RemotingCommand unregResponse = defaultRequestProcessor.processRequest(ctx, unregRequest);
-
-        assertThat(unregResponse.getCode()).isEqualTo(ResponseCode.SUCCESS);
-        assertThat(unregResponse.getRemark()).isNull();
-
-        RouteInfoManager routes = namesrvController.getRouteInfoManager();
-        Field brokerAddrTable = RouteInfoManager.class.getDeclaredField("brokerAddrTable");
-        brokerAddrTable.setAccessible(true);
-
-        assertThat((Map) brokerAddrTable.get(routes)).isNotEmpty();
-    }
-
-    @Test
-    public void testGetRouteInfoByTopic() throws RemotingCommandException {
-        ChannelHandlerContext ctx = mock(ChannelHandlerContext.class);
-        when(ctx.channel()).thenReturn(null);
-        RemotingCommand request = getRemotingCommand(RequestCode.GET_ROUTEINFO_BY_TOPIC);
-        RemotingCommand remotingCommandSuccess = defaultRequestProcessor.processRequest(ctx, request);
-        assertThat(remotingCommandSuccess.getCode()).isEqualTo(ResponseCode.SUCCESS);
-        request.getExtFields().put("topic", "test");
-        RemotingCommand remotingCommandNoTopicRouteInfo = defaultRequestProcessor.processRequest(ctx, request);
-        assertThat(remotingCommandNoTopicRouteInfo.getCode()).isEqualTo(ResponseCode.TOPIC_NOT_EXIST);
-    }
-
-    @Test
-    public void testGetBrokerClusterInfo() throws RemotingCommandException {
-        ChannelHandlerContext ctx = mock(ChannelHandlerContext.class);
-        when(ctx.channel()).thenReturn(null);
-        RemotingCommand request = getRemotingCommand(RequestCode.GET_BROKER_CLUSTER_INFO);
-        RemotingCommand remotingCommand = defaultRequestProcessor.processRequest(ctx, request);
-        assertThat(remotingCommand.getCode()).isEqualTo(ResponseCode.SUCCESS);
-    }
-
-    @Test
-    public void testWipeWritePermOfBroker() throws RemotingCommandException {
-        ChannelHandlerContext ctx = mock(ChannelHandlerContext.class);
-        when(ctx.channel()).thenReturn(null);
-        RemotingCommand request = getRemotingCommand(RequestCode.WIPE_WRITE_PERM_OF_BROKER);
-        RemotingCommand remotingCommand = defaultRequestProcessor.processRequest(ctx, request);
-        assertThat(remotingCommand.getCode()).isEqualTo(ResponseCode.SUCCESS);
-    }
-
-    @Test
-    public void testGetAllTopicListFromNameserver() throws RemotingCommandException {
-        ChannelHandlerContext ctx = mock(ChannelHandlerContext.class);
-        when(ctx.channel()).thenReturn(null);
-        RemotingCommand request = getRemotingCommand(RequestCode.GET_ALL_TOPIC_LIST_FROM_NAMESERVER);
-        RemotingCommand remotingCommand = defaultRequestProcessor.processRequest(ctx, request);
-        assertThat(remotingCommand.getCode()).isEqualTo(ResponseCode.SUCCESS);
-    }
-
-    @Test
-    public void testDeleteTopicInNamesrv() throws RemotingCommandException {
-        ChannelHandlerContext ctx = mock(ChannelHandlerContext.class);
-        when(ctx.channel()).thenReturn(null);
-        RemotingCommand request = getRemotingCommand(RequestCode.DELETE_TOPIC_IN_NAMESRV);
-        RemotingCommand remotingCommand = defaultRequestProcessor.processRequest(ctx, request);
-        assertThat(remotingCommand.getCode()).isEqualTo(ResponseCode.SUCCESS);
-    }
-
-    @Test
-    public void testGetKVListByNamespace() throws RemotingCommandException {
-        ChannelHandlerContext ctx = mock(ChannelHandlerContext.class);
-        when(ctx.channel()).thenReturn(null);
-        RemotingCommand request = getRemotingCommand(RequestCode.GET_KVLIST_BY_NAMESPACE);
-        request.addExtField("namespace", "default-namespace-1");
-        RemotingCommand remotingCommand = defaultRequestProcessor.processRequest(ctx, request);
-        assertThat(remotingCommand.getCode()).isEqualTo(ResponseCode.QUERY_NOT_FOUND);
-        namesrvController.getKvConfigManager().putKVConfig("default-namespace-1", "key", "value");
-        RemotingCommand remotingCommandSuccess = defaultRequestProcessor.processRequest(ctx, request);
-        assertThat(remotingCommandSuccess.getCode()).isEqualTo(ResponseCode.SUCCESS);
-    }
-
-    @Test
-    public void testGetTopicsByCluster() throws RemotingCommandException {
-        ChannelHandlerContext ctx = mock(ChannelHandlerContext.class);
-        when(ctx.channel()).thenReturn(null);
-        RemotingCommand request = getRemotingCommand(RequestCode.GET_TOPICS_BY_CLUSTER);
-        request.addExtField("cluster", "default-cluster");
-        RemotingCommand remotingCommand = defaultRequestProcessor.processRequest(ctx, request);
-        assertThat(remotingCommand.getCode()).isEqualTo(ResponseCode.SUCCESS);
-    }
-
-    @Test
-    public void testGetSystemTopicListFromNs() throws RemotingCommandException {
-        ChannelHandlerContext ctx = mock(ChannelHandlerContext.class);
-        when(ctx.channel()).thenReturn(null);
-        RemotingCommand request = getRemotingCommand(RequestCode.GET_SYSTEM_TOPIC_LIST_FROM_NS);
-        request.addExtField("cluster", "default-cluster");
-        RemotingCommand remotingCommand = defaultRequestProcessor.processRequest(ctx, request);
-        assertThat(remotingCommand.getCode()).isEqualTo(ResponseCode.SUCCESS);
-    }
-
-    @Test
-    public void testGetUnitTopicList() throws RemotingCommandException {
-        ChannelHandlerContext ctx = mock(ChannelHandlerContext.class);
-        when(ctx.channel()).thenReturn(null);
-        RemotingCommand request = getRemotingCommand(RequestCode.GET_UNIT_TOPIC_LIST);
-        request.addExtField("cluster", "default-cluster");
-        RemotingCommand remotingCommand = defaultRequestProcessor.processRequest(ctx, request);
-        assertThat(remotingCommand.getCode()).isEqualTo(ResponseCode.SUCCESS);
-    }
-
-    @Test
-    public void testGetHasUnitSubTopicList() throws RemotingCommandException {
-        ChannelHandlerContext ctx = mock(ChannelHandlerContext.class);
-        when(ctx.channel()).thenReturn(null);
-        RemotingCommand request = getRemotingCommand(RequestCode.GET_HAS_UNIT_SUB_TOPIC_LIST);
-        request.addExtField("cluster", "default-cluster");
-        RemotingCommand remotingCommand = defaultRequestProcessor.processRequest(ctx, request);
-        assertThat(remotingCommand.getCode()).isEqualTo(ResponseCode.SUCCESS);
-    }
-
-    @Test
-    public void testGetHasUnitSubUnUnitTopicList() throws RemotingCommandException {
-        ChannelHandlerContext ctx = mock(ChannelHandlerContext.class);
-        when(ctx.channel()).thenReturn(null);
-        RemotingCommand request = getRemotingCommand(RequestCode.GET_HAS_UNIT_SUB_UNUNIT_TOPIC_LIST);
-        request.addExtField("cluster", "default-cluster");
-        RemotingCommand remotingCommand = defaultRequestProcessor.processRequest(ctx, request);
-        assertThat(remotingCommand.getCode()).isEqualTo(ResponseCode.SUCCESS);
-    }
-
-    @Test
-    public void testUpdateConfig() throws RemotingCommandException {
-        ChannelHandlerContext ctx = mock(ChannelHandlerContext.class);
-        when(ctx.channel()).thenReturn(null);
-        RemotingCommand request = getRemotingCommand(RequestCode.UPDATE_NAMESRV_CONFIG);
-        request.addExtField("cluster", "default-cluster");
-        Map<String, String> propertiesMap = new HashMap<>();
-        propertiesMap.put("key", "value");
-        request.setBody(propertiesMap.toString().getBytes());
-        RemotingCommand remotingCommand = defaultRequestProcessor.processRequest(ctx, request);
-        assertThat(remotingCommand.getCode()).isEqualTo(ResponseCode.SUCCESS);
-    }
-
-    @Test
-    public void testGetConfig() throws RemotingCommandException {
-        ChannelHandlerContext ctx = mock(ChannelHandlerContext.class);
-        when(ctx.channel()).thenReturn(null);
-        RemotingCommand request = getRemotingCommand(RequestCode.GET_NAMESRV_CONFIG);
-        request.addExtField("cluster", "default-cluster");
-        RemotingCommand remotingCommand = defaultRequestProcessor.processRequest(ctx, request);
-        assertThat(remotingCommand.getCode()).isEqualTo(ResponseCode.SUCCESS);
-    }
-
-    private RemotingCommand getRemotingCommand(int code) {
-        RegisterBrokerRequestHeader header = new RegisterBrokerRequestHeader();
-        header.setBrokerName("broker");
-        RemotingCommand request = RemotingCommand.createRequestCommand(code, header);
-        request.addExtField("brokerName", "broker");
-        request.addExtField("brokerAddr", "10.10.1.1");
-        request.addExtField("clusterName", "cluster");
-        request.addExtField("haServerAddr", "10.10.2.1");
-        request.addExtField("brokerId", "2333");
-        request.addExtField("topic", "unit-test");
-        return request;
-    }
-
-    private static RemotingCommand genSampleRegisterCmd(boolean reg) {
-        RegisterBrokerRequestHeader header = new RegisterBrokerRequestHeader();
-        header.setBrokerName("broker");
-        RemotingCommand request = RemotingCommand.createRequestCommand(
-            reg ? RequestCode.REGISTER_BROKER : RequestCode.UNREGISTER_BROKER, header);
-        request.addExtField("brokerName", "broker");
-        request.addExtField("brokerAddr", "10.10.1.1");
-        request.addExtField("clusterName", "cluster");
-        request.addExtField("haServerAddr", "10.10.2.1");
-        request.addExtField("brokerId", "2333");
-        return request;
-    }
-
-    private static void setFinalStatic(Field field, Object newValue) throws Exception {
-        field.setAccessible(true);
-        Field modifiersField = Field.class.getDeclaredField("modifiers");
-        modifiersField.setAccessible(true);
-        modifiersField.setInt(field, field.getModifiers() & ~Modifier.FINAL);
-        field.set(null, newValue);
-    }
-
-    private void registerRouteInfoManager() {
-        TopicConfigSerializeWrapper topicConfigSerializeWrapper = new TopicConfigSerializeWrapper();
-        ConcurrentHashMap<String, TopicConfig> topicConfigConcurrentHashMap = new ConcurrentHashMap<>();
-        TopicConfig topicConfig = new TopicConfig();
-        topicConfig.setWriteQueueNums(8);
-        topicConfig.setTopicName("unit-test");
-        topicConfig.setPerm(6);
-        topicConfig.setReadQueueNums(8);
-        topicConfig.setOrder(false);
-        topicConfigConcurrentHashMap.put("unit-test", topicConfig);
-        topicConfigSerializeWrapper.setTopicConfigTable(topicConfigConcurrentHashMap);
-        Channel channel = mock(Channel.class);
-        RegisterBrokerResult registerBrokerResult = routeInfoManager.registerBroker("default-cluster", "127.0.0.1:10911", "default-broker", 0, "127.0.0.1:1001",
-            topicConfigSerializeWrapper, new ArrayList<String>(), channel);
-
-    }
-}
\ No newline at end of file
diff --git a/remoting/src/main/java/org/apache/rocketmq/remoting/netty/AsyncNettyRequestProcessor.java b/remoting/src/main/java/org/apache/rocketmq/remoting/netty/AsyncNettyRequestProcessor.java
deleted file mode 100644
index db333f8..0000000
--- a/remoting/src/main/java/org/apache/rocketmq/remoting/netty/AsyncNettyRequestProcessor.java
+++ /dev/null
@@ -1,29 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one or more
- * contributor license agreements.  See the NOTICE file distributed with
- * this work for additional information regarding copyright ownership.
- * The ASF licenses this file to You under the Apache License, Version 2.0
- * (the "License"); you may not use this file except in compliance with
- * the License.  You may obtain a copy of the License at
- *
- *     http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-
-package org.apache.rocketmq.remoting.netty;
-
-import io.netty.channel.ChannelHandlerContext;
-import org.apache.rocketmq.remoting.protocol.RemotingCommand;
-
-public abstract class AsyncNettyRequestProcessor implements NettyRequestProcessor {
-
-    public void asyncProcessRequest(ChannelHandlerContext ctx, RemotingCommand request, RemotingResponseCallback responseCallback) throws Exception {
-        RemotingCommand response = processRequest(ctx, request);
-        responseCallback.callback(response);
-    }
-}
diff --git a/store/src/main/java/org/apache/rocketmq/store/MessageExtBatch.java b/store/src/main/java/org/apache/rocketmq/store/MessageExtBatch.java
deleted file mode 100644
index e62dfb4..0000000
--- a/store/src/main/java/org/apache/rocketmq/store/MessageExtBatch.java
+++ /dev/null
@@ -1,51 +0,0 @@
-/**
- * Licensed to the Apache Software Foundation (ASF) under one or more
- * contributor license agreements.  See the NOTICE file distributed with
- * this work for additional information regarding copyright ownership.
- * The ASF licenses this file to You under the Apache License, Version 2.0
- * (the "License"); you may not use this file except in compliance with
- * the License.  You may obtain a copy of the License at
- *
- * http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-
-package org.apache.rocketmq.store;
-
-import java.nio.ByteBuffer;
-
-public class MessageExtBatch extends MessageExtBrokerInner {
-
-    private static final long serialVersionUID = -2353110995348498537L;
-    /**
-     * Inner batch means the batch dose not need to be unwrapped
-     */
-    private boolean isInnerBatch = false;
-    public ByteBuffer wrap() {
-        assert getBody() != null;
-        return ByteBuffer.wrap(getBody(), 0, getBody().length);
-    }
-
-    public boolean isInnerBatch() {
-        return isInnerBatch;
-    }
-
-    public void setInnerBatch(boolean innerBatch) {
-        isInnerBatch = innerBatch;
-    }
-
-    private ByteBuffer encodedBuff;
-
-    public ByteBuffer getEncodedBuff() {
-        return encodedBuff;
-    }
-
-    public void setEncodedBuff(ByteBuffer encodedBuff) {
-        this.encodedBuff = encodedBuff;
-    }
-}
diff --git a/store/src/main/java/org/apache/rocketmq/store/MessageExtBrokerInner.java b/store/src/main/java/org/apache/rocketmq/store/MessageExtBrokerInner.java
deleted file mode 100644
index df7e6e5..0000000
--- a/store/src/main/java/org/apache/rocketmq/store/MessageExtBrokerInner.java
+++ /dev/null
@@ -1,64 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one or more
- * contributor license agreements.  See the NOTICE file distributed with
- * this work for additional information regarding copyright ownership.
- * The ASF licenses this file to You under the Apache License, Version 2.0
- * (the "License"); you may not use this file except in compliance with
- * the License.  You may obtain a copy of the License at
- *
- *     http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-package org.apache.rocketmq.store;
-
-import java.nio.ByteBuffer;
-
-import org.apache.rocketmq.common.TopicFilterType;
-import org.apache.rocketmq.common.message.MessageExt;
-
-public class MessageExtBrokerInner extends MessageExt {
-    private static final long serialVersionUID = 7256001576878700634L;
-    private String propertiesString;
-    private long tagsCode;
-
-    private ByteBuffer encodedBuff;
-
-    public ByteBuffer getEncodedBuff() {
-        return encodedBuff;
-    }
-
-    public void setEncodedBuff(ByteBuffer encodedBuff) {
-        this.encodedBuff = encodedBuff;
-    }
-
-    public static long tagsString2tagsCode(final TopicFilterType filter, final String tags) {
-        if (null == tags || tags.length() == 0) { return 0; }
-
-        return tags.hashCode();
-    }
-
-    public static long tagsString2tagsCode(final String tags) {
-        return tagsString2tagsCode(null, tags);
-    }
-
-    public String getPropertiesString() {
-        return propertiesString;
-    }
-
-    public void setPropertiesString(String propertiesString) {
-        this.propertiesString = propertiesString;
-    }
-
-    public long getTagsCode() {
-        return tagsCode;
-    }
-
-    public void setTagsCode(long tagsCode) {
-        this.tagsCode = tagsCode;
-    }
-}
diff --git a/store/src/main/java/org/apache/rocketmq/store/schedule/DelayOffsetSerializeWrapper.java b/store/src/main/java/org/apache/rocketmq/store/schedule/DelayOffsetSerializeWrapper.java
deleted file mode 100644
index 7021992..0000000
--- a/store/src/main/java/org/apache/rocketmq/store/schedule/DelayOffsetSerializeWrapper.java
+++ /dev/null
@@ -1,34 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one or more
- * contributor license agreements.  See the NOTICE file distributed with
- * this work for additional information regarding copyright ownership.
- * The ASF licenses this file to You under the Apache License, Version 2.0
- * (the "License"); you may not use this file except in compliance with
- * the License.  You may obtain a copy of the License at
- *
- *     http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-package org.apache.rocketmq.store.schedule;
-
-import java.util.concurrent.ConcurrentHashMap;
-import java.util.concurrent.ConcurrentMap;
-import org.apache.rocketmq.remoting.protocol.RemotingSerializable;
-
-public class DelayOffsetSerializeWrapper extends RemotingSerializable {
-    private ConcurrentMap<Integer /* level */, Long/* offset */> offsetTable =
-        new ConcurrentHashMap<Integer, Long>(32);
-
-    public ConcurrentMap<Integer, Long> getOffsetTable() {
-        return offsetTable;
-    }
-
-    public void setOffsetTable(ConcurrentMap<Integer, Long> offsetTable) {
-        this.offsetTable = offsetTable;
-    }
-}
diff --git a/store/src/main/java/org/apache/rocketmq/store/schedule/ScheduleMessageService.java b/store/src/main/java/org/apache/rocketmq/store/schedule/ScheduleMessageService.java
deleted file mode 100644
index 4d058ad..0000000
--- a/store/src/main/java/org/apache/rocketmq/store/schedule/ScheduleMessageService.java
+++ /dev/null
@@ -1,819 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one or more
- * contributor license agreements.  See the NOTICE file distributed with
- * this work for additional information regarding copyright ownership.
- * The ASF licenses this file to You under the Apache License, Version 2.0
- * (the "License"); you may not use this file except in compliance with
- * the License.  You may obtain a copy of the License at
- *
- *     http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-package org.apache.rocketmq.store.schedule;
-
-import java.util.Comparator;
-import java.util.HashMap;
-import java.util.Iterator;
-import java.util.List;
-import java.util.Map;
-import java.util.Queue;
-import java.util.concurrent.CompletableFuture;
-import java.util.concurrent.ConcurrentHashMap;
-import java.util.concurrent.ConcurrentMap;
-import java.util.concurrent.ExecutionException;
-import java.util.concurrent.LinkedBlockingQueue;
-import java.util.concurrent.ScheduledExecutorService;
-import java.util.concurrent.ScheduledThreadPoolExecutor;
-import java.util.concurrent.TimeUnit;
-import java.util.concurrent.atomic.AtomicBoolean;
-import java.util.stream.Collectors;
-import org.apache.rocketmq.common.ConfigManager;
-import org.apache.rocketmq.common.MixAll;
-import org.apache.rocketmq.common.ThreadFactoryImpl;
-import org.apache.rocketmq.common.TopicFilterType;
-import org.apache.rocketmq.common.constant.LoggerName;
-import org.apache.rocketmq.common.topic.TopicValidator;
-import org.apache.rocketmq.logging.InternalLogger;
-import org.apache.rocketmq.logging.InternalLoggerFactory;
-import org.apache.rocketmq.common.message.MessageAccessor;
-import org.apache.rocketmq.common.message.MessageConst;
-import org.apache.rocketmq.common.message.MessageDecoder;
-import org.apache.rocketmq.common.message.MessageExt;
-import org.apache.rocketmq.common.running.RunningStats;
-import org.apache.rocketmq.store.DefaultMessageStore;
-import org.apache.rocketmq.store.MessageExtBrokerInner;
-import org.apache.rocketmq.store.MessageStore;
-import org.apache.rocketmq.store.PutMessageResult;
-import org.apache.rocketmq.store.PutMessageStatus;
-import org.apache.rocketmq.store.config.StorePathConfigHelper;
-import org.apache.rocketmq.store.queue.ConsumeQueueInterface;
-import org.apache.rocketmq.store.queue.CqUnit;
-import org.apache.rocketmq.store.queue.ReferredIterator;
-
-public class ScheduleMessageService extends ConfigManager {
-    private static final InternalLogger log = InternalLoggerFactory.getLogger(LoggerName.STORE_LOGGER_NAME);
-
-    private static final long FIRST_DELAY_TIME = 1000L;
-    private static final long DELAY_FOR_A_WHILE = 100L;
-    private static final long DELAY_FOR_A_PERIOD = 10000L;
-    private static final long WAIT_FOR_SHUTDOWN = 5000L;
-    private static final long DELAY_FOR_A_SLEEP = 10L;
-
-    private final ConcurrentMap<Integer /* level */, Long/* delay timeMillis */> delayLevelTable =
-        new ConcurrentHashMap<Integer, Long>(32);
-
-    private final ConcurrentMap<Integer /* level */, Long/* offset */> offsetTable =
-        new ConcurrentHashMap<Integer, Long>(32);
-    private final DefaultMessageStore defaultMessageStore;
-    private final AtomicBoolean started = new AtomicBoolean(false);
-    private ScheduledExecutorService deliverExecutorService;
-    private MessageStore writeMessageStore;
-    private int maxDelayLevel;
-    private boolean enableAsyncDeliver = false;
-    private ScheduledExecutorService handleExecutorService;
-    private final Map<Integer /* level */, LinkedBlockingQueue<PutResultProcess>> deliverPendingTable =
-        new ConcurrentHashMap<>(32);
-
-    public ScheduleMessageService(final DefaultMessageStore defaultMessageStore) {
-        this.defaultMessageStore = defaultMessageStore;
-        this.writeMessageStore = defaultMessageStore;
-        if (defaultMessageStore != null) {
-            this.enableAsyncDeliver = defaultMessageStore.getMessageStoreConfig().isEnableScheduleAsyncDeliver();
-        }
-    }
-
-    public static int queueId2DelayLevel(final int queueId) {
-        return queueId + 1;
-    }
-
-    public static int delayLevel2QueueId(final int delayLevel) {
-        return delayLevel - 1;
-    }
-
-    /**
-     * @param writeMessageStore the writeMessageStore to set
-     */
-    public void setWriteMessageStore(MessageStore writeMessageStore) {
-        this.writeMessageStore = writeMessageStore;
-    }
-
-    public void buildRunningStats(HashMap<String, String> stats) {
-        Iterator<Map.Entry<Integer, Long>> it = this.offsetTable.entrySet().iterator();
-        while (it.hasNext()) {
-            Map.Entry<Integer, Long> next = it.next();
-            int queueId = delayLevel2QueueId(next.getKey());
-            long delayOffset = next.getValue();
-            long maxOffset = this.defaultMessageStore.getMaxOffsetInQueue(TopicValidator.RMQ_SYS_SCHEDULE_TOPIC, queueId);
-            String value = String.format("%d,%d", delayOffset, maxOffset);
-            String key = String.format("%s_%d", RunningStats.scheduleMessageOffset.name(), next.getKey());
-            stats.put(key, value);
-        }
-    }
-
-    private void updateOffset(int delayLevel, long offset) {
-        this.offsetTable.put(delayLevel, offset);
-    }
-
-    public long computeDeliverTimestamp(final int delayLevel, final long storeTimestamp) {
-        Long time = this.delayLevelTable.get(delayLevel);
-        if (time != null) {
-            return time + storeTimestamp;
-        }
-
-        return storeTimestamp + 1000;
-    }
-
-    public void start() {
-        if (started.compareAndSet(false, true)) {
-            super.load();
-            this.deliverExecutorService = new ScheduledThreadPoolExecutor(this.maxDelayLevel, new ThreadFactoryImpl("ScheduleMessageTimerThread_"));
-            if (this.enableAsyncDeliver) {
-                this.handleExecutorService = new ScheduledThreadPoolExecutor(this.maxDelayLevel, new ThreadFactoryImpl("ScheduleMessageExecutorHandleThread_"));
-            }
-            for (Map.Entry<Integer, Long> entry : this.delayLevelTable.entrySet()) {
-                Integer level = entry.getKey();
-                Long timeDelay = entry.getValue();
-                Long offset = this.offsetTable.get(level);
-                if (null == offset) {
-                    offset = 0L;
-                }
-
-                if (timeDelay != null) {
-                    if (this.enableAsyncDeliver) {
-                        this.handleExecutorService.schedule(new HandlePutResultTask(level), FIRST_DELAY_TIME, TimeUnit.MILLISECONDS);
-                    }
-                    this.deliverExecutorService.schedule(new DeliverDelayedMessageTimerTask(level, offset), FIRST_DELAY_TIME, TimeUnit.MILLISECONDS);
-                }
-            }
-
-            this.deliverExecutorService.scheduleAtFixedRate(new Runnable() {
-
-                @Override
-                public void run() {
-                    try {
-                        if (started.get()) {
-                            ScheduleMessageService.this.persist();
-                        }
-                    } catch (Throwable e) {
-                        log.error("scheduleAtFixedRate flush exception", e);
-                    }
-                }
-            }, 10000, this.defaultMessageStore.getMessageStoreConfig().getFlushDelayOffsetInterval(), TimeUnit.MILLISECONDS);
-        }
-    }
-
-    public void shutdown() {
-        if (this.started.compareAndSet(true, false) && null != this.deliverExecutorService) {
-            this.deliverExecutorService.shutdown();
-            try {
-                this.deliverExecutorService.awaitTermination(WAIT_FOR_SHUTDOWN, TimeUnit.MILLISECONDS);
-            } catch (InterruptedException e) {
-                log.error("deliverExecutorService awaitTermination error", e);
-            }
-
-            if (this.handleExecutorService != null) {
-                this.handleExecutorService.shutdown();
-                try {
-                    this.handleExecutorService.awaitTermination(WAIT_FOR_SHUTDOWN, TimeUnit.MILLISECONDS);
-                } catch (InterruptedException e) {
-                    log.error("handleExecutorService awaitTermination error", e);
-                }
-            }
-
-            if (this.deliverPendingTable != null) {
-                for (int i = 1; i <= this.deliverPendingTable.size(); i++) {
-                    log.warn("deliverPendingTable level: {}, size: {}", i, this.deliverPendingTable.get(i).size());
-                }
-            }
-
-            this.persist();
-        }
-    }
-
-    public boolean isStarted() {
-        return started.get();
-    }
-
-    public int getMaxDelayLevel() {
-        return maxDelayLevel;
-    }
-
-    @Override
-    public String encode() {
-        return this.encode(false);
-    }
-
-    @Override
-    public boolean load() {
-        boolean result = super.load();
-        result = result && this.parseDelayLevel();
-        result = result && this.correctDelayOffset();
-        return result;
-    }
-
-    public boolean correctDelayOffset() {
-        try {
-            for (int delayLevel : delayLevelTable.keySet()) {
-                ConsumeQueueInterface cq =
-                    ScheduleMessageService.this.defaultMessageStore.getQueueStore().findOrCreateConsumeQueue(TopicValidator.RMQ_SYS_SCHEDULE_TOPIC,
-                        delayLevel2QueueId(delayLevel));
-                Long currentDelayOffset = offsetTable.get(delayLevel);
-                if (currentDelayOffset == null || cq == null) {
-                    continue;
-                }
-                long correctDelayOffset = currentDelayOffset;
-                long cqMinOffset = cq.getMinOffsetInQueue();
-                long cqMaxOffset = cq.getMaxOffsetInQueue();
-                if (currentDelayOffset < cqMinOffset) {
-                    correctDelayOffset = cqMinOffset;
-                    log.error("schedule CQ offset invalid. offset={}, cqMinOffset={}, cqMaxOffset={}, queueId={}",
-                        currentDelayOffset, cqMinOffset, cqMaxOffset, cq.getQueueId());
-                }
-
-                if (currentDelayOffset > cqMaxOffset) {
-                    correctDelayOffset = cqMaxOffset;
-                    log.error("schedule CQ offset invalid. offset={}, cqMinOffset={}, cqMaxOffset={}, queueId={}",
-                        currentDelayOffset, cqMinOffset, cqMaxOffset, cq.getQueueId());
-                }
-                if (correctDelayOffset != currentDelayOffset) {
-                    log.error("correct delay offset [ delayLevel {} ] from {} to {}", delayLevel, currentDelayOffset, correctDelayOffset);
-                    offsetTable.put(delayLevel, correctDelayOffset);
-                }
-            }
-        } catch (Exception e) {
-            log.error("correctDelayOffset exception", e);
-            return false;
-        }
-        return true;
-    }
-
-    @Override
-    public String configFilePath() {
-        return StorePathConfigHelper.getDelayOffsetStorePath(this.defaultMessageStore.getMessageStoreConfig()
-            .getStorePathRootDir());
-    }
-
-    @Override
-    public void decode(String jsonString) {
-        if (jsonString != null) {
-            DelayOffsetSerializeWrapper delayOffsetSerializeWrapper =
-                DelayOffsetSerializeWrapper.fromJson(jsonString, DelayOffsetSerializeWrapper.class);
-            if (delayOffsetSerializeWrapper != null) {
-                this.offsetTable.putAll(delayOffsetSerializeWrapper.getOffsetTable());
-            }
-        }
-    }
-
-    @Override
-    public String encode(final boolean prettyFormat) {
-        DelayOffsetSerializeWrapper delayOffsetSerializeWrapper = new DelayOffsetSerializeWrapper();
-        delayOffsetSerializeWrapper.setOffsetTable(this.offsetTable);
-        return delayOffsetSerializeWrapper.toJson(prettyFormat);
-    }
-
-    public boolean parseDelayLevel() {
-        HashMap<String, Long> timeUnitTable = new HashMap<String, Long>();
-        timeUnitTable.put("s", 1000L);
-        timeUnitTable.put("m", 1000L * 60);
-        timeUnitTable.put("h", 1000L * 60 * 60);
-        timeUnitTable.put("d", 1000L * 60 * 60 * 24);
-
-        String levelString = this.defaultMessageStore.getMessageStoreConfig().getMessageDelayLevel();
-        try {
-            String[] levelArray = levelString.split(" ");
-            for (int i = 0; i < levelArray.length; i++) {
-                String value = levelArray[i];
-                String ch = value.substring(value.length() - 1);
-                Long tu = timeUnitTable.get(ch);
-
-                int level = i + 1;
-                if (level > this.maxDelayLevel) {
-                    this.maxDelayLevel = level;
-                }
-                long num = Long.parseLong(value.substring(0, value.length() - 1));
-                long delayTimeMillis = tu * num;
-                this.delayLevelTable.put(level, delayTimeMillis);
-                if (this.enableAsyncDeliver) {
-                    this.deliverPendingTable.put(level, new LinkedBlockingQueue<>());
-                }
-            }
-        } catch (Exception e) {
-            log.error("parseDelayLevel exception", e);
-            log.info("levelString String = {}", levelString);
-            return false;
-        }
-
-        return true;
-    }
-
-    private MessageExtBrokerInner messageTimeup(MessageExt msgExt) {
-        MessageExtBrokerInner msgInner = new MessageExtBrokerInner();
-        msgInner.setBody(msgExt.getBody());
-        msgInner.setFlag(msgExt.getFlag());
-        MessageAccessor.setProperties(msgInner, msgExt.getProperties());
-
-        TopicFilterType topicFilterType = MessageExt.parseTopicFilterType(msgInner.getSysFlag());
-        long tagsCodeValue =
-            MessageExtBrokerInner.tagsString2tagsCode(topicFilterType, msgInner.getTags());
-        msgInner.setTagsCode(tagsCodeValue);
-        msgInner.setPropertiesString(MessageDecoder.messageProperties2String(msgExt.getProperties()));
-
-        msgInner.setSysFlag(msgExt.getSysFlag());
-        msgInner.setBornTimestamp(msgExt.getBornTimestamp());
-        msgInner.setBornHost(msgExt.getBornHost());
-        msgInner.setStoreHost(msgExt.getStoreHost());
-        msgInner.setReconsumeTimes(msgExt.getReconsumeTimes());
-
-        msgInner.setWaitStoreMsgOK(false);
-        MessageAccessor.clearProperty(msgInner, MessageConst.PROPERTY_DELAY_TIME_LEVEL);
-
-        msgInner.setTopic(msgInner.getProperty(MessageConst.PROPERTY_REAL_TOPIC));
-
-        String queueIdStr = msgInner.getProperty(MessageConst.PROPERTY_REAL_QUEUE_ID);
-        int queueId = Integer.parseInt(queueIdStr);
-        msgInner.setQueueId(queueId);
-
-        return msgInner;
-    }
-
-    public int computeDelayLevel(long timeMillis) {
-        long intervalMillis = timeMillis - System.currentTimeMillis();
-        List<Map.Entry<Integer, Long>> sortedLevels = delayLevelTable.entrySet().stream().sorted(Comparator.comparingLong(Map.Entry::getValue)).collect(Collectors.toList());
-        for (Map.Entry<Integer, Long> entry : sortedLevels) {
-            if (entry.getValue() > intervalMillis) {
-                return entry.getKey();
-            }
-        }
-        return sortedLevels.get(sortedLevels.size() - 1).getKey();
-    }
-
-    class DeliverDelayedMessageTimerTask implements Runnable {
-        private final int delayLevel;
-        private final long offset;
-
-        public DeliverDelayedMessageTimerTask(int delayLevel, long offset) {
-            this.delayLevel = delayLevel;
-            this.offset = offset;
-        }
-
-        @Override
-        public void run() {
-            try {
-                if (isStarted()) {
-                    this.executeOnTimeup();
-                }
-            } catch (Exception e) {
-                // XXX: warn and notify me
-                log.error("ScheduleMessageService, executeOnTimeup exception", e);
-                this.scheduleNextTimerTask(this.offset, DELAY_FOR_A_PERIOD);
-            }
-        }
-
-        /**
-         * @return
-         */
-        private long correctDeliverTimestamp(final long now, final long deliverTimestamp) {
-
-            long result = deliverTimestamp;
-
-            long maxTimestamp = now + ScheduleMessageService.this.delayLevelTable.get(this.delayLevel);
-            if (deliverTimestamp > maxTimestamp) {
-                result = now;
-            }
-
-            return result;
-        }
-
-        public void executeOnTimeup() {
-            ConsumeQueueInterface cq =
-                ScheduleMessageService.this.defaultMessageStore.getConsumeQueue(TopicValidator.RMQ_SYS_SCHEDULE_TOPIC,
-                    delayLevel2QueueId(delayLevel));
-
-            if (cq == null) {
-                this.scheduleNextTimerTask(this.offset, DELAY_FOR_A_WHILE);
-                return;
-            }
-
-            ReferredIterator<CqUnit> bufferCQ = cq.iterateFrom(this.offset);
-            if (bufferCQ == null) {
-                long resetOffset;
-                if ((resetOffset = cq.getMinOffsetInQueue()) > this.offset) {
-                    log.error("schedule CQ offset invalid. offset={}, cqMinOffset={}, queueId={}",
-                        this.offset, resetOffset, cq.getQueueId());
-                } else if ((resetOffset = cq.getMaxOffsetInQueue()) < this.offset) {
-                    log.error("schedule CQ offset invalid. offset={}, cqMaxOffset={}, queueId={}",
-                        this.offset, resetOffset, cq.getQueueId());
-                } else {
-                    resetOffset = this.offset;
-                }
-
-                this.scheduleNextTimerTask(resetOffset, DELAY_FOR_A_WHILE);
-                return;
-            }
-
-            long nextOffset = this.offset;
-            try {
-                while (bufferCQ.hasNext() && isStarted()) {
-                    CqUnit cqUnit = bufferCQ.next();
-                    long offsetPy = cqUnit.getPos();
-                    int sizePy = cqUnit.getSize();
-                    long tagsCode = cqUnit.getTagsCode();
-
-                    if (!cqUnit.isTagsCodeValid()) {
-                        //can't find ext content.So re compute tags code.
-                        log.error("[BUG] can't find consume queue extend file content!addr={}, offsetPy={}, sizePy={}",
-                            tagsCode, offsetPy, sizePy);
-                        long msgStoreTime = defaultMessageStore.getCommitLog().pickupStoreTimestamp(offsetPy, sizePy);
-                        tagsCode = computeDeliverTimestamp(delayLevel, msgStoreTime);
-                    }
-
-                    long now = System.currentTimeMillis();
-                    long deliverTimestamp = this.correctDeliverTimestamp(now, tagsCode);
-
-                    long currOffset = cqUnit.getQueueOffset();
-                    assert cqUnit.getBatchNum() == 1;
-                    nextOffset = currOffset + cqUnit.getBatchNum();
-
-                    long countdown = deliverTimestamp - now;
-                    if (countdown > 0) {
-                        this.scheduleNextTimerTask(currOffset, DELAY_FOR_A_WHILE);
-                        ScheduleMessageService.this.updateOffset(this.delayLevel, currOffset);
-                        return;
-                    }
-
-                    MessageExt msgExt = ScheduleMessageService.this.defaultMessageStore.lookMessageByOffset(offsetPy, sizePy);
-                    if (msgExt == null) {
-                        continue;
-                    }
-
-                    MessageExtBrokerInner msgInner = ScheduleMessageService.this.messageTimeup(msgExt);
-                    if (TopicValidator.RMQ_SYS_TRANS_HALF_TOPIC.equals(msgInner.getTopic())) {
-                        log.error("[BUG] the real topic of schedule msg is {}, discard the msg. msg={}",
-                            msgInner.getTopic(), msgInner);
-                        continue;
-                    }
-
-                    boolean deliverSuc;
-                    if (ScheduleMessageService.this.enableAsyncDeliver) {
-                        deliverSuc = this.asyncDeliver(msgInner, msgExt.getMsgId(), offset, offsetPy, sizePy);
-                    } else {
-                        deliverSuc = this.syncDeliver(msgInner, msgExt.getMsgId(), offset, offsetPy, sizePy);
-                    }
-
-                    if (!deliverSuc) {
-                        this.scheduleNextTimerTask(nextOffset, DELAY_FOR_A_WHILE);
-                        return;
-                    }
-                }
-            } catch (Exception e) {
-                log.error("ScheduleMessageService, messageTimeup execute error, offset = {}", nextOffset, e);
-            } finally {
-                bufferCQ.release();
-            }
-
-            this.scheduleNextTimerTask(nextOffset, DELAY_FOR_A_WHILE);
-        }
-
-        public void scheduleNextTimerTask(long offset, long delay) {
-            ScheduleMessageService.this.deliverExecutorService.schedule(new DeliverDelayedMessageTimerTask(
-                this.delayLevel, offset), delay, TimeUnit.MILLISECONDS);
-        }
-
-        private boolean syncDeliver(MessageExtBrokerInner msgInner, String msgId, long offset, long offsetPy,
-            int sizePy) {
-            PutResultProcess resultProcess = deliverMessage(msgInner, msgId, offset, offsetPy, sizePy, false);
-            PutMessageResult result = resultProcess.get();
-            boolean sendStatus = result != null && result.getPutMessageStatus() == PutMessageStatus.PUT_OK;
-            if (sendStatus) {
-                ScheduleMessageService.this.updateOffset(this.delayLevel, resultProcess.getNextOffset());
-            }
-            return sendStatus;
-        }
-
-        private boolean asyncDeliver(MessageExtBrokerInner msgInner, String msgId, long offset, long offsetPy,
-            int sizePy) {
-            Queue<PutResultProcess> processesQueue = ScheduleMessageService.this.deliverPendingTable.get(this.delayLevel);
-
-            //Flow Control
-            int currentPendingNum = processesQueue.size();
-            int maxPendingLimit = ScheduleMessageService.this.defaultMessageStore.getMessageStoreConfig()
-                .getScheduleAsyncDeliverMaxPendingLimit();
-            if (currentPendingNum > maxPendingLimit) {
-                log.warn("Asynchronous deliver triggers flow control, " +
-                    "currentPendingNum={}, maxPendingLimit={}", currentPendingNum, maxPendingLimit);
-                return false;
-            }
-
-            //Blocked
-            PutResultProcess firstProcess = processesQueue.peek();
-            if (firstProcess != null && firstProcess.need2Blocked()) {
-                log.warn("Asynchronous deliver block. info={}", firstProcess.toString());
-                return false;
-            }
-
-            PutResultProcess resultProcess = deliverMessage(msgInner, msgId, offset, offsetPy, sizePy, true);
-            processesQueue.add(resultProcess);
-            return true;
-        }
-
-        private PutResultProcess deliverMessage(MessageExtBrokerInner msgInner, String msgId, long offset,
-            long offsetPy, int sizePy, boolean autoResend) {
-            CompletableFuture<PutMessageResult> future =
-                ScheduleMessageService.this.writeMessageStore.asyncPutMessage(msgInner);
-            return new PutResultProcess()
-                .setTopic(msgInner.getTopic())
-                .setDelayLevel(this.delayLevel)
-                .setOffset(offset)
-                .setPhysicOffset(offsetPy)
-                .setPhysicSize(sizePy)
-                .setMsgId(msgId)
-                .setAutoResend(autoResend)
-                .setFuture(future)
-                .thenProcess();
-        }
-    }
-
-    public class HandlePutResultTask implements Runnable {
-        private final int delayLevel;
-
-        public HandlePutResultTask(int delayLevel) {
-            this.delayLevel = delayLevel;
-        }
-
-        @Override
-        public void run() {
-            LinkedBlockingQueue<PutResultProcess> pendingQueue =
-                ScheduleMessageService.this.deliverPendingTable.get(this.delayLevel);
-
-            PutResultProcess putResultProcess;
-            while ((putResultProcess = pendingQueue.peek()) != null) {
-                try {
-                    switch (putResultProcess.getStatus()) {
-                        case SUCCESS:
-                            ScheduleMessageService.this.updateOffset(this.delayLevel, putResultProcess.getNextOffset());
-                            pendingQueue.remove();
-                            break;
-                        case RUNNING:
-                            break;
-                        case EXCEPTION:
-                            if (!isStarted()) {
-                                log.warn("HandlePutResultTask shutdown, info={}", putResultProcess.toString());
-                                return;
-                            }
-                            log.warn("putResultProcess error, info={}", putResultProcess.toString());
-                            putResultProcess.onException();
-                            break;
-                        case SKIP:
-                            log.warn("putResultProcess skip, info={}", putResultProcess.toString());
-                            pendingQueue.remove();
-                            break;
-                    }
-                } catch (Exception e) {
-                    log.error("HandlePutResultTask exception. info={}", putResultProcess.toString(), e);
-                    putResultProcess.onException();
-                }
-            }
-
-            if (isStarted()) {
-                ScheduleMessageService.this.handleExecutorService
-                    .schedule(new HandlePutResultTask(this.delayLevel), DELAY_FOR_A_SLEEP, TimeUnit.MILLISECONDS);
-            }
-        }
-    }
-
-    public class PutResultProcess {
-        private String topic;
-        private long offset;
-        private long physicOffset;
-        private int physicSize;
-        private int delayLevel;
-        private String msgId;
-        private boolean autoResend = false;
-        private CompletableFuture<PutMessageResult> future;
-
-        private volatile int resendCount = 0;
-        private volatile ProcessStatus status = ProcessStatus.RUNNING;
-
-        public PutResultProcess setTopic(String topic) {
-            this.topic = topic;
-            return this;
-        }
-
-        public PutResultProcess setOffset(long offset) {
-            this.offset = offset;
-            return this;
-        }
-
-        public PutResultProcess setPhysicOffset(long physicOffset) {
-            this.physicOffset = physicOffset;
-            return this;
-        }
-
-        public PutResultProcess setPhysicSize(int physicSize) {
-            this.physicSize = physicSize;
-            return this;
-        }
-
-        public PutResultProcess setDelayLevel(int delayLevel) {
-            this.delayLevel = delayLevel;
-            return this;
-        }
-
-        public PutResultProcess setMsgId(String msgId) {
-            this.msgId = msgId;
-            return this;
-        }
-
-        public PutResultProcess setAutoResend(boolean autoResend) {
-            this.autoResend = autoResend;
-            return this;
-        }
-
-        public PutResultProcess setFuture(CompletableFuture<PutMessageResult> future) {
-            this.future = future;
-            return this;
-        }
-
-        public String getTopic() {
-            return topic;
-        }
-
-        public long getOffset() {
-            return offset;
-        }
-
-        public long getNextOffset() {
-            return offset + 1;
-        }
-
-        public long getPhysicOffset() {
-            return physicOffset;
-        }
-
-        public int getPhysicSize() {
-            return physicSize;
-        }
-
-        public Integer getDelayLevel() {
-            return delayLevel;
-        }
-
-        public String getMsgId() {
-            return msgId;
-        }
-
-        public boolean isAutoResend() {
-            return autoResend;
-        }
-
-        public CompletableFuture<PutMessageResult> getFuture() {
-            return future;
-        }
-
-        public int getResendCount() {
-            return resendCount;
-        }
-
-        public PutResultProcess thenProcess() {
-            this.future.thenAccept(result -> {
-                this.handleResult(result);
-            });
-
-            this.future.exceptionally(e -> {
-                log.error("ScheduleMessageService put message exceptionally, info: {}",
-                    PutResultProcess.this.toString(), e);
-
-                onException();
-                return null;
-            });
-            return this;
-        }
-
-        private void handleResult(PutMessageResult result) {
-            if (result != null && result.getPutMessageStatus() == PutMessageStatus.PUT_OK) {
-                onSuccess(result);
-            } else {
-                log.warn("ScheduleMessageService put message failed. info: {}.", result);
-                onException();
-            }
-        }
-
-        public void onSuccess(PutMessageResult result) {
-            this.status = ProcessStatus.SUCCESS;
-            if (ScheduleMessageService.this.defaultMessageStore.getMessageStoreConfig().isEnableScheduleMessageStats()) {
-                ScheduleMessageService.this.defaultMessageStore.getBrokerStatsManager().incQueueGetNums(MixAll.SCHEDULE_CONSUMER_GROUP, TopicValidator.RMQ_SYS_SCHEDULE_TOPIC, delayLevel - 1, result.getAppendMessageResult().getMsgNum());
-                ScheduleMessageService.this.defaultMessageStore.getBrokerStatsManager().incQueueGetSize(MixAll.SCHEDULE_CONSUMER_GROUP, TopicValidator.RMQ_SYS_SCHEDULE_TOPIC, delayLevel - 1, result.getAppendMessageResult().getWroteBytes());
-                ScheduleMessageService.this.defaultMessageStore.getBrokerStatsManager().incGroupGetNums(MixAll.SCHEDULE_CONSUMER_GROUP, TopicValidator.RMQ_SYS_SCHEDULE_TOPIC, result.getAppendMessageResult().getMsgNum());
-                ScheduleMessageService.this.defaultMessageStore.getBrokerStatsManager().incGroupGetSize(MixAll.SCHEDULE_CONSUMER_GROUP, TopicValidator.RMQ_SYS_SCHEDULE_TOPIC, result.getAppendMessageResult().getWroteBytes());
-                ScheduleMessageService.this.defaultMessageStore.getBrokerStatsManager().incTopicPutNums(this.topic, result.getAppendMessageResult().getMsgNum(), 1);
-                ScheduleMessageService.this.defaultMessageStore.getBrokerStatsManager().incTopicPutSize(this.topic, result.getAppendMessageResult().getWroteBytes());
-                ScheduleMessageService.this.defaultMessageStore.getBrokerStatsManager().incBrokerPutNums(result.getAppendMessageResult().getMsgNum());
-            }
-        }
-
-        public void onException() {
-            log.warn("ScheduleMessageService onException, info: {}", this.toString());
-            if (this.autoResend) {
-                this.resend();
-            } else {
-                this.status = ProcessStatus.SKIP;
-            }
-        }
-
-        public ProcessStatus getStatus() {
-            return this.status;
-        }
-
-        public PutMessageResult get() {
-            try {
-                return this.future.get();
-            } catch (InterruptedException | ExecutionException e) {
-                return new PutMessageResult(PutMessageStatus.UNKNOWN_ERROR, null);
-            }
-        }
-
-        private void resend() {
-            log.info("Resend message, info: {}", this.toString());
-
-            // Gradually increase the resend interval.
-            try {
-                Thread.sleep(Math.min(this.resendCount++ * 100, 60 * 1000));
-            } catch (InterruptedException e) {
-                e.printStackTrace();
-            }
-
-            try {
-                MessageExt msgExt = ScheduleMessageService.this.defaultMessageStore.lookMessageByOffset(this.physicOffset, this.physicSize);
-                if (msgExt == null) {
-                    log.warn("ScheduleMessageService resend not found message. info: {}", this.toString());
-                    this.status = need2Skip() ? ProcessStatus.SKIP : ProcessStatus.EXCEPTION;
-                    return;
-                }
-
-                MessageExtBrokerInner msgInner = ScheduleMessageService.this.messageTimeup(msgExt);
-                PutMessageResult result = ScheduleMessageService.this.writeMessageStore.putMessage(msgInner);
-                this.handleResult(result);
-                if (result != null && result.getPutMessageStatus() == PutMessageStatus.PUT_OK) {
-                    log.info("Resend message success, info: {}", this.toString());
-                }
-            } catch (Exception e) {
-                this.status = ProcessStatus.EXCEPTION;
-                log.error("Resend message error, info: {}", this.toString(), e);
-            }
-        }
-
-        public boolean need2Blocked() {
-            int maxResendNum2Blocked = ScheduleMessageService.this.defaultMessageStore.getMessageStoreConfig()
-                .getScheduleAsyncDeliverMaxResendNum2Blocked();
-            return this.resendCount > maxResendNum2Blocked;
-        }
-
-        public boolean need2Skip() {
-            int maxResendNum2Blocked = ScheduleMessageService.this.defaultMessageStore.getMessageStoreConfig()
-                .getScheduleAsyncDeliverMaxResendNum2Blocked();
-            return this.resendCount > maxResendNum2Blocked * 2;
-        }
-
-        @Override
-        public String toString() {
-            return "PutResultProcess{" +
-                "topic='" + topic + '\'' +
-                ", offset=" + offset +
-                ", physicOffset=" + physicOffset +
-                ", physicSize=" + physicSize +
-                ", delayLevel=" + delayLevel +
-                ", msgId='" + msgId + '\'' +
-                ", autoResend=" + autoResend +
-                ", resendCount=" + resendCount +
-                ", status=" + status +
-                '}';
-        }
-    }
-
-    public enum ProcessStatus {
-        /**
-         * In process, the processing result has not yet been returned.
-         */
-        RUNNING,
-
-        /**
-         * Put message success.
-         */
-        SUCCESS,
-
-        /**
-         * Put message exception. When autoResend is true, the message will be resend.
-         */
-        EXCEPTION,
-
-        /**
-         * Skip put message. When the message cannot be looked, the message will be skipped.
-         */
-        SKIP,
-    }
-}
diff --git a/store/src/test/java/org/apache/rocketmq/store/ScheduleMessageServiceTest.java b/store/src/test/java/org/apache/rocketmq/store/ScheduleMessageServiceTest.java
deleted file mode 100644
index 1c0451c..0000000
--- a/store/src/test/java/org/apache/rocketmq/store/ScheduleMessageServiceTest.java
+++ /dev/null
@@ -1,194 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one or more
- * contributor license agreements.  See the NOTICE file distributed with
- * this work for additional information regarding copyright ownership.
- * The ASF licenses this file to You under the Apache License, Version 2.0
- * (the "License"); you may not use this file except in compliance with
- * the License.  You may obtain a copy of the License at
- *
- *     http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-
-package org.apache.rocketmq.store;
-
-import java.lang.reflect.Field;
-import java.util.ArrayList;
-import java.util.Collections;
-import java.util.List;
-import java.util.Map;
-import java.util.Random;
-import java.util.concurrent.CompletableFuture;
-import java.util.concurrent.ConcurrentHashMap;
-import java.util.concurrent.ConcurrentMap;
-import java.util.concurrent.LinkedBlockingQueue;
-import java.util.concurrent.ScheduledExecutorService;
-import java.util.concurrent.ScheduledThreadPoolExecutor;
-import java.util.concurrent.TimeUnit;
-import java.util.concurrent.atomic.AtomicBoolean;
-import org.apache.rocketmq.common.BrokerConfig;
-import org.apache.rocketmq.common.ThreadFactoryImpl;
-import org.apache.rocketmq.common.message.MessageConst;
-import org.apache.rocketmq.common.message.MessageExt;
-import org.apache.rocketmq.store.config.FlushDiskType;
-import org.apache.rocketmq.store.config.MessageStoreConfig;
-import org.apache.rocketmq.store.schedule.ScheduleMessageService;
-import org.apache.rocketmq.store.stats.BrokerStatsManager;
-import org.junit.Assert;
-import org.junit.Test;
-
-import static org.junit.Assert.assertEquals;
-import static org.mockito.ArgumentMatchers.any;
-import static org.mockito.ArgumentMatchers.anyInt;
-import static org.mockito.ArgumentMatchers.anyLong;
-import static org.mockito.Mockito.mock;
-import static org.mockito.Mockito.when;
-
-public class ScheduleMessageServiceTest {
-
-    private Random random = new Random();
-
-    @Test
-    public void testCorrectDelayOffset_whenInit() throws Exception {
-
-        ConcurrentMap<Integer /* level */, Long/* offset */> offsetTable = null;
-
-        DefaultMessageStore defaultMessageStore = new DefaultMessageStore(buildMessageStoreConfig(),
-            new BrokerStatsManager("simpleTest", true), null, new BrokerConfig());
-        ScheduleMessageService scheduleMessageService = new ScheduleMessageService(defaultMessageStore);
-        scheduleMessageService.parseDelayLevel();
-
-        ConcurrentMap<Integer /* level */, Long/* offset */> offsetTable1 = new ConcurrentHashMap<>();
-        for (int i = 1; i <= 18; i++) {
-            offsetTable1.put(i, random.nextLong());
-        }
-
-        Field field = scheduleMessageService.getClass().getDeclaredField("offsetTable");
-        field.setAccessible(true);
-        field.set(scheduleMessageService, offsetTable1);
-
-        String jsonStr = scheduleMessageService.encode();
-        scheduleMessageService.decode(jsonStr);
-
-        offsetTable = (ConcurrentMap<Integer, Long>) field.get(scheduleMessageService);
-
-        for (Map.Entry<Integer, Long> entry : offsetTable.entrySet()) {
-            assertEquals(entry.getValue(), offsetTable1.get(entry.getKey()));
-        }
-
-        scheduleMessageService.correctDelayOffset();
-
-        offsetTable = (ConcurrentMap<Integer, Long>) field.get(scheduleMessageService);
-
-        for (long offset : offsetTable.values()) {
-            assertEquals(0, offset);
-        }
-
-    }
-
-    private MessageStoreConfig buildMessageStoreConfig() throws Exception {
-        MessageStoreConfig messageStoreConfig = new MessageStoreConfig();
-        messageStoreConfig.setMappedFileSizeCommitLog(1024 * 1024 * 10);
-        messageStoreConfig.setMappedFileSizeConsumeQueue(1024 * 1024 * 10);
-        messageStoreConfig.setMaxHashSlotNum(10000);
-        messageStoreConfig.setMaxIndexNum(100 * 100);
-        messageStoreConfig.setFlushDiskType(FlushDiskType.SYNC_FLUSH);
-        messageStoreConfig.setFlushIntervalConsumeQueue(1);
-        return messageStoreConfig;
-    }
-
-    @Test
-    public void testHandlePutResultTask() throws Exception {
-        DefaultMessageStore messageStore = mock(DefaultMessageStore.class);
-        MessageStoreConfig config = buildMessageStoreConfig();
-        config.setEnableScheduleMessageStats(false);
-        config.setEnableScheduleAsyncDeliver(true);
-        when(messageStore.getMessageStoreConfig()).thenReturn(config);
-        ScheduleMessageService scheduleMessageService = new ScheduleMessageService(messageStore);
-        scheduleMessageService.parseDelayLevel();
-
-        Field field = scheduleMessageService.getClass().getDeclaredField("deliverPendingTable");
-        field.setAccessible(true);
-        Map<Integer /* level */, LinkedBlockingQueue<ScheduleMessageService.PutResultProcess>> deliverPendingTable =
-            (Map<Integer, LinkedBlockingQueue<ScheduleMessageService.PutResultProcess>>) field.get(scheduleMessageService);
-
-        field = scheduleMessageService.getClass().getDeclaredField("offsetTable");
-        field.setAccessible(true);
-        ConcurrentMap<Integer /* level */, Long/* offset */> offsetTable =
-            (ConcurrentMap<Integer /* level */, Long/* offset */>) field.get(scheduleMessageService);
-        for (int i = 1; i <= scheduleMessageService.getMaxDelayLevel(); i++) {
-            offsetTable.put(i, 0L);
-        }
-
-        int deliverThreadPoolNums = Runtime.getRuntime().availableProcessors();
-        ScheduledExecutorService handleExecutorService = new ScheduledThreadPoolExecutor(deliverThreadPoolNums,
-            new ThreadFactoryImpl("ScheduleMessageExecutorHandleThread_"));
-        field = scheduleMessageService.getClass().getDeclaredField("handleExecutorService");
-        field.setAccessible(true);
-        field.set(scheduleMessageService, handleExecutorService);
-
-        field = scheduleMessageService.getClass().getDeclaredField("started");
-        field.setAccessible(true);
-        AtomicBoolean started = (AtomicBoolean) field.get(scheduleMessageService);
-        started.set(true);
-
-        for (int level = 1; level <= scheduleMessageService.getMaxDelayLevel(); level++) {
-            ScheduleMessageService.HandlePutResultTask handlePutResultTask = scheduleMessageService.new HandlePutResultTask(level);
-            handleExecutorService.schedule(handlePutResultTask, 10L, TimeUnit.MILLISECONDS);
-        }
-
-        MessageExt messageExt = new MessageExt();
-        messageExt.putUserProperty("init", "test");
-        messageExt.getProperties().put(MessageConst.PROPERTY_REAL_QUEUE_ID, "0");
-        when(messageStore.lookMessageByOffset(anyLong(), anyInt())).thenReturn(messageExt);
-        when(messageStore.putMessage(any())).thenReturn(new PutMessageResult(PutMessageStatus.PUT_OK, null));
-
-        int msgNum = 100;
-        int totalMsgNum = msgNum * scheduleMessageService.getMaxDelayLevel();
-        List<CompletableFuture<PutMessageResult>> putMsgFutrueList = new ArrayList<>(totalMsgNum);
-        for (int level = 1; level <= scheduleMessageService.getMaxDelayLevel(); level++) {
-            for (int num = 0; num < msgNum; num++) {
-                CompletableFuture<PutMessageResult> future = new CompletableFuture<>();
-                ScheduleMessageService.PutResultProcess putResultProcess = scheduleMessageService.new PutResultProcess();
-                putResultProcess = putResultProcess
-                    .setOffset(num)
-                    .setAutoResend(true)
-                    .setFuture(future)
-                    .thenProcess();
-                deliverPendingTable.get(level).add(putResultProcess);
-                putMsgFutrueList.add(future);
-            }
-        }
-
-        Collections.shuffle(putMsgFutrueList);
-        Random random = new Random();
-        for (CompletableFuture<PutMessageResult> future : putMsgFutrueList) {
-            PutMessageStatus status;
-            if (random.nextInt(1000) % 2 == 0) {
-                status = PutMessageStatus.PUT_OK;
-            } else {
-                status = PutMessageStatus.OS_PAGECACHE_BUSY;
-            }
-
-            if (random.nextInt(1000) % 2 == 0) {
-                PutMessageResult result = new PutMessageResult(status, null);
-                future.complete(result);
-            } else {
-                future.completeExceptionally(new Throwable("complete exceptionally"));
-            }
-        }
-
-        Thread.sleep(1000);
-        for (int level = 1; level <= scheduleMessageService.getMaxDelayLevel(); level++) {
-            Assert.assertEquals(0, deliverPendingTable.get(level).size());
-            Assert.assertEquals(msgNum, offsetTable.get(level).longValue());
-        }
-
-        scheduleMessageService.shutdown();
-    }
-}
diff --git a/store/src/test/java/org/apache/rocketmq/store/schedule/ScheduleMessageServiceTest.java b/store/src/test/java/org/apache/rocketmq/store/schedule/ScheduleMessageServiceTest.java
deleted file mode 100644
index de3cf7f..0000000
--- a/store/src/test/java/org/apache/rocketmq/store/schedule/ScheduleMessageServiceTest.java
+++ /dev/null
@@ -1,235 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one or more
- * contributor license agreements.  See the NOTICE file distributed with
- * this work for additional information regarding copyright ownership.
- * The ASF licenses this file to You under the Apache License, Version 2.0
- * (the "License"); you may not use this file except in compliance with
- * the License.  You may obtain a copy of the License at
- *
- *     http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-
-package org.apache.rocketmq.store.schedule;
-
-import org.apache.rocketmq.common.BrokerConfig;
-import org.apache.rocketmq.common.UtilAll;
-import org.apache.rocketmq.common.message.MessageDecoder;
-import org.apache.rocketmq.common.message.MessageExt;
-import org.apache.rocketmq.store.*;
-import org.apache.rocketmq.store.config.MessageStoreConfig;
-import org.apache.rocketmq.store.stats.BrokerStatsManager;
-import org.junit.After;
-import org.junit.Before;
-import org.junit.Test;
-
-import java.io.File;
-import java.net.InetAddress;
-import java.net.InetSocketAddress;
-import java.net.SocketAddress;
-import java.net.UnknownHostException;
-import java.nio.ByteBuffer;
-import java.util.HashMap;
-import java.util.List;
-import java.util.Map;
-import java.util.UUID;
-import java.util.concurrent.TimeUnit;
-
-import static org.apache.rocketmq.store.stats.BrokerStatsManager.BROKER_PUT_NUMS;
-import static org.apache.rocketmq.store.stats.BrokerStatsManager.TOPIC_PUT_NUMS;
-import static org.apache.rocketmq.store.stats.BrokerStatsManager.TOPIC_PUT_SIZE;
-import static org.assertj.core.api.Assertions.assertThat;
-
-
-public class ScheduleMessageServiceTest {
-
-
-    /**
-     * t
-     * defaultMessageDelayLevel = "1s 5s 10s 30s 1m 2m 3m 4m 5m 6m 7m 8m 9m 10m 20m 30m 1h 2h"
-     */
-    String testMessageDelayLevel = "5s 8s";
-    /**
-     * choose delay level
-     */
-    int delayLevel = 2;
-
-    private static final String storePath = System.getProperty("user.home") + File.separator + "schedule_test#" + UUID.randomUUID();
-    private static final int commitLogFileSize = 1024;
-    private static final int cqFileSize = 10;
-    private static final int cqExtFileSize = 10 * (ConsumeQueueExt.CqExtUnit.MIN_EXT_UNIT_SIZE + 64);
-
-    private static SocketAddress bornHost;
-    private static SocketAddress storeHost;
-    DefaultMessageStore messageStore;
-    MessageStoreConfig messageStoreConfig;
-    BrokerConfig brokerConfig;
-    ScheduleMessageService scheduleMessageService;
-
-    static String sendMessage = " ------- schedule message test -------";
-    static String topic = "schedule_topic_test";
-    static String messageGroup = "delayGroupTest";
-
-
-    static {
-        try {
-            bornHost = new InetSocketAddress(InetAddress.getLocalHost(), 8123);
-        } catch (UnknownHostException e) {
-            e.printStackTrace();
-        }
-        try {
-            storeHost = new InetSocketAddress(InetAddress.getByName("127.0.0.1"), 0);
-        } catch (UnknownHostException e) {
-            e.printStackTrace();
-        }
-    }
-
-
-    @Before
-    public void init() throws Exception {
-        messageStoreConfig = new MessageStoreConfig();
-        messageStoreConfig.setMessageDelayLevel(testMessageDelayLevel);
-        messageStoreConfig.setMappedFileSizeCommitLog(commitLogFileSize);
-        messageStoreConfig.setMappedFileSizeConsumeQueue(cqFileSize);
-        messageStoreConfig.setMappedFileSizeConsumeQueueExt(cqExtFileSize);
-        messageStoreConfig.setMessageIndexEnable(false);
-        messageStoreConfig.setEnableConsumeQueueExt(true);
-        messageStoreConfig.setStorePathRootDir(storePath);
-        messageStoreConfig.setStorePathCommitLog(storePath + File.separator + "commitlog");
-
-        brokerConfig = new BrokerConfig();
-        BrokerStatsManager manager = new BrokerStatsManager(brokerConfig.getBrokerClusterName(), brokerConfig.isEnableDetailStat());
-        messageStore = new DefaultMessageStore(messageStoreConfig, manager, new MyMessageArrivingListener(), new BrokerConfig());
-
-        assertThat(messageStore.load()).isTrue();
-
-        messageStore.start();
-        scheduleMessageService = messageStore.getScheduleMessageService();
-    }
-
-
-    @Test
-    public void deliverDelayedMessageTimerTaskTest() throws Exception {
-        assertThat(messageStore.getMessageStoreConfig().isEnableScheduleMessageStats()).isTrue();
-
-        assertThat(messageStore.getBrokerStatsManager().getStatsItem(TOPIC_PUT_NUMS, topic)).isNull();
-
-        MessageExtBrokerInner msg = buildMessage();
-        int realQueueId = msg.getQueueId();
-        // set delayLevel,and send delay message
-        msg.setDelayTimeLevel(delayLevel);
-        PutMessageResult result = messageStore.putMessage(msg);
-        assertThat(result.isOk()).isTrue();
-
-        // make sure consumerQueue offset = commitLog offset
-        StoreTestUtil.waitCommitLogReput(messageStore);
-
-        // consumer message
-        int delayQueueId = ScheduleMessageService.delayLevel2QueueId(delayLevel);
-        assertThat(delayQueueId).isEqualTo(delayLevel - 1);
-
-        Long offset = result.getAppendMessageResult().getLogicsOffset();
-
-        // now, no message in queue,must wait > delayTime
-        GetMessageResult messageResult = getMessage(realQueueId, offset);
-        assertThat(messageResult.getStatus()).isEqualTo(GetMessageStatus.NO_MESSAGE_IN_QUEUE);
-
-        // timer run maybe delay, then consumer message again
-        // and wait offsetTable
-        TimeUnit.SECONDS.sleep(10);
-        scheduleMessageService.buildRunningStats(new HashMap<String, String>());
-
-        messageResult = getMessage(realQueueId, offset);
-        // now,found the message
-        assertThat(messageResult.getStatus()).isEqualTo(GetMessageStatus.FOUND);
-
-        // get the stats change
-        assertThat(messageStore.getBrokerStatsManager().getStatsItem(BROKER_PUT_NUMS, brokerConfig.getBrokerClusterName()).getValue().sum()).isEqualTo(1);
-        assertThat(messageStore.getBrokerStatsManager().getStatsItem(TOPIC_PUT_NUMS, topic).getValue().sum()).isEqualTo(1L);
-        assertThat(messageStore.getBrokerStatsManager().getStatsItem(TOPIC_PUT_SIZE, topic).getValue().sum()).isEqualTo(messageResult.getBufferTotalSize());
-
-        // get the message body
-        ByteBuffer byteBuffer = ByteBuffer.allocate(messageResult.getBufferTotalSize());
-        List<ByteBuffer> byteBufferList = messageResult.getMessageBufferList();
-        for (ByteBuffer bb : byteBufferList) {
-            byteBuffer.put(bb);
-        }
-
-        // warp and decode the message
-        byteBuffer = ByteBuffer.wrap(byteBuffer.array());
-        List<MessageExt> msgList = MessageDecoder.decodes(byteBuffer);
-        String retryMsg = new String(msgList.get(0).getBody());
-        assertThat(sendMessage).isEqualTo(retryMsg);
-
-        //  method will wait 10s,so I run it by myself
-        scheduleMessageService.persist();
-
-        // add mapFile release
-        messageResult.release();
-
-    }
-
-    /**
-     * add some [error/no use] code test
-     */
-    @Test
-    public void otherTest() {
-        // the method no use ,why need ?
-        int queueId = ScheduleMessageService.queueId2DelayLevel(delayLevel);
-        assertThat(queueId).isEqualTo(delayLevel + 1);
-
-        // error delayLevelTest
-        Long time = scheduleMessageService.computeDeliverTimestamp(999, 0);
-        assertThat(time).isEqualTo(1000);
-
-        // just decode
-        scheduleMessageService.decode(new DelayOffsetSerializeWrapper().toJson());
-    }
-
-
-    private GetMessageResult getMessage(int queueId, Long offset) {
-        return messageStore.getMessage(messageGroup, topic,
-                queueId, offset, 1, null);
-
-    }
-
-
-    @After
-    public void shutdown() throws InterruptedException {
-        messageStore.shutdown();
-        messageStore.destroy();
-        File file = new File(messageStoreConfig.getStorePathRootDir());
-        UtilAll.deleteFile(file);
-    }
-
-
-    public MessageExtBrokerInner buildMessage() {
-
-        byte[] msgBody = sendMessage.getBytes();
-        MessageExtBrokerInner msg = new MessageExtBrokerInner();
-        msg.setTopic(topic);
-        msg.setTags("schedule_tag");
-        msg.setKeys("schedule_key");
-        msg.setBody(msgBody);
-        msg.setSysFlag(0);
-        msg.setBornTimestamp(System.currentTimeMillis());
-        msg.setStoreHost(storeHost);
-        msg.setBornHost(bornHost);
-        return msg;
-    }
-
-
-    private class MyMessageArrivingListener implements MessageArrivingListener {
-        @Override
-        public void arriving(String topic, int queueId, long logicOffset, long tagsCode, long msgStoreTime,
-                             byte[] filterBitMap, Map<String, String> properties) {
-        }
-    }
-
-
-}

[rocketmq] 06/07: feature(test):[RIP-31]Add ITs for BrokerContainer (#3980)

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

jinrongtong pushed a commit to branch 5.0.0-beta-tmp
in repository https://gitbox.apache.org/repos/asf/rocketmq.git

commit 00d2d1c40c622d267b20af1c73bfd37ff74f62fd
Author: rongtong <ji...@163.com>
AuthorDate: Mon Mar 14 14:07:18 2022 +0800

    feature(test):[RIP-31]Add ITs for BrokerContainer (#3980)
---
 test/pom.xml                                       |   4 +
 .../rocketmq/test/util/MQAdminTestUtils.java       |   2 +-
 .../org/apache/rocketmq/test/base/BaseConf.java    |  26 +-
 .../test/client/producer/batch/BatchSendIT.java    |   2 +-
 .../client/producer/order/OrderMsgRebalanceIT.java |   2 +-
 .../test/container/AddAndRemoveBrokerIT.java       |  81 +++
 .../rocketmq/test/container/BrokerFailoverIT.java  |  89 +++
 .../test/container/BrokerMemberGroupIT.java        |  69 +++
 .../container/ContainerIntegrationTestBase.java    | 666 +++++++++++++++++++++
 .../test/container/GetMaxOffsetFromSlaveIT.java    |  98 +++
 .../test/container/GetMetadataReverseIT.java       | 231 +++++++
 .../test/container/PullMultipleReplicasIT.java     | 199 ++++++
 .../test/container/PushMultipleReplicasIT.java     | 112 ++++
 .../test/container/RebalanceLockOnSlaveIT.java     | 208 +++++++
 .../container/ScheduleSlaveActingMasterIT.java     | 220 +++++++
 .../test/container/ScheduledMessageIT.java         | 152 +++++
 .../test/container/SendMultipleReplicasIT.java     | 157 +++++
 .../rocketmq/test/container/SlaveBrokerIT.java     | 115 ++++
 .../test/container/SyncConsumerOffsetIT.java       | 146 +++++
 .../rocketmq/test/statictopic/StaticTopicIT.java   |  15 +-
 20 files changed, 2568 insertions(+), 26 deletions(-)

diff --git a/test/pom.xml b/test/pom.xml
index 2bdab7c..3cf4f2d 100644
--- a/test/pom.xml
+++ b/test/pom.xml
@@ -41,6 +41,10 @@
             <artifactId>rocketmq-namesrv</artifactId>
         </dependency>
         <dependency>
+            <groupId>${project.groupId}</groupId>
+            <artifactId>rocketmq-container</artifactId>
+        </dependency>
+        <dependency>
             <groupId>com.google.truth</groupId>
             <artifactId>truth</artifactId>
             <version>0.30</version>
diff --git a/test/src/main/java/org/apache/rocketmq/test/util/MQAdminTestUtils.java b/test/src/main/java/org/apache/rocketmq/test/util/MQAdminTestUtils.java
index 20149d4..7f6a2b6 100644
--- a/test/src/main/java/org/apache/rocketmq/test/util/MQAdminTestUtils.java
+++ b/test/src/main/java/org/apache/rocketmq/test/util/MQAdminTestUtils.java
@@ -140,7 +140,7 @@ public class MQAdminTestUtils {
         if (clusterInfo == null) {
             return false;
         } else {
-            HashMap<String, BrokerData> brokers = clusterInfo.getBrokerAddrTable();
+            Map<String, BrokerData> brokers = clusterInfo.getBrokerAddrTable();
             for (String brokerName : brokers.keySet()) {
                 HashMap<Long, String> brokerIps = brokers.get(brokerName).getBrokerAddrs();
                 for (long brokerId : brokerIps.keySet()) {
diff --git a/test/src/test/java/org/apache/rocketmq/test/base/BaseConf.java b/test/src/test/java/org/apache/rocketmq/test/base/BaseConf.java
index c523fd9..8420fdd 100644
--- a/test/src/test/java/org/apache/rocketmq/test/base/BaseConf.java
+++ b/test/src/test/java/org/apache/rocketmq/test/base/BaseConf.java
@@ -96,17 +96,17 @@ public class BaseConf {
     }
 
     // This method can't be placed in the static block of BaseConf, which seems to lead to a strange dead lock.
-    public static void waitBrokerRegistered(final String nsAddr, final String clusterName,
-        final int expectedBrokerNum) {
+    public static void waitBrokerRegistered(final String nsAddr, final String clusterName, final int expectedBrokerNum) {
         final DefaultMQAdminExt mqAdminExt = new DefaultMQAdminExt(500);
         mqAdminExt.setNamesrvAddr(nsAddr);
         try {
             mqAdminExt.start();
+            Thread.sleep(10000);
             await().atMost(30, TimeUnit.SECONDS).until(() -> {
                 List<BrokerData> brokerDatas = mqAdminExt.examineTopicRouteInfo(clusterName).getBrokerDatas();
                 return brokerDatas.size() == expectedBrokerNum;
             });
-            for (BrokerController brokerController : brokerControllerList) {
+            for (BrokerController brokerController: brokerControllerList) {
                 brokerController.getBrokerOuterAPI().refreshMetadata();
             }
         } catch (Exception e) {
@@ -144,6 +144,7 @@ public class BaseConf {
         return mqAdminExt;
     }
 
+
     public static RMQNormalProducer getProducer(String nsAddr, String topic) {
         return getProducer(nsAddr, topic, false);
     }
@@ -157,8 +158,7 @@ public class BaseConf {
         return producer;
     }
 
-    public static RMQTransactionalProducer getTransactionalProducer(String nsAddr, String topic,
-        TransactionListener transactionListener) {
+    public static RMQTransactionalProducer getTransactionalProducer(String nsAddr, String topic, TransactionListener transactionListener) {
         RMQTransactionalProducer producer = new RMQTransactionalProducer(nsAddr, topic, false, transactionListener);
         if (debug) {
             producer.setDebug();
@@ -168,9 +168,9 @@ public class BaseConf {
     }
 
     public static RMQNormalProducer getProducer(String nsAddr, String topic, String producerGoup,
-        String instanceName) {
+                                                String instanceName) {
         RMQNormalProducer producer = new RMQNormalProducer(nsAddr, topic, producerGoup,
-            instanceName);
+                instanceName);
         if (debug) {
             producer.setDebug();
         }
@@ -188,31 +188,31 @@ public class BaseConf {
     }
 
     public static RMQNormalConsumer getConsumer(String nsAddr, String topic, String subExpression,
-        AbstractListener listener) {
+                                                AbstractListener listener) {
         return getConsumer(nsAddr, topic, subExpression, listener, false);
     }
 
     public static RMQNormalConsumer getConsumer(String nsAddr, String topic, String subExpression,
-        AbstractListener listener, boolean useTLS) {
+                                                AbstractListener listener, boolean useTLS) {
         String consumerGroup = initConsumerGroup();
         return getConsumer(nsAddr, consumerGroup, topic, subExpression, listener, useTLS);
     }
 
     public static RMQNormalConsumer getConsumer(String nsAddr, String consumerGroup, String topic,
-        String subExpression, AbstractListener listener) {
+                                                String subExpression, AbstractListener listener) {
         return getConsumer(nsAddr, consumerGroup, topic, subExpression, listener, false);
     }
 
     public static RMQNormalConsumer getConsumer(String nsAddr, String consumerGroup, String topic,
-        String subExpression, AbstractListener listener, boolean useTLS) {
+                                                String subExpression, AbstractListener listener, boolean useTLS) {
         RMQNormalConsumer consumer = ConsumerFactory.getRMQNormalConsumer(nsAddr, consumerGroup,
-            topic, subExpression, listener, useTLS);
+                topic, subExpression, listener, useTLS);
         if (debug) {
             consumer.setDebug();
         }
         mqClients.add(consumer);
         log.info(String.format("consumer[%s] start,topic[%s],subExpression[%s]", consumerGroup,
-            topic, subExpression));
+                topic, subExpression));
         return consumer;
     }
 
diff --git a/test/src/test/java/org/apache/rocketmq/test/client/producer/batch/BatchSendIT.java b/test/src/test/java/org/apache/rocketmq/test/client/producer/batch/BatchSendIT.java
index 3a649ed..283dcbe 100644
--- a/test/src/test/java/org/apache/rocketmq/test/client/producer/batch/BatchSendIT.java
+++ b/test/src/test/java/org/apache/rocketmq/test/client/producer/batch/BatchSendIT.java
@@ -137,6 +137,7 @@ public class BatchSendIT extends BaseConf {
         Thread.sleep(300);
         {
             DefaultMQPullConsumer defaultMQPullConsumer = ConsumerFactory.getRMQPullConsumer(nsAddr, "group");
+            System.out.println(defaultMQPullConsumer.maxOffset(messageQueue));
 
             PullResult pullResult = defaultMQPullConsumer.pullBlockIfNotFound(messageQueue, "*", 5, batchCount * batchNum);
             Assert.assertEquals(PullStatus.FOUND, pullResult.getPullStatus());
@@ -181,7 +182,6 @@ public class BatchSendIT extends BaseConf {
         DefaultMQProducer producer = ProducerFactory.getRMQProducer(nsAddr);
         MessageQueue messageQueue = producer.fetchPublishMessageQueues(batchTopic).iterator().next();
 
-
         int batchCount = 10;
         int batchNum = 10;
         for (int i = 0; i < batchCount; i++) {
diff --git a/test/src/test/java/org/apache/rocketmq/test/client/producer/order/OrderMsgRebalanceIT.java b/test/src/test/java/org/apache/rocketmq/test/client/producer/order/OrderMsgRebalanceIT.java
index bae5397..eff70a0 100644
--- a/test/src/test/java/org/apache/rocketmq/test/client/producer/order/OrderMsgRebalanceIT.java
+++ b/test/src/test/java/org/apache/rocketmq/test/client/producer/order/OrderMsgRebalanceIT.java
@@ -81,7 +81,7 @@ public class OrderMsgRebalanceIT extends BaseConf {
     }
 
     @Test
-    public void testFourConsuemrBalance() {
+    public void testFourConsumerBalance() {
         int msgSize = 20;
         RMQNormalConsumer consumer1 = getConsumer(nsAddr, topic, "*", new RMQOrderListener());
         RMQNormalConsumer consumer2 = getConsumer(nsAddr, consumer1.getConsumerGroup(), topic,
diff --git a/test/src/test/java/org/apache/rocketmq/test/container/AddAndRemoveBrokerIT.java b/test/src/test/java/org/apache/rocketmq/test/container/AddAndRemoveBrokerIT.java
new file mode 100644
index 0000000..c19a787
--- /dev/null
+++ b/test/src/test/java/org/apache/rocketmq/test/container/AddAndRemoveBrokerIT.java
@@ -0,0 +1,81 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.rocketmq.test.container;
+
+import org.apache.rocketmq.container.BrokerContainer;
+import org.apache.rocketmq.client.exception.MQBrokerException;
+import org.apache.rocketmq.common.protocol.ResponseCode;
+import org.apache.rocketmq.remoting.exception.RemotingConnectException;
+import org.apache.rocketmq.remoting.exception.RemotingSendRequestException;
+import org.apache.rocketmq.remoting.exception.RemotingTimeoutException;
+import org.junit.AfterClass;
+import org.junit.BeforeClass;
+import org.junit.Test;
+
+import static org.assertj.core.api.Assertions.assertThat;
+
+public class AddAndRemoveBrokerIT extends ContainerIntegrationTestBase {
+    private static BrokerContainer brokerContainer4;
+
+    @BeforeClass
+    public static void beforeClass() {
+        brokerContainer4 = createAndStartBrokerContainer(nsAddr);
+    }
+
+    @AfterClass
+    public static void afterClass() {
+        brokerContainer4.shutdown();
+    }
+
+    @Test
+    public void addBrokerTest()
+        throws InterruptedException, RemotingTimeoutException, RemotingSendRequestException,
+        RemotingConnectException {
+        String remark = null;
+        int code = 0;
+        try {
+            defaultMQAdminExt.addBrokerToContainer(brokerContainer4.getBrokerContainerAddr(), "");
+        } catch (MQBrokerException e) {
+            code = e.getResponseCode();
+            remark = e.getErrorMessage();
+        }
+        assertThat(code).isEqualTo(ResponseCode.SYSTEM_ERROR);
+        assertThat(remark).isEqualTo("addBroker properties empty");
+    }
+
+    @Test
+    public void removeBrokerTest()
+        throws InterruptedException, RemotingTimeoutException, RemotingSendRequestException, RemotingConnectException{
+
+        boolean exceptionCaught = false;
+
+        try {
+            defaultMQAdminExt.removeBrokerFromContainer(brokerContainer1.getBrokerContainerAddr(),
+                master3With3Replicas.getBrokerConfig().getBrokerClusterName(),
+                master3With3Replicas.getBrokerConfig().getBrokerName(), 1);
+        } catch (MQBrokerException e) {
+            exceptionCaught = true;
+        }
+
+        assertThat(exceptionCaught).isFalse();
+        assertThat(brokerContainer1.getSlaveBrokers().size()).isEqualTo(1);
+
+        createAndAddSlave(1, brokerContainer1, master3With3Replicas);
+        awaitUntilSlaveOK();
+    }
+}
diff --git a/test/src/test/java/org/apache/rocketmq/test/container/BrokerFailoverIT.java b/test/src/test/java/org/apache/rocketmq/test/container/BrokerFailoverIT.java
new file mode 100644
index 0000000..b4e4baf
--- /dev/null
+++ b/test/src/test/java/org/apache/rocketmq/test/container/BrokerFailoverIT.java
@@ -0,0 +1,89 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.rocketmq.test.container;
+
+import java.time.Duration;
+import org.apache.rocketmq.container.InnerSalveBrokerController;
+import org.apache.rocketmq.common.protocol.RequestCode;
+import org.apache.rocketmq.remoting.RPCHook;
+import org.apache.rocketmq.remoting.protocol.RemotingCommand;
+import org.junit.Test;
+
+import static org.assertj.core.api.Assertions.assertThat;
+import static org.awaitility.Awaitility.await;
+
+public class BrokerFailoverIT extends ContainerIntegrationTestBase {
+
+    @Test
+    public void testBrokerFailoverWithoutCompatible() {
+        changeCompatibleMode(false);
+        awaitUntilSlaveOK();
+        testBrokerFailover(false);
+    }
+
+    @Test
+    public void testBrokerFailoverWithCompatible() {
+        changeCompatibleMode(true);
+        awaitUntilSlaveOK();
+        testBrokerFailover(true);
+    }
+
+    private void testBrokerFailover(boolean compatibleMode) {
+        await().atMost(Duration.ofSeconds(10)).until(() ->
+            master1With3Replicas.getMessageStore().getAliveReplicaNumInGroup() == 3
+                && master2With3Replicas.getMessageStore().getAliveReplicaNumInGroup() == 3
+                && master3With3Replicas.getMessageStore().getAliveReplicaNumInGroup() == 3);
+
+        InnerSalveBrokerController targetSlave = getSlaveFromContainerByName(brokerContainer2, master1With3Replicas.getBrokerConfig().getBrokerName());
+
+        assertThat(targetSlave).isNotNull();
+
+        brokerContainer1.registerClientRPCHook(new RPCHook() {
+            @Override
+            public void doBeforeRequest(String remoteAddr, RemotingCommand request) {
+                if (request.getCode() == (compatibleMode ? RequestCode.QUERY_DATA_VERSION : RequestCode.BROKER_HEARTBEAT)) {
+                    request.setCode(-1);
+                }
+            }
+
+            @Override
+            public void doAfterResponse(String remoteAddr, RemotingCommand request,
+                RemotingCommand response) {
+
+            }
+
+            @Override
+            public void doAfterRpcFailure(String remoteAddr, RemotingCommand request, Boolean remoteTimeout) {
+
+            }
+        });
+
+        InnerSalveBrokerController finalTargetSlave = targetSlave;
+        await().atMost(Duration.ofSeconds(60)).until(() ->
+            finalTargetSlave.getMessageStore().getAliveReplicaNumInGroup() == 2
+                && master2With3Replicas.getMessageStore().getAliveReplicaNumInGroup() == 2
+                && master3With3Replicas.getMessageStore().getAliveReplicaNumInGroup() == 2);
+
+        brokerContainer1.clearClientRPCHook();
+
+        await().atMost(Duration.ofSeconds(60)).until(() ->
+            master1With3Replicas.getMessageStore().getAliveReplicaNumInGroup() == 3
+                && master2With3Replicas.getMessageStore().getAliveReplicaNumInGroup() == 3
+                && master3With3Replicas.getMessageStore().getAliveReplicaNumInGroup() == 3);
+    }
+}
diff --git a/test/src/test/java/org/apache/rocketmq/test/container/BrokerMemberGroupIT.java b/test/src/test/java/org/apache/rocketmq/test/container/BrokerMemberGroupIT.java
new file mode 100644
index 0000000..ec9b6b1
--- /dev/null
+++ b/test/src/test/java/org/apache/rocketmq/test/container/BrokerMemberGroupIT.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.test.container;
+
+import java.time.Duration;
+import org.apache.rocketmq.common.BrokerConfig;
+import org.apache.rocketmq.common.protocol.body.BrokerMemberGroup;
+import org.junit.Test;
+
+import static org.awaitility.Awaitility.await;
+
+public class BrokerMemberGroupIT extends ContainerIntegrationTestBase {
+    @Test
+    public void testSyncBrokerMemberGroup() throws Exception {
+        await().atMost(Duration.ofSeconds(5)).until(() -> {
+            final BrokerConfig brokerConfig = master1With3Replicas.getBrokerConfig();
+            final BrokerMemberGroup memberGroup = master1With3Replicas.getBrokerOuterAPI()
+                .syncBrokerMemberGroup(brokerConfig.getBrokerClusterName(), brokerConfig.getBrokerName());
+
+            return memberGroup.getBrokerAddrs().size() == 3;
+        });
+
+        await().atMost(Duration.ofSeconds(5)).until(() -> {
+            final BrokerConfig brokerConfig = master3With3Replicas.getBrokerConfig();
+            final BrokerMemberGroup memberGroup = master3With3Replicas.getBrokerOuterAPI()
+                .syncBrokerMemberGroup(brokerConfig.getBrokerClusterName(), brokerConfig.getBrokerName());
+
+            return memberGroup.getBrokerAddrs().size() == 3;
+        });
+
+        removeSlaveBroker(1, brokerContainer1, master3With3Replicas);
+        removeSlaveBroker(1, brokerContainer2, master1With3Replicas);
+
+        await().atMost(Duration.ofSeconds(5)).until(() -> {
+            final BrokerConfig brokerConfig = master1With3Replicas.getBrokerConfig();
+            final BrokerMemberGroup memberGroup = master1With3Replicas.getBrokerOuterAPI()
+                .syncBrokerMemberGroup(brokerConfig.getBrokerClusterName(), brokerConfig.getBrokerName());
+
+            return memberGroup.getBrokerAddrs().size() == 2 && memberGroup.getBrokerAddrs().get(1L) == null;
+        });
+
+        await().atMost(Duration.ofSeconds(5)).until(() -> {
+            final BrokerConfig brokerConfig = master3With3Replicas.getBrokerConfig();
+            final BrokerMemberGroup memberGroup = master3With3Replicas.getBrokerOuterAPI()
+                .syncBrokerMemberGroup(brokerConfig.getBrokerClusterName(), brokerConfig.getBrokerName());
+            return memberGroup.getBrokerAddrs().size() == 2 && memberGroup.getBrokerAddrs().get(1L) == null;
+        });
+
+        createAndAddSlave(1, brokerContainer2, master1With3Replicas);
+        createAndAddSlave(1, brokerContainer1, master3With3Replicas);
+
+        awaitUntilSlaveOK();
+    }
+}
diff --git a/test/src/test/java/org/apache/rocketmq/test/container/ContainerIntegrationTestBase.java b/test/src/test/java/org/apache/rocketmq/test/container/ContainerIntegrationTestBase.java
new file mode 100644
index 0000000..fab4c46
--- /dev/null
+++ b/test/src/test/java/org/apache/rocketmq/test/container/ContainerIntegrationTestBase.java
@@ -0,0 +1,666 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.rocketmq.test.container;
+
+import ch.qos.logback.classic.LoggerContext;
+import ch.qos.logback.classic.joran.JoranConfigurator;
+import io.netty.channel.ChannelHandlerContext;
+import java.io.File;
+import java.io.IOException;
+import java.nio.file.Files;
+import java.time.Duration;
+import java.util.ArrayList;
+import java.util.HashSet;
+import java.util.List;
+import java.util.Random;
+import java.util.Set;
+import java.util.UUID;
+import java.util.concurrent.ConcurrentHashMap;
+import java.util.concurrent.ConcurrentMap;
+import java.util.concurrent.TimeUnit;
+import java.util.concurrent.atomic.AtomicBoolean;
+import java.util.concurrent.atomic.AtomicInteger;
+import org.apache.commons.lang3.builder.EqualsBuilder;
+import org.apache.commons.lang3.builder.HashCodeBuilder;
+import org.apache.rocketmq.container.BrokerContainer;
+import org.apache.rocketmq.container.InnerSalveBrokerController;
+import org.apache.rocketmq.broker.BrokerController;
+import org.apache.rocketmq.client.consumer.DefaultMQPullConsumer;
+import org.apache.rocketmq.client.consumer.DefaultMQPushConsumer;
+import org.apache.rocketmq.client.producer.DefaultMQProducer;
+import org.apache.rocketmq.client.producer.TransactionCheckListener;
+import org.apache.rocketmq.client.producer.TransactionMQProducer;
+import org.apache.rocketmq.common.BrokerConfig;
+import org.apache.rocketmq.container.BrokerContainerConfig;
+import org.apache.rocketmq.common.BrokerIdentity;
+import org.apache.rocketmq.common.MQVersion;
+import org.apache.rocketmq.common.TopicConfig;
+import org.apache.rocketmq.common.UtilAll;
+import org.apache.rocketmq.common.message.MessageQueue;
+import org.apache.rocketmq.common.namesrv.NamesrvConfig;
+import org.apache.rocketmq.common.protocol.RequestCode;
+import org.apache.rocketmq.common.protocol.header.namesrv.RegisterBrokerRequestHeader;
+import org.apache.rocketmq.common.subscription.SubscriptionGroupConfig;
+import org.apache.rocketmq.logging.InternalLogger;
+import org.apache.rocketmq.logging.InternalLoggerFactory;
+import org.apache.rocketmq.namesrv.NamesrvController;
+import org.apache.rocketmq.remoting.netty.NettyClientConfig;
+import org.apache.rocketmq.remoting.netty.NettyRequestProcessor;
+import org.apache.rocketmq.remoting.netty.NettyServerConfig;
+import org.apache.rocketmq.remoting.protocol.RemotingCommand;
+import org.apache.rocketmq.store.config.BrokerRole;
+import org.apache.rocketmq.store.config.MessageStoreConfig;
+import org.apache.rocketmq.store.ha.HAConnection;
+import org.apache.rocketmq.store.ha.HAConnectionState;
+import org.apache.rocketmq.tools.admin.DefaultMQAdminExt;
+import org.junit.Assert;
+import org.junit.BeforeClass;
+import org.slf4j.LoggerFactory;
+
+import static org.awaitility.Awaitility.await;
+
+/**
+ * ContainerIntegrationTestBase will setup a rocketmq ha cluster contains two broker group:
+ * <li>BrokerA contains two replicas</li>
+ * <li>BrokerB contains three replicas</li>
+ */
+public class ContainerIntegrationTestBase {
+    private static final AtomicBoolean CLUSTER_SET_UP = new AtomicBoolean(false);
+    private static final List<File> TMP_FILE_LIST = new ArrayList<>();
+    private static final Random RANDOM = new Random();
+    protected static String nsAddr;
+
+    protected static final String THREE_REPLICAS_TOPIC = "SEND_MESSAGE_TEST_TOPIC_THREE_REPLICAS";
+
+    protected static final List<BrokerContainer> brokerContainerList = new ArrayList<>();
+    protected static final List<NamesrvController> namesrvControllers = new ArrayList<>();
+
+    protected static final String BROKER_NAME_PREFIX = "TestBrokerName_";
+    protected static final int COMMIT_LOG_SIZE = 128 * 1024;
+    protected static final int INDEX_NUM = 1000;
+    protected static final AtomicInteger BROKER_INDEX = new AtomicInteger(0);
+
+    protected static BrokerContainer brokerContainer1;
+    protected static BrokerContainer brokerContainer2;
+    protected static BrokerContainer brokerContainer3;
+    protected static BrokerController master1With3Replicas;
+    protected static BrokerController master2With3Replicas;
+    protected static BrokerController master3With3Replicas;
+    protected static NamesrvController namesrvController;
+
+    protected static DefaultMQAdminExt defaultMQAdminExt;
+
+    private final static InternalLogger LOG = InternalLoggerFactory.getLogger(ContainerIntegrationTestBase.class);
+    private static ConcurrentMap<BrokerConfig, MessageStoreConfig> slaveStoreConfigCache = new ConcurrentHashMap<>();
+
+    protected static ConcurrentMap<BrokerConfigLite, BrokerController> isolatedBrokers = new ConcurrentHashMap<>();
+    private static final Set<Integer> PORTS_IN_USE = new HashSet<>();
+
+    @BeforeClass
+    public static void setUp() throws Exception {
+        if (CLUSTER_SET_UP.compareAndSet(false, true)) {
+            System.setProperty(RemotingCommand.REMOTING_VERSION_KEY, Integer.toString(MQVersion.CURRENT_VERSION));
+            System.setProperty("rocketmq.broker.diskSpaceCleanForciblyRatio", "0.99");
+            System.setProperty("rocketmq.broker.diskSpaceWarningLevelRatio", "0.99");
+
+            LoggerContext lc = (LoggerContext) LoggerFactory.getILoggerFactory();
+            JoranConfigurator configurator = new JoranConfigurator();
+            configurator.setContext(lc);
+            lc.reset();
+            //https://logback.qos.ch/manual/configuration.html
+            lc.setPackagingDataEnabled(false);
+
+            configurator.doConfigure("../distribution/conf/logback_broker.xml");
+            configurator.doConfigure("../distribution/conf/logback_namesrv.xml");
+
+            setUpCluster();
+            setUpTopic();
+            registerCleaner();
+
+            System.out.printf("cluster setup complete%n");
+        }
+    }
+
+    private static void setUpTopic() {
+        createTopic(THREE_REPLICAS_TOPIC);
+    }
+
+    private static void createTopic(String topic) {
+        createTopicTo(master1With3Replicas, topic);
+        createTopicTo(master2With3Replicas, topic);
+        createTopicTo(master3With3Replicas, topic);
+    }
+
+    private static void setUpCluster() throws Exception {
+        namesrvController = createAndStartNamesrv();
+        nsAddr = "127.0.0.1:" + namesrvController.getNettyServerConfig().getListenPort();
+        System.out.printf("namesrv addr: %s%n", nsAddr);
+
+        /*
+         *     BrokerContainer1      |      BrokerContainer2      |      BrokerContainer3
+         *
+         *   master1With3Replicas(m)      master2With3Replicas(m)      master3With3Replicas(m)
+         *   master3With3Replicas(s0)     master1With3Replicas(s0)     master2With3Replicas(s0)
+         *   master2With3Replicas(s1)     master3With3Replicas(s1)     master1With3Replicas(s1)
+         */
+
+        brokerContainer1 = createAndStartBrokerContainer(nsAddr);
+        brokerContainer2 = createAndStartBrokerContainer(nsAddr);
+        brokerContainer3 = createAndStartBrokerContainer(nsAddr);
+        // Create three broker groups, two contains two replicas, another contains three replicas
+        master1With3Replicas = createAndAddMaster(brokerContainer1, new BrokerGroupConfig(), BROKER_INDEX.getAndIncrement());
+        master2With3Replicas = createAndAddMaster(brokerContainer2, new BrokerGroupConfig(), BROKER_INDEX.getAndIncrement());
+        master3With3Replicas = createAndAddMaster(brokerContainer3, new BrokerGroupConfig(), BROKER_INDEX.getAndIncrement());
+
+        createAndAddSlave(1, brokerContainer1, master3With3Replicas);
+        createAndAddSlave(1, brokerContainer2, master1With3Replicas);
+        createAndAddSlave(1, brokerContainer3, master2With3Replicas);
+
+        createAndAddSlave(2, brokerContainer1, master2With3Replicas);
+        createAndAddSlave(2, brokerContainer2, master3With3Replicas);
+        createAndAddSlave(2, brokerContainer3, master1With3Replicas);
+
+        awaitUntilSlaveOK();
+
+        defaultMQAdminExt = new DefaultMQAdminExt("HATest_Admin_Group");
+        defaultMQAdminExt.setNamesrvAddr(nsAddr);
+        defaultMQAdminExt.start();
+    }
+
+    protected static void createTopicTo(BrokerController masterBroker, String topicName, int rqn, int wqn) {
+        try {
+            TopicConfig topicConfig = new TopicConfig(topicName, rqn, wqn, 6, 0);
+            defaultMQAdminExt.createAndUpdateTopicConfig(masterBroker.getBrokerAddr(), topicConfig);
+
+            triggerSlaveSync(masterBroker.getBrokerConfig().getBrokerName(), brokerContainer1);
+            triggerSlaveSync(masterBroker.getBrokerConfig().getBrokerName(), brokerContainer2);
+            triggerSlaveSync(masterBroker.getBrokerConfig().getBrokerName(), brokerContainer3);
+        } catch (Exception e) {
+            e.printStackTrace();
+            throw new RuntimeException("Create topic to broker failed", e);
+        }
+    }
+
+    protected static void createGroup(BrokerController masterBroker, String groupName) {
+        try {
+            SubscriptionGroupConfig config = new SubscriptionGroupConfig();
+            config.setGroupName(groupName);
+
+            masterBroker.getSubscriptionGroupManager().updateSubscriptionGroupConfig(config);
+
+            triggerSlaveSync(masterBroker.getBrokerConfig().getBrokerName(), brokerContainer1);
+            triggerSlaveSync(masterBroker.getBrokerConfig().getBrokerName(), brokerContainer2);
+            triggerSlaveSync(masterBroker.getBrokerConfig().getBrokerName(), brokerContainer3);
+        } catch (Exception e) {
+            e.printStackTrace();
+            throw new RuntimeException("Create group to broker failed", e);
+        }
+    }
+
+    private static void triggerSlaveSync(String brokerName, BrokerContainer brokerContainer) {
+        for (InnerSalveBrokerController slaveBroker : brokerContainer.getSlaveBrokers()) {
+            if (slaveBroker.getBrokerConfig().getBrokerName().equals(brokerName)) {
+                slaveBroker.getSlaveSynchronize().syncAll();
+                slaveBroker.registerBrokerAll(true, false, true);
+            }
+        }
+    }
+
+    protected static void createTopicTo(BrokerController brokerController, String topicName) {
+        createTopicTo(brokerController, topicName, 8, 8);
+    }
+
+    private static void registerCleaner() {
+        Runtime.getRuntime().addShutdownHook(new Thread(() -> {
+            if (CLUSTER_SET_UP.compareAndSet(true, false)) {
+                System.out.printf("clean up%n");
+                defaultMQAdminExt.shutdown();
+
+                for (final BrokerContainer brokerContainer : brokerContainerList) {
+                    brokerContainer.shutdown();
+                    for (BrokerController brokerController : brokerContainer.getBrokerControllers()) {
+                        brokerController.getMessageStore().destroy();
+                    }
+                }
+
+                for (final NamesrvController namesrvController : namesrvControllers) {
+                    namesrvController.shutdown();
+                }
+
+                for (final File file : TMP_FILE_LIST) {
+                    UtilAll.deleteFile(file);
+                }
+            }
+        }));
+    }
+
+    private static File createBaseDir(String prefix) {
+        final File file;
+        try {
+            file = Files.createTempDirectory(prefix).toFile();
+            TMP_FILE_LIST.add(file);
+            System.out.printf("create file at %s%n", file.getAbsolutePath());
+            return file;
+        } catch (IOException e) {
+            throw new RuntimeException("Couldn't create tmp folder", e);
+        }
+    }
+
+    public static NamesrvController createAndStartNamesrv() {
+        String baseDir = createBaseDir("test-cluster-namesrv").getAbsolutePath();
+        NamesrvConfig namesrvConfig = new NamesrvConfig();
+        NettyServerConfig nameServerNettyServerConfig = new NettyServerConfig();
+        namesrvConfig.setKvConfigPath(baseDir + File.separator + "namesrv" + File.separator + "kvConfig.json");
+        namesrvConfig.setConfigStorePath(baseDir + File.separator + "namesrv" + File.separator + "namesrv.properties");
+        namesrvConfig.setSupportActingMaster(true);
+        namesrvConfig.setScanNotActiveBrokerInterval(1000);
+
+        nameServerNettyServerConfig.setListenPort(generatePort(10000, 10000));
+        NamesrvController namesrvController = new NamesrvController(namesrvConfig, nameServerNettyServerConfig);
+        try {
+            Assert.assertTrue(namesrvController.initialize());
+            LOG.info("Name Server Start:{}", nameServerNettyServerConfig.getListenPort());
+            namesrvController.start();
+        } catch (Exception e) {
+            LOG.info("Name Server start failed");
+            e.printStackTrace();
+            System.exit(1);
+        }
+
+        namesrvController.getRemotingServer().registerProcessor(RequestCode.REGISTER_BROKER, new NettyRequestProcessor() {
+            @Override
+            public RemotingCommand processRequest(final ChannelHandlerContext ctx,
+                final RemotingCommand request) throws Exception {
+                final RegisterBrokerRequestHeader requestHeader = request.decodeCommandCustomHeader(RegisterBrokerRequestHeader.class);
+                final BrokerConfigLite liteConfig = new BrokerConfigLite(requestHeader.getClusterName(),
+                    requestHeader.getBrokerName(),
+                    requestHeader.getBrokerAddr(),
+                    requestHeader.getBrokerId());
+                if (isolatedBrokers.containsKey(liteConfig)) {
+                    // return response with SYSTEM_ERROR
+                    return RemotingCommand.createResponseCommand(null);
+                }
+                return namesrvController.getRemotingServer().getDefaultProcessorPair().getObject1().processRequest(ctx, request);
+            }
+
+            @Override
+            public boolean rejectRequest() {
+                return false;
+            }
+        }, null);
+
+        namesrvControllers.add(namesrvController);
+        return namesrvController;
+
+    }
+
+    public static BrokerContainer createAndStartBrokerContainer(String nsAddr) {
+        BrokerContainerConfig brokerContainerConfig = new BrokerContainerConfig();
+        NettyServerConfig nettyServerConfig = new NettyServerConfig();
+        NettyClientConfig nettyClientConfig = new NettyClientConfig();
+        brokerContainerConfig.setNamesrvAddr(nsAddr);
+        brokerContainerConfig.setCompatibleWithOldNameSrv(false);
+
+        nettyServerConfig.setListenPort(generatePort(20000, 10000));
+        BrokerContainer brokerContainer = new BrokerContainer(brokerContainerConfig, nettyServerConfig, nettyClientConfig);
+        try {
+            Assert.assertTrue(brokerContainer.initialize());
+            LOG.info("Broker container Start, listen on {}.", nettyServerConfig.getListenPort());
+            brokerContainer.start();
+        } catch (Exception e) {
+            LOG.info("Broker container start failed", e);
+            e.printStackTrace();
+            System.exit(1);
+        }
+        brokerContainerList.add(brokerContainer);
+        return brokerContainer;
+    }
+
+    private static int generatePort(int base, int range) {
+        int result = base + RANDOM.nextInt(range);
+        while (PORTS_IN_USE.contains(result) || PORTS_IN_USE.contains(result - 2)) {
+            result = base + RANDOM.nextInt(range);
+        }
+        PORTS_IN_USE.add(result);
+        PORTS_IN_USE.add(result - 2);
+        return result;
+    }
+
+    public static BrokerController createAndAddMaster(BrokerContainer brokerContainer,
+        BrokerGroupConfig brokerGroupConfig, int brokerIndex) throws Exception {
+        BrokerConfig brokerConfig = new BrokerConfig();
+        MessageStoreConfig storeConfig = new MessageStoreConfig();
+        brokerConfig.setBrokerName(BROKER_NAME_PREFIX + brokerIndex);
+        brokerConfig.setBrokerIP1("127.0.0.1");
+        brokerConfig.setBrokerIP2("127.0.0.1");
+        brokerConfig.setBrokerId(0);
+        brokerConfig.setEnablePropertyFilter(true);
+        brokerConfig.setEnableSlaveActingMaster(brokerGroupConfig.enableSlaveActingMaster);
+        brokerConfig.setEnableRemoteEscape(brokerGroupConfig.enableRemoteEscape);
+        brokerConfig.setSlaveReadEnable(brokerGroupConfig.slaveReadEnable);
+        brokerConfig.setLockInStrictMode(true);
+        brokerConfig.setConsumerOffsetUpdateVersionStep(10);
+        brokerConfig.setDelayOffsetUpdateVersionStep(10);
+        brokerConfig.setListenPort(generatePort(brokerContainer.getRemotingServer().localListenPort(), 10000));
+
+        String baseDir = createBaseDir(brokerConfig.getBrokerName() + "_" + brokerConfig.getBrokerId()).getAbsolutePath();
+        storeConfig.setStorePathRootDir(baseDir);
+        storeConfig.setStorePathCommitLog(baseDir + File.separator + "commitlog");
+        storeConfig.setHaListenPort(generatePort(30000, 10000));
+        storeConfig.setMappedFileSizeCommitLog(COMMIT_LOG_SIZE);
+        storeConfig.setMaxIndexNum(INDEX_NUM);
+        storeConfig.setMaxHashSlotNum(INDEX_NUM * 4);
+        storeConfig.setTotalReplicas(brokerGroupConfig.totalReplicas);
+        storeConfig.setInSyncReplicas(brokerGroupConfig.inSyncReplicas);
+        storeConfig.setMinInSyncReplicas(brokerGroupConfig.minReplicas);
+        storeConfig.setEnableAutoInSyncReplicas(brokerGroupConfig.autoReplicas);
+        storeConfig.setBrokerRole(BrokerRole.SYNC_MASTER);
+        storeConfig.setSyncFlushTimeout(10 * 1000);
+
+        System.out.printf("start master %s with port %d-%d%n", brokerConfig.getCanonicalName(), brokerConfig.getListenPort(), storeConfig.getHaListenPort());
+        BrokerController brokerController = null;
+        try {
+            brokerController = brokerContainer.addBroker(brokerConfig, storeConfig);
+            Assert.assertNotNull(brokerController);
+            brokerController.start();
+            TMP_FILE_LIST.add(new File(brokerController.getTopicConfigManager().configFilePath()));
+            TMP_FILE_LIST.add(new File(brokerController.getSubscriptionGroupManager().configFilePath()));
+            LOG.info("Broker Start name:{} addr:{}", brokerConfig.getBrokerName(), brokerController.getBrokerAddr());
+        } catch (Exception e) {
+            LOG.info("Broker start failed", e);
+            e.printStackTrace();
+            System.exit(1);
+        }
+
+        return brokerController;
+    }
+
+    protected static DefaultMQProducer createProducer(String producerGroup) {
+        DefaultMQProducer producer = new DefaultMQProducer(producerGroup);
+        producer.setInstanceName(UUID.randomUUID().toString());
+        producer.setNamesrvAddr(nsAddr);
+        return producer;
+    }
+
+    protected static TransactionMQProducer createTransactionProducer(String producerGroup,
+        TransactionCheckListener transactionCheckListener) {
+        TransactionMQProducer producer = new TransactionMQProducer(producerGroup);
+        producer.setInstanceName(UUID.randomUUID().toString());
+        producer.setNamesrvAddr(nsAddr);
+        producer.setTransactionCheckListener(transactionCheckListener);
+        return producer;
+    }
+
+    protected static DefaultMQPullConsumer createPullConsumer(String consumerGroup) {
+        DefaultMQPullConsumer consumer = new DefaultMQPullConsumer(consumerGroup);
+        consumer.setInstanceName(UUID.randomUUID().toString());
+        consumer.setNamesrvAddr(nsAddr);
+        return consumer;
+    }
+
+    protected static DefaultMQPushConsumer createPushConsumer(String consumerGroup) {
+        DefaultMQPushConsumer consumer = new DefaultMQPushConsumer(consumerGroup);
+        consumer.setInstanceName(UUID.randomUUID().toString());
+        consumer.setNamesrvAddr(nsAddr);
+        return consumer;
+    }
+
+    protected static void createAndAddSlave(int slaveBrokerId, BrokerContainer brokerContainer,
+        BrokerController master) {
+        BrokerConfig slaveBrokerConfig = new BrokerConfig();
+        slaveBrokerConfig.setBrokerName(master.getBrokerConfig().getBrokerName());
+        slaveBrokerConfig.setBrokerId(slaveBrokerId);
+        slaveBrokerConfig.setBrokerClusterName(master.getBrokerConfig().getBrokerClusterName());
+
+        slaveBrokerConfig.setBrokerIP1("127.0.0.1");
+        slaveBrokerConfig.setBrokerIP2("127.0.0.1");
+        slaveBrokerConfig.setEnablePropertyFilter(true);
+        slaveBrokerConfig.setSlaveReadEnable(true);
+        slaveBrokerConfig.setEnableSlaveActingMaster(true);
+        slaveBrokerConfig.setEnableRemoteEscape(true);
+        slaveBrokerConfig.setLockInStrictMode(true);
+        slaveBrokerConfig.setListenPort(generatePort(brokerContainer.getRemotingServer().localListenPort(), 10000));
+        slaveBrokerConfig.setConsumerOffsetUpdateVersionStep(10);
+        slaveBrokerConfig.setDelayOffsetUpdateVersionStep(10);
+
+        MessageStoreConfig storeConfig = slaveStoreConfigCache.get(slaveBrokerConfig);
+
+        if (storeConfig == null) {
+            storeConfig = new MessageStoreConfig();
+            String baseDir = createBaseDir(slaveBrokerConfig.getBrokerName() + "_" + slaveBrokerConfig.getBrokerId()).getAbsolutePath();
+            storeConfig.setStorePathRootDir(baseDir);
+            storeConfig.setStorePathCommitLog(baseDir + File.separator + "commitlog");
+            storeConfig.setHaListenPort(generatePort(master.getMessageStoreConfig().getHaListenPort(), 10000));
+            storeConfig.setMappedFileSizeCommitLog(COMMIT_LOG_SIZE);
+            storeConfig.setMaxIndexNum(INDEX_NUM);
+            storeConfig.setMaxHashSlotNum(INDEX_NUM * 4);
+            storeConfig.setTotalReplicas(master.getMessageStoreConfig().getTotalReplicas());
+            storeConfig.setInSyncReplicas(master.getMessageStoreConfig().getInSyncReplicas());
+            storeConfig.setMinInSyncReplicas(master.getMessageStoreConfig().getMinInSyncReplicas());
+            storeConfig.setBrokerRole(BrokerRole.SLAVE);
+            slaveStoreConfigCache.put(slaveBrokerConfig, storeConfig);
+        }
+
+        System.out.printf("start slave %s with port %d-%d%n", slaveBrokerConfig.getCanonicalName(), slaveBrokerConfig.getListenPort(), storeConfig.getHaListenPort());
+
+        try {
+            BrokerController brokerController = brokerContainer.addBroker(slaveBrokerConfig, storeConfig);
+            Assert.assertNotNull(brokerContainer);
+            brokerController.start();
+            TMP_FILE_LIST.add(new File(brokerController.getTopicConfigManager().configFilePath()));
+            TMP_FILE_LIST.add(new File(brokerController.getSubscriptionGroupManager().configFilePath()));
+            LOG.info("Add slave name:{} addr:{}", slaveBrokerConfig.getBrokerName(), brokerController.getBrokerAddr());
+        } catch (Exception e) {
+            e.printStackTrace();
+            throw new RuntimeException("Couldn't add slave broker", e);
+        }
+    }
+
+    protected static void removeSlaveBroker(int slaveBrokerId, BrokerContainer brokerContainer,
+        BrokerController master) throws Exception {
+        BrokerIdentity brokerIdentity = new BrokerIdentity(master.getBrokerConfig().getBrokerClusterName(),
+            master.getBrokerConfig().getBrokerName(), slaveBrokerId);
+
+        brokerContainer.removeBroker(brokerIdentity);
+    }
+
+    protected static void awaitUntilSlaveOK() {
+        await().atMost(100, TimeUnit.SECONDS)
+            .until(() -> {
+                boolean isOk = master1With3Replicas.getMessageStore().getHaService().getConnectionCount().get() == 2
+                    && master1With3Replicas.getMessageStore().getAliveReplicaNumInGroup() == 3;
+                for (HAConnection haConnection : master1With3Replicas.getMessageStore().getHaService().getConnectionList()) {
+                    isOk &= haConnection.getCurrentState().equals(HAConnectionState.TRANSFER);
+                }
+                return isOk;
+            });
+
+        await().atMost(100, TimeUnit.SECONDS)
+            .until(() -> {
+                boolean isOk = master2With3Replicas.getMessageStore().getHaService().getConnectionCount().get() == 2
+                    && master2With3Replicas.getMessageStore().getAliveReplicaNumInGroup() == 3;
+                for (HAConnection haConnection : master2With3Replicas.getMessageStore().getHaService().getConnectionList()) {
+                    isOk &= haConnection.getCurrentState().equals(HAConnectionState.TRANSFER);
+                }
+                return isOk;
+            });
+
+        await().atMost(100, TimeUnit.SECONDS)
+            .until(() -> {
+                boolean isOk = master3With3Replicas.getMessageStore().getHaService().getConnectionCount().get() == 2
+                    && master3With3Replicas.getMessageStore().getAliveReplicaNumInGroup() == 3;
+                for (HAConnection haConnection : master3With3Replicas.getMessageStore().getHaService().getConnectionList()) {
+                    isOk &= haConnection.getCurrentState().equals(HAConnectionState.TRANSFER);
+                }
+                return isOk;
+            });
+
+        try {
+            Thread.sleep(2000);
+        } catch (InterruptedException e) {
+            e.printStackTrace();
+        }
+    }
+
+    protected static void isolateBroker(BrokerController brokerController) {
+        final BrokerConfig config = brokerController.getBrokerConfig();
+
+        BrokerConfigLite liteConfig = new BrokerConfigLite(config.getBrokerClusterName(),
+            config.getBrokerName(),
+            brokerController.getBrokerAddr(),
+            config.getBrokerId());
+
+        // Reject register requests from the specific broker
+        isolatedBrokers.putIfAbsent(liteConfig, brokerController);
+
+        // UnRegister the specific broker immediately
+        namesrvController.getRouteInfoManager().unregisterBroker(liteConfig.getClusterName(),
+            liteConfig.getBrokerAddr(),
+            liteConfig.getBrokerName(),
+            liteConfig.getBrokerId());
+    }
+
+    protected static void cancelIsolatedBroker(BrokerController brokerController) {
+        final BrokerConfig config = brokerController.getBrokerConfig();
+
+        BrokerConfigLite liteConfig = new BrokerConfigLite(config.getBrokerClusterName(),
+            config.getBrokerName(),
+            brokerController.getBrokerAddr(),
+            config.getBrokerId());
+
+        isolatedBrokers.remove(liteConfig);
+        brokerController.registerBrokerAll(true, false, true);
+
+        await().atMost(Duration.ofMinutes(1)).until(() -> namesrvController.getRouteInfoManager()
+            .getBrokerMemberGroup(liteConfig.getClusterName(), liteConfig.brokerName).getBrokerAddrs()
+            .containsKey(liteConfig.getBrokerId()));
+    }
+
+    protected static InnerSalveBrokerController getSlaveFromContainerByName(BrokerContainer brokerContainer,
+        String brokerName) {
+        InnerSalveBrokerController targetSlave = null;
+        for (InnerSalveBrokerController slave : brokerContainer.getSlaveBrokers()) {
+            if (slave.getBrokerConfig().getBrokerName().equals(brokerName)) {
+                targetSlave = slave;
+            }
+        }
+
+        return targetSlave;
+    }
+
+    protected static void changeCompatibleMode(boolean compatibleMode) {
+        brokerContainer1.getBrokerContainerConfig().setCompatibleWithOldNameSrv(compatibleMode);
+        brokerContainer2.getBrokerContainerConfig().setCompatibleWithOldNameSrv(compatibleMode);
+        brokerContainer3.getBrokerContainerConfig().setCompatibleWithOldNameSrv(compatibleMode);
+    }
+
+    protected static Set<MessageQueue> filterMessageQueue(Set<MessageQueue> mqSet, String topic) {
+        Set<MessageQueue> targetMqSet = new HashSet<>();
+        if (topic != null) {
+            for (MessageQueue mq : mqSet) {
+                if (mq.getTopic().equals(topic)) {
+                    targetMqSet.add(mq);
+                }
+            }
+        }
+
+        return targetMqSet;
+    }
+
+    public static class BrokerGroupConfig {
+        int totalReplicas = 3;
+        int minReplicas = 1;
+        int inSyncReplicas = 2;
+        boolean autoReplicas = true;
+        boolean enableSlaveActingMaster = true;
+        boolean enableRemoteEscape = true;
+        boolean slaveReadEnable = true;
+
+        public BrokerGroupConfig() {
+        }
+
+        public BrokerGroupConfig(final int totalReplicas, final int minReplicas, final int inSyncReplicas,
+            final boolean autoReplicas, boolean enableSlaveActingMaster, boolean slaveReadEnable) {
+            this.totalReplicas = totalReplicas;
+            this.minReplicas = minReplicas;
+            this.inSyncReplicas = inSyncReplicas;
+            this.autoReplicas = autoReplicas;
+            this.enableSlaveActingMaster = enableSlaveActingMaster;
+            this.slaveReadEnable = slaveReadEnable;
+        }
+    }
+
+    static class BrokerConfigLite {
+        private String clusterName;
+        private String brokerName;
+        private String brokerAddr;
+        private long brokerId;
+
+        public BrokerConfigLite(final String clusterName, final String brokerName, final String brokerAddr,
+            final long brokerId) {
+            this.clusterName = clusterName;
+            this.brokerName = brokerName;
+            this.brokerAddr = brokerAddr;
+            this.brokerId = brokerId;
+        }
+
+        public String getClusterName() {
+            return clusterName;
+        }
+
+        public String getBrokerName() {
+            return brokerName;
+        }
+
+        public String getBrokerAddr() {
+            return brokerAddr;
+        }
+
+        public long getBrokerId() {
+            return brokerId;
+        }
+
+        @Override
+        public boolean equals(final Object o) {
+            if (this == o)
+                return true;
+
+            if (o == null || getClass() != o.getClass())
+                return false;
+
+            final BrokerConfigLite lite = (BrokerConfigLite) o;
+
+            return new EqualsBuilder()
+                .append(clusterName, lite.clusterName)
+                .append(brokerName, lite.brokerName)
+                .append(brokerAddr, lite.brokerAddr)
+                .append(brokerId, lite.brokerId)
+                .isEquals();
+        }
+
+        @Override
+        public int hashCode() {
+            return new HashCodeBuilder(17, 37)
+                .append(clusterName)
+                .append(brokerName)
+                .append(brokerAddr)
+                .append(brokerId)
+                .toHashCode();
+        }
+    }
+}
diff --git a/test/src/test/java/org/apache/rocketmq/test/container/GetMaxOffsetFromSlaveIT.java b/test/src/test/java/org/apache/rocketmq/test/container/GetMaxOffsetFromSlaveIT.java
new file mode 100644
index 0000000..0b55d77
--- /dev/null
+++ b/test/src/test/java/org/apache/rocketmq/test/container/GetMaxOffsetFromSlaveIT.java
@@ -0,0 +1,98 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.rocketmq.test.container;
+
+import java.io.UnsupportedEncodingException;
+import java.util.HashMap;
+import java.util.Map;
+import java.util.concurrent.TimeUnit;
+import org.apache.rocketmq.client.exception.MQBrokerException;
+import org.apache.rocketmq.client.exception.MQClientException;
+import org.apache.rocketmq.client.impl.producer.TopicPublishInfo;
+import org.apache.rocketmq.client.producer.DefaultMQProducer;
+import org.apache.rocketmq.client.producer.SendResult;
+import org.apache.rocketmq.client.producer.SendStatus;
+import org.apache.rocketmq.common.message.Message;
+import org.apache.rocketmq.common.message.MessageQueue;
+import org.apache.rocketmq.remoting.common.RemotingHelper;
+import org.apache.rocketmq.remoting.exception.RemotingException;
+import org.apache.rocketmq.store.DefaultMessageStore;
+import org.junit.AfterClass;
+import org.junit.BeforeClass;
+import org.junit.Test;
+
+import static org.assertj.core.api.Java6Assertions.assertThat;
+import static org.awaitility.Awaitility.await;
+
+public class GetMaxOffsetFromSlaveIT extends ContainerIntegrationTestBase {
+    private static DefaultMQProducer mqProducer;
+
+    private final byte[] MESSAGE_BODY = ("Hello RocketMQ ").getBytes(RemotingHelper.DEFAULT_CHARSET);
+
+    public GetMaxOffsetFromSlaveIT() throws UnsupportedEncodingException {
+    }
+
+    @BeforeClass
+    public static void beforeClass() throws MQClientException {
+        mqProducer = createProducer(GetMaxOffsetFromSlaveIT.class.getSimpleName() + "_Producer");
+        mqProducer.start();
+    }
+
+    @AfterClass
+    public static void afterClass() {
+        if (mqProducer != null) {
+            mqProducer.shutdown();
+        }
+    }
+
+    @Test
+    public void testGetMaxOffsetFromSlave() throws InterruptedException, RemotingException, MQClientException, MQBrokerException {
+        awaitUntilSlaveOK();
+        mqProducer.getDefaultMQProducerImpl().getmQClientFactory().updateTopicRouteInfoFromNameServer(THREE_REPLICAS_TOPIC);
+
+        for (int i = 0; i < 100; i++) {
+            Message msg = new Message(THREE_REPLICAS_TOPIC, MESSAGE_BODY);
+            SendResult sendResult = mqProducer.send(msg, 10000);
+            assertThat(sendResult.getSendStatus()).isEqualTo(SendStatus.SEND_OK);
+        }
+
+        Map<Integer, Long> maxOffsetMap = new HashMap<>();
+        TopicPublishInfo publishInfo = mqProducer.getDefaultMQProducerImpl().getTopicPublishInfoTable().get(THREE_REPLICAS_TOPIC);
+        assertThat(publishInfo).isNotNull();
+        for (MessageQueue mq : publishInfo.getMessageQueueList()) {
+            maxOffsetMap.put(mq.getQueueId(), mqProducer.getDefaultMQProducerImpl().
+                maxOffset(new MessageQueue(THREE_REPLICAS_TOPIC, master3With3Replicas.getBrokerConfig().getBrokerName(), mq.getQueueId())));
+        }
+
+        isolateBroker(master3With3Replicas);
+
+        mqProducer.getDefaultMQProducerImpl().getmQClientFactory().updateTopicRouteInfoFromNameServer(THREE_REPLICAS_TOPIC);
+        assertThat(mqProducer.getDefaultMQProducerImpl().getmQClientFactory().findBrokerAddressInPublish(
+            master3With3Replicas.getBrokerConfig().getBrokerName())).isNotNull();
+
+        for (MessageQueue mq : publishInfo.getMessageQueueList()) {
+            assertThat(mqProducer.getDefaultMQProducerImpl().maxOffset(
+                new MessageQueue(THREE_REPLICAS_TOPIC, master3With3Replicas.getBrokerConfig().getBrokerName(), mq.getQueueId())))
+                .isEqualTo(maxOffsetMap.get(mq.getQueueId()));
+        }
+
+        cancelIsolatedBroker(master3With3Replicas);
+        await().atMost(100, TimeUnit.SECONDS)
+            .until(() -> ((DefaultMessageStore) master3With3Replicas.getMessageStore()).getHaService().getConnectionCount().get() == 2);
+    }
+}
diff --git a/test/src/test/java/org/apache/rocketmq/test/container/GetMetadataReverseIT.java b/test/src/test/java/org/apache/rocketmq/test/container/GetMetadataReverseIT.java
new file mode 100644
index 0000000..7138f42
--- /dev/null
+++ b/test/src/test/java/org/apache/rocketmq/test/container/GetMetadataReverseIT.java
@@ -0,0 +1,231 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.rocketmq.test.container;
+
+import java.io.UnsupportedEncodingException;
+import java.time.Duration;
+import java.util.Map;
+import java.util.Random;
+import java.util.concurrent.atomic.AtomicInteger;
+import org.apache.rocketmq.container.InnerSalveBrokerController;
+import org.apache.rocketmq.client.consumer.DefaultMQPushConsumer;
+import org.apache.rocketmq.client.consumer.listener.ConsumeConcurrentlyStatus;
+import org.apache.rocketmq.client.consumer.listener.MessageListenerConcurrently;
+import org.apache.rocketmq.client.producer.DefaultMQProducer;
+import org.apache.rocketmq.client.producer.SendResult;
+import org.apache.rocketmq.client.producer.SendStatus;
+import org.apache.rocketmq.common.BrokerIdentity;
+import org.apache.rocketmq.common.consumer.ConsumeFromWhere;
+import org.apache.rocketmq.common.message.Message;
+import org.apache.rocketmq.common.message.MessageQueue;
+import org.junit.AfterClass;
+import org.junit.BeforeClass;
+import org.junit.Ignore;
+import org.junit.Test;
+
+import static org.awaitility.Awaitility.await;
+
+@Ignore
+public class GetMetadataReverseIT extends ContainerIntegrationTestBase {
+
+    private static DefaultMQProducer producer;
+
+    private static final String CONSUMER_GROUP = GetMetadataReverseIT.class.getSimpleName() + "_Consumer";
+
+    private static final int MESSAGE_COUNT = 32;
+
+    private final static Random random = new Random();
+
+    public GetMetadataReverseIT() throws UnsupportedEncodingException {
+
+    }
+
+    @BeforeClass
+    public static void beforeClass() throws Throwable {
+        producer = createProducer(PushMultipleReplicasIT.class.getSimpleName() + "_PRODUCER");
+        producer.setSendMsgTimeout(15 * 1000);
+        producer.start();
+    }
+
+    @AfterClass
+    public static void afterClass() throws Exception {
+        producer.shutdown();
+    }
+
+    @Test
+    public void testGetMetadataReverse_consumerOffset() throws Exception {
+        String topic = GetMetadataReverseIT.class.getSimpleName() + "_consumerOffset" + random.nextInt(65535);
+        createTopicTo(master1With3Replicas, topic, 1, 1);
+        // Wait topic synchronization
+        await().atMost(Duration.ofMinutes(1)).until(() -> {
+            InnerSalveBrokerController slaveBroker = brokerContainer2.getSlaveBrokers().iterator().next();
+            return slaveBroker.getTopicConfigManager().selectTopicConfig(topic) != null;
+        });
+
+        int sendSuccess = 0;
+        for (int i = 0; i < MESSAGE_COUNT; i++) {
+            Message msg = new Message(topic, Integer.toString(i).getBytes());
+            SendResult sendResult = producer.send(msg);
+            if (sendResult.getSendStatus() == SendStatus.SEND_OK) {
+                sendSuccess++;
+            }
+        }
+        final int finalSendSuccess = sendSuccess;
+        await().atMost(Duration.ofMinutes(1)).until(() -> finalSendSuccess >= MESSAGE_COUNT);
+        System.out.printf("send success%n");
+
+        isolateBroker(master1With3Replicas);
+        brokerContainer1.removeBroker(new BrokerIdentity(
+            master1With3Replicas.getBrokerConfig().getBrokerClusterName(),
+            master1With3Replicas.getBrokerConfig().getBrokerName(),
+            master1With3Replicas.getBrokerConfig().getBrokerId()));
+
+        System.out.printf("Remove master%n");
+
+        DefaultMQPushConsumer pushConsumer = createPushConsumer(CONSUMER_GROUP);
+        pushConsumer.subscribe(topic, "*");
+        pushConsumer.setConsumeFromWhere(ConsumeFromWhere.CONSUME_FROM_FIRST_OFFSET);
+        AtomicInteger receivedMsgCount = new AtomicInteger(0);
+        pushConsumer.registerMessageListener((MessageListenerConcurrently) (msgs, context) -> {
+            receivedMsgCount.addAndGet(msgs.size());
+            msgs.forEach(x -> System.out.printf(x + "%n"));
+            return ConsumeConcurrentlyStatus.CONSUME_SUCCESS;
+        });
+        pushConsumer.start();
+        await().atMost(Duration.ofMinutes(3)).until(() -> receivedMsgCount.get() >= MESSAGE_COUNT);
+
+        await().atMost(Duration.ofMinutes(1)).until(() -> {
+            pushConsumer.getDefaultMQPushConsumerImpl().persistConsumerOffset();
+            Map<Integer, Long> slaveOffsetTable = null;
+            for (InnerSalveBrokerController slave : brokerContainer2.getSlaveBrokers()) {
+                if (slave.getBrokerConfig().getBrokerName().equals(master1With3Replicas.getBrokerConfig().getBrokerName())) {
+                    slaveOffsetTable = slave.getConsumerOffsetManager().queryOffset(CONSUMER_GROUP, topic);
+                }
+            }
+
+            if (slaveOffsetTable != null) {
+                long totalOffset = 0;
+                for (final Long offset : slaveOffsetTable.values()) {
+                    totalOffset += offset;
+                }
+
+                return totalOffset >= MESSAGE_COUNT;
+            }
+            return false;
+        });
+
+        //Add back master
+        master1With3Replicas = brokerContainer1.addBroker(master1With3Replicas.getBrokerConfig(), master1With3Replicas.getMessageStoreConfig());
+        master1With3Replicas.start();
+        cancelIsolatedBroker(master1With3Replicas);
+        System.out.printf("Add back master%n");
+
+        awaitUntilSlaveOK();
+
+        await().atMost(Duration.ofMinutes(1)).until(() -> {
+            Map<Integer, Long> offsetTable = master1With3Replicas.getConsumerOffsetManager().queryOffset(CONSUMER_GROUP, topic);
+            long totalOffset = 0;
+            if (offsetTable != null) {
+                for (final Long offset : offsetTable.values()) {
+                    totalOffset += offset;
+                }
+            }
+            return totalOffset >= MESSAGE_COUNT;
+        });
+
+        pushConsumer.shutdown();
+    }
+
+    @Test
+    public void testGetMetadataReverse_delayOffset() throws Exception {
+        String topic = GetMetadataReverseIT.class.getSimpleName() + "_delayOffset" + random.nextInt(65535);
+        createTopicTo(master1With3Replicas, topic, 1, 1);
+        createTopicTo(master2With3Replicas, topic, 1, 1);
+        createTopicTo(master3With3Replicas, topic, 1, 1);
+        // Wait topic synchronization
+        await().atMost(Duration.ofMinutes(1)).until(() -> {
+            InnerSalveBrokerController slaveBroker = brokerContainer2.getSlaveBrokers().iterator().next();
+            return slaveBroker.getTopicConfigManager().selectTopicConfig(topic) != null;
+        });
+        int delayLevel = 4;
+
+        DefaultMQPushConsumer pushConsumer = createPushConsumer(CONSUMER_GROUP);
+        pushConsumer.subscribe(topic, "*");
+        AtomicInteger receivedMsgCount = new AtomicInteger(0);
+        pushConsumer.registerMessageListener((MessageListenerConcurrently) (msgs, context) -> {
+            receivedMsgCount.addAndGet(msgs.size());
+            msgs.forEach(x -> System.out.printf(x + "%n"));
+            return ConsumeConcurrentlyStatus.CONSUME_SUCCESS;
+        });
+        pushConsumer.start();
+
+        MessageQueue messageQueue = new MessageQueue(topic, master1With3Replicas.getBrokerConfig().getBrokerName(), 0);
+        int sendSuccess = 0;
+        for (int i = 0; i < MESSAGE_COUNT; i++) {
+            Message msg = new Message(topic, Integer.toString(i).getBytes());
+            msg.setDelayTimeLevel(delayLevel);
+            SendResult sendResult = producer.send(msg, messageQueue);
+            if (sendResult.getSendStatus() == SendStatus.SEND_OK) {
+                sendSuccess++;
+            }
+        }
+        final int finalSendSuccess = sendSuccess;
+        await().atMost(Duration.ofMinutes(1)).until(() -> finalSendSuccess >= MESSAGE_COUNT);
+        System.out.printf("send success%n");
+
+        isolateBroker(master1With3Replicas);
+        brokerContainer1.removeBroker(new BrokerIdentity(
+            master1With3Replicas.getBrokerConfig().getBrokerClusterName(),
+            master1With3Replicas.getBrokerConfig().getBrokerName(),
+            master1With3Replicas.getBrokerConfig().getBrokerId()));
+
+        System.out.printf("Remove master%n");
+
+        await().atMost(Duration.ofMinutes(1)).until(() -> receivedMsgCount.get() >= MESSAGE_COUNT);
+
+        await().atMost(Duration.ofMinutes(1)).until(() -> {
+            pushConsumer.getDefaultMQPushConsumerImpl().persistConsumerOffset();
+            Map<Integer, Long> OffsetTable = master2With3Replicas.getConsumerOffsetManager().queryOffset(CONSUMER_GROUP, topic);
+            if (OffsetTable != null) {
+                long totalOffset = 0;
+                for (final Long offset : OffsetTable.values()) {
+                    totalOffset += offset;
+                }
+                return totalOffset >= MESSAGE_COUNT;
+
+            }
+            return false;
+        });
+
+        //Add back master
+        master1With3Replicas = brokerContainer1.addBroker(master1With3Replicas.getBrokerConfig(), master1With3Replicas.getMessageStoreConfig());
+        master1With3Replicas.start();
+        cancelIsolatedBroker(master1With3Replicas);
+        System.out.printf("Add back master%n");
+
+        awaitUntilSlaveOK();
+
+        await().atMost(Duration.ofMinutes(1)).until(() -> {
+            Map<Integer, Long> offsetTable = master1With3Replicas.getScheduleMessageService().getOffsetTable();
+            System.out.println("" + offsetTable.get(delayLevel));
+            return offsetTable.get(delayLevel) >= MESSAGE_COUNT;
+        });
+
+        pushConsumer.shutdown();
+    }
+}
diff --git a/test/src/test/java/org/apache/rocketmq/test/container/PullMultipleReplicasIT.java b/test/src/test/java/org/apache/rocketmq/test/container/PullMultipleReplicasIT.java
new file mode 100644
index 0000000..0181645
--- /dev/null
+++ b/test/src/test/java/org/apache/rocketmq/test/container/PullMultipleReplicasIT.java
@@ -0,0 +1,199 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.rocketmq.test.container;
+
+import java.io.UnsupportedEncodingException;
+import java.lang.reflect.Field;
+import java.net.InetSocketAddress;
+import java.time.Duration;
+import java.util.List;
+
+import org.apache.commons.lang3.RandomStringUtils;
+import org.apache.rocketmq.client.impl.consumer.DefaultMQPullConsumerImpl;
+import org.apache.rocketmq.client.impl.factory.MQClientInstance;
+import org.apache.rocketmq.container.InnerSalveBrokerController;
+import org.apache.rocketmq.client.consumer.DefaultMQPullConsumer;
+import org.apache.rocketmq.client.consumer.PullResult;
+import org.apache.rocketmq.client.consumer.PullStatus;
+import org.apache.rocketmq.client.exception.MQBrokerException;
+import org.apache.rocketmq.client.exception.MQClientException;
+import org.apache.rocketmq.client.producer.DefaultMQProducer;
+import org.apache.rocketmq.client.producer.SendResult;
+import org.apache.rocketmq.client.producer.SendStatus;
+import org.apache.rocketmq.common.MixAll;
+import org.apache.rocketmq.common.message.Message;
+import org.apache.rocketmq.common.message.MessageExt;
+import org.apache.rocketmq.common.message.MessageQueue;
+import org.apache.rocketmq.remoting.common.RemotingHelper;
+import org.apache.rocketmq.remoting.exception.RemotingException;
+import org.apache.rocketmq.store.DefaultMessageStore;
+import org.junit.AfterClass;
+import org.junit.BeforeClass;
+import org.junit.Test;
+
+import static org.assertj.core.api.Java6Assertions.assertThat;
+import static org.awaitility.Awaitility.await;
+
+public class PullMultipleReplicasIT extends ContainerIntegrationTestBase {
+    private static DefaultMQPullConsumer pullConsumer;
+    private static DefaultMQProducer producer;
+    private static MQClientInstance mqClientInstance;
+
+    private final String MESSAGE_STRING = RandomStringUtils.random(1024);
+    private final byte[] MESSAGE_BODY = MESSAGE_STRING.getBytes(RemotingHelper.DEFAULT_CHARSET);
+
+    public PullMultipleReplicasIT() throws UnsupportedEncodingException {
+    }
+
+    @BeforeClass
+    public static void beforeClass() throws Exception {
+
+        pullConsumer = createPullConsumer(PullMultipleReplicasIT.class.getSimpleName() + "_Consumer");
+        pullConsumer.start();
+
+        Field field = DefaultMQPullConsumerImpl.class.getDeclaredField("mQClientFactory");
+        field.setAccessible(true);
+        mqClientInstance = (MQClientInstance) field.get(pullConsumer.getDefaultMQPullConsumerImpl());
+
+        producer = createProducer(PullMultipleReplicasIT.class.getSimpleName() + "_Producer");
+        producer.setSendMsgTimeout(15 * 1000);
+        producer.start();
+    }
+
+    @AfterClass
+    public static void afterClass() {
+        producer.shutdown();
+        pullConsumer.shutdown();
+    }
+
+    @Test
+    public void testPullMessageFromSlave() throws InterruptedException, RemotingException, MQClientException, MQBrokerException, UnsupportedEncodingException {
+        awaitUntilSlaveOK();
+
+        Message msg = new Message(THREE_REPLICAS_TOPIC, MESSAGE_BODY);
+        SendResult sendResult = producer.send(msg);
+        assertThat(sendResult.getSendStatus()).isEqualTo(SendStatus.SEND_OK);
+
+        final MessageQueue messageQueue = sendResult.getMessageQueue();
+        final long queueOffset = sendResult.getQueueOffset();
+
+        final PullResult[] pullResult = {null};
+        await().atMost(Duration.ofSeconds(5)).until(() -> {
+            pullResult[0] = pullConsumer.pull(messageQueue, "*", queueOffset, 1);
+            return pullResult[0].getPullStatus() == PullStatus.FOUND;
+        });
+
+        List<MessageExt> msgFoundList = pullResult[0].getMsgFoundList();
+        assertThat(msgFoundList.size()).isEqualTo(1);
+        assertThat(new String(msgFoundList.get(0).getBody(), RemotingHelper.DEFAULT_CHARSET)).isEqualTo(MESSAGE_STRING);
+
+        // Pull the same message from the slave broker
+        pullConsumer.getDefaultMQPullConsumerImpl().getPullAPIWrapper().updatePullFromWhichNode(messageQueue, 1);
+
+        await().atMost(Duration.ofSeconds(5)).until(() -> {
+            pullResult[0] = pullConsumer.pull(messageQueue, "*", queueOffset, 1);
+            return pullResult[0].getPullStatus() == PullStatus.FOUND;
+        });
+
+        msgFoundList = pullResult[0].getMsgFoundList();
+        assertThat(msgFoundList.size()).isEqualTo(1);
+        assertThat(new String(msgFoundList.get(0).getBody(), RemotingHelper.DEFAULT_CHARSET)).isEqualTo(MESSAGE_STRING);
+
+        // Pull the same message from the slave broker
+        pullConsumer.getDefaultMQPullConsumerImpl().getPullAPIWrapper().updatePullFromWhichNode(messageQueue, 2);
+
+        await().atMost(Duration.ofSeconds(5)).until(() -> {
+            pullResult[0] = pullConsumer.pull(messageQueue, "*", queueOffset, 1);
+            return pullResult[0].getPullStatus() == PullStatus.FOUND;
+        });
+
+        msgFoundList = pullResult[0].getMsgFoundList();
+        assertThat(msgFoundList.size()).isEqualTo(1);
+        assertThat(new String(msgFoundList.get(0).getBody(), RemotingHelper.DEFAULT_CHARSET)).isEqualTo(MESSAGE_STRING);
+
+        pullConsumer.getDefaultMQPullConsumerImpl().getPullAPIWrapper().updatePullFromWhichNode(messageQueue, 0);
+    }
+
+    @Test
+    public void testSendMessageBackToSlave() throws InterruptedException, RemotingException, MQClientException, MQBrokerException, UnsupportedEncodingException {
+        awaitUntilSlaveOK();
+
+        String clusterTopic = "TOPIC_ON_BROKER2_AND_BROKER3_FOR_MESSAGE_BACK";
+        createTopicTo(master1With3Replicas, clusterTopic);
+        createTopicTo(master3With3Replicas, clusterTopic);
+
+        Message msg = new Message(clusterTopic, MESSAGE_BODY);
+        producer.setSendMsgTimeout(10 * 1000);
+
+        final MessageQueue[] selectedQueue = new MessageQueue[1];
+        await().atMost(Duration.ofSeconds(5)).until(() -> {
+            for (final MessageQueue queue : producer.fetchPublishMessageQueues(clusterTopic)) {
+                if (queue.getBrokerName().equals(master3With3Replicas.getBrokerConfig().getBrokerName())) {
+                    selectedQueue[0] = queue;
+                }
+            }
+            return selectedQueue[0] != null;
+        });
+
+        SendResult sendResult = producer.send(msg, selectedQueue[0]);
+        assertThat(sendResult.getSendStatus()).isEqualTo(SendStatus.SEND_OK);
+
+        final MessageQueue messageQueue = sendResult.getMessageQueue();
+        final long queueOffset = sendResult.getQueueOffset();
+
+        final PullResult[] pullResult = {null};
+        await().atMost(Duration.ofSeconds(60)).until(() -> {
+            pullResult[0] = pullConsumer.pull(messageQueue, "*", queueOffset, 1);
+            return pullResult[0].getPullStatus() == PullStatus.FOUND;
+        });
+
+        await().atMost(Duration.ofSeconds(60)).until(() -> {
+            DefaultMessageStore messageStore = (DefaultMessageStore) master3With3Replicas.getMessageStore();
+            return messageStore.getHaService().inSyncSlaveNums(messageStore.getMaxPhyOffset()) == 2;
+        });
+
+        InnerSalveBrokerController slaveBroker = null;
+        for (InnerSalveBrokerController slave : brokerContainer1.getSlaveBrokers()) {
+            if (slave.getBrokerConfig().getBrokerName().equals(master3With3Replicas.getBrokerConfig().getBrokerName())) {
+                slaveBroker = slave;
+            }
+        }
+
+        assertThat(slaveBroker).isNotNull();
+
+        MessageExt backMessage = pullResult[0].getMsgFoundList().get(0);
+
+        // Message will be sent to the master broker(master1With3Replicas) beside a slave broker of master3With3Replicas
+        backMessage.setStoreHost(new InetSocketAddress(slaveBroker.getBrokerConfig().getBrokerIP1(), slaveBroker.getBrokerConfig().getListenPort()));
+        pullConsumer.sendMessageBack(backMessage, 0);
+
+        String retryTopic = MixAll.getRetryTopic(pullConsumer.getConsumerGroup());
+        // Retry topic only has one queue by default
+        MessageQueue newMsgQueue = new MessageQueue(retryTopic, master1With3Replicas.getBrokerConfig().getBrokerName(), 0);
+        await().atMost(Duration.ofSeconds(60)).until(() -> {
+            pullResult[0] = pullConsumer.pull(newMsgQueue, "*", 0, 1);
+            return pullResult[0].getPullStatus() == PullStatus.FOUND;
+        });
+
+        List<MessageExt> msgFoundList = pullResult[0].getMsgFoundList();
+        assertThat(msgFoundList.size()).isEqualTo(1);
+        assertThat(new String(msgFoundList.get(0).getBody(), RemotingHelper.DEFAULT_CHARSET)).isEqualTo(MESSAGE_STRING);
+
+        awaitUntilSlaveOK();
+    }
+}
diff --git a/test/src/test/java/org/apache/rocketmq/test/container/PushMultipleReplicasIT.java b/test/src/test/java/org/apache/rocketmq/test/container/PushMultipleReplicasIT.java
new file mode 100644
index 0000000..abe1d5e
--- /dev/null
+++ b/test/src/test/java/org/apache/rocketmq/test/container/PushMultipleReplicasIT.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.test.container;
+
+import java.io.UnsupportedEncodingException;
+import java.time.Duration;
+import java.util.Map;
+import java.util.concurrent.atomic.AtomicInteger;
+import org.apache.rocketmq.container.InnerSalveBrokerController;
+import org.apache.rocketmq.client.consumer.DefaultMQPushConsumer;
+import org.apache.rocketmq.client.consumer.listener.ConsumeConcurrentlyStatus;
+import org.apache.rocketmq.client.consumer.listener.MessageListenerConcurrently;
+import org.apache.rocketmq.client.exception.MQClientException;
+import org.apache.rocketmq.client.producer.DefaultMQProducer;
+import org.apache.rocketmq.common.consumer.ConsumeFromWhere;
+import org.apache.rocketmq.common.message.Message;
+import org.junit.AfterClass;
+import org.junit.BeforeClass;
+import org.junit.Test;
+
+import static org.awaitility.Awaitility.await;
+
+public class PushMultipleReplicasIT extends ContainerIntegrationTestBase {
+    private static DefaultMQProducer producer;
+
+    private static final String TOPIC = PushMultipleReplicasIT.class.getSimpleName() + "_TOPIC";
+    private static final String REDIRECT_TOPIC = PushMultipleReplicasIT.class.getSimpleName() + "_REDIRECT_TOPIC";
+    private static final String CONSUMER_GROUP = PushMultipleReplicasIT.class.getSimpleName() + "_Consumer";
+    private static final int MESSAGE_COUNT = 32;
+
+    public PushMultipleReplicasIT() throws UnsupportedEncodingException {
+    }
+
+    @BeforeClass
+    public static void beforeClass() throws Throwable {
+        createTopicTo(master1With3Replicas, TOPIC,1, 1);
+        producer = createProducer(PushMultipleReplicasIT.class.getSimpleName() + "_PRODUCER");
+        producer.setSendMsgTimeout(15 * 1000);
+        producer.start();
+
+        for (int i = 0; i < MESSAGE_COUNT; i++) {
+            producer.send(new Message(TOPIC, Integer.toString(i).getBytes()));
+        }
+
+        createTopicTo(master3With3Replicas, REDIRECT_TOPIC, 1, 1);
+    }
+
+    @AfterClass
+    public static void afterClass() throws Exception {
+        producer.shutdown();
+    }
+
+    @Test
+    public void consumeMessageFromSlave_PushConsumer() throws MQClientException {
+        // Wait topic synchronization
+        await().atMost(Duration.ofMinutes(1)).until(() -> {
+            InnerSalveBrokerController slaveBroker = brokerContainer2.getSlaveBrokers().iterator().next();
+           return slaveBroker.getTopicConfigManager().selectTopicConfig(TOPIC) != null;
+        });
+        isolateBroker(master1With3Replicas);
+        DefaultMQPushConsumer pushConsumer = createPushConsumer(CONSUMER_GROUP);
+        pushConsumer.subscribe(TOPIC, "*");
+        pushConsumer.setConsumeFromWhere(ConsumeFromWhere.CONSUME_FROM_FIRST_OFFSET);
+        AtomicInteger receivedMsgCount = new AtomicInteger(0);
+        pushConsumer.registerMessageListener((MessageListenerConcurrently) (msgs, context) -> {
+            receivedMsgCount.addAndGet(msgs.size());
+            return ConsumeConcurrentlyStatus.CONSUME_SUCCESS;
+        });
+        pushConsumer.start();
+        await().atMost(Duration.ofMinutes(5)).until(() -> receivedMsgCount.get() >= MESSAGE_COUNT);
+
+        await().atMost(Duration.ofMinutes(1)).until(() -> {
+            pushConsumer.getDefaultMQPushConsumerImpl().persistConsumerOffset();
+            Map<Integer, Long> slaveOffsetTable = null;
+            for (InnerSalveBrokerController slave : brokerContainer2.getSlaveBrokers()) {
+                if (slave.getBrokerConfig().getBrokerName().equals(master1With3Replicas.getBrokerConfig().getBrokerName())) {
+                    slaveOffsetTable = slave.getConsumerOffsetManager().queryOffset(CONSUMER_GROUP, TOPIC);
+                }
+            }
+
+            if (slaveOffsetTable != null) {
+                long totalOffset = 0;
+                for (final Long offset : slaveOffsetTable.values()) {
+                    totalOffset += offset;
+                }
+
+                return totalOffset >= MESSAGE_COUNT;
+            }
+            return false;
+        });
+
+        pushConsumer.shutdown();
+        cancelIsolatedBroker(master1With3Replicas);
+
+        awaitUntilSlaveOK();
+    }
+}
diff --git a/test/src/test/java/org/apache/rocketmq/test/container/RebalanceLockOnSlaveIT.java b/test/src/test/java/org/apache/rocketmq/test/container/RebalanceLockOnSlaveIT.java
new file mode 100644
index 0000000..41345b1
--- /dev/null
+++ b/test/src/test/java/org/apache/rocketmq/test/container/RebalanceLockOnSlaveIT.java
@@ -0,0 +1,208 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.rocketmq.test.container;
+
+import java.time.Duration;
+import java.util.ArrayList;
+import java.util.List;
+import java.util.Set;
+import java.util.concurrent.TimeUnit;
+import org.apache.rocketmq.client.consumer.DefaultMQPushConsumer;
+import org.apache.rocketmq.client.consumer.listener.ConsumeOrderlyStatus;
+import org.apache.rocketmq.client.consumer.listener.MessageListenerOrderly;
+import org.apache.rocketmq.client.exception.MQClientException;
+import org.apache.rocketmq.client.impl.FindBrokerResult;
+import org.apache.rocketmq.client.producer.DefaultMQProducer;
+import org.apache.rocketmq.common.MixAll;
+import org.apache.rocketmq.common.ServiceState;
+import org.apache.rocketmq.common.consumer.ConsumeFromWhere;
+import org.apache.rocketmq.common.message.MessageQueue;
+import org.junit.AfterClass;
+import org.junit.BeforeClass;
+import org.junit.Ignore;
+import org.junit.Test;
+
+import static org.assertj.core.api.Assertions.assertThat;
+import static org.awaitility.Awaitility.await;
+
+/**
+ * Test lock on slave when acting master enabled
+ */
+public class RebalanceLockOnSlaveIT extends ContainerIntegrationTestBase {
+    private static final String THREE_REPLICA_CONSUMER_GROUP = "SyncConsumerOffsetIT_ConsumerThreeReplica";
+
+    private static DefaultMQProducer mqProducer;
+    private static DefaultMQPushConsumer mqConsumerThreeReplica1;
+    private static DefaultMQPushConsumer mqConsumerThreeReplica2;
+    private static DefaultMQPushConsumer mqConsumerThreeReplica3;
+
+    public RebalanceLockOnSlaveIT() {
+    }
+
+    @BeforeClass
+    public static void beforeClass() throws Exception {
+        mqProducer = createProducer("SyncConsumerOffsetIT_Producer");
+        mqProducer.start();
+
+        mqConsumerThreeReplica1 = createPushConsumer(THREE_REPLICA_CONSUMER_GROUP);
+        mqConsumerThreeReplica1.setConsumeFromWhere(ConsumeFromWhere.CONSUME_FROM_LAST_OFFSET);
+        mqConsumerThreeReplica1.subscribe(THREE_REPLICAS_TOPIC, "*");
+
+        mqConsumerThreeReplica2 = createPushConsumer(THREE_REPLICA_CONSUMER_GROUP);
+        mqConsumerThreeReplica2.setConsumeFromWhere(ConsumeFromWhere.CONSUME_FROM_LAST_OFFSET);
+        mqConsumerThreeReplica2.subscribe(THREE_REPLICAS_TOPIC, "*");
+
+        mqConsumerThreeReplica3 = createPushConsumer(THREE_REPLICA_CONSUMER_GROUP);
+        mqConsumerThreeReplica3.setConsumeFromWhere(ConsumeFromWhere.CONSUME_FROM_LAST_OFFSET);
+        mqConsumerThreeReplica3.subscribe(THREE_REPLICAS_TOPIC, "*");
+    }
+
+    @AfterClass
+    public static void afterClass() {
+        if (mqProducer != null) {
+            mqProducer.shutdown();
+        }
+    }
+
+    @Test
+    public void lockFromSlave() throws Exception {
+        awaitUntilSlaveOK();
+
+        mqConsumerThreeReplica3.registerMessageListener((MessageListenerOrderly) (msgs, context) -> ConsumeOrderlyStatus.SUCCESS);
+        mqConsumerThreeReplica3.start();
+
+        final Set<MessageQueue> mqSet = mqConsumerThreeReplica3.fetchSubscribeMessageQueues(THREE_REPLICAS_TOPIC);
+
+        assertThat(targetTopicMqCount(mqSet, THREE_REPLICAS_TOPIC)).isEqualTo(24);
+
+        for (MessageQueue mq : mqSet) {
+            await().atMost(Duration.ofSeconds(60)).until(() -> mqConsumerThreeReplica3.getDefaultMQPushConsumerImpl().getRebalanceImpl().lock(mq));
+        }
+
+        isolateBroker(master3With3Replicas);
+
+        mqConsumerThreeReplica3.getDefaultMQPushConsumerImpl().getmQClientFactory().updateTopicRouteInfoFromNameServer(THREE_REPLICAS_TOPIC);
+        FindBrokerResult result = mqConsumerThreeReplica3.getDefaultMQPushConsumerImpl().getmQClientFactory().findBrokerAddressInSubscribe(
+            master3With3Replicas.getBrokerConfig().getBrokerName(), MixAll.MASTER_ID, true);
+        assertThat(result).isNotNull();
+
+        for (MessageQueue mq : mqSet) {
+            if (mq.getBrokerName().equals(master3With3Replicas.getBrokerConfig().getBrokerName())) {
+                await().atMost(Duration.ofSeconds(60)).until(() -> mqConsumerThreeReplica3.getDefaultMQPushConsumerImpl().getRebalanceImpl().lock(mq));
+            }
+        }
+
+        removeSlaveBroker(1, brokerContainer1, master3With3Replicas);
+        assertThat(brokerContainer1.getSlaveBrokers().size()).isEqualTo(1);
+
+        mqConsumerThreeReplica3.getDefaultMQPushConsumerImpl().getmQClientFactory().updateTopicRouteInfoFromNameServer(THREE_REPLICAS_TOPIC);
+
+        for (MessageQueue mq : mqSet) {
+            if (mq.getBrokerName().equals(master3With3Replicas.getBrokerConfig().getBrokerName())) {
+                await().atMost(Duration.ofSeconds(60)).until(() -> !mqConsumerThreeReplica3.getDefaultMQPushConsumerImpl().getRebalanceImpl().lock(mq));
+            }
+        }
+
+        cancelIsolatedBroker(master3With3Replicas);
+        createAndAddSlave(1, brokerContainer1, master3With3Replicas);
+        awaitUntilSlaveOK();
+
+        mqConsumerThreeReplica3.shutdown();
+        await().atMost(100, TimeUnit.SECONDS).until(() -> mqConsumerThreeReplica3.getDefaultMQPushConsumerImpl().getServiceState() == ServiceState.SHUTDOWN_ALREADY);
+    }
+
+    @Ignore
+    @Test
+    public void multiConsumerLockFromSlave() throws MQClientException, InterruptedException {
+        awaitUntilSlaveOK();
+
+        mqConsumerThreeReplica1.registerMessageListener((MessageListenerOrderly) (msgs, context) -> ConsumeOrderlyStatus.SUCCESS);
+        mqConsumerThreeReplica1.start();
+
+        mqConsumerThreeReplica1.getDefaultMQPushConsumerImpl().doRebalance();
+        Set<MessageQueue> mqSet1 = filterMessageQueue(mqConsumerThreeReplica1.getDefaultMQPushConsumerImpl().getRebalanceImpl().getProcessQueueTable().keySet(), THREE_REPLICAS_TOPIC);
+
+        assertThat(mqSet1.size()).isEqualTo(24);
+
+        isolateBroker(master3With3Replicas);
+
+        System.out.printf("%s isolated%n", master3With3Replicas.getBrokerConfig().getCanonicalName());
+
+        Thread.sleep(5000);
+
+        mqConsumerThreeReplica2.registerMessageListener((MessageListenerOrderly) (msgs, context) -> ConsumeOrderlyStatus.SUCCESS);
+        mqConsumerThreeReplica2.start();
+
+        Thread.sleep(5000);
+
+        mqConsumerThreeReplica1.getDefaultMQPushConsumerImpl().getmQClientFactory().updateTopicRouteInfoFromNameServer(THREE_REPLICAS_TOPIC);
+        mqConsumerThreeReplica2.getDefaultMQPushConsumerImpl().getmQClientFactory().updateTopicRouteInfoFromNameServer(THREE_REPLICAS_TOPIC);
+
+        assertThat(mqConsumerThreeReplica1.getDefaultMQPushConsumerImpl().getmQClientFactory().findBrokerAddressInSubscribe(
+            master3With3Replicas.getBrokerConfig().getBrokerName(), MixAll.MASTER_ID, true)).isNotNull();
+
+        mqConsumerThreeReplica2.getDefaultMQPushConsumerImpl().getmQClientFactory().findBrokerAddressInSubscribe(
+            master3With3Replicas.getBrokerConfig().getBrokerName(), MixAll.MASTER_ID, true);
+        assertThat(mqConsumerThreeReplica2.getDefaultMQPushConsumerImpl().getmQClientFactory().findBrokerAddressInSubscribe(
+            master3With3Replicas.getBrokerConfig().getBrokerName(), MixAll.MASTER_ID, true)).isNotNull();
+
+        mqConsumerThreeReplica1.getDefaultMQPushConsumerImpl().doRebalance();
+        mqConsumerThreeReplica2.getDefaultMQPushConsumerImpl().doRebalance();
+
+        Set<MessageQueue> mqSet2 = filterMessageQueue(mqConsumerThreeReplica2.getDefaultMQPushConsumerImpl().getRebalanceImpl().getProcessQueueTable().keySet(), THREE_REPLICAS_TOPIC);
+
+        mqSet1 = filterMessageQueue(mqConsumerThreeReplica1.getDefaultMQPushConsumerImpl().getRebalanceImpl().getProcessQueueTable().keySet(), THREE_REPLICAS_TOPIC);
+
+        List<MessageQueue> mqList = new ArrayList<>();
+
+        for (MessageQueue mq : mqSet2) {
+            if (mq.getTopic().equals(THREE_REPLICAS_TOPIC)) {
+                mqList.add(mq);
+            }
+        }
+
+        for (MessageQueue mq : mqSet1) {
+            if (mq.getTopic().equals(THREE_REPLICAS_TOPIC)) {
+                mqList.add(mq);
+            }
+        }
+
+        await().atMost(Duration.ofSeconds(30)).until(() -> mqList.size() == 24);
+
+        cancelIsolatedBroker(master3With3Replicas);
+        awaitUntilSlaveOK();
+
+        mqConsumerThreeReplica1.shutdown();
+        mqConsumerThreeReplica2.shutdown();
+
+        await().atMost(100, TimeUnit.SECONDS).until(() ->
+            mqConsumerThreeReplica1.getDefaultMQPushConsumerImpl().getServiceState() == ServiceState.SHUTDOWN_ALREADY &&
+                mqConsumerThreeReplica2.getDefaultMQPushConsumerImpl().getServiceState() == ServiceState.SHUTDOWN_ALREADY
+        );
+    }
+
+    private static int targetTopicMqCount(Set<MessageQueue> mqSet, String topic) {
+        int count = 0;
+        for (MessageQueue mq : mqSet) {
+            if (mq.getTopic().equals(topic)) {
+                count++;
+            }
+        }
+        return count;
+    }
+}
diff --git a/test/src/test/java/org/apache/rocketmq/test/container/ScheduleSlaveActingMasterIT.java b/test/src/test/java/org/apache/rocketmq/test/container/ScheduleSlaveActingMasterIT.java
new file mode 100644
index 0000000..c2850f3
--- /dev/null
+++ b/test/src/test/java/org/apache/rocketmq/test/container/ScheduleSlaveActingMasterIT.java
@@ -0,0 +1,220 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.rocketmq.test.container;
+
+import java.io.UnsupportedEncodingException;
+import java.time.Duration;
+import java.util.Random;
+import java.util.concurrent.atomic.AtomicInteger;
+import org.apache.commons.lang3.RandomStringUtils;
+import org.apache.rocketmq.client.consumer.DefaultMQPushConsumer;
+import org.apache.rocketmq.client.consumer.listener.ConsumeConcurrentlyStatus;
+import org.apache.rocketmq.client.consumer.listener.MessageListenerConcurrently;
+import org.apache.rocketmq.client.producer.DefaultMQProducer;
+import org.apache.rocketmq.client.producer.SendResult;
+import org.apache.rocketmq.client.producer.SendStatus;
+import org.apache.rocketmq.common.BrokerIdentity;
+import org.apache.rocketmq.common.message.Message;
+import org.apache.rocketmq.common.message.MessageQueue;
+import org.apache.rocketmq.remoting.common.RemotingHelper;
+import org.junit.AfterClass;
+import org.junit.BeforeClass;
+import org.junit.Ignore;
+import org.junit.Test;
+
+import static org.awaitility.Awaitility.await;
+
+//The test is correct, but it takes too much time, so it is ignored for the time being
+@Ignore
+public class ScheduleSlaveActingMasterIT extends ContainerIntegrationTestBase {
+
+    private static final String CONSUME_GROUP = ScheduleSlaveActingMasterIT.class.getSimpleName() + "_Consumer";
+    private final static int MESSAGE_COUNT = 32;
+    private final static Random random = new Random();
+    private static DefaultMQProducer producer;
+    private final static String MESSAGE_STRING = RandomStringUtils.random(1024);
+    private static byte[] MESSAGE_BODY;
+
+    static {
+        try {
+            MESSAGE_BODY = MESSAGE_STRING.getBytes(RemotingHelper.DEFAULT_CHARSET);
+        } catch (UnsupportedEncodingException e) {
+            e.printStackTrace();
+        }
+    }
+
+    void createTopic(String topic) {
+        createTopicTo(master1With3Replicas, topic, 1, 1);
+        createTopicTo(master2With3Replicas, topic, 1, 1);
+        createTopicTo(master3With3Replicas, topic, 1, 1);
+    }
+
+    @BeforeClass
+    public static void beforeClass() throws Throwable {
+        producer = createProducer(ScheduleSlaveActingMasterIT.class.getSimpleName() + "_PRODUCER");
+        producer.setSendMsgTimeout(5000);
+        producer.start();
+    }
+
+    @AfterClass
+    public static void afterClass() throws Exception {
+        producer.shutdown();
+    }
+
+    @Test
+    public void testLocalActing_delayMsg() throws Exception {
+        awaitUntilSlaveOK();
+        String topic = ScheduleSlaveActingMasterIT.class.getSimpleName() + random.nextInt(65535);
+        createTopic(topic);
+        DefaultMQPushConsumer pushConsumer = createPushConsumer(CONSUME_GROUP);
+        pushConsumer.subscribe(topic, "*");
+        AtomicInteger receivedMsgCount = new AtomicInteger(0);
+        AtomicInteger inTimeMsgCount = new AtomicInteger(0);
+        pushConsumer.registerMessageListener((MessageListenerConcurrently) (msgs, context) -> {
+            long period = System.currentTimeMillis() - msgs.get(0).getBornTimestamp();
+            if (Math.abs(period - 30000) <= 4000) {
+                inTimeMsgCount.addAndGet(msgs.size());
+            }
+            receivedMsgCount.addAndGet(msgs.size());
+            msgs.forEach(x -> System.out.printf(x + "%n"));
+            return ConsumeConcurrentlyStatus.CONSUME_SUCCESS;
+        });
+        pushConsumer.start();
+
+        MessageQueue messageQueue = new MessageQueue(topic, master1With3Replicas.getBrokerConfig().getBrokerName(), 0);
+        int sendSuccess = 0;
+        for (int i = 0; i < MESSAGE_COUNT; i++) {
+            Message msg = new Message(topic, MESSAGE_BODY);
+            msg.setDelayTimeLevel(4);
+            SendResult sendResult = producer.send(msg, messageQueue);
+            if (sendResult.getSendStatus() == SendStatus.SEND_OK) {
+                sendSuccess++;
+            }
+        }
+        final int finalSendSuccess = sendSuccess;
+        await().atMost(Duration.ofMinutes(1)).until(() -> finalSendSuccess >= MESSAGE_COUNT);
+        System.out.printf("send success%n");
+
+        isolateBroker(master1With3Replicas);
+        brokerContainer1.removeBroker(new BrokerIdentity(
+            master1With3Replicas.getBrokerConfig().getBrokerClusterName(),
+            master1With3Replicas.getBrokerConfig().getBrokerName(),
+            master1With3Replicas.getBrokerConfig().getBrokerId()));
+
+        System.out.printf("Remove master1%n");
+
+        await().atMost(Duration.ofMinutes(1)).until(() -> receivedMsgCount.get() >= MESSAGE_COUNT && inTimeMsgCount.get() >= MESSAGE_COUNT * 0.95);
+
+        System.out.printf("consumer received %d msg, %d in time%n", receivedMsgCount.get(), inTimeMsgCount.get());
+
+        pushConsumer.shutdown();
+
+        //Add back master
+        master1With3Replicas = brokerContainer1.addBroker(master1With3Replicas.getBrokerConfig(), master1With3Replicas.getMessageStoreConfig());
+        master1With3Replicas.start();
+        cancelIsolatedBroker(master1With3Replicas);
+        System.out.printf("Add back master1%n");
+
+        awaitUntilSlaveOK();
+        // sleep a while to recover
+        Thread.sleep(30000);
+    }
+
+    @Test
+    public void testRemoteActing_delayMsg() throws Exception {
+        awaitUntilSlaveOK();
+
+        String topic = ScheduleSlaveActingMasterIT.class.getSimpleName() + random.nextInt(65535);
+        createTopic(topic);
+        AtomicInteger receivedMsgCount = new AtomicInteger(0);
+        AtomicInteger inTimeMsgCount = new AtomicInteger(0);
+        AtomicInteger master3MsgCount = new AtomicInteger(0);
+
+        MessageQueue messageQueue = new MessageQueue(topic, master1With3Replicas.getBrokerConfig().getBrokerName(), 0);
+        int sendSuccess = 0;
+        for (int i = 0; i < MESSAGE_COUNT; i++) {
+            Message msg = new Message(topic, MESSAGE_BODY);
+            msg.setDelayTimeLevel(4);
+            SendResult sendResult = producer.send(msg, messageQueue);
+            if (sendResult.getSendStatus() == SendStatus.SEND_OK) {
+                sendSuccess++;
+            }
+        }
+        final int finalSendSuccess = sendSuccess;
+        await().atMost(Duration.ofMinutes(1)).until(() -> finalSendSuccess >= MESSAGE_COUNT);
+        long sendCompleteTimeStamp = System.currentTimeMillis();
+        System.out.printf("send success%n");
+
+        DefaultMQPushConsumer pushConsumer = createPushConsumer(CONSUME_GROUP);
+        pushConsumer.subscribe(topic, "*");
+        pushConsumer.registerMessageListener((MessageListenerConcurrently) (msgs, context) -> {
+            long period = System.currentTimeMillis() - sendCompleteTimeStamp;
+            // Remote Acting lead to born timestamp, msgId changed, it need to polish.
+            if (Math.abs(period - 30000) <= 4000) {
+                inTimeMsgCount.addAndGet(msgs.size());
+            }
+            if (msgs.get(0).getBrokerName().equals(master3With3Replicas.getBrokerConfig().getBrokerName())) {
+                master3MsgCount.addAndGet(msgs.size());
+            }
+            receivedMsgCount.addAndGet(msgs.size());
+            msgs.forEach(x -> System.out.printf("cost " + period + " " + x + "%n"));
+            return ConsumeConcurrentlyStatus.CONSUME_SUCCESS;
+        });
+        pushConsumer.start();
+
+        isolateBroker(master1With3Replicas);
+        BrokerIdentity master1BrokerIdentity = new BrokerIdentity(
+            master1With3Replicas.getBrokerConfig().getBrokerClusterName(),
+            master1With3Replicas.getBrokerConfig().getBrokerName(),
+            master1With3Replicas.getBrokerConfig().getBrokerId());
+
+        brokerContainer1.removeBroker(master1BrokerIdentity);
+        System.out.printf("Remove master1%n");
+
+        isolateBroker(master2With3Replicas);
+        BrokerIdentity master2BrokerIdentity = new BrokerIdentity(
+            master2With3Replicas.getBrokerConfig().getBrokerClusterName(),
+            master2With3Replicas.getBrokerConfig().getBrokerName(),
+            master2With3Replicas.getBrokerConfig().getBrokerId());
+        brokerContainer2.removeBroker(master2BrokerIdentity);
+        System.out.printf("Remove master2%n");
+
+        await().atMost(Duration.ofMinutes(2)).until(() -> receivedMsgCount.get() >= MESSAGE_COUNT && master3MsgCount.get() >= MESSAGE_COUNT && inTimeMsgCount.get() >= MESSAGE_COUNT * 0.95);
+
+        System.out.printf("consumer received %d msg, %d in time%n", receivedMsgCount.get(), inTimeMsgCount.get());
+
+        pushConsumer.shutdown();
+
+        //Add back master
+        master1With3Replicas = brokerContainer1.addBroker(master1With3Replicas.getBrokerConfig(), master1With3Replicas.getMessageStoreConfig());
+        master1With3Replicas.start();
+        cancelIsolatedBroker(master1With3Replicas);
+        System.out.printf("Add back master1%n");
+
+        //Add back master
+        master2With3Replicas = brokerContainer2.addBroker(master2With3Replicas.getBrokerConfig(), master2With3Replicas.getMessageStoreConfig());
+        master2With3Replicas.start();
+        cancelIsolatedBroker(master2With3Replicas);
+        System.out.printf("Add back master2%n");
+
+        awaitUntilSlaveOK();
+        // sleep a while to recover
+        Thread.sleep(30000);
+    }
+
+}
diff --git a/test/src/test/java/org/apache/rocketmq/test/container/ScheduledMessageIT.java b/test/src/test/java/org/apache/rocketmq/test/container/ScheduledMessageIT.java
new file mode 100644
index 0000000..6e5c9a9
--- /dev/null
+++ b/test/src/test/java/org/apache/rocketmq/test/container/ScheduledMessageIT.java
@@ -0,0 +1,152 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.rocketmq.test.container;
+
+import java.io.UnsupportedEncodingException;
+import java.time.Duration;
+import java.util.Random;
+import java.util.concurrent.TimeUnit;
+import java.util.concurrent.atomic.AtomicInteger;
+import org.apache.commons.lang3.RandomStringUtils;
+import org.apache.rocketmq.client.consumer.DefaultMQPushConsumer;
+import org.apache.rocketmq.client.consumer.listener.ConsumeConcurrentlyStatus;
+import org.apache.rocketmq.client.consumer.listener.MessageListenerConcurrently;
+import org.apache.rocketmq.client.exception.MQBrokerException;
+import org.apache.rocketmq.client.exception.MQClientException;
+import org.apache.rocketmq.client.producer.DefaultMQProducer;
+import org.apache.rocketmq.common.message.Message;
+import org.apache.rocketmq.remoting.common.RemotingHelper;
+import org.apache.rocketmq.remoting.exception.RemotingException;
+import org.apache.rocketmq.store.DefaultMessageStore;
+import org.junit.AfterClass;
+import org.junit.BeforeClass;
+import org.junit.Ignore;
+import org.junit.Test;
+
+import static org.assertj.core.api.Java6Assertions.assertThat;
+import static org.awaitility.Awaitility.await;
+
+public class ScheduledMessageIT extends ContainerIntegrationTestBase {
+    private static DefaultMQProducer producer;
+
+    private static final String CONSUME_GROUP = ScheduledMessageIT.class.getSimpleName() + "_Consumer";
+    private static final String MESSAGE_STRING = RandomStringUtils.random(1024);
+    private static byte[] MESSAGE_BODY;
+
+    static {
+        try {
+            MESSAGE_BODY = MESSAGE_STRING.getBytes(RemotingHelper.DEFAULT_CHARSET);
+        } catch (UnsupportedEncodingException ignored) {
+        }
+    }
+
+    private static final String TOPIC_PREFIX = ScheduledMessageIT.class.getSimpleName() + "_TOPIC";
+    private static Random random = new Random();
+    private static final int MESSAGE_COUNT = 128;
+
+    public ScheduledMessageIT() throws UnsupportedEncodingException {
+    }
+
+    void createTopic(String topic) {
+        createTopicTo(master1With3Replicas, topic, 1, 1);
+        createTopicTo(master2With3Replicas, topic, 1, 1);
+        createTopicTo(master3With3Replicas, topic, 1, 1);
+    }
+
+    @BeforeClass
+    public static void beforeClass() throws Throwable {
+        producer = createProducer(ScheduledMessageIT.class.getSimpleName() + "_PRODUCER");
+        producer.setSendMsgTimeout(5000);
+        producer.start();
+    }
+
+    @AfterClass
+    public static void afterClass() throws Exception {
+        producer.shutdown();
+    }
+
+    @Ignore
+    @Test
+    public void consumeScheduledMsg() throws MQClientException, RemotingException, InterruptedException, MQBrokerException {
+        String topic = TOPIC_PREFIX + random.nextInt(65535);
+        createTopic(topic);
+        DefaultMQPushConsumer pushConsumer = createPushConsumer(CONSUME_GROUP + random.nextInt(65535));
+        pushConsumer.subscribe(topic, "*");
+        AtomicInteger receivedMsgCount = new AtomicInteger(0);
+        AtomicInteger inTimeMsgCount = new AtomicInteger(0);
+        pushConsumer.registerMessageListener((MessageListenerConcurrently) (msgs, context) -> {
+            long period = System.currentTimeMillis() - msgs.get(0).getBornTimestamp();
+            if (Math.abs(period - 5000) <= 1000) {
+                inTimeMsgCount.addAndGet(msgs.size());
+            }
+            receivedMsgCount.addAndGet(msgs.size());
+            msgs.forEach(x -> System.out.printf(receivedMsgCount.get()+" cost " + period + " " + x + "%n"));
+
+            return ConsumeConcurrentlyStatus.CONSUME_SUCCESS;
+        });
+        pushConsumer.start();
+
+        for (int i = 0; i < MESSAGE_COUNT; i++) {
+            Message msg = new Message(topic, MESSAGE_BODY);
+            msg.setDelayTimeLevel(2);
+            producer.send(msg);
+        }
+
+        await().atMost(Duration.ofSeconds(MESSAGE_COUNT * 2)).until(() -> receivedMsgCount.get() >= MESSAGE_COUNT && inTimeMsgCount.get() >= MESSAGE_COUNT * 0.95);
+
+        System.out.printf("consumer received %d msg, %d in time%n", receivedMsgCount.get(), inTimeMsgCount.get());
+
+        pushConsumer.shutdown();
+    }
+
+    @Test
+    public void consumeScheduledMsgFromSlave() throws MQClientException, RemotingException, InterruptedException, MQBrokerException {
+        String topic = TOPIC_PREFIX + random.nextInt(65535);
+        createTopic(topic);
+        DefaultMQPushConsumer pushConsumer = createPushConsumer(CONSUME_GROUP + random.nextInt(65535));
+        pushConsumer.subscribe(topic, "*");
+        AtomicInteger receivedMsgCount = new AtomicInteger(0);
+        pushConsumer.registerMessageListener((MessageListenerConcurrently) (msgs, context) -> {
+            receivedMsgCount.addAndGet(msgs.size());
+            msgs.forEach(x -> System.out.printf(x + "%n"));
+            return ConsumeConcurrentlyStatus.CONSUME_SUCCESS;
+        });
+
+        for (int i = 0; i < MESSAGE_COUNT; i++) {
+            Message msg = new Message(topic, String.valueOf(i).getBytes());
+            msg.setDelayTimeLevel(2);
+            producer.send(msg);
+        }
+
+        isolateBroker(master1With3Replicas);
+
+        producer.getDefaultMQProducerImpl().getmQClientFactory().updateTopicRouteInfoFromNameServer(topic);
+        assertThat(producer.getDefaultMQProducerImpl().getmQClientFactory().findBrokerAddressInPublish(topic)).isNull();
+
+        pushConsumer.start();
+
+        await().atMost(Duration.ofSeconds(MESSAGE_COUNT * 2)).until(() -> receivedMsgCount.get() >= MESSAGE_COUNT);
+
+        pushConsumer.shutdown();
+        cancelIsolatedBroker(master1With3Replicas);
+
+        await().atMost(100, TimeUnit.SECONDS)
+            .until(() -> ((DefaultMessageStore) master1With3Replicas.getMessageStore()).getHaService().getConnectionCount().get() == 2);
+    }
+
+}
diff --git a/test/src/test/java/org/apache/rocketmq/test/container/SendMultipleReplicasIT.java b/test/src/test/java/org/apache/rocketmq/test/container/SendMultipleReplicasIT.java
new file mode 100644
index 0000000..ea35781
--- /dev/null
+++ b/test/src/test/java/org/apache/rocketmq/test/container/SendMultipleReplicasIT.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.test.container;
+
+import java.io.UnsupportedEncodingException;
+import java.util.List;
+import java.util.concurrent.TimeUnit;
+import org.apache.rocketmq.client.exception.MQBrokerException;
+import org.apache.rocketmq.client.exception.MQClientException;
+import org.apache.rocketmq.client.producer.DefaultMQProducer;
+import org.apache.rocketmq.client.producer.SendResult;
+import org.apache.rocketmq.client.producer.SendStatus;
+import org.apache.rocketmq.common.message.Message;
+import org.apache.rocketmq.common.message.MessageQueue;
+import org.apache.rocketmq.remoting.common.RemotingHelper;
+import org.apache.rocketmq.remoting.exception.RemotingException;
+import org.apache.rocketmq.store.DefaultMessageStore;
+import org.junit.AfterClass;
+import org.junit.BeforeClass;
+import org.junit.Test;
+
+import static org.assertj.core.api.Java6Assertions.assertThat;
+import static org.awaitility.Awaitility.await;
+
+public class SendMultipleReplicasIT extends ContainerIntegrationTestBase {
+    private static DefaultMQProducer mqProducer;
+
+    private final byte[] MESSAGE_BODY = ("Hello RocketMQ ").getBytes(RemotingHelper.DEFAULT_CHARSET);
+
+    public SendMultipleReplicasIT() throws UnsupportedEncodingException {
+    }
+
+    @BeforeClass
+    public static void beforeClass() throws Exception {
+        mqProducer = createProducer("SendMultipleReplicasMessageIT_Producer");
+        mqProducer.setSendMsgTimeout(15 * 1000);
+        mqProducer.start();
+    }
+
+    @AfterClass
+    public static void afterClass() {
+        if (mqProducer != null) {
+            mqProducer.shutdown();
+        }
+    }
+
+    @Test
+    public void sendMessageToBrokerGroup() throws MQClientException, RemotingException, InterruptedException, MQBrokerException {
+        awaitUntilSlaveOK();
+
+        // Send message to broker group with three replicas
+        Message msg = new Message(THREE_REPLICAS_TOPIC, MESSAGE_BODY);
+        SendResult sendResult = mqProducer.send(msg);
+        assertThat(sendResult.getSendStatus()).isEqualTo(SendStatus.SEND_OK);
+    }
+
+    @Test
+    public void sendMessage_Auto_Replicas_Success() throws Exception {
+        await().atMost(100, TimeUnit.SECONDS)
+            .until(() -> ((DefaultMessageStore) master1With3Replicas.getMessageStore()).getHaService().getConnectionCount().get() == 2
+                && master1With3Replicas.getMessageStore().getAliveReplicaNumInGroup() == 3);
+        // Broker with 3 replicas configured as 3-2-1 auto replicas mode
+        Message msg = new Message(THREE_REPLICAS_TOPIC, MESSAGE_BODY);
+        SendResult sendResult = mqProducer.send(msg);
+        assertThat(sendResult.getSendStatus()).isEqualTo(SendStatus.SEND_OK);
+
+        // Remove two slave broker
+        removeSlaveBroker(1, brokerContainer2, master1With3Replicas);
+        removeSlaveBroker(2, brokerContainer3, master1With3Replicas);
+        await().atMost(100, TimeUnit.SECONDS)
+            .until(() ->
+                ((DefaultMessageStore) master1With3Replicas.getMessageStore()).getHaService().getConnectionCount().get() == 0
+                    && master1With3Replicas.getMessageStore().getAliveReplicaNumInGroup() == 1);
+
+        master1With3Replicas.getMessageStoreConfig().setEnableAutoInSyncReplicas(true);
+        List<MessageQueue> mqList = mqProducer.getDefaultMQProducerImpl().fetchPublishMessageQueues(THREE_REPLICAS_TOPIC);
+        MessageQueue targetMq = null;
+        for (MessageQueue mq : mqList) {
+            if (mq.getBrokerName().equals(master1With3Replicas.getBrokerConfig().getBrokerName())) {
+                targetMq = mq;
+            }
+        }
+
+        assertThat(targetMq).isNotNull();
+        // Although this broker group only has one slave broker, send will be success in auto mode.
+        msg = new Message(THREE_REPLICAS_TOPIC, MESSAGE_BODY);
+        sendResult = mqProducer.send(msg, targetMq);
+        assertThat(sendResult.getSendStatus()).isEqualTo(SendStatus.SEND_OK);
+
+        // Recover the cluster state
+        createAndAddSlave(1, brokerContainer2, master1With3Replicas);
+        createAndAddSlave(2, brokerContainer3, master1With3Replicas);
+        await().atMost(100, TimeUnit.SECONDS)
+            .until(() -> ((DefaultMessageStore) master1With3Replicas.getMessageStore()).getHaService().getConnectionCount().get() == 2
+                && master1With3Replicas.getMessageStore().getAliveReplicaNumInGroup() == 3);
+    }
+
+    @Test
+    public void sendMessage_Auto_Replicas_Failed()
+        throws Exception {
+        await().atMost(100, TimeUnit.SECONDS)
+            .until(() -> ((DefaultMessageStore) master1With3Replicas.getMessageStore()).getHaService().getConnectionCount().get() == 2
+                && master1With3Replicas.getMessageStore().getAliveReplicaNumInGroup() == 3);
+        // Broker with 3 replicas configured as 3-2-1 auto replicas mode
+        // Remove two slave broker
+        removeSlaveBroker(1, brokerContainer2, master1With3Replicas);
+        removeSlaveBroker(2, brokerContainer3, master1With3Replicas);
+        await().atMost(100, TimeUnit.SECONDS)
+            .until(() ->
+                ((DefaultMessageStore) master1With3Replicas.getMessageStore()).getHaService().getConnectionCount().get() == 0
+                    && master1With3Replicas.getMessageStore().getAliveReplicaNumInGroup() == 1);
+
+        // Disable the auto mode
+        master1With3Replicas.getMessageStoreConfig().setEnableAutoInSyncReplicas(false);
+
+        List<MessageQueue> mqList = mqProducer.getDefaultMQProducerImpl().fetchPublishMessageQueues(THREE_REPLICAS_TOPIC);
+        MessageQueue targetMq = null;
+        for (MessageQueue mq : mqList) {
+            if (mq.getBrokerName().equals(master1With3Replicas.getBrokerConfig().getBrokerName())) {
+                targetMq = mq;
+            }
+        }
+
+        assertThat(targetMq).isNotNull();
+
+        Message msg = new Message(THREE_REPLICAS_TOPIC, MESSAGE_BODY);
+        boolean exceptionCaught = false;
+        try {
+            mqProducer.send(msg, targetMq);
+        } catch (MQBrokerException e) {
+            exceptionCaught = true;
+        }
+
+        assertThat(exceptionCaught).isTrue();
+        // Recover the cluster state
+        createAndAddSlave(1, brokerContainer2, master1With3Replicas);
+        createAndAddSlave(2, brokerContainer3, master1With3Replicas);
+        await().atMost(100, TimeUnit.SECONDS)
+            .until(() -> ((DefaultMessageStore) master1With3Replicas.getMessageStore()).getHaService().getConnectionCount().get() == 2
+                && master1With3Replicas.getMessageStore().getAliveReplicaNumInGroup() == 3);
+    }
+}
diff --git a/test/src/test/java/org/apache/rocketmq/test/container/SlaveBrokerIT.java b/test/src/test/java/org/apache/rocketmq/test/container/SlaveBrokerIT.java
new file mode 100644
index 0000000..3e6cd8d
--- /dev/null
+++ b/test/src/test/java/org/apache/rocketmq/test/container/SlaveBrokerIT.java
@@ -0,0 +1,115 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.rocketmq.test.container;
+
+import java.time.Duration;
+import java.util.concurrent.TimeUnit;
+import org.apache.rocketmq.common.protocol.body.ClusterInfo;
+import org.apache.rocketmq.store.DefaultMessageStore;
+import org.junit.Test;
+
+import static org.assertj.core.api.Assertions.assertThat;
+import static org.awaitility.Awaitility.await;
+
+public class SlaveBrokerIT extends ContainerIntegrationTestBase {
+    @Test
+    public void reAddSlaveBroker() throws Exception {
+        await().atMost(Duration.ofMinutes(1)).until(() -> {
+            ClusterInfo clusterInfo = defaultMQAdminExt.examineBrokerClusterInfo();
+
+            if (clusterInfo.getClusterAddrTable().get(master1With3Replicas.getBrokerConfig().getBrokerClusterName()).size() != 3) {
+                return false;
+            }
+
+            if (clusterInfo.getBrokerAddrTable().get(master1With3Replicas.getBrokerConfig().getBrokerName()).getBrokerAddrs().size() != 3) {
+                return false;
+            }
+
+            if (clusterInfo.getBrokerAddrTable().get(master2With3Replicas.getBrokerConfig().getBrokerName()).getBrokerAddrs().size() != 3) {
+                return false;
+            }
+
+            if (clusterInfo.getBrokerAddrTable().get(master3With3Replicas.getBrokerConfig().getBrokerName()).getBrokerAddrs().size() != 3) {
+                return false;
+            }
+
+            return true;
+        });
+
+        // Remove one replicas from each broker group
+        removeSlaveBroker(1, brokerContainer1, master3With3Replicas);
+        removeSlaveBroker(1, brokerContainer2, master1With3Replicas);
+        removeSlaveBroker(1, brokerContainer3, master2With3Replicas);
+
+        await().atMost(Duration.ofMinutes(1)).until(() -> {
+            // Test cluster info again
+            ClusterInfo clusterInfo = defaultMQAdminExt.examineBrokerClusterInfo();
+            assertThat(clusterInfo.getBrokerAddrTable().get(master1With3Replicas.getBrokerConfig().getBrokerName()).getBrokerAddrs().size())
+                .isEqualTo(2);
+
+            assertThat(clusterInfo.getBrokerAddrTable().get(master2With3Replicas.getBrokerConfig().getBrokerName()).getBrokerAddrs().size())
+                .isEqualTo(2);
+
+            assertThat(clusterInfo.getBrokerAddrTable().get(master3With3Replicas.getBrokerConfig().getBrokerName()).getBrokerAddrs().size())
+                .isEqualTo(2);
+            return true;
+        });
+
+        // ReAdd the slave broker
+        createAndAddSlave(1, brokerContainer1, master3With3Replicas);
+        createAndAddSlave(1, brokerContainer2, master1With3Replicas);
+        createAndAddSlave(1, brokerContainer3, master2With3Replicas);
+
+        // Trigger a register action
+        //for (final SlaveBrokerController slaveBrokerController : brokerContainer1.getSlaveBrokers()) {
+        //    slaveBrokerController.registerBrokerAll(false, false, true);
+        //}
+        //
+        //for (final SlaveBrokerController slaveBrokerController : brokerContainer2.getSlaveBrokers()) {
+        //    slaveBrokerController.registerBrokerAll(false, false, true);
+        //}
+
+        await().atMost(Duration.ofMinutes(1)).until(() -> {
+            ClusterInfo clusterInfo = defaultMQAdminExt.examineBrokerClusterInfo();
+
+            return clusterInfo.getBrokerAddrTable()
+                .get(master1With3Replicas.getBrokerConfig().getBrokerName()).getBrokerAddrs().size() == 3
+                && clusterInfo.getBrokerAddrTable()
+                .get(master2With3Replicas.getBrokerConfig().getBrokerName()).getBrokerAddrs().size() == 3
+                && clusterInfo.getBrokerAddrTable()
+                .get(master2With3Replicas.getBrokerConfig().getBrokerName()).getBrokerAddrs().size() == 3;
+        });
+    }
+
+    @Test
+    public void reAddSlaveBroker_ConnectionCheck() throws Exception {
+        await().atMost(100, TimeUnit.SECONDS)
+            .until(() -> ((DefaultMessageStore) master3With3Replicas.getMessageStore()).getHaService().getConnectionCount().get() == 2);
+
+        removeSlaveBroker(1, brokerContainer1, master3With3Replicas);
+        createAndAddSlave(1, brokerContainer1, master3With3Replicas);
+
+        await().atMost(100, TimeUnit.SECONDS)
+            .until(() -> ((DefaultMessageStore) master3With3Replicas.getMessageStore()).getHaService().getConnectionCount().get() == 2);
+
+        await().atMost(100, TimeUnit.SECONDS)
+            .until(() -> ((DefaultMessageStore) master3With3Replicas.getMessageStore()).getHaService().inSyncSlaveNums(0) == 2);
+
+        Thread.sleep(1000 * 101);
+    }
+}
diff --git a/test/src/test/java/org/apache/rocketmq/test/container/SyncConsumerOffsetIT.java b/test/src/test/java/org/apache/rocketmq/test/container/SyncConsumerOffsetIT.java
new file mode 100644
index 0000000..7b243f5
--- /dev/null
+++ b/test/src/test/java/org/apache/rocketmq/test/container/SyncConsumerOffsetIT.java
@@ -0,0 +1,146 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.rocketmq.test.container;
+
+import java.io.UnsupportedEncodingException;
+import java.util.Arrays;
+import java.util.HashMap;
+import java.util.List;
+import java.util.Map;
+import java.util.Set;
+import java.util.concurrent.CountDownLatch;
+import java.util.concurrent.TimeUnit;
+
+import org.apache.rocketmq.broker.BrokerController;
+import org.apache.rocketmq.container.BrokerContainer;
+import org.apache.rocketmq.container.InnerSalveBrokerController;
+import org.apache.rocketmq.client.consumer.DefaultMQPushConsumer;
+import org.apache.rocketmq.client.consumer.listener.ConsumeConcurrentlyStatus;
+import org.apache.rocketmq.client.consumer.listener.MessageListenerConcurrently;
+import org.apache.rocketmq.client.exception.MQBrokerException;
+import org.apache.rocketmq.client.exception.MQClientException;
+import org.apache.rocketmq.client.producer.DefaultMQProducer;
+import org.apache.rocketmq.client.producer.SendResult;
+import org.apache.rocketmq.client.producer.SendStatus;
+import org.apache.rocketmq.common.consumer.ConsumeFromWhere;
+import org.apache.rocketmq.common.message.Message;
+import org.apache.rocketmq.common.message.MessageQueue;
+import org.apache.rocketmq.remoting.common.RemotingHelper;
+import org.apache.rocketmq.remoting.exception.RemotingException;
+import org.junit.AfterClass;
+import org.junit.BeforeClass;
+import org.junit.Test;
+
+import static org.assertj.core.api.Java6Assertions.assertThat;
+import static org.awaitility.Awaitility.await;
+
+public class SyncConsumerOffsetIT extends ContainerIntegrationTestBase {
+    private static final String THREE_REPLICA_CONSUMER_GROUP = "SyncConsumerOffsetIT_ConsumerThreeReplica";
+    private static final String TEST_SYNC_TOPIC = SyncConsumerOffsetIT.class.getSimpleName() + "_topic";
+
+    private static DefaultMQProducer mqProducer;
+    private static DefaultMQPushConsumer mqConsumerThreeReplica;
+
+    private final byte[] MESSAGE_BODY = ("Hello RocketMQ").getBytes(RemotingHelper.DEFAULT_CHARSET);
+
+    public SyncConsumerOffsetIT() throws UnsupportedEncodingException {
+    }
+
+    @BeforeClass
+    public static void beforeClass() throws Exception {
+        createTopicTo(master3With3Replicas, TEST_SYNC_TOPIC);
+
+        mqProducer = createProducer("SyncConsumerOffsetIT_Producer");
+        mqProducer.setSendMsgTimeout(15 * 1000);
+        mqProducer.start();
+
+        mqConsumerThreeReplica = createPushConsumer(THREE_REPLICA_CONSUMER_GROUP);
+        mqConsumerThreeReplica.setConsumeFromWhere(ConsumeFromWhere.CONSUME_FROM_LAST_OFFSET);
+        mqConsumerThreeReplica.subscribe(TEST_SYNC_TOPIC, "*");
+    }
+
+    @AfterClass
+    public static void afterClass() {
+        mqProducer.shutdown();
+        mqConsumerThreeReplica.shutdown();
+    }
+
+    @Test
+    public void syncConsumerOffsetWith3Replicas() throws InterruptedException, RemotingException, MQClientException, MQBrokerException {
+        syncConsumeOffsetInner(TEST_SYNC_TOPIC, mqConsumerThreeReplica,
+            master3With3Replicas, Arrays.asList(brokerContainer1, brokerContainer2));
+    }
+
+    private void syncConsumeOffsetInner(String topic, DefaultMQPushConsumer consumer, BrokerController master,
+        List<BrokerContainer> slaveContainers) throws InterruptedException, RemotingException, MQClientException, MQBrokerException {
+        awaitUntilSlaveOK();
+        String group = THREE_REPLICA_CONSUMER_GROUP;
+
+        int msgCount = 100;
+        for (int i = 0; i < msgCount; i++) {
+            Message msg = new Message(topic, MESSAGE_BODY);
+            SendResult sendResult = mqProducer.send(msg);
+            assertThat(sendResult.getSendStatus()).isEqualTo(SendStatus.SEND_OK);
+        }
+
+        CountDownLatch countDownLatch = new CountDownLatch(msgCount);
+        consumer.registerMessageListener((MessageListenerConcurrently) (msgs, context) -> {
+            countDownLatch.countDown();
+            return ConsumeConcurrentlyStatus.CONSUME_SUCCESS;
+        });
+        consumer.start();
+        boolean ok = countDownLatch.await(100, TimeUnit.SECONDS);
+        assertThat(ok).isEqualTo(true);
+        System.out.printf("consume complete%n");
+
+        final Set<MessageQueue> mqSet = filterMessageQueue(consumer.fetchSubscribeMessageQueues(topic), topic);
+
+        await().atMost(120, TimeUnit.SECONDS).until(() -> {
+            Map<Integer, Long> consumerOffsetMap = new HashMap<>();
+            long offsetTotal = 0L;
+            for (MessageQueue mq : mqSet) {
+                long queueOffset = master.getConsumerOffsetManager().queryOffset(group, topic, mq.getQueueId());
+                if (queueOffset < 0) {
+                    continue;
+                }
+                offsetTotal += queueOffset;
+                consumerOffsetMap.put(mq.getQueueId(), queueOffset);
+            }
+
+            if (offsetTotal < 100) {
+                return false;
+            }
+            boolean syncOk = true;
+
+            for (BrokerContainer brokerContainer : slaveContainers) {
+                for (InnerSalveBrokerController slave : brokerContainer.getSlaveBrokers()) {
+                    if (!slave.getBrokerConfig().getBrokerName().equals(master.getBrokerConfig().getBrokerName())) {
+                        continue;
+                    }
+                    for (MessageQueue mq : mqSet) {
+                        long slaveOffset = slave.getConsumerOffsetManager().queryOffset(group, topic, mq.getQueueId());
+                        boolean check = slaveOffset == consumerOffsetMap.get(mq.getQueueId());
+                        syncOk &= check;
+                    }
+                }
+            }
+
+            return syncOk;
+        });
+    }
+}
diff --git a/test/src/test/java/org/apache/rocketmq/test/statictopic/StaticTopicIT.java b/test/src/test/java/org/apache/rocketmq/test/statictopic/StaticTopicIT.java
index b3d2d47..a6e3e75 100644
--- a/test/src/test/java/org/apache/rocketmq/test/statictopic/StaticTopicIT.java
+++ b/test/src/test/java/org/apache/rocketmq/test/statictopic/StaticTopicIT.java
@@ -19,11 +19,7 @@ package org.apache.rocketmq.test.statictopic;
 
 import com.google.common.collect.ImmutableList;
 import com.google.common.collect.ImmutableSet;
-import org.apache.commons.cli.CommandLine;
-import org.apache.commons.cli.Options;
-import org.apache.commons.cli.PosixParser;
 import org.apache.log4j.Logger;
-import org.apache.rocketmq.acl.common.AclUtils;
 import org.apache.rocketmq.broker.BrokerController;
 import org.apache.rocketmq.common.MixAll;
 import org.apache.rocketmq.common.admin.ConsumeStats;
@@ -36,22 +32,20 @@ import org.apache.rocketmq.common.statictopic.LogicQueueMappingItem;
 import org.apache.rocketmq.common.statictopic.TopicConfigAndQueueMapping;
 import org.apache.rocketmq.common.statictopic.TopicQueueMappingOne;
 import org.apache.rocketmq.common.statictopic.TopicQueueMappingUtils;
-import org.apache.rocketmq.srvutil.ServerUtil;
 import org.apache.rocketmq.test.base.BaseConf;
 import org.apache.rocketmq.test.client.rmq.RMQNormalConsumer;
 import org.apache.rocketmq.test.client.rmq.RMQNormalProducer;
 import org.apache.rocketmq.test.listener.rmq.concurrent.RMQNormalListener;
 import org.apache.rocketmq.test.util.MQAdminTestUtils;
 import org.apache.rocketmq.test.util.MQRandomUtils;
+import org.apache.rocketmq.test.util.TestUtils;
 import org.apache.rocketmq.test.util.VerifyUtils;
 import org.apache.rocketmq.tools.admin.DefaultMQAdminExt;
 import org.apache.rocketmq.tools.admin.MQAdminUtils;
-import org.apache.rocketmq.tools.command.MQAdminStartup;
 import org.junit.After;
 import org.junit.Assert;
 import org.junit.Before;
 import org.junit.FixMethodOrder;
-import org.junit.Ignore;
 import org.junit.Test;
 
 import java.util.ArrayList;
@@ -210,8 +204,8 @@ public class StaticTopicIT extends BaseConf {
 
     private void consumeMessagesAndCheck(RMQNormalProducer producer, RMQNormalConsumer consumer, String topic, int queueNum, int msgEachQueue, int startGen, int genNum) {
         consumer.getListener().waitForMessageConsume(producer.getAllMsgBody(), 30000);
-//        System.out.println("produce:" + producer.getAllMsgBody().size());
-//        System.out.println("consume:" + consumer.getListener().getAllMsgBody().size());
+        /*System.out.println("produce:" + producer.getAllMsgBody().size());
+        System.out.println("consume:" + consumer.getListener().getAllMsgBody().size());*/
 
         assertThat(VerifyUtils.getFilterdMessage(producer.getAllMsgBody(),
                 consumer.getListener().getAllMsgBody()))
@@ -284,7 +278,6 @@ public class StaticTopicIT extends BaseConf {
             sendMessagesAndCheck(producer, targetBrokers, topic, queueNum, msgEachQueue, 0);
             consumeMessagesAndCheck(producer, consumer, topic, queueNum, msgEachQueue, 0, 1);
         }
-
         //remapping the static topic
         {
             Set<String> targetBrokers = ImmutableSet.of(broker2Name);
@@ -344,6 +337,8 @@ public class StaticTopicIT extends BaseConf {
             Thread.sleep(500);
             sendMessagesAndCheck(producer, targetBrokers, topic, queueNum, msgEachQueue, (i + 1) * TopicQueueMappingUtils.DEFAULT_BLOCK_SEQ_SIZE);
         }
+
+        TestUtils.waitForSeconds(20);
         consumeStats = defaultMQAdminExt.examineConsumeStats(group);
 
         messageQueues = producer.getMessageQueue();

[rocketmq] 02/07: feature(container&remoting):[RIP-31][PART-A]Support RocketMQ BrokerContainer (#3976)

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

jinrongtong pushed a commit to branch 5.0.0-beta-tmp
in repository https://gitbox.apache.org/repos/asf/rocketmq.git

commit 27ab1dfdcd9c97583656de3cf4bb44c6a57dcc02
Author: rongtong <ji...@163.com>
AuthorDate: Mon Mar 14 14:05:53 2022 +0800

    feature(container&remoting):[RIP-31][PART-A]Support RocketMQ BrokerContainer (#3976)
---
 container/pom.xml                                  |  36 ++
 .../apache/rocketmq/container/BrokerBootHook.java  |  48 ++
 .../apache/rocketmq/container/BrokerContainer.java | 435 +++++++++++++++++++
 .../rocketmq/container/BrokerContainerConfig.java  |  84 ++++
 .../container/BrokerContainerProcessor.java        | 273 ++++++++++++
 .../rocketmq/container/BrokerContainerStartup.java | 482 +++++++++++++++++++++
 .../rocketmq/container/BrokerPreOnlineService.java | 277 ++++++++++++
 .../ContainerClientHouseKeepingService.java        | 104 +++++
 .../rocketmq/container/IBrokerContainer.java       | 142 ++++++
 .../rocketmq/container/InnerBrokerController.java  | 378 ++++++++++++++++
 .../container/InnerSalveBrokerController.java      | 160 +++++++
 .../logback/BrokerLogbackConfigurator.java         | 187 ++++++++
 .../container/BrokerContainerStartupTest.java      | 151 +++++++
 .../rocketmq/container/BrokerContainerTest.java    | 339 +++++++++++++++
 .../rocketmq/container/BrokerPreOnlineTest.java    | 102 +++++
 pom.xml                                            |  20 +-
 .../java/org/apache/rocketmq/remoting/RPCHook.java |   4 +-
 .../apache/rocketmq/remoting/RemotingClient.java   |  13 +-
 .../apache/rocketmq/remoting/RemotingServer.java   |   6 +
 .../apache/rocketmq/remoting/RemotingService.java  |   5 +
 .../rocketmq/remoting/common/RemotingHelper.java   |  38 ++
 .../rocketmq/remoting/common/RemotingUtil.java     |  18 +-
 .../rocketmq/remoting/netty/NettyClientConfig.java |  38 ++
 .../remoting/netty/NettyRemotingAbstract.java      | 115 +++--
 .../remoting/netty/NettyRemotingClient.java        | 396 +++++++++++++----
 .../remoting/netty/NettyRemotingServer.java        | 158 ++++++-
 .../remoting/netty/NettyRequestProcessor.java      |   1 -
 .../rocketmq/remoting/netty/ResponseFuture.java    |  42 +-
 .../remoting/protocol/RemotingCommand.java         |   8 +-
 .../rocketmq/remoting/RemotingServerTest.java      |   5 +-
 .../rocketmq/remoting/SubRemotingServerTest.java   | 109 +++++
 .../remoting/netty/NettyRemotingAbstractTest.java  |   6 +-
 .../remoting/protocol/RemotingCommandTest.java     |   1 +
 33 files changed, 3995 insertions(+), 186 deletions(-)

diff --git a/container/pom.xml b/container/pom.xml
new file mode 100644
index 0000000..a7f13d1
--- /dev/null
+++ b/container/pom.xml
@@ -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.
+  -->
+<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>
+        <groupId>org.apache.rocketmq</groupId>
+        <artifactId>rocketmq-all</artifactId>
+        <version>5.0.0-BETA-SNAPSHOT</version>
+    </parent>
+
+    <modelVersion>4.0.0</modelVersion>
+    <packaging>jar</packaging>
+    <artifactId>rocketmq-container</artifactId>
+    <name>rocketmq-container ${project.version}</name>
+
+    <dependencies>
+        <dependency>
+            <groupId>org.apache.rocketmq</groupId>
+            <artifactId>rocketmq-broker</artifactId>
+        </dependency>
+    </dependencies>
+</project>
diff --git a/container/src/main/java/org/apache/rocketmq/container/BrokerBootHook.java b/container/src/main/java/org/apache/rocketmq/container/BrokerBootHook.java
new file mode 100644
index 0000000..fe126af
--- /dev/null
+++ b/container/src/main/java/org/apache/rocketmq/container/BrokerBootHook.java
@@ -0,0 +1,48 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.rocketmq.container;
+
+import java.util.Properties;
+import org.apache.rocketmq.broker.BrokerController;
+
+public interface BrokerBootHook {
+    /**
+     * Name of the hook.
+     *
+     * @return name of the hook
+     */
+    String hookName();
+
+    /**
+     * Code to execute before broker start.
+     *
+     * @param brokerController broker to start
+     * @param properties broker properties
+     * @throws Exception when execute hook
+     */
+    void executeBeforeStart(BrokerController brokerController, Properties properties) throws Exception;
+
+    /**
+     * Code to execute after broker start.
+     *
+     * @param brokerController broker to start
+     * @param properties broker properties
+     * @throws Exception when execute hook
+     */
+    void executeAfterStart(BrokerController brokerController, Properties properties) throws Exception;
+}
diff --git a/container/src/main/java/org/apache/rocketmq/container/BrokerContainer.java b/container/src/main/java/org/apache/rocketmq/container/BrokerContainer.java
new file mode 100644
index 0000000..50b8bc4
--- /dev/null
+++ b/container/src/main/java/org/apache/rocketmq/container/BrokerContainer.java
@@ -0,0 +1,435 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.rocketmq.container;
+
+import java.util.ArrayList;
+import java.util.Collection;
+import java.util.List;
+import java.util.concurrent.ConcurrentHashMap;
+import java.util.concurrent.ConcurrentMap;
+import java.util.concurrent.ExecutorService;
+import java.util.concurrent.LinkedBlockingQueue;
+import java.util.concurrent.ScheduledExecutorService;
+import java.util.concurrent.ScheduledThreadPoolExecutor;
+import java.util.concurrent.ThreadPoolExecutor;
+import java.util.concurrent.TimeUnit;
+
+import org.apache.commons.lang3.concurrent.BasicThreadFactory;
+import org.apache.rocketmq.broker.BrokerController;
+import org.apache.rocketmq.broker.BrokerPathConfigHelper;
+import org.apache.rocketmq.container.logback.BrokerLogbackConfigurator;
+import org.apache.rocketmq.broker.out.BrokerOuterAPI;
+import org.apache.rocketmq.common.AbstractBrokerRunnable;
+import org.apache.rocketmq.common.BrokerConfig;
+import org.apache.rocketmq.common.BrokerIdentity;
+import org.apache.rocketmq.common.Configuration;
+import org.apache.rocketmq.common.MixAll;
+import org.apache.rocketmq.common.ThreadFactoryImpl;
+import org.apache.rocketmq.common.constant.LoggerName;
+import org.apache.rocketmq.common.utils.ThreadUtils;
+import org.apache.rocketmq.logging.InternalLogger;
+import org.apache.rocketmq.logging.InternalLoggerFactory;
+import org.apache.rocketmq.remoting.RPCHook;
+import org.apache.rocketmq.remoting.RemotingServer;
+import org.apache.rocketmq.remoting.netty.NettyClientConfig;
+import org.apache.rocketmq.remoting.netty.NettyRemotingServer;
+import org.apache.rocketmq.remoting.netty.NettyServerConfig;
+import org.apache.rocketmq.store.config.BrokerRole;
+import org.apache.rocketmq.store.config.MessageStoreConfig;
+
+public class BrokerContainer implements IBrokerContainer {
+    private static final InternalLogger LOG = InternalLoggerFactory.getLogger(LoggerName.BROKER_LOGGER_NAME);
+
+    private final ScheduledExecutorService scheduledExecutorService = new ScheduledThreadPoolExecutor(1,
+        new BasicThreadFactory.Builder()
+            .namingPattern("BrokerContainerScheduledThread")
+            .daemon(true)
+            .build());
+    private final NettyServerConfig nettyServerConfig;
+    private final NettyClientConfig nettyClientConfig;
+    private final BrokerOuterAPI brokerOuterAPI;
+    private final ContainerClientHouseKeepingService containerClientHouseKeepingService;
+
+    private final ConcurrentMap<BrokerIdentity, InnerSalveBrokerController> slaveBrokerControllers = new ConcurrentHashMap<>();
+    private final ConcurrentMap<BrokerIdentity, InnerBrokerController> masterBrokerControllers = new ConcurrentHashMap<>();
+    private final List<BrokerBootHook> brokerBootHookList = new ArrayList<>();
+    private final BrokerContainerProcessor brokerContainerProcessor;
+    private final Configuration configuration;
+    private final BrokerContainerConfig brokerContainerConfig;
+
+    private RemotingServer remotingServer;
+    private RemotingServer fastRemotingServer;
+    private ExecutorService brokerContainerExecutor;
+
+    public BrokerContainer(
+        final BrokerContainerConfig brokerContainerConfig,
+        final NettyServerConfig nettyServerConfig,
+        final NettyClientConfig nettyClientConfig
+    ) {
+        this.brokerContainerConfig = brokerContainerConfig;
+        this.nettyServerConfig = nettyServerConfig;
+        this.nettyClientConfig = nettyClientConfig;
+
+        this.brokerOuterAPI = new BrokerOuterAPI(nettyClientConfig);
+
+        this.brokerContainerProcessor = new BrokerContainerProcessor(this);
+        this.brokerContainerProcessor.registerBrokerBootHook(this.brokerBootHookList);
+        this.containerClientHouseKeepingService = new ContainerClientHouseKeepingService(this);
+
+        this.configuration = new Configuration(
+            LOG,
+            BrokerPathConfigHelper.getBrokerConfigPath(),
+            this.brokerContainerConfig, this.nettyServerConfig, this.nettyClientConfig);
+    }
+
+    @Override
+    public String getBrokerContainerAddr() {
+        return this.brokerContainerConfig.getBrokerContainerIP() + ":" + this.nettyServerConfig.getListenPort();
+    }
+
+    @Override
+    public BrokerContainerConfig getBrokerContainerConfig() {
+        return brokerContainerConfig;
+    }
+
+    @Override
+    public NettyServerConfig getNettyServerConfig() {
+        return nettyServerConfig;
+    }
+
+    public NettyClientConfig getNettyClientConfig() {
+        return nettyClientConfig;
+    }
+
+    @Override
+    public BrokerOuterAPI getBrokerOuterAPI() {
+        return brokerOuterAPI;
+    }
+
+    @Override
+    public RemotingServer getRemotingServer() {
+        return remotingServer;
+    }
+
+    public Configuration getConfiguration() {
+        return this.configuration;
+    }
+
+    public boolean initialize() {
+        this.remotingServer = new NettyRemotingServer(this.nettyServerConfig, this.containerClientHouseKeepingService);
+        this.fastRemotingServer = this.remotingServer.newRemotingServer(this.nettyServerConfig.getListenPort() - 2);
+
+        this.brokerContainerExecutor = new ThreadPoolExecutor(
+            1,
+            1,
+            1000 * 60,
+            TimeUnit.MILLISECONDS,
+            new LinkedBlockingQueue<>(10000),
+            new ThreadFactoryImpl("SharedBrokerThread_"));
+
+        this.registerProcessor();
+
+        if (this.brokerContainerConfig.getNamesrvAddr() != null) {
+            this.brokerOuterAPI.updateNameServerAddressList(this.brokerContainerConfig.getNamesrvAddr());
+            LOG.info("Set user specified name server address: {}", this.brokerContainerConfig.getNamesrvAddr());
+            // also auto update namesrv if specify
+            this.scheduledExecutorService.scheduleAtFixedRate(new AbstractBrokerRunnable(BrokerIdentity.BROKER_CONTAINER_IDENTITY) {
+                @Override
+                public void run2() {
+                    try {
+                        BrokerContainer.this.brokerOuterAPI.updateNameServerAddressList(BrokerContainer.this.brokerContainerConfig.getNamesrvAddr());
+                    } catch (Throwable e) {
+                        LOG.error("ScheduledTask fetchNameServerAddr exception", e);
+                    }
+                }
+            }, 1000 * 10, 1000 * 60 * 2, TimeUnit.MILLISECONDS);
+        } else if (this.brokerContainerConfig.isFetchNamesrvAddrByAddressServer()) {
+            this.scheduledExecutorService.scheduleAtFixedRate(new AbstractBrokerRunnable(BrokerIdentity.BROKER_CONTAINER_IDENTITY) {
+
+                @Override
+                public void run2() {
+                    try {
+                        BrokerContainer.this.brokerOuterAPI.fetchNameServerAddr();
+                    } catch (Throwable e) {
+                        LOG.error("ScheduledTask fetchNameServerAddr exception", e);
+                    }
+                }
+            }, 1000 * 10, 1000 * 60 * 2, TimeUnit.MILLISECONDS);
+        }
+
+        this.scheduledExecutorService.scheduleAtFixedRate(new AbstractBrokerRunnable(BrokerIdentity.BROKER_CONTAINER_IDENTITY) {
+            @Override
+            public void run2() {
+                try {
+                    BrokerContainer.this.brokerOuterAPI.refreshMetadata();
+                } catch (Exception e) {
+                    LOG.error("ScheduledTask refresh metadata exception", e);
+                }
+            }
+        }, 1, 5, TimeUnit.SECONDS);
+
+        return true;
+    }
+
+    private void registerProcessor() {
+        remotingServer.registerDefaultProcessor(brokerContainerProcessor, this.brokerContainerExecutor);
+        fastRemotingServer.registerDefaultProcessor(brokerContainerProcessor, this.brokerContainerExecutor);
+    }
+
+    @Override
+    public void start() throws Exception {
+        if (this.remotingServer != null) {
+            this.remotingServer.start();
+        }
+
+        if (this.fastRemotingServer != null) {
+            this.fastRemotingServer.start();
+        }
+
+        if (this.brokerOuterAPI != null) {
+            this.brokerOuterAPI.start();
+        }
+    }
+
+    @Override
+    public void shutdown() {
+        // Shutdown slave brokers
+        for (InnerSalveBrokerController slaveBrokerController : slaveBrokerControllers.values()) {
+            slaveBrokerController.shutdown();
+        }
+
+        slaveBrokerControllers.clear();
+
+        // Shutdown master brokers
+        for (BrokerController masterBrokerController : masterBrokerControllers.values()) {
+            masterBrokerController.shutdown();
+        }
+
+        masterBrokerControllers.clear();
+
+        // Shutdown the remoting server with a high priority to avoid further traffic
+        if (this.remotingServer != null) {
+            this.remotingServer.shutdown();
+        }
+
+        if (this.fastRemotingServer != null) {
+            this.fastRemotingServer.shutdown();
+        }
+
+        // Shutdown the request executors
+        ThreadUtils.shutdown(this.brokerContainerExecutor);
+
+        if (this.brokerOuterAPI != null) {
+            this.brokerOuterAPI.shutdown();
+        }
+    }
+
+    public void registerClientRPCHook(RPCHook rpcHook) {
+        this.getBrokerOuterAPI().registerRPCHook(rpcHook);
+    }
+
+    public void clearClientRPCHook() {
+        this.getBrokerOuterAPI().clearRPCHook();
+    }
+
+    public List<BrokerBootHook> getBrokerBootHookList() {
+        return brokerBootHookList;
+    }
+
+    public void registerBrokerBootHook(BrokerBootHook brokerBootHook) {
+        this.brokerBootHookList.add(brokerBootHook);
+        LOG.info("register BrokerBootHook, {}", brokerBootHook.hookName());
+    }
+
+    @Override
+    public InnerBrokerController addBroker(final BrokerConfig brokerConfig,
+        final MessageStoreConfig storeConfig) throws Exception {
+        if (brokerConfig.getBrokerId() == MixAll.MASTER_ID && storeConfig.getBrokerRole() != BrokerRole.SLAVE) {
+            return this.addMasterBroker(brokerConfig, storeConfig);
+        }
+        if (brokerConfig.getBrokerId() != MixAll.MASTER_ID && storeConfig.getBrokerRole() == BrokerRole.SLAVE) {
+            return this.addSlaveBroker(brokerConfig, storeConfig);
+        }
+
+        return null;
+    }
+
+    public InnerBrokerController addMasterBroker(final BrokerConfig masterBrokerConfig,
+        final MessageStoreConfig storeConfig) throws Exception {
+
+        masterBrokerConfig.setInBrokerContainer(true);
+        if (storeConfig.isDuplicationEnable()) {
+            LOG.error("Can not add broker to container when duplicationEnable is true currently");
+            throw new Exception("Can not add broker to container when duplicationEnable is true currently");
+        }
+        InnerBrokerController masterBroker = new InnerBrokerController(this, masterBrokerConfig, storeConfig);
+        BrokerIdentity brokerIdentity = new BrokerIdentity(masterBrokerConfig.getBrokerClusterName(),
+            masterBrokerConfig.getBrokerName(), masterBrokerConfig.getBrokerId());
+        final BrokerController previousBroker = masterBrokerControllers.putIfAbsent(brokerIdentity, masterBroker);
+        if (previousBroker == null) {
+            // New master broker added, start it
+            try {
+                BrokerLogbackConfigurator.doConfigure(masterBrokerConfig);
+                final boolean initResult = masterBroker.initialize();
+                if (!initResult) {
+                    masterBroker.shutdown();
+                    masterBrokerControllers.remove(brokerIdentity);
+                    throw new Exception("Failed to init master broker " + masterBrokerConfig.getCanonicalName());
+                }
+
+                for (InnerSalveBrokerController slaveBroker : this.getSlaveBrokers()) {
+                    if (slaveBroker.getMessageStore().getMasterStoreInProcess() == null) {
+                        slaveBroker.getMessageStore().setMasterStoreInProcess(masterBroker.getMessageStore());
+                    }
+                }
+            } catch (Exception e) {
+                // Remove the failed master broker and throw the exception
+                masterBroker.shutdown();
+                masterBrokerControllers.remove(brokerIdentity);
+                throw new Exception("Failed to initialize master broker " + masterBrokerConfig.getCanonicalName(), e);
+            }
+            return masterBroker;
+        }
+        throw new Exception(masterBrokerConfig.getCanonicalName() + " has already been added to current broker");
+    }
+
+    /**
+     * This function will create a slave broker along with the main broker, and start it with a different port.
+     *
+     * @param slaveBrokerConfig the specific slave broker config
+     * @throws Exception is thrown if an error occurs
+     */
+    public InnerSalveBrokerController addSlaveBroker(final BrokerConfig slaveBrokerConfig,
+        final MessageStoreConfig storeConfig) throws Exception {
+
+        slaveBrokerConfig.setInBrokerContainer(true);
+        if (storeConfig.isDuplicationEnable()) {
+            LOG.error("Can not add broker to container when duplicationEnable is true currently");
+            throw new Exception("Can not add broker to container when duplicationEnable is true currently");
+        }
+
+        int ratio = storeConfig.getAccessMessageInMemoryMaxRatio() - 10;
+        storeConfig.setAccessMessageInMemoryMaxRatio(Math.max(ratio, 0));
+        InnerSalveBrokerController slaveBroker = new InnerSalveBrokerController(this, slaveBrokerConfig, storeConfig);
+        BrokerIdentity brokerIdentity = new BrokerIdentity(slaveBrokerConfig.getBrokerClusterName(),
+            slaveBrokerConfig.getBrokerName(), slaveBrokerConfig.getBrokerId());
+        final InnerSalveBrokerController previousBroker = slaveBrokerControllers.putIfAbsent(brokerIdentity, slaveBroker);
+        if (previousBroker == null) {
+            // New slave broker added, start it
+            try {
+                BrokerLogbackConfigurator.doConfigure(slaveBrokerConfig);
+                final boolean initResult = slaveBroker.initialize();
+                if (!initResult) {
+                    slaveBroker.shutdown();
+                    slaveBrokerControllers.remove(brokerIdentity);
+                    throw new Exception("Failed to init slave broker " + slaveBrokerConfig.getCanonicalName());
+                }
+                BrokerController masterBroker = this.peekMasterBroker();
+                if (slaveBroker.getMessageStore().getMasterStoreInProcess() == null && masterBroker != null) {
+                    slaveBroker.getMessageStore().setMasterStoreInProcess(masterBroker.getMessageStore());
+                }
+            } catch (Exception e) {
+                // Remove the failed slave broker and throw the exception
+                slaveBroker.shutdown();
+                slaveBrokerControllers.remove(brokerIdentity);
+                throw new Exception("Failed to initialize slave broker " + slaveBrokerConfig.getCanonicalName(), e);
+            }
+            return slaveBroker;
+        }
+        throw new Exception(slaveBrokerConfig.getCanonicalName() + " has already been added to current broker");
+    }
+
+    @Override
+    public BrokerController removeBroker(final BrokerIdentity brokerIdentity) throws Exception {
+
+        InnerSalveBrokerController slaveBroker = slaveBrokerControllers.remove(brokerIdentity);
+        if (slaveBroker != null) {
+            slaveBroker.shutdown();
+            return slaveBroker;
+        }
+
+        BrokerController masterBroker = masterBrokerControllers.remove(brokerIdentity);
+
+        BrokerController nextMasterBroker = this.peekMasterBroker();
+        for (InnerSalveBrokerController slave : this.getSlaveBrokers()) {
+            if (nextMasterBroker == null) {
+                slave.getMessageStore().setMasterStoreInProcess(null);
+            } else {
+                slave.getMessageStore().setMasterStoreInProcess(nextMasterBroker.getMessageStore());
+            }
+
+        }
+
+        if (masterBroker != null) {
+            masterBroker.shutdown();
+            return masterBroker;
+        }
+
+        return null;
+    }
+
+    @Override
+    public BrokerController getBroker(final BrokerIdentity brokerIdentity) {
+        InnerSalveBrokerController slaveBroker = slaveBrokerControllers.get(brokerIdentity);
+        if (slaveBroker != null) {
+            return slaveBroker;
+        }
+
+        return masterBrokerControllers.get(brokerIdentity);
+    }
+
+    @Override
+    public Collection<InnerBrokerController> getMasterBrokers() {
+        return masterBrokerControllers.values();
+    }
+
+    @Override
+    public Collection<InnerSalveBrokerController> getSlaveBrokers() {
+        return slaveBrokerControllers.values();
+    }
+
+    @Override
+    public List<BrokerController> getBrokerControllers() {
+        List<BrokerController> brokerControllers = new ArrayList<>();
+        brokerControllers.addAll(this.getMasterBrokers());
+        brokerControllers.addAll(this.getSlaveBrokers());
+        return brokerControllers;
+    }
+
+    @Override
+    public BrokerController peekMasterBroker() {
+        if (!masterBrokerControllers.isEmpty()) {
+            return masterBrokerControllers.values().iterator().next();
+        }
+        return null;
+    }
+
+    public BrokerController findBrokerControllerByBrokerName(String brokerName) {
+        for (BrokerController brokerController : masterBrokerControllers.values()) {
+            if (brokerController.getBrokerConfig().getBrokerName().equals(brokerName)) {
+                return brokerController;
+            }
+        }
+
+        for (BrokerController brokerController : slaveBrokerControllers.values()) {
+            if (brokerController.getBrokerConfig().getBrokerName().equals(brokerName)) {
+                return brokerController;
+            }
+        }
+        return null;
+    }
+}
diff --git a/container/src/main/java/org/apache/rocketmq/container/BrokerContainerConfig.java b/container/src/main/java/org/apache/rocketmq/container/BrokerContainerConfig.java
new file mode 100644
index 0000000..28a5242
--- /dev/null
+++ b/container/src/main/java/org/apache/rocketmq/container/BrokerContainerConfig.java
@@ -0,0 +1,84 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.rocketmq.container;
+
+import org.apache.rocketmq.common.MixAll;
+import org.apache.rocketmq.common.annotation.ImportantField;
+import org.apache.rocketmq.remoting.common.RemotingUtil;
+
+public class BrokerContainerConfig {
+
+    private String rocketmqHome = System.getProperty(MixAll.ROCKETMQ_HOME_PROPERTY, System.getenv(MixAll.ROCKETMQ_HOME_ENV));
+
+    @ImportantField
+    private String namesrvAddr = System.getProperty(MixAll.NAMESRV_ADDR_PROPERTY, System.getenv(MixAll.NAMESRV_ADDR_ENV));
+
+    @ImportantField
+    private boolean fetchNamesrvAddrByAddressServer = false;
+
+    @ImportantField
+    private String brokerContainerIP = RemotingUtil.getLocalAddress();
+
+    private String brokerConfigPaths = null;
+
+    private boolean compatibleWithOldNameSrv = true;
+
+    public String getRocketmqHome() {
+        return rocketmqHome;
+    }
+
+    public void setRocketmqHome(String rocketmqHome) {
+        this.rocketmqHome = rocketmqHome;
+    }
+
+    public String getNamesrvAddr() {
+        return namesrvAddr;
+    }
+
+    public void setNamesrvAddr(String namesrvAddr) {
+        this.namesrvAddr = namesrvAddr;
+    }
+
+    public boolean isFetchNamesrvAddrByAddressServer() {
+        return fetchNamesrvAddrByAddressServer;
+    }
+
+    public void setFetchNamesrvAddrByAddressServer(boolean fetchNamesrvAddrByAddressServer) {
+        this.fetchNamesrvAddrByAddressServer = fetchNamesrvAddrByAddressServer;
+    }
+
+    public String getBrokerContainerIP() {
+        return brokerContainerIP;
+    }
+
+    public String getBrokerConfigPaths() {
+        return brokerConfigPaths;
+    }
+
+    public void setBrokerConfigPaths(String brokerConfigPaths) {
+        this.brokerConfigPaths = brokerConfigPaths;
+    }
+
+    public boolean isCompatibleWithOldNameSrv() {
+        return compatibleWithOldNameSrv;
+    }
+
+    public void setCompatibleWithOldNameSrv(boolean compatibleWithOldNameSrv) {
+        this.compatibleWithOldNameSrv = compatibleWithOldNameSrv;
+    }
+}
diff --git a/container/src/main/java/org/apache/rocketmq/container/BrokerContainerProcessor.java b/container/src/main/java/org/apache/rocketmq/container/BrokerContainerProcessor.java
new file mode 100644
index 0000000..6893882
--- /dev/null
+++ b/container/src/main/java/org/apache/rocketmq/container/BrokerContainerProcessor.java
@@ -0,0 +1,273 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.rocketmq.container;
+
+import io.netty.channel.ChannelHandlerContext;
+
+import java.io.UnsupportedEncodingException;
+import java.util.List;
+import java.util.Properties;
+import org.apache.rocketmq.broker.BrokerController;
+import org.apache.rocketmq.broker.BrokerStartup;
+import org.apache.rocketmq.common.BrokerConfig;
+import org.apache.rocketmq.common.BrokerIdentity;
+import org.apache.rocketmq.common.MixAll;
+import org.apache.rocketmq.common.constant.LoggerName;
+import org.apache.rocketmq.common.protocol.RequestCode;
+import org.apache.rocketmq.common.protocol.ResponseCode;
+import org.apache.rocketmq.common.protocol.header.AddBrokerRequestHeader;
+import org.apache.rocketmq.common.protocol.header.GetBrokerConfigResponseHeader;
+import org.apache.rocketmq.common.protocol.header.RemoveBrokerRequestHeader;
+import org.apache.rocketmq.logging.InternalLogger;
+import org.apache.rocketmq.logging.InternalLoggerFactory;
+import org.apache.rocketmq.remoting.common.RemotingHelper;
+import org.apache.rocketmq.remoting.exception.RemotingCommandException;
+import org.apache.rocketmq.remoting.netty.NettyRequestProcessor;
+import org.apache.rocketmq.remoting.protocol.RemotingCommand;
+import org.apache.rocketmq.store.config.MessageStoreConfig;
+
+public class BrokerContainerProcessor implements NettyRequestProcessor {
+    private static final InternalLogger LOGGER = InternalLoggerFactory.getLogger(LoggerName.BROKER_LOGGER_NAME);
+    private final BrokerContainer brokerContainer;
+    private List<BrokerBootHook> brokerBootHookList;
+
+    public BrokerContainerProcessor(BrokerContainer brokerContainer) {
+        this.brokerContainer = brokerContainer;
+    }
+
+    @Override
+    public RemotingCommand processRequest(ChannelHandlerContext ctx, RemotingCommand request) throws Exception {
+        switch (request.getCode()) {
+            case RequestCode.ADD_BROKER:
+                return this.addBroker(ctx, request);
+            case RequestCode.REMOVE_BROKER:
+                return this.removeBroker(ctx, request);
+            case RequestCode.GET_BROKER_CONFIG:
+                return this.getBrokerConfig(ctx, request);
+            case RequestCode.UPDATE_BROKER_CONFIG:
+                return this.updateBrokerConfig(ctx, request);
+            default:
+                break;
+        }
+        return null;
+    }
+
+    @Override public boolean rejectRequest() {
+        return false;
+    }
+
+    private synchronized RemotingCommand addBroker(ChannelHandlerContext ctx,
+        RemotingCommand request) throws Exception {
+        final RemotingCommand response = RemotingCommand.createResponseCommand(null);
+        final AddBrokerRequestHeader requestHeader = request.decodeCommandCustomHeader(AddBrokerRequestHeader.class);
+
+        LOGGER.info("addBroker called by {}", RemotingHelper.parseChannelRemoteAddr(ctx.channel()));
+
+        Properties brokerProperties = null;
+        String configPath = requestHeader.getConfigPath();
+
+        if (configPath != null && !configPath.isEmpty()) {
+            BrokerStartup.SystemConfigFileHelper configFileHelper = new BrokerStartup.SystemConfigFileHelper();
+            configFileHelper.setFile(configPath);
+
+            try {
+                brokerProperties = configFileHelper.loadConfig();
+            } catch (Exception e) {
+                LOGGER.error("addBroker load config from {} failed, {}", configPath, e);
+            }
+        } else {
+            byte[] body = request.getBody();
+            if (body != null) {
+                String bodyStr = new String(body, MixAll.DEFAULT_CHARSET);
+                brokerProperties = MixAll.string2Properties(bodyStr);
+            }
+        }
+
+        if (brokerProperties == null) {
+            LOGGER.error("addBroker properties empty");
+            response.setCode(ResponseCode.SYSTEM_ERROR);
+            response.setRemark("addBroker properties empty");
+            return response;
+        }
+
+        BrokerConfig brokerConfig = new BrokerConfig();
+        MessageStoreConfig messageStoreConfig = new MessageStoreConfig();
+        MixAll.properties2Object(brokerProperties, brokerConfig);
+        MixAll.properties2Object(brokerProperties, messageStoreConfig);
+
+        messageStoreConfig.setHaListenPort(brokerConfig.getListenPort() + 1);
+
+        if (configPath != null && !configPath.isEmpty()) {
+            brokerConfig.setBrokerConfigPath(configPath);
+        }
+
+        switch (messageStoreConfig.getBrokerRole()) {
+            case ASYNC_MASTER:
+            case SYNC_MASTER:
+                brokerConfig.setBrokerId(MixAll.MASTER_ID);
+                break;
+            case SLAVE:
+                if (brokerConfig.getBrokerId() <= 0) {
+                    response.setCode(ResponseCode.SYSTEM_ERROR);
+                    response.setRemark("slave broker id must be > 0");
+                    return response;
+                }
+                break;
+            default:
+                break;
+
+        }
+
+        if (messageStoreConfig.getTotalReplicas() < messageStoreConfig.getInSyncReplicas()
+            || messageStoreConfig.getTotalReplicas() < messageStoreConfig.getMinInSyncReplicas()
+            || messageStoreConfig.getInSyncReplicas() < messageStoreConfig.getMinInSyncReplicas()) {
+            response.setCode(ResponseCode.SYSTEM_ERROR);
+            response.setRemark("invalid replicas number");
+            return response;
+        }
+
+        BrokerController brokerController;
+        try {
+            brokerController = this.brokerContainer.addBroker(brokerConfig, messageStoreConfig);
+        } catch (Exception e) {
+            LOGGER.error("addBroker exception {}", e);
+            response.setCode(ResponseCode.SYSTEM_ERROR);
+            response.setRemark(e.getMessage());
+            return response;
+        }
+        if (brokerController != null) {
+            brokerController.getConfiguration().registerConfig(brokerProperties);
+            try {
+                for (BrokerBootHook brokerBootHook : brokerBootHookList) {
+                    brokerBootHook.executeBeforeStart(brokerController, brokerProperties);
+                }
+                brokerController.start();
+
+                for (BrokerBootHook brokerBootHook : brokerBootHookList) {
+                    brokerBootHook.executeAfterStart(brokerController, brokerProperties);
+                }
+            } catch (Exception e) {
+                LOGGER.error("start broker exception {}", e);
+                BrokerIdentity brokerIdentity = new BrokerIdentity(brokerConfig.getBrokerClusterName(),
+                    brokerConfig.getBrokerName(),
+                    brokerConfig.getBrokerId());
+                this.brokerContainer.removeBroker(brokerIdentity);
+                brokerController.shutdown();
+                response.setCode(ResponseCode.SYSTEM_ERROR);
+                response.setRemark("start broker failed, " + e);
+                return response;
+            }
+            response.setCode(ResponseCode.SUCCESS);
+            response.setRemark(null);
+        } else {
+            response.setCode(ResponseCode.SYSTEM_ERROR);
+            response.setRemark("add broker return null");
+        }
+
+        return response;
+    }
+
+    private synchronized RemotingCommand removeBroker(ChannelHandlerContext ctx,
+        RemotingCommand request) throws RemotingCommandException {
+        final RemotingCommand response = RemotingCommand.createResponseCommand(null);
+        final RemoveBrokerRequestHeader requestHeader = request.decodeCommandCustomHeader(RemoveBrokerRequestHeader.class);
+
+        LOGGER.info("removeBroker called by {}", RemotingHelper.parseChannelRemoteAddr(ctx.channel()));
+
+        BrokerIdentity brokerIdentity = new BrokerIdentity(requestHeader.getBrokerClusterName(), requestHeader.getBrokerName(), requestHeader.getBrokerId());
+
+        BrokerController brokerController;
+        try {
+            brokerController = this.brokerContainer.removeBroker(brokerIdentity);
+        } catch (Exception e) {
+            response.setCode(ResponseCode.SYSTEM_ERROR);
+            response.setRemark(e.getMessage());
+            return response;
+        }
+
+        if (brokerController != null) {
+            response.setCode(ResponseCode.SUCCESS);
+            response.setRemark(null);
+        } else {
+            response.setCode(ResponseCode.BROKER_NOT_EXIST);
+            response.setRemark("Broker not exist");
+        }
+        return response;
+    }
+
+    public void registerBrokerBootHook(List<BrokerBootHook> brokerBootHookList) {
+        this.brokerBootHookList = brokerBootHookList;
+    }
+
+    private RemotingCommand updateBrokerConfig(ChannelHandlerContext ctx, RemotingCommand request) {
+        final RemotingCommand response = RemotingCommand.createResponseCommand(null);
+
+        LOGGER.info("updateSharedBrokerConfig called by {}", RemotingHelper.parseChannelRemoteAddr(ctx.channel()));
+
+        byte[] body = request.getBody();
+        if (body != null) {
+            try {
+                String bodyStr = new String(body, MixAll.DEFAULT_CHARSET);
+                Properties properties = MixAll.string2Properties(bodyStr);
+                if (properties != null) {
+                    LOGGER.info("updateSharedBrokerConfig, new config: [{}] client: {} ", properties, ctx.channel().remoteAddress());
+                    this.brokerContainer.getConfiguration().update(properties);
+                } else {
+                    LOGGER.error("string2Properties error");
+                    response.setCode(ResponseCode.SYSTEM_ERROR);
+                    response.setRemark("string2Properties error");
+                    return response;
+                }
+            } catch (UnsupportedEncodingException e) {
+                LOGGER.error("", e);
+                response.setCode(ResponseCode.SYSTEM_ERROR);
+                response.setRemark("UnsupportedEncodingException " + e);
+                return response;
+            }
+        }
+
+        response.setCode(ResponseCode.SUCCESS);
+        response.setRemark(null);
+        return response;
+    }
+
+    private RemotingCommand getBrokerConfig(ChannelHandlerContext ctx, RemotingCommand request) {
+
+        final RemotingCommand response = RemotingCommand.createResponseCommand(GetBrokerConfigResponseHeader.class);
+        final GetBrokerConfigResponseHeader responseHeader = (GetBrokerConfigResponseHeader) response.readCustomHeader();
+
+        String content = this.brokerContainer.getConfiguration().getAllConfigsFormatString();
+        if (content != null && content.length() > 0) {
+            try {
+                response.setBody(content.getBytes(MixAll.DEFAULT_CHARSET));
+            } catch (UnsupportedEncodingException e) {
+                LOGGER.error("", e);
+
+                response.setCode(ResponseCode.SYSTEM_ERROR);
+                response.setRemark("UnsupportedEncodingException " + e);
+                return response;
+            }
+        }
+
+        responseHeader.setVersion(this.brokerContainer.getConfiguration().getDataVersionJson());
+
+        response.setCode(ResponseCode.SUCCESS);
+        response.setRemark(null);
+        return response;
+    }
+}
diff --git a/container/src/main/java/org/apache/rocketmq/container/BrokerContainerStartup.java b/container/src/main/java/org/apache/rocketmq/container/BrokerContainerStartup.java
new file mode 100644
index 0000000..f4f5d4a
--- /dev/null
+++ b/container/src/main/java/org/apache/rocketmq/container/BrokerContainerStartup.java
@@ -0,0 +1,482 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.rocketmq.container;
+
+import java.io.BufferedInputStream;
+import java.io.FileInputStream;
+import java.io.InputStream;
+import java.util.ArrayList;
+import java.util.List;
+import java.util.Properties;
+import java.util.concurrent.atomic.AtomicInteger;
+
+import ch.qos.logback.classic.LoggerContext;
+import ch.qos.logback.classic.joran.JoranConfigurator;
+import org.apache.commons.cli.CommandLine;
+import org.apache.commons.cli.DefaultParser;
+import org.apache.commons.cli.Option;
+import org.apache.commons.cli.Options;
+import org.apache.rocketmq.broker.BrokerController;
+import org.apache.rocketmq.broker.BrokerPathConfigHelper;
+import org.apache.rocketmq.common.BrokerConfig;
+import org.apache.rocketmq.common.MQVersion;
+import org.apache.rocketmq.common.MixAll;
+import org.apache.rocketmq.common.constant.LoggerName;
+import org.apache.rocketmq.logging.InternalLogger;
+import org.apache.rocketmq.logging.InternalLoggerFactory;
+import org.apache.rocketmq.remoting.common.RemotingUtil;
+import org.apache.rocketmq.remoting.common.TlsMode;
+import org.apache.rocketmq.remoting.netty.NettyClientConfig;
+import org.apache.rocketmq.remoting.netty.NettyServerConfig;
+import org.apache.rocketmq.remoting.netty.NettySystemConfig;
+import org.apache.rocketmq.remoting.netty.TlsSystemConfig;
+import org.apache.rocketmq.remoting.protocol.RemotingCommand;
+import org.apache.rocketmq.srvutil.ServerUtil;
+import org.apache.rocketmq.store.config.MessageStoreConfig;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+public class BrokerContainerStartup {
+    private static final char BROKER_CONTAINER_CONFIG_OPTION = 'c';
+    private static final char BROKER_CONFIG_OPTION = 'b';
+    private static final char PRINT_PROPERTIES_OPTION = 'p';
+    private static final String PRINT_IMPORTANT_PROPERTIES_OPTION = "pm";
+    public static Properties properties = null;
+    public static CommandLine commandLine = null;
+    public static String configFile = null;
+    public static InternalLogger log;
+    public static SystemConfigFileHelper configFileHelper = new SystemConfigFileHelper();
+    public static String rocketmqHome = null;
+    public static JoranConfigurator configurator = new JoranConfigurator();
+
+    public static void main(String[] args) {
+        final BrokerContainer brokerContainer = startBrokerContainer(createBrokerContainer(args));
+        createAndStartBrokers(brokerContainer);
+    }
+
+    /**
+     * Start brokerController, for old version compatibility.
+     *
+     * @param controller brokerController
+     * @return brokerController
+     */
+    public static BrokerController start(BrokerController controller) {
+        try {
+
+            controller.start();
+
+            String tip = "The broker[" + controller.getBrokerConfig().getBrokerName() + ", "
+                + controller.getBrokerAddr() + "] boot success. serializeType=" + RemotingCommand.getSerializeTypeConfigInThisServer();
+
+            if (null != controller.getBrokerConfig().getNamesrvAddr()) {
+                tip += " and name server is " + controller.getBrokerConfig().getNamesrvAddr();
+            }
+
+            log.info(tip);
+            System.out.printf("%s%n", tip);
+            return controller;
+        } catch (Throwable e) {
+            e.printStackTrace();
+            System.exit(-1);
+        }
+
+        return null;
+    }
+
+    public static BrokerController createBrokerController(String[] args) {
+        final BrokerContainer sharedBrokerController = startBrokerContainer(createBrokerContainer(args, true));
+        return createAndInitializeBroker(sharedBrokerController, configFile, properties);
+    }
+
+    public static List<BrokerController> createAndStartBrokers(BrokerContainer brokerContainer) {
+        String[] configPaths = parseBrokerConfigPath();
+        List<BrokerController> brokerControllerList = new ArrayList<>();
+
+        if (configPaths != null && configPaths.length > 0) {
+            SystemConfigFileHelper configFileHelper = new SystemConfigFileHelper();
+            for (String configPath : configPaths) {
+                System.out.printf("Start broker from config file path %s%n", configPath);
+                configFileHelper.setFile(configPath);
+
+                Properties brokerProperties = null;
+                try {
+                    brokerProperties = configFileHelper.loadConfig();
+                } catch (Exception e) {
+                    e.printStackTrace();
+                    System.exit(-1);
+                }
+
+                final BrokerController brokerController = createAndInitializeBroker(brokerContainer, configPath, brokerProperties);
+                if (brokerController != null) {
+                    brokerControllerList.add(brokerController);
+                    startBrokerController(brokerContainer, brokerController, brokerProperties);
+                }
+            }
+        }
+
+        return brokerControllerList;
+    }
+
+    public static String[] parseBrokerConfigPath() {
+        String brokerConfigList = null;
+        if (commandLine.hasOption(BROKER_CONFIG_OPTION)) {
+            brokerConfigList = commandLine.getOptionValue(BROKER_CONFIG_OPTION);
+
+        } else if (commandLine.hasOption(BROKER_CONTAINER_CONFIG_OPTION)) {
+            String brokerContainerConfigPath = commandLine.getOptionValue(BROKER_CONTAINER_CONFIG_OPTION);
+            if (brokerContainerConfigPath != null) {
+                BrokerContainerConfig brokerContainerConfig = new BrokerContainerConfig();
+                SystemConfigFileHelper configFileHelper = new SystemConfigFileHelper();
+                configFileHelper.setFile(brokerContainerConfigPath);
+                Properties brokerContainerProperties = null;
+                try {
+                    brokerContainerProperties = configFileHelper.loadConfig();
+                } catch (Exception e) {
+                    e.printStackTrace();
+                    System.exit(-1);
+                }
+                if (brokerContainerProperties != null) {
+                    MixAll.properties2Object(brokerContainerProperties, brokerContainerConfig);
+                }
+                brokerConfigList = brokerContainerConfig.getBrokerConfigPaths();
+            }
+        }
+
+        if (brokerConfigList != null) {
+            return brokerConfigList.split(":");
+        }
+        return null;
+    }
+
+    public static BrokerController createAndInitializeBroker(BrokerContainer brokerContainer,
+        String filePath, Properties brokerProperties) {
+
+        final BrokerConfig brokerConfig = new BrokerConfig();
+        final MessageStoreConfig messageStoreConfig = new MessageStoreConfig();
+
+        if (brokerProperties != null) {
+            properties2SystemEnv(brokerProperties);
+            MixAll.properties2Object(brokerProperties, brokerConfig);
+            MixAll.properties2Object(brokerProperties, messageStoreConfig);
+        }
+
+        messageStoreConfig.setHaListenPort(brokerConfig.getListenPort() + 1);
+
+        MixAll.properties2Object(ServerUtil.commandLine2Properties(commandLine), brokerConfig);
+
+        switch (messageStoreConfig.getBrokerRole()) {
+            case ASYNC_MASTER:
+            case SYNC_MASTER:
+                brokerConfig.setBrokerId(MixAll.MASTER_ID);
+                break;
+            case SLAVE:
+                if (brokerConfig.getBrokerId() <= 0) {
+                    System.out.printf("Slave's brokerId must be > 0%n");
+                    System.exit(-3);
+                }
+
+                break;
+            default:
+                break;
+        }
+
+        if (messageStoreConfig.getTotalReplicas() < messageStoreConfig.getInSyncReplicas()
+            || messageStoreConfig.getTotalReplicas() < messageStoreConfig.getMinInSyncReplicas()
+            || messageStoreConfig.getInSyncReplicas() < messageStoreConfig.getMinInSyncReplicas()) {
+            System.out.printf("invalid replicas number%n");
+            System.exit(-3);
+        }
+
+        brokerConfig.setBrokerConfigPath(filePath);
+
+        log = InternalLoggerFactory.getLogger(brokerConfig.getLoggerIdentifier() + LoggerName.BROKER_LOGGER_NAME);
+        MixAll.printObjectProperties(log, brokerConfig);
+        MixAll.printObjectProperties(log, messageStoreConfig);
+
+        try {
+            BrokerController brokerController = brokerContainer.addBroker(brokerConfig, messageStoreConfig);
+            if (brokerController != null) {
+                brokerController.getConfiguration().registerConfig(brokerProperties);
+                return brokerController;
+            } else {
+                System.out.printf("Add broker [%s-%s] failed.%n", brokerConfig.getBrokerName(), brokerConfig.getBrokerId());
+            }
+        } catch (Exception e) {
+            e.printStackTrace();
+            System.exit(-1);
+        }
+        return null;
+    }
+
+    public static BrokerContainer startBrokerContainer(BrokerContainer brokerContainer) {
+        try {
+
+            brokerContainer.start();
+
+            String tip = "The broker container boot success. serializeType=" + RemotingCommand.getSerializeTypeConfigInThisServer();
+
+            if (null != brokerContainer.getBrokerContainerConfig().getNamesrvAddr()) {
+                tip += " and name server is " + brokerContainer.getBrokerContainerConfig().getNamesrvAddr();
+            }
+
+            log.info(tip);
+            System.out.printf("%s%n", tip);
+            return brokerContainer;
+        } catch (Throwable e) {
+            e.printStackTrace();
+            System.exit(-1);
+        }
+
+        return null;
+    }
+
+    public static void startBrokerController(BrokerContainer brokerContainer,
+        BrokerController brokerController, Properties brokerProperties) {
+        try {
+            for (BrokerBootHook hook : brokerContainer.getBrokerBootHookList()) {
+                hook.executeBeforeStart(brokerController, brokerProperties);
+            }
+
+            brokerController.start();
+
+            for (BrokerBootHook hook : brokerContainer.getBrokerBootHookList()) {
+                hook.executeAfterStart(brokerController, brokerProperties);
+            }
+
+            String tip = String.format("Broker [%s-%s] boot success. serializeType=%s",
+                brokerController.getBrokerConfig().getBrokerName(),
+                brokerController.getBrokerConfig().getBrokerId(),
+                RemotingCommand.getSerializeTypeConfigInThisServer());
+
+            log.info(tip);
+            System.out.printf("%s%n", tip);
+        } catch (Throwable e) {
+            e.printStackTrace();
+            System.exit(-1);
+        }
+    }
+
+    public static void shutdown(final BrokerContainer controller) {
+        if (null != controller) {
+            controller.shutdown();
+        }
+    }
+
+    public static BrokerContainer createBrokerContainer(String[] args) {
+        return createBrokerContainer(args, false);
+    }
+
+    public static BrokerContainer createBrokerContainer(String[] args, boolean useDefaultPort) {
+        System.setProperty(RemotingCommand.REMOTING_VERSION_KEY, Integer.toString(MQVersion.CURRENT_VERSION));
+
+        if (null == System.getProperty(NettySystemConfig.COM_ROCKETMQ_REMOTING_SOCKET_SNDBUF_SIZE)) {
+            NettySystemConfig.socketSndbufSize = 131072;
+        }
+
+        if (null == System.getProperty(NettySystemConfig.COM_ROCKETMQ_REMOTING_SOCKET_RCVBUF_SIZE)) {
+            NettySystemConfig.socketRcvbufSize = 131072;
+        }
+
+        try {
+            //PackageConflictDetect.detectFastjson();
+            Options options = ServerUtil.buildCommandlineOptions(new Options());
+            commandLine = ServerUtil.parseCmdLine("mqbroker", args, buildCommandlineOptions(options),
+                new DefaultParser());
+            if (null == commandLine) {
+                System.exit(-1);
+            }
+
+            final BrokerContainerConfig brokerConfig = new BrokerContainerConfig();
+            final NettyServerConfig nettyServerConfig = new NettyServerConfig();
+            final NettyClientConfig nettyClientConfig = new NettyClientConfig();
+
+            nettyClientConfig.setUseTLS(Boolean.parseBoolean(System.getProperty(TlsSystemConfig.TLS_ENABLE,
+                String.valueOf(TlsSystemConfig.tlsMode == TlsMode.ENFORCING))));
+            nettyServerConfig.setListenPort(10811);
+
+            if (commandLine.hasOption(BROKER_CONTAINER_CONFIG_OPTION)) {
+                String file = commandLine.getOptionValue(BROKER_CONTAINER_CONFIG_OPTION);
+                if (file != null) {
+                    configFileHelper.setFile(file);
+                    configFile = file;
+                    BrokerPathConfigHelper.setBrokerConfigPath(file);
+                }
+            }
+
+            properties = configFileHelper.loadConfig();
+            if (properties != null) {
+                properties2SystemEnv(properties);
+                MixAll.properties2Object(properties, brokerConfig);
+                MixAll.properties2Object(properties, nettyServerConfig);
+                MixAll.properties2Object(properties, nettyClientConfig);
+            }
+
+            if (useDefaultPort) {
+                nettyServerConfig.setListenPort(10811);
+            }
+
+            MixAll.properties2Object(ServerUtil.commandLine2Properties(commandLine), brokerConfig);
+
+            if (null == brokerConfig.getRocketmqHome()) {
+                System.out.printf("Please set the %s variable in your environment to match the location of the RocketMQ installation", MixAll.ROCKETMQ_HOME_ENV);
+                System.exit(-2);
+            }
+            rocketmqHome = brokerConfig.getRocketmqHome();
+
+            String namesrvAddr = brokerConfig.getNamesrvAddr();
+            if (null != namesrvAddr) {
+                try {
+                    String[] addrArray = namesrvAddr.split(";");
+                    for (String addr : addrArray) {
+                        RemotingUtil.string2SocketAddress(addr);
+                    }
+                } catch (Exception e) {
+                    System.out.printf(
+                        "The Name Server Address[%s] illegal, please set it as follows, \"127.0.0.1:9876;192.168.0.1:9876\"%n",
+                        namesrvAddr);
+                    System.exit(-3);
+                }
+            }
+
+            LoggerContext lc = (LoggerContext) LoggerFactory.getILoggerFactory();
+            configurator.setContext(lc);
+            lc.reset();
+            //https://logback.qos.ch/manual/configuration.html
+            lc.setPackagingDataEnabled(false);
+
+            configurator.doConfigure(rocketmqHome + "/conf/logback_broker.xml");
+
+            if (commandLine.hasOption(PRINT_PROPERTIES_OPTION)) {
+                InternalLogger console = InternalLoggerFactory.getLogger(LoggerName.BROKER_CONSOLE_NAME);
+                MixAll.printObjectProperties(console, brokerConfig);
+                MixAll.printObjectProperties(console, nettyServerConfig);
+                MixAll.printObjectProperties(console, nettyClientConfig);
+                System.exit(0);
+            } else if (commandLine.hasOption(PRINT_IMPORTANT_PROPERTIES_OPTION)) {
+                InternalLogger console = InternalLoggerFactory.getLogger(LoggerName.BROKER_CONSOLE_NAME);
+                MixAll.printObjectProperties(console, brokerConfig, true);
+                MixAll.printObjectProperties(console, nettyServerConfig, true);
+                MixAll.printObjectProperties(console, nettyClientConfig, true);
+                System.exit(0);
+            }
+
+            log = InternalLoggerFactory.getLogger(LoggerName.BROKER_LOGGER_NAME);
+            MixAll.printObjectProperties(log, brokerConfig);
+            MixAll.printObjectProperties(log, nettyServerConfig);
+            MixAll.printObjectProperties(log, nettyClientConfig);
+
+            final BrokerContainer brokerContainer = new BrokerContainer(
+                brokerConfig,
+                nettyServerConfig,
+                nettyClientConfig);
+            // remember all configs to prevent discard
+            brokerContainer.getConfiguration().registerConfig(properties);
+
+            boolean initResult = brokerContainer.initialize();
+            if (!initResult) {
+                brokerContainer.shutdown();
+                System.exit(-3);
+            }
+
+            Runtime.getRuntime().addShutdownHook(new Thread(new Runnable() {
+                private volatile boolean hasShutdown = false;
+                private AtomicInteger shutdownTimes = new AtomicInteger(0);
+
+                @Override
+                public void run() {
+                    synchronized (this) {
+                        log.info("Shutdown hook was invoked, {}", this.shutdownTimes.incrementAndGet());
+                        if (!this.hasShutdown) {
+                            this.hasShutdown = true;
+                            long beginTime = System.currentTimeMillis();
+                            brokerContainer.shutdown();
+                            long consumingTimeTotal = System.currentTimeMillis() - beginTime;
+                            log.info("Shutdown hook over, consuming total time(ms): {}", consumingTimeTotal);
+                        }
+                    }
+                }
+            }, "ShutdownHook"));
+
+            return brokerContainer;
+        } catch (Throwable e) {
+            e.printStackTrace();
+            System.exit(-1);
+        }
+
+        return null;
+    }
+
+    private static void properties2SystemEnv(Properties properties) {
+        if (properties == null) {
+            return;
+        }
+        String rmqAddressServerDomain = properties.getProperty("rmqAddressServerDomain", MixAll.WS_DOMAIN_NAME);
+        String rmqAddressServerSubGroup = properties.getProperty("rmqAddressServerSubGroup", MixAll.WS_DOMAIN_SUBGROUP);
+        System.setProperty("rocketmq.namesrv.domain", rmqAddressServerDomain);
+        System.setProperty("rocketmq.namesrv.domain.subgroup", rmqAddressServerSubGroup);
+    }
+
+    private static Options buildCommandlineOptions(final Options options) {
+        Option opt = new Option("c", "configFile", true, "Config file for shared broker");
+        opt.setRequired(false);
+        options.addOption(opt);
+
+        opt = new Option("p", "printConfigItem", false, "Print all config item");
+        opt.setRequired(false);
+        options.addOption(opt);
+
+        opt = new Option("m", "printImportantConfig", false, "Print important config item");
+        opt.setRequired(false);
+        options.addOption(opt);
+
+        opt = new Option("b", "brokerConfigFiles", true, "The path of broker config files, split by ':'");
+        opt.setRequired(false);
+        options.addOption(opt);
+
+        return options;
+    }
+
+    public static class SystemConfigFileHelper {
+        private static final Logger LOGGER = LoggerFactory.getLogger(SystemConfigFileHelper.class);
+
+        private String file;
+
+        public SystemConfigFileHelper() {
+        }
+
+        public Properties loadConfig() throws Exception {
+            InputStream in = new BufferedInputStream(new FileInputStream(file));
+            Properties properties = new Properties();
+            properties.load(in);
+            in.close();
+            return properties;
+        }
+
+        public void update(Properties properties) throws Exception {
+            LOGGER.error("[SystemConfigFileHelper] update no thing.");
+        }
+
+        public void setFile(String file) {
+            this.file = file;
+        }
+
+        public String getFile() {
+            return file;
+        }
+    }
+
+}
diff --git a/container/src/main/java/org/apache/rocketmq/container/BrokerPreOnlineService.java b/container/src/main/java/org/apache/rocketmq/container/BrokerPreOnlineService.java
new file mode 100644
index 0000000..40a0187
--- /dev/null
+++ b/container/src/main/java/org/apache/rocketmq/container/BrokerPreOnlineService.java
@@ -0,0 +1,277 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.rocketmq.container;
+
+import java.io.IOException;
+import java.util.ArrayList;
+import java.util.Collections;
+import java.util.HashMap;
+import java.util.List;
+import java.util.Map;
+import java.util.concurrent.CompletableFuture;
+
+import org.apache.rocketmq.broker.plugin.BrokerAttachedPlugin;
+import org.apache.rocketmq.common.BrokerSyncInfo;
+import org.apache.rocketmq.common.MixAll;
+import org.apache.rocketmq.common.ServiceThread;
+import org.apache.rocketmq.common.constant.LoggerName;
+import org.apache.rocketmq.common.protocol.body.BrokerMemberGroup;
+import org.apache.rocketmq.common.protocol.body.ConsumerOffsetSerializeWrapper;
+import org.apache.rocketmq.logging.InternalLogger;
+import org.apache.rocketmq.logging.InternalLoggerFactory;
+import org.apache.rocketmq.remoting.common.RemotingHelper;
+import org.apache.rocketmq.store.config.StorePathConfigHelper;
+import org.apache.rocketmq.broker.schedule.DelayOffsetSerializeWrapper;
+import org.apache.rocketmq.store.ha.HAConnectionState;
+import org.apache.rocketmq.store.ha.HAConnectionStateNotificationRequest;
+
+public class BrokerPreOnlineService extends ServiceThread {
+    private static final InternalLogger LOGGER = InternalLoggerFactory.getLogger(LoggerName.BROKER_LOGGER_NAME);
+    private final InnerBrokerController brokerController;
+
+    private int waitBrokerIndex = 0;
+
+    public BrokerPreOnlineService(InnerBrokerController brokerController) {
+        this.brokerController = brokerController;
+    }
+
+    @Override
+    public String getServiceName() {
+        if (this.brokerController != null && this.brokerController.getBrokerConfig().isInBrokerContainer()) {
+            return brokerController.getBrokerConfig().getLoggerIdentifier() + BrokerPreOnlineService.class.getSimpleName();
+        }
+        return BrokerPreOnlineService.class.getSimpleName();
+    }
+
+    @Override
+    public void run() {
+        LOGGER.info(this.getServiceName() + " service started");
+
+        while (!this.isStopped()) {
+            if (!this.brokerController.isIsolated()) {
+                LOGGER.info("broker {} is online", this.brokerController.getBrokerConfig().getCanonicalName());
+                break;
+            }
+            try {
+                boolean isSuccess = this.prepareForBrokerOnline();
+                if (!isSuccess) {
+                    this.waitForRunning(1000);
+                } else {
+                    break;
+                }
+            } catch (Exception e) {
+                LOGGER.error("Broker preOnline error, ", e);
+            }
+        }
+
+        LOGGER.info(this.getServiceName() + " service end");
+    }
+
+    CompletableFuture<Boolean> waitForHaHandshakeComplete(String brokerAddr) {
+        LOGGER.info("wait for handshake completion with {}", brokerAddr);
+        HAConnectionStateNotificationRequest request =
+            new HAConnectionStateNotificationRequest(HAConnectionState.TRANSFER, RemotingHelper.parseHostFromAddress(brokerAddr), true);
+        if (this.brokerController.getMessageStore().getHaService() != null) {
+            this.brokerController.getMessageStore().getHaService().putGroupConnectionStateRequest(request);
+        } else {
+            LOGGER.error("HAService is null, maybe broker config is wrong. For example, duplicationEnable is true");
+            request.getRequestFuture().complete(false);
+        }
+        return request.getRequestFuture();
+    }
+
+    private boolean futureWaitAction(boolean result, BrokerMemberGroup brokerMemberGroup) {
+        if (!result) {
+            LOGGER.error("wait for handshake completion failed, HA connection lost");
+            return false;
+        }
+        if (this.brokerController.getBrokerConfig().getBrokerId() != MixAll.MASTER_ID) {
+            LOGGER.info("slave preOnline complete, start service");
+            long minBrokerId = getMinBrokerId(brokerMemberGroup.getBrokerAddrs());
+            this.brokerController.startService(minBrokerId, brokerMemberGroup.getBrokerAddrs().get(minBrokerId));
+        }
+        return true;
+    }
+
+    private boolean prepareForMasterOnline(BrokerMemberGroup brokerMemberGroup) {
+        List<Long> brokerIdList = new ArrayList<>(brokerMemberGroup.getBrokerAddrs().keySet());
+        Collections.sort(brokerIdList);
+        while (true) {
+            if (waitBrokerIndex >= brokerIdList.size()) {
+                LOGGER.info("master preOnline complete, start service");
+                this.brokerController.startService(MixAll.MASTER_ID, this.brokerController.getBrokerAddr());
+                return true;
+            }
+
+            String brokerAddrToWait = brokerMemberGroup.getBrokerAddrs().get(brokerIdList.get(waitBrokerIndex));
+
+            try {
+                this.brokerController.getBrokerOuterAPI().
+                    sendBrokerHaInfo(brokerAddrToWait, this.brokerController.getHAServerAddr(),
+                        this.brokerController.getMessageStore().getBrokerInitMaxOffset(), this.brokerController.getBrokerAddr());
+            } catch (Exception e) {
+                LOGGER.error("send ha address to {} exception, {}", brokerAddrToWait, e);
+                return false;
+            }
+
+            CompletableFuture<Boolean> haHandshakeFuture = waitForHaHandshakeComplete(brokerAddrToWait)
+                .thenApply(result -> futureWaitAction(result, brokerMemberGroup));
+
+            try {
+                if (!haHandshakeFuture.get()) {
+                    return false;
+                }
+            } catch (Exception e) {
+                LOGGER.error("Wait handshake completion exception, {}", e);
+                return false;
+            }
+
+            if (syncMetadataReverse(brokerAddrToWait)) {
+                waitBrokerIndex++;
+            } else {
+                return false;
+            }
+        }
+    }
+
+    private boolean syncMetadataReverse(String brokerAddr) {
+        try {
+            LOGGER.info("Get metadata reverse from {}", brokerAddr);
+
+            String delayOffset = this.brokerController.getBrokerOuterAPI().getAllDelayOffset(brokerAddr);
+            DelayOffsetSerializeWrapper delayOffsetSerializeWrapper =
+                DelayOffsetSerializeWrapper.fromJson(delayOffset, DelayOffsetSerializeWrapper.class);
+
+            ConsumerOffsetSerializeWrapper consumerOffsetSerializeWrapper = this.brokerController.getBrokerOuterAPI().getAllConsumerOffset(brokerAddr);
+
+            if (null != consumerOffsetSerializeWrapper && brokerController.getConsumerOffsetManager().getDataVersion().compare(consumerOffsetSerializeWrapper.getDataVersion()) <= 0) {
+                LOGGER.info("{}'s consumerOffset data version is larger than master broker, {}'s consumerOffset will be used.", brokerAddr, brokerAddr);
+                this.brokerController.getConsumerOffsetManager().getOffsetTable()
+                    .putAll(consumerOffsetSerializeWrapper.getOffsetTable());
+                this.brokerController.getConsumerOffsetManager().getDataVersion().assignNewOne(consumerOffsetSerializeWrapper.getDataVersion());
+                this.brokerController.getConsumerOffsetManager().persist();
+            }
+
+            if (null != delayOffset && brokerController.getScheduleMessageService().getDataVersion().compare(delayOffsetSerializeWrapper.getDataVersion()) <= 0) {
+                LOGGER.info("{}'s scheduleMessageService data version is larger than master broker, {}'s delayOffset will be used.", brokerAddr, brokerAddr);
+                String fileName =
+                    StorePathConfigHelper.getDelayOffsetStorePath(this.brokerController
+                        .getMessageStoreConfig().getStorePathRootDir());
+                try {
+                    MixAll.string2File(delayOffset, fileName);
+                    this.brokerController.getScheduleMessageService().load();
+                } catch (IOException e) {
+                    LOGGER.error("Persist file Exception, {}", fileName, e);
+                }
+            }
+
+            for (BrokerAttachedPlugin brokerAttachedPlugin : brokerController.getBrokerAttachedPlugins()) {
+                if (brokerAttachedPlugin != null) {
+                    brokerAttachedPlugin.syncMetadataReverse(brokerAddr);
+                }
+            }
+
+        } catch (Exception e) {
+            LOGGER.error("GetMetadataReverse Failed", e);
+            return false;
+        }
+
+        return true;
+    }
+
+    private boolean prepareForSlaveOnline(BrokerMemberGroup brokerMemberGroup) {
+        BrokerSyncInfo brokerSyncInfo;
+        try {
+            brokerSyncInfo = this.brokerController.getBrokerOuterAPI()
+                .retrieveBrokerHaInfo(brokerMemberGroup.getBrokerAddrs().get(MixAll.MASTER_ID));
+        } catch (Exception e) {
+            LOGGER.error("retrieve master ha info exception, {}", e);
+            return false;
+        }
+
+        if (this.brokerController.getMessageStore().getMasterFlushedOffset() == 0
+            && this.brokerController.getMessageStoreConfig().isSyncMasterFlushOffsetWhenStartup()) {
+            LOGGER.info("Set master flush offset in slave to {}", brokerSyncInfo.getMasterFlushOffset());
+            this.brokerController.getMessageStore().setMasterFlushedOffset(brokerSyncInfo.getMasterFlushOffset());
+        }
+
+        if (brokerSyncInfo.getMasterHaAddress() != null) {
+            this.brokerController.getMessageStore().updateHaMasterAddress(brokerSyncInfo.getMasterHaAddress());
+            this.brokerController.getMessageStore().updateMasterAddress(brokerSyncInfo.getMasterAddress());
+        } else {
+            LOGGER.info("fetch master ha address return null, start service directly");
+            long minBrokerId = getMinBrokerId(brokerMemberGroup.getBrokerAddrs());
+            this.brokerController.startService(minBrokerId, brokerMemberGroup.getBrokerAddrs().get(minBrokerId));
+            return true;
+        }
+
+        CompletableFuture<Boolean> haHandshakeFuture = waitForHaHandshakeComplete(brokerSyncInfo.getMasterHaAddress())
+            .thenApply(result -> futureWaitAction(result, brokerMemberGroup));
+
+        try {
+            if (!haHandshakeFuture.get()) {
+                return false;
+            }
+        } catch (Exception e) {
+            LOGGER.error("Wait handshake completion exception, {}", e);
+            return false;
+        }
+
+        return true;
+    }
+
+    private boolean prepareForBrokerOnline() {
+        BrokerMemberGroup brokerMemberGroup;
+        try {
+            brokerMemberGroup = this.brokerController.getBrokerOuterAPI().syncBrokerMemberGroup(
+                this.brokerController.getBrokerConfig().getBrokerClusterName(),
+                this.brokerController.getBrokerConfig().getBrokerName(),
+                this.brokerController.getBrokerContainer().getBrokerContainerConfig().isCompatibleWithOldNameSrv());
+        } catch (Exception e) {
+            LOGGER.error("syncBrokerMemberGroup from namesrv error, start service failed, will try later, ", e);
+            return false;
+        }
+
+        if (brokerMemberGroup != null && !brokerMemberGroup.getBrokerAddrs().isEmpty()) {
+            long minBrokerId = getMinBrokerId(brokerMemberGroup.getBrokerAddrs());
+
+            if (this.brokerController.getBrokerConfig().getBrokerId() == MixAll.MASTER_ID) {
+                return prepareForMasterOnline(brokerMemberGroup);
+            } else if (minBrokerId == MixAll.MASTER_ID) {
+                return prepareForSlaveOnline(brokerMemberGroup);
+            } else {
+                LOGGER.info("no master online, start service directly");
+                this.brokerController.startService(minBrokerId, brokerMemberGroup.getBrokerAddrs().get(minBrokerId));
+            }
+        } else {
+            LOGGER.info("no other broker online, will start service directly");
+            this.brokerController.startService(this.brokerController.getBrokerConfig().getBrokerId(), this.brokerController.getBrokerAddr());
+        }
+
+        return true;
+    }
+
+    private long getMinBrokerId(Map<Long, String> brokerAddrMap) {
+        Map<Long, String> brokerAddrMapCopy = new HashMap<>(brokerAddrMap);
+        brokerAddrMapCopy.remove(this.brokerController.getBrokerConfig().getBrokerId());
+        if (!brokerAddrMapCopy.isEmpty()) {
+            return Collections.min(brokerAddrMapCopy.keySet());
+        }
+        return this.brokerController.getBrokerConfig().getBrokerId();
+    }
+}
diff --git a/container/src/main/java/org/apache/rocketmq/container/ContainerClientHouseKeepingService.java b/container/src/main/java/org/apache/rocketmq/container/ContainerClientHouseKeepingService.java
new file mode 100644
index 0000000..dc9f463
--- /dev/null
+++ b/container/src/main/java/org/apache/rocketmq/container/ContainerClientHouseKeepingService.java
@@ -0,0 +1,104 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.rocketmq.container;
+
+import io.netty.channel.Channel;
+import java.util.Collection;
+import org.apache.rocketmq.broker.BrokerController;
+import org.apache.rocketmq.remoting.ChannelEventListener;
+
+public class ContainerClientHouseKeepingService implements ChannelEventListener {
+    private final IBrokerContainer brokerContainer;
+
+    public ContainerClientHouseKeepingService(final IBrokerContainer brokerContainer) {
+        this.brokerContainer = brokerContainer;
+    }
+
+    @Override public void onChannelConnect(String remoteAddr, Channel channel) {
+        onChannelOperation(CallbackCode.CONNECT, remoteAddr, channel);
+    }
+
+    @Override public void onChannelClose(String remoteAddr, Channel channel) {
+        onChannelOperation(CallbackCode.CLOSE, remoteAddr, channel);
+    }
+
+    @Override public void onChannelException(String remoteAddr, Channel channel) {
+        onChannelOperation(CallbackCode.EXCEPTION, remoteAddr, channel);
+    }
+
+    @Override public void onChannelIdle(String remoteAddr, Channel channel) {
+        onChannelOperation(CallbackCode.IDLE, remoteAddr, channel);
+    }
+
+    private void onChannelOperation(CallbackCode callbackCode, String remoteAddr, Channel channel) {
+        Collection<InnerBrokerController> masterBrokers = this.brokerContainer.getMasterBrokers();
+        Collection<InnerSalveBrokerController> slaveBrokers = this.brokerContainer.getSlaveBrokers();
+
+        for (BrokerController masterBroker : masterBrokers) {
+            brokerOperation(masterBroker, callbackCode, remoteAddr, channel);
+        }
+
+        for (InnerSalveBrokerController slaveBroker : slaveBrokers) {
+            brokerOperation(slaveBroker, callbackCode, remoteAddr, channel);
+        }
+    }
+
+    private void brokerOperation(BrokerController brokerController, CallbackCode callbackCode, String remoteAddr,
+        Channel channel) {
+        if (callbackCode == CallbackCode.CONNECT) {
+            brokerController.getBrokerStatsManager().incChannelConnectNum();
+            return;
+        }
+        boolean removed = brokerController.getProducerManager().doChannelCloseEvent(remoteAddr, channel);
+        removed &= brokerController.getConsumerManager().doChannelCloseEvent(remoteAddr, channel);
+        if (removed) {
+            switch (callbackCode) {
+                case CLOSE:
+                    brokerController.getBrokerStatsManager().incChannelCloseNum();
+                    break;
+                case EXCEPTION:
+                    brokerController.getBrokerStatsManager().incChannelExceptionNum();
+                    break;
+                case IDLE:
+                    brokerController.getBrokerStatsManager().incChannelIdleNum();
+                    break;
+                default:
+                    break;
+            }
+        }
+    }
+
+    public enum CallbackCode {
+        /**
+         * onChannelConnect
+         */
+        CONNECT,
+        /**
+         * onChannelClose
+         */
+        CLOSE,
+        /**
+         * onChannelException
+         */
+        EXCEPTION,
+        /**
+         * onChannelIdle
+         */
+        IDLE
+    }
+}
diff --git a/container/src/main/java/org/apache/rocketmq/container/IBrokerContainer.java b/container/src/main/java/org/apache/rocketmq/container/IBrokerContainer.java
new file mode 100644
index 0000000..d3cdc05
--- /dev/null
+++ b/container/src/main/java/org/apache/rocketmq/container/IBrokerContainer.java
@@ -0,0 +1,142 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.rocketmq.container;
+
+import java.util.Collection;
+import java.util.List;
+
+import org.apache.rocketmq.broker.BrokerController;
+import org.apache.rocketmq.broker.out.BrokerOuterAPI;
+import org.apache.rocketmq.common.BrokerConfig;
+import org.apache.rocketmq.common.BrokerIdentity;
+import org.apache.rocketmq.remoting.RemotingServer;
+import org.apache.rocketmq.remoting.netty.NettyClientConfig;
+import org.apache.rocketmq.remoting.netty.NettyServerConfig;
+import org.apache.rocketmq.store.config.MessageStoreConfig;
+
+/**
+ * An interface for broker container to hold multiple master and slave brokers.
+ */
+public interface IBrokerContainer {
+
+    /**
+     * Start broker container
+     */
+    void start() throws Exception;
+
+    /**
+     * Shutdown broker container and all the brokers inside.
+     */
+    void shutdown();
+
+    /**
+     * Add a broker to this container with specific broker config.
+     *
+     * @param brokerConfig the specified broker config
+     * @param storeConfig the specified store config
+     * @return the added BrokerController or null if the broker already exists
+     * @throws Exception when initialize broker
+     */
+    BrokerController addBroker(BrokerConfig brokerConfig, MessageStoreConfig storeConfig) throws Exception;
+
+    /**
+     * Remove the broker from this container associated with the specific broker identity
+     *
+     * @param brokerIdentity the specific broker identity
+     * @return the removed BrokerController or null if the broker doesn't exists
+     */
+    BrokerController removeBroker(BrokerIdentity brokerIdentity) throws Exception;
+
+    /**
+     * Return the broker controller associated with the specific broker identity
+     *
+     * @param brokerIdentity the specific broker identity
+     * @return the associated messaging broker or null
+     */
+    BrokerController getBroker(BrokerIdentity brokerIdentity);
+
+    /**
+     * Return all the master brokers belong to this container
+     *
+     * @return the master broker list
+     */
+    Collection<InnerBrokerController> getMasterBrokers();
+
+    /**
+     * Return all the slave brokers belong to this container
+     *
+     * @return the slave broker list
+     */
+    Collection<InnerSalveBrokerController> getSlaveBrokers();
+
+    /**
+     * Return all broker controller in this container
+     *
+     * @return all broker controller
+     */
+    List<BrokerController> getBrokerControllers();
+
+    /**
+     * Return the address of broker container.
+     *
+     * @return broker container address.
+     */
+    String getBrokerContainerAddr();
+
+    /**
+     * Peek the first master broker in container.
+     *
+     * @return the first master broker in container
+     */
+    BrokerController peekMasterBroker();
+
+    /**
+     * Return the config of the broker container
+     *
+     * @return the broker container config
+     */
+    BrokerContainerConfig getBrokerContainerConfig();
+
+    /**
+     * Get netty server config.
+     *
+     * @return netty server config
+     */
+    NettyServerConfig getNettyServerConfig();
+
+    /**
+     * Get netty client config.
+     *
+     * @return netty client config
+     */
+    NettyClientConfig getNettyClientConfig();
+
+    /**
+     * Return the shared BrokerOuterAPI
+     *
+     * @return the shared BrokerOuterAPI
+     */
+    BrokerOuterAPI getBrokerOuterAPI();
+
+    /**
+     * Return the shared RemotingServer
+     *
+     * @return the shared RemotingServer
+     */
+    RemotingServer getRemotingServer();
+}
diff --git a/container/src/main/java/org/apache/rocketmq/container/InnerBrokerController.java b/container/src/main/java/org/apache/rocketmq/container/InnerBrokerController.java
new file mode 100644
index 0000000..4603164
--- /dev/null
+++ b/container/src/main/java/org/apache/rocketmq/container/InnerBrokerController.java
@@ -0,0 +1,378 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.rocketmq.container;
+
+import java.util.ArrayList;
+import java.util.List;
+import java.util.Map;
+import java.util.concurrent.ScheduledExecutorService;
+import java.util.concurrent.ScheduledFuture;
+import java.util.concurrent.ScheduledThreadPoolExecutor;
+import java.util.concurrent.TimeUnit;
+
+import org.apache.rocketmq.broker.BrokerController;
+import org.apache.rocketmq.broker.out.BrokerOuterAPI;
+import org.apache.rocketmq.common.AbstractBrokerRunnable;
+import org.apache.rocketmq.common.BrokerConfig;
+import org.apache.rocketmq.common.MixAll;
+import org.apache.rocketmq.common.ThreadFactoryImpl;
+import org.apache.rocketmq.common.namesrv.RegisterBrokerResult;
+import org.apache.rocketmq.common.protocol.body.TopicConfigSerializeWrapper;
+import org.apache.rocketmq.remoting.netty.NettyClientConfig;
+import org.apache.rocketmq.remoting.netty.NettyServerConfig;
+import org.apache.rocketmq.store.MessageStore;
+import org.apache.rocketmq.store.config.MessageStoreConfig;
+
+public class InnerBrokerController extends BrokerController {
+    private ScheduledExecutorService syncBrokerMemberGroupExecutorService;
+    private ScheduledExecutorService brokerHeartbeatExecutorService;
+    protected volatile long minBrokerIdInGroup = 0;
+    protected volatile String minBrokerAddrInGroup = null;
+    protected final List<ScheduledFuture<?>> scheduledFutures = new ArrayList<>();
+    private BrokerPreOnlineService brokerPreOnlineService;
+    protected BrokerContainer brokerContainer;
+    protected volatile boolean isIsolated = false;
+
+    public InnerBrokerController(
+        final BrokerContainer brokerContainer,
+        final BrokerConfig brokerConfig,
+        final MessageStoreConfig messageStoreConfig
+    ) {
+        super(brokerConfig, messageStoreConfig);
+        this.brokerContainer = brokerContainer;
+        this.brokerOuterAPI = this.brokerContainer.getBrokerOuterAPI();
+
+        if (!this.brokerConfig.isSkipPreOnline()) {
+            this.brokerPreOnlineService = new BrokerPreOnlineService(this);
+        }
+    }
+
+    @Override
+    protected void initializeRemotingServer() {
+        this.remotingServer = this.brokerContainer.getRemotingServer().newRemotingServer(brokerConfig.getListenPort());
+        this.fastRemotingServer = this.brokerContainer.getRemotingServer().newRemotingServer(brokerConfig.getListenPort() - 2);
+    }
+
+    /**
+     * Initialize resources for master which will be re-used by slave.
+     */
+    @Override
+    protected void initializeResources() {
+        super.initializeResources();
+        this.syncBrokerMemberGroupExecutorService = new ScheduledThreadPoolExecutor(1,
+            new ThreadFactoryImpl("BrokerControllerSyncBrokerScheduledThread", brokerConfig));
+        this.brokerHeartbeatExecutorService = new ScheduledThreadPoolExecutor(1,
+            new ThreadFactoryImpl("rokerControllerHeartbeatScheduledThread", brokerConfig));
+    }
+
+    @Override
+    protected void initializeScheduledTasks() {
+        initializeBrokerScheduledTasks();
+    }
+
+    @Override
+    public void shutdown() {
+
+        shutdownBasicService();
+
+        if (this.remotingServer != null) {
+            this.brokerContainer.getRemotingServer().removeRemotingServer(brokerConfig.getListenPort());
+        }
+
+        if (this.fastRemotingServer != null) {
+            this.brokerContainer.getRemotingServer().removeRemotingServer(brokerConfig.getListenPort() - 2);
+        }
+
+        for (ScheduledFuture<?> scheduledFuture : scheduledFutures) {
+            scheduledFuture.cancel(true);
+        }
+
+        if (this.brokerPreOnlineService != null && !this.brokerPreOnlineService.isStopped()) {
+            this.brokerPreOnlineService.shutdown();
+        }
+
+        shutdownScheduledExecutorService(this.syncBrokerMemberGroupExecutorService);
+        shutdownScheduledExecutorService(this.brokerHeartbeatExecutorService);
+
+    }
+
+    @Override
+    public String getBrokerAddr() {
+        return this.brokerConfig.getBrokerIP1() + ":" + this.brokerConfig.getListenPort();
+    }
+
+    @Override
+    public void start() throws Exception {
+
+        this.shouldStartTime = System.currentTimeMillis() + messageStoreConfig.getDisappearTimeAfterStart();
+
+        if (messageStoreConfig.getTotalReplicas() > 1) {
+            isIsolated = true;
+        }
+
+        startBasicService();
+
+        if (this.brokerPreOnlineService != null) {
+            this.brokerPreOnlineService.start();
+        }
+
+        if (!isIsolated) {
+            this.registerBrokerAll(true, false, true);
+        }
+
+        scheduledFutures.add(this.scheduledExecutorService.scheduleAtFixedRate(new AbstractBrokerRunnable(this.brokerConfig) {
+            @Override
+            public void run2() {
+                try {
+                    if (System.currentTimeMillis() < shouldStartTime) {
+                        BrokerController.LOG.info("Register to namesrv after {}", shouldStartTime);
+                        return;
+                    }
+                    if (isIsolated) {
+                        BrokerController.LOG.info("Skip register for broker is isolated");
+                        return;
+                    }
+                    InnerBrokerController.this.registerBrokerAll(true, false, brokerConfig.isForceRegister());
+                } catch (Throwable e) {
+                    BrokerController.LOG.error("registerBrokerAll Exception", e);
+                }
+            }
+        }, 1000 * 10, Math.max(10000, Math.min(brokerConfig.getRegisterNameServerPeriod(), 60000)), TimeUnit.MILLISECONDS));
+
+        if (this.brokerConfig.isEnableSlaveActingMaster()) {
+            scheduledFutures.add(this.brokerHeartbeatExecutorService.scheduleAtFixedRate(new AbstractBrokerRunnable(this.brokerConfig) {
+                @Override
+                public void run2() {
+                    if (isIsolated) {
+                        return;
+                    }
+                    try {
+                        InnerBrokerController.this.sendHeartbeat();
+                    } catch (Exception e) {
+                        BrokerController.LOG.error("sendHeartbeat Exception", e);
+                    }
+
+                }
+            }, 1000, brokerConfig.getBrokerHeartbeatInterval(), TimeUnit.MILLISECONDS));
+
+            scheduledFutures.add(this.syncBrokerMemberGroupExecutorService.scheduleAtFixedRate(new AbstractBrokerRunnable(this.brokerConfig) {
+                @Override public void run2() {
+                    try {
+                        InnerBrokerController.this.syncBrokerMemberGroup();
+                    } catch (Throwable e) {
+                        BrokerController.LOG.error("sync BrokerMemberGroup error. ", e);
+                    }
+                }
+            }, 1000, this.brokerConfig.getSyncBrokerMemberGroupPeriod(), TimeUnit.MILLISECONDS));
+
+        }
+
+        if (!isIsolated && !messageStoreConfig.isEnableDLegerCommitLog()
+            && !messageStoreConfig.isDuplicationEnable() && !this.brokerConfig.isEnableSlaveActingMaster()) {
+            changeSpecialServiceStatus(this.brokerConfig.getBrokerId() == MixAll.MASTER_ID);
+        }
+
+        if (brokerConfig.isSkipPreOnline()) {
+            startServiceWithoutCondition();
+        }
+    }
+
+    private void sendHeartbeat() {
+        if (this.brokerContainer.getBrokerContainerConfig().isCompatibleWithOldNameSrv()) {
+            this.brokerOuterAPI.sendHeartbeatViaDataVersion(
+                this.brokerConfig.getBrokerClusterName(),
+                this.getBrokerAddr(),
+                this.brokerConfig.getBrokerName(),
+                this.brokerConfig.getBrokerId(),
+                this.brokerConfig.getSendHeartbeatTimeoutMillis(),
+                this.getTopicConfigManager().getDataVersion(),
+                this.brokerConfig.isInBrokerContainer());
+        } else {
+            this.brokerOuterAPI.sendHeartbeat(
+                this.brokerConfig.getBrokerClusterName(),
+                this.getBrokerAddr(),
+                this.brokerConfig.getBrokerName(),
+                this.brokerConfig.getBrokerId(),
+                this.brokerConfig.getSendHeartbeatTimeoutMillis(),
+                this.brokerConfig.isInBrokerContainer());
+        }
+    }
+
+    public void syncBrokerMemberGroup() {
+        try {
+            brokerMemberGroup = this.brokerContainer.getBrokerOuterAPI()
+                .syncBrokerMemberGroup(this.brokerConfig.getBrokerClusterName(), this.brokerConfig.getBrokerName(), this.brokerContainer.getBrokerContainerConfig().isCompatibleWithOldNameSrv());
+        } catch (Exception e) {
+            BrokerController.LOG.error("syncBrokerMemberGroup from namesrv failed, ", e);
+            return;
+        }
+        if (brokerMemberGroup == null || brokerMemberGroup.getBrokerAddrs().size() == 0) {
+            BrokerController.LOG.warn("Couldn't find any broker member from namesrv in {}/{}", this.brokerConfig.getBrokerClusterName(), this.brokerConfig.getBrokerName());
+            return;
+        }
+        this.messageStore.setAliveReplicaNumInGroup(calcAliveBrokerNumInGroup(brokerMemberGroup.getBrokerAddrs()));
+
+        if (!this.isIsolated) {
+            long minBrokerId = brokerMemberGroup.minimumBrokerId();
+            this.updateMinBroker(minBrokerId, brokerMemberGroup.getBrokerAddrs().get(minBrokerId));
+        }
+    }
+
+    private int calcAliveBrokerNumInGroup(Map<Long, String> brokerAddrTable) {
+        if (brokerAddrTable.containsKey(this.brokerConfig.getBrokerId())) {
+            return brokerAddrTable.size();
+        } else {
+            return brokerAddrTable.size() + 1;
+        }
+    }
+
+    @Override
+    protected void doRegisterBrokerAll(boolean checkOrderConfig, boolean oneway,
+        TopicConfigSerializeWrapper topicConfigWrapper) {
+
+        if (shutdown) {
+            BrokerController.LOG.info("BrokerController#doResterBrokerAll: broker has shutdown, no need to register any more.");
+            return;
+        }
+        List<RegisterBrokerResult> registerBrokerResultList = this.brokerOuterAPI.registerBrokerAll(
+            this.brokerConfig.getBrokerClusterName(),
+            this.getBrokerAddr(),
+            this.brokerConfig.getBrokerName(),
+            this.brokerConfig.getBrokerId(),
+            this.getHAServerAddr(),
+            topicConfigWrapper,
+            this.filterServerManager.buildNewFilterServerList(),
+            oneway,
+            this.brokerConfig.getRegisterBrokerTimeoutMills(),
+            this.brokerConfig.isEnableSlaveActingMaster(),
+            this.brokerConfig.isCompressedRegister(),
+            this.brokerConfig.isEnableSlaveActingMaster() ? this.brokerConfig.getBrokerNotActiveTimeoutMillis() : null,
+            this.brokerConfig.isInBrokerContainer());
+
+        handleRegisterBrokerResult(registerBrokerResultList, checkOrderConfig);
+    }
+
+    @Override
+    public String getNameServerList() {
+        if (this.brokerContainer.getBrokerContainerConfig().getNamesrvAddr() != null) {
+            this.brokerContainer.getBrokerOuterAPI().updateNameServerAddressList(brokerContainer.getBrokerContainerConfig().getNamesrvAddr());
+            return this.brokerContainer.getBrokerContainerConfig().getNamesrvAddr();
+        } else if (this.brokerContainer.getBrokerContainerConfig().isFetchNamesrvAddrByAddressServer()) {
+            return this.brokerContainer.getBrokerOuterAPI().fetchNameServerAddr();
+        }
+        return null;
+    }
+
+    @Override
+    public String getHAServerAddr() {
+        return this.brokerConfig.getBrokerIP2() + ":" + this.messageStoreConfig.getHaListenPort();
+    }
+
+    @Override
+    public long getMinBrokerIdInGroup() {
+        return this.minBrokerIdInGroup;
+    }
+
+    @Override
+    public boolean isSpecialServiceRunning() {
+        if (isScheduleServiceStart() && isTransactionCheckServiceStart()) {
+            return true;
+        }
+
+        return this.ackMessageProcessor != null && this.ackMessageProcessor.isPopReviveServiceRunning();
+    }
+
+    @Override
+    public int getListenPort() {
+        return this.brokerConfig.getListenPort();
+    }
+
+    public BrokerOuterAPI getBrokerOuterAPI() {
+        return brokerContainer.getBrokerOuterAPI();
+    }
+
+    public void startService(long minBrokerId, String minBrokerAddr) {
+        BrokerController.LOG.info("{} start service, min broker id is {}, min broker addr: {}",
+            this.brokerConfig.getCanonicalName(), minBrokerId, minBrokerAddr);
+        this.minBrokerIdInGroup = minBrokerId;
+        this.minBrokerAddrInGroup = minBrokerAddr;
+
+        this.changeSpecialServiceStatus(this.brokerConfig.getBrokerId() == minBrokerId);
+        this.registerBrokerAll(true, false, brokerConfig.isForceRegister());
+
+        isIsolated = false;
+    }
+
+    public void startServiceWithoutCondition() {
+        BrokerController.LOG.info("{} start service", this.brokerConfig.getCanonicalName());
+
+        this.changeSpecialServiceStatus(this.brokerConfig.getBrokerId() == MixAll.MASTER_ID);
+        this.registerBrokerAll(true, false, brokerConfig.isForceRegister());
+
+        isIsolated = false;
+    }
+
+    public void stopService() {
+        BrokerController.LOG.info("{} stop service", this.getBrokerConfig().getCanonicalName());
+        isIsolated = true;
+        this.changeSpecialServiceStatus(false);
+        this.closeChannels();
+    }
+
+    public synchronized void closeChannels() {
+        this.brokerContainer.getBrokerOuterAPI().getRemotingClient().closeChannels();
+    }
+
+    public BrokerContainer getBrokerContainer() {
+        return this.brokerContainer;
+    }
+
+    public boolean isIsolated() {
+        return this.isIsolated;
+    }
+
+    public NettyServerConfig getNettyServerConfig() {
+        return brokerContainer.getNettyServerConfig();
+    }
+
+    public NettyClientConfig getNettyClientConfig() {
+        return brokerContainer.getNettyClientConfig();
+    }
+
+    public MessageStore getMessageStoreByBrokerName(String brokerName) {
+        if (this.brokerConfig.getBrokerName().equals(brokerName)) {
+            return this.getMessageStore();
+        }
+        BrokerController brokerController = this.brokerContainer.findBrokerControllerByBrokerName(brokerName);
+        if (brokerController != null) {
+            return brokerController.getMessageStore();
+        }
+        return null;
+    }
+
+    @Override
+    public BrokerController peekMasterBroker() {
+        if (brokerConfig.getBrokerId() == MixAll.MASTER_ID) {
+            return this;
+        }
+        return this.brokerContainer.peekMasterBroker();
+    }
+
+    public BrokerPreOnlineService getBrokerPreOnlineService() {
+        return brokerPreOnlineService;
+    }
+}
diff --git a/container/src/main/java/org/apache/rocketmq/container/InnerSalveBrokerController.java b/container/src/main/java/org/apache/rocketmq/container/InnerSalveBrokerController.java
new file mode 100644
index 0000000..e00242a
--- /dev/null
+++ b/container/src/main/java/org/apache/rocketmq/container/InnerSalveBrokerController.java
@@ -0,0 +1,160 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.rocketmq.container;
+
+import com.google.common.base.Preconditions;
+
+import java.util.Arrays;
+import java.util.concurrent.TimeUnit;
+import java.util.concurrent.locks.Lock;
+import java.util.concurrent.locks.ReentrantLock;
+
+import org.apache.rocketmq.broker.BrokerController;
+import org.apache.rocketmq.common.BrokerConfig;
+import org.apache.rocketmq.common.BrokerSyncInfo;
+import org.apache.rocketmq.common.MixAll;
+import org.apache.rocketmq.store.config.MessageStoreConfig;
+
+public class InnerSalveBrokerController extends InnerBrokerController {
+
+    private final Lock lock = new ReentrantLock();
+
+    public InnerSalveBrokerController(final BrokerContainer brokerContainer,
+        final BrokerConfig brokerConfig,
+        final MessageStoreConfig storeConfig) {
+        super(brokerContainer, brokerConfig, storeConfig);
+        // Check configs
+        checkSlaveBrokerConfig();
+    }
+
+    private void checkSlaveBrokerConfig() {
+        Preconditions.checkNotNull(brokerConfig.getBrokerClusterName());
+        Preconditions.checkNotNull(brokerConfig.getBrokerName());
+        Preconditions.checkArgument(brokerConfig.getBrokerId() != MixAll.MASTER_ID);
+    }
+
+    private void onMasterOffline() {
+        // close channels with master broker
+        String masterAddr = this.slaveSynchronize.getMasterAddr();
+        if (masterAddr != null) {
+            this.brokerOuterAPI.getRemotingClient().closeChannels(
+                Arrays.asList(masterAddr, MixAll.brokerVIPChannel(true, masterAddr)));
+        }
+        // master not available, stop sync
+        this.slaveSynchronize.setMasterAddr(null);
+        this.messageStore.updateHaMasterAddress(null);
+    }
+
+    private void onMasterOnline(String masterAddr, String masterHaAddr) {
+        boolean needSyncMasterFlushOffset = this.messageStore.getMasterFlushedOffset() == 0
+            && this.messageStoreConfig.isSyncMasterFlushOffsetWhenStartup();
+        if (masterHaAddr == null || needSyncMasterFlushOffset) {
+            try {
+                BrokerSyncInfo brokerSyncInfo = this.brokerOuterAPI.retrieveBrokerHaInfo(masterAddr);
+
+                if (needSyncMasterFlushOffset) {
+                    LOG.info("Set master flush offset in slave to {}", brokerSyncInfo.getMasterFlushOffset());
+                    this.messageStore.setMasterFlushedOffset(brokerSyncInfo.getMasterFlushOffset());
+                }
+
+                if (masterHaAddr == null) {
+                    this.messageStore.updateHaMasterAddress(brokerSyncInfo.getMasterHaAddress());
+                    this.messageStore.updateMasterAddress(brokerSyncInfo.getMasterAddress());
+                }
+            } catch (Exception e) {
+                LOG.error("retrieve master ha info exception, {}", e);
+            }
+        }
+
+        // set master HA address.
+        if (masterHaAddr != null) {
+            this.messageStore.updateHaMasterAddress(masterHaAddr);
+        }
+
+        // wakeup HAClient
+        this.messageStore.wakeupHAClient();
+    }
+
+    private void onMinBrokerChange(long minBrokerId, String minBrokerAddr, String offlineBrokerAddr,
+        String masterHaAddr) {
+        LOG.info("Min broker changed, old: {}-{}, new {}-{}",
+            this.minBrokerIdInGroup, this.minBrokerAddrInGroup, minBrokerId, minBrokerAddr);
+
+        this.minBrokerIdInGroup = minBrokerId;
+        this.minBrokerAddrInGroup = minBrokerAddr;
+
+        this.changeSpecialServiceStatus(this.brokerConfig.getBrokerId() == this.minBrokerIdInGroup);
+
+        if (offlineBrokerAddr != null && offlineBrokerAddr.equals(this.slaveSynchronize.getMasterAddr())) {
+            // master offline
+            onMasterOffline();
+        }
+
+        if (minBrokerId == MixAll.MASTER_ID && minBrokerAddr != null) {
+            // master online
+            onMasterOnline(minBrokerAddr, masterHaAddr);
+        }
+
+        // notify PullRequest on hold to pull from master.
+        if (this.minBrokerIdInGroup == MixAll.MASTER_ID) {
+            this.pullRequestHoldService.notifyMasterOnline();
+        }
+    }
+
+    @Override
+    public void updateMinBroker(long minBrokerId, String minBrokerAddr) {
+        if (lock.tryLock()) {
+            try {
+                if (minBrokerId != this.minBrokerIdInGroup) {
+                    String offlineBrokerAddr = null;
+                    if (minBrokerId > this.minBrokerIdInGroup) {
+                        offlineBrokerAddr = this.minBrokerAddrInGroup;
+                    }
+                    onMinBrokerChange(minBrokerId, minBrokerAddr, offlineBrokerAddr, null);
+                }
+            } finally {
+                lock.unlock();
+            }
+
+        }
+    }
+
+    @Override
+    public void updateMinBroker(long minBrokerId, String minBrokerAddr, String offlineBrokerAddr,
+        String masterHaAddr) {
+        try {
+            if (lock.tryLock(3000, TimeUnit.MILLISECONDS)) {
+                try {
+                    if (minBrokerId != this.minBrokerIdInGroup) {
+                        onMinBrokerChange(minBrokerId, minBrokerAddr, offlineBrokerAddr, masterHaAddr);
+                    }
+                } finally {
+                    lock.unlock();
+                }
+
+            }
+        } catch (InterruptedException e) {
+            LOG.error("Update min broker error, {}", e);
+        }
+    }
+
+    @Override
+    public BrokerController peekMasterBroker() {
+        return this.brokerContainer.peekMasterBroker();
+    }
+}
diff --git a/container/src/main/java/org/apache/rocketmq/container/logback/BrokerLogbackConfigurator.java b/container/src/main/java/org/apache/rocketmq/container/logback/BrokerLogbackConfigurator.java
new file mode 100644
index 0000000..d4b4a75
--- /dev/null
+++ b/container/src/main/java/org/apache/rocketmq/container/logback/BrokerLogbackConfigurator.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.container.logback;
+
+import ch.qos.logback.classic.AsyncAppender;
+import ch.qos.logback.classic.LoggerContext;
+import ch.qos.logback.classic.encoder.PatternLayoutEncoder;
+import ch.qos.logback.classic.spi.ILoggingEvent;
+import ch.qos.logback.core.Appender;
+import ch.qos.logback.core.encoder.Encoder;
+import ch.qos.logback.core.rolling.FixedWindowRollingPolicy;
+import ch.qos.logback.core.rolling.RollingFileAppender;
+import ch.qos.logback.core.rolling.RollingPolicy;
+import ch.qos.logback.core.rolling.SizeAndTimeBasedFNATP;
+import ch.qos.logback.core.rolling.SizeBasedTriggeringPolicy;
+import ch.qos.logback.core.rolling.TimeBasedFileNamingAndTriggeringPolicy;
+import ch.qos.logback.core.rolling.TimeBasedRollingPolicy;
+
+import java.lang.reflect.Field;
+import java.util.HashSet;
+import java.util.Set;
+
+import ch.qos.logback.core.util.FileSize;
+import org.apache.rocketmq.common.BrokerConfig;
+import org.apache.rocketmq.common.constant.LoggerName;
+import org.apache.rocketmq.logging.InternalLogger;
+import org.apache.rocketmq.logging.InternalLoggerFactory;
+import org.slf4j.LoggerFactory;
+
+public class BrokerLogbackConfigurator {
+    private static final InternalLogger LOG = InternalLoggerFactory.getLogger(LoggerName.BROKER_LOGGER_NAME);
+
+    private static final Set<String> CONFIGURED_BROKER_LIST = new HashSet<>();
+
+    public static final String ROCKETMQ_LOGS = "rocketmqlogs";
+    public static final String ROCKETMQ_PREFIX = "Rocketmq";
+    public static final String SUFFIX_CONSOLE = "Console";
+    public static final String SUFFIX_APPENDER = "Appender";
+    public static final String SUFFIX_INNER_APPENDER = "_inner";
+
+    public static void doConfigure(BrokerConfig brokerConfig) {
+        if (!CONFIGURED_BROKER_LIST.contains(brokerConfig.getCanonicalName())) {
+            try {
+                LoggerContext lc = (LoggerContext) LoggerFactory.getILoggerFactory();
+                for (ch.qos.logback.classic.Logger tempLogger : lc.getLoggerList()) {
+                    String loggerName = tempLogger.getName();
+                    if (loggerName.startsWith(ROCKETMQ_PREFIX)
+                        && !loggerName.endsWith(SUFFIX_CONSOLE)
+                        && !loggerName.equals(LoggerName.ACCOUNT_LOGGER_NAME)
+                        && !loggerName.equals(LoggerName.COMMERCIAL_LOGGER_NAME)
+                        && !loggerName.equals(LoggerName.CONSUMER_STATS_LOGGER_NAME)) {
+                        ch.qos.logback.classic.Logger logger = lc.getLogger(brokerConfig.getLoggerIdentifier() + loggerName);
+                        logger.setAdditive(tempLogger.isAdditive());
+                        logger.setLevel(tempLogger.getLevel());
+                        String appenderName = loggerName + SUFFIX_APPENDER;
+                        Appender<ILoggingEvent> tempAppender = tempLogger.getAppender(appenderName);
+                        if (tempAppender instanceof AsyncAppender) {
+                            AsyncAppender tempAsyncAppender = (AsyncAppender) tempAppender;
+                            AsyncAppender asyncAppender = new AsyncAppender();
+                            asyncAppender.setName(brokerConfig.getLoggerIdentifier() + appenderName);
+                            asyncAppender.setContext(tempAsyncAppender.getContext());
+
+                            String innerAppenderName = appenderName + SUFFIX_INNER_APPENDER;
+                            Appender<ILoggingEvent> tempInnerAppender = tempAsyncAppender.getAppender(innerAppenderName);
+                            if (!(tempInnerAppender instanceof RollingFileAppender)) {
+                                continue;
+                            }
+                            asyncAppender.addAppender(configureRollingFileAppender((RollingFileAppender<ILoggingEvent>) tempInnerAppender,
+                                brokerConfig, innerAppenderName));
+                            asyncAppender.start();
+                            logger.addAppender(asyncAppender);
+                        } else if (tempAppender instanceof RollingFileAppender) {
+                            logger.addAppender(configureRollingFileAppender((RollingFileAppender<ILoggingEvent>) tempAppender,
+                                brokerConfig, appenderName));
+                        }
+                    }
+                }
+            } catch (Exception e) {
+                LOG.error("Configure logback for broker {} failed, will use default broker log config instead. {}", brokerConfig.getCanonicalName(), e);
+                return;
+            }
+
+            CONFIGURED_BROKER_LIST.add(brokerConfig.getCanonicalName());
+        }
+    }
+
+    private static RollingFileAppender<ILoggingEvent> configureRollingFileAppender(
+        RollingFileAppender<ILoggingEvent> tempRollingFileAppender, BrokerConfig brokerConfig, String appenderName)
+        throws NoSuchFieldException, IllegalAccessException {
+        RollingFileAppender<ILoggingEvent> rollingFileAppender = new RollingFileAppender<>();
+
+        // configure appender name
+        rollingFileAppender.setName(brokerConfig.getLoggerIdentifier() + appenderName);
+
+        // configure file name
+        rollingFileAppender.setFile(tempRollingFileAppender.getFile().replaceAll(ROCKETMQ_LOGS, brokerConfig.getCanonicalName() + "_" + ROCKETMQ_LOGS));
+
+        // configure append
+        rollingFileAppender.setAppend(true);
+
+        // configure prudent
+        rollingFileAppender.setPrudent(tempRollingFileAppender.isPrudent());
+
+        // configure rollingPolicy
+        RollingPolicy originalRollingPolicy = tempRollingFileAppender.getRollingPolicy();
+        if (originalRollingPolicy instanceof TimeBasedRollingPolicy) {
+            TimeBasedRollingPolicy<ILoggingEvent> tempRollingPolicy = (TimeBasedRollingPolicy<ILoggingEvent>) originalRollingPolicy;
+            TimeBasedRollingPolicy<ILoggingEvent> rollingPolicy = new TimeBasedRollingPolicy<>();
+            rollingPolicy.setContext(tempRollingPolicy.getContext());
+            rollingPolicy.setFileNamePattern(tempRollingPolicy.getFileNamePattern());
+            SizeAndTimeBasedFNATP<ILoggingEvent> sizeAndTimeBasedFNATP = new SizeAndTimeBasedFNATP<>();
+            sizeAndTimeBasedFNATP.setContext(tempRollingPolicy.getContext());
+            TimeBasedFileNamingAndTriggeringPolicy<ILoggingEvent> timeBasedFileNamingAndTriggeringPolicy =
+                tempRollingPolicy.getTimeBasedFileNamingAndTriggeringPolicy();
+            if (timeBasedFileNamingAndTriggeringPolicy instanceof SizeAndTimeBasedFNATP) {
+                SizeAndTimeBasedFNATP<ILoggingEvent> originalSizeAndTimeBasedFNATP =
+                    (SizeAndTimeBasedFNATP<ILoggingEvent>) timeBasedFileNamingAndTriggeringPolicy;
+                Field field = originalSizeAndTimeBasedFNATP.getClass().getDeclaredField("maxFileSize");
+                field.setAccessible(true);
+                sizeAndTimeBasedFNATP.setMaxFileSize((FileSize) field.get(originalSizeAndTimeBasedFNATP));
+                sizeAndTimeBasedFNATP.setTimeBasedRollingPolicy(rollingPolicy);
+            }
+            rollingPolicy.setTimeBasedFileNamingAndTriggeringPolicy(sizeAndTimeBasedFNATP);
+            rollingPolicy.setMaxHistory(tempRollingPolicy.getMaxHistory());
+            rollingPolicy.setParent(rollingFileAppender);
+            rollingPolicy.start();
+            rollingFileAppender.setRollingPolicy(rollingPolicy);
+        } else if (originalRollingPolicy instanceof FixedWindowRollingPolicy) {
+            FixedWindowRollingPolicy tempRollingPolicy = (FixedWindowRollingPolicy) originalRollingPolicy;
+            FixedWindowRollingPolicy rollingPolicy = new FixedWindowRollingPolicy();
+            rollingPolicy.setContext(tempRollingPolicy.getContext());
+            rollingPolicy.setFileNamePattern(tempRollingPolicy.getFileNamePattern().replaceAll(ROCKETMQ_LOGS, brokerConfig.getCanonicalName() + "_" + ROCKETMQ_LOGS));
+            rollingPolicy.setMaxIndex(tempRollingPolicy.getMaxIndex());
+            rollingPolicy.setMinIndex(tempRollingPolicy.getMinIndex());
+            rollingPolicy.setParent(rollingFileAppender);
+            rollingPolicy.start();
+            rollingFileAppender.setRollingPolicy(rollingPolicy);
+        }
+
+        // configure triggerPolicy
+        if (tempRollingFileAppender.getTriggeringPolicy() instanceof SizeBasedTriggeringPolicy) {
+            SizeBasedTriggeringPolicy<ILoggingEvent> tempTriggerPolicy = (SizeBasedTriggeringPolicy<ILoggingEvent>) tempRollingFileAppender.getTriggeringPolicy();
+            SizeBasedTriggeringPolicy<ILoggingEvent> triggerPolicy = new SizeBasedTriggeringPolicy<>();
+            triggerPolicy.setContext(tempTriggerPolicy.getContext());
+            Field field = triggerPolicy.getClass().getDeclaredField("maxFileSize");
+            field.setAccessible(true);
+            triggerPolicy.setMaxFileSize((FileSize) field.get(triggerPolicy));
+            triggerPolicy.start();
+            rollingFileAppender.setTriggeringPolicy(triggerPolicy);
+        }
+
+        // configure encoder
+        Encoder<ILoggingEvent> tempEncoder = tempRollingFileAppender.getEncoder();
+        if (tempEncoder instanceof PatternLayoutEncoder) {
+            PatternLayoutEncoder tempPatternLayoutEncoder = (PatternLayoutEncoder) tempEncoder;
+            PatternLayoutEncoder patternLayoutEncoder = new PatternLayoutEncoder();
+            patternLayoutEncoder.setContext(tempPatternLayoutEncoder.getContext());
+            patternLayoutEncoder.setPattern(tempPatternLayoutEncoder.getPattern());
+            patternLayoutEncoder.setCharset(tempPatternLayoutEncoder.getCharset());
+            patternLayoutEncoder.start();
+
+            rollingFileAppender.setEncoder(patternLayoutEncoder);
+        }
+
+        // configure context
+        rollingFileAppender.setContext(tempRollingFileAppender.getContext());
+
+        rollingFileAppender.start();
+
+        return rollingFileAppender;
+    }
+}
diff --git a/container/src/test/java/org/apache/rocketmq/container/BrokerContainerStartupTest.java b/container/src/test/java/org/apache/rocketmq/container/BrokerContainerStartupTest.java
new file mode 100644
index 0000000..8138098
--- /dev/null
+++ b/container/src/test/java/org/apache/rocketmq/container/BrokerContainerStartupTest.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.container;
+
+import java.io.File;
+import java.io.IOException;
+import java.nio.file.Files;
+import java.util.ArrayList;
+import java.util.List;
+import java.util.Properties;
+
+import org.apache.rocketmq.broker.BrokerController;
+import org.apache.rocketmq.common.BrokerConfig;
+import org.apache.rocketmq.common.MixAll;
+import org.apache.rocketmq.common.UtilAll;
+import org.apache.rocketmq.remoting.netty.NettyClientConfig;
+import org.apache.rocketmq.remoting.netty.NettyServerConfig;
+import org.apache.rocketmq.store.config.MessageStoreConfig;
+import org.assertj.core.util.Arrays;
+import org.junit.After;
+import org.junit.Before;
+import org.junit.Test;
+import org.junit.runner.RunWith;
+import org.mockito.Mock;
+import org.mockito.junit.MockitoJUnitRunner;
+
+import static org.assertj.core.api.Java6Assertions.assertThat;
+
+@RunWith(MockitoJUnitRunner.class)
+public class BrokerContainerStartupTest {
+    private static final List<File> TMP_FILE_LIST = new ArrayList<>();
+    private static final String BROKER_NAME_PREFIX = "TestBroker";
+    private static final String SHARED_BROKER_NAME_PREFIX = "TestBrokerContainer";
+    private static String brokerConfigPath;
+    private static String brokerContainerConfigPath;
+
+    @Mock
+    private BrokerConfig brokerConfig;
+    private String storePathRootDir = "store/test";
+    @Mock
+    private NettyClientConfig nettyClientConfig;
+    @Mock
+    private NettyServerConfig nettyServerConfig;
+
+    @Before
+    public void init() throws IOException {
+        String brokerName = BROKER_NAME_PREFIX + "_" + System.currentTimeMillis();
+        BrokerConfig brokerConfig = new BrokerConfig();
+        brokerConfig.setBrokerName(brokerName);
+        if (brokerConfig.getRocketmqHome() == null) {
+            brokerConfig.setRocketmqHome("../distribution");
+        }
+        MessageStoreConfig storeConfig = new MessageStoreConfig();
+        String baseDir = createBaseDir(brokerConfig.getBrokerName() + "_" + brokerConfig.getBrokerId()).getAbsolutePath();
+        storeConfig.setStorePathRootDir(baseDir);
+        storeConfig.setStorePathCommitLog(baseDir + File.separator + "commitlog");
+
+        brokerConfigPath = "/tmp/" + brokerName;
+        brokerConfig.setBrokerConfigPath(brokerConfigPath);
+        File file = new File(brokerConfigPath);
+        TMP_FILE_LIST.add(file);
+        Properties brokerConfigProp = MixAll.object2Properties(brokerConfig);
+        Properties storeConfigProp = MixAll.object2Properties(storeConfig);
+
+        for (Object key : storeConfigProp.keySet()) {
+            brokerConfigProp.put(key, storeConfigProp.get(key));
+        }
+        MixAll.string2File(MixAll.properties2String(brokerConfigProp), brokerConfigPath);
+
+        brokerContainerConfigPath = "/tmp/" + SHARED_BROKER_NAME_PREFIX + System.currentTimeMillis();
+        BrokerContainerConfig brokerContainerConfig = new BrokerContainerConfig();
+        brokerContainerConfig.setBrokerConfigPaths(brokerConfigPath);
+        if (brokerContainerConfig.getRocketmqHome() == null) {
+            brokerContainerConfig.setRocketmqHome("../distribution");
+        }
+        File file1 = new File(brokerContainerConfigPath);
+        TMP_FILE_LIST.add(file1);
+        Properties brokerContainerConfigProp = MixAll.object2Properties(brokerContainerConfig);
+        MixAll.string2File(MixAll.properties2String(brokerContainerConfigProp), brokerContainerConfigPath);
+    }
+
+    @After
+    public void destory() {
+        for (File file : TMP_FILE_LIST) {
+            UtilAll.deleteFile(file);
+        }
+    }
+
+    @Test
+    public void testStartBroker1() {
+        BrokerContainer brokerContainer = BrokerContainerStartup.startBrokerContainer(
+            BrokerContainerStartup.createBrokerContainer(Arrays.array("-c", brokerContainerConfigPath)));
+        assertThat(brokerContainer).isNotNull();
+        List<BrokerController> brokers = BrokerContainerStartup.createAndStartBrokers(brokerContainer);
+        assertThat(brokers.size()).isEqualTo(1);
+
+        brokerContainer.shutdown();
+        assertThat(brokerContainer.getBrokerControllers().size()).isEqualTo(0);
+    }
+
+    @Test
+    public void testStartBroker2() {
+        InnerBrokerController brokerController = (InnerBrokerController) BrokerContainerStartup.start(BrokerContainerStartup.createBrokerController(Arrays.array("-c", brokerConfigPath)));
+        assertThat(brokerController).isNotNull();
+
+        assertThat(brokerController.getBrokerContainer().getBrokerControllers().size()).isEqualTo(1);
+
+        brokerController.getBrokerContainer().shutdown();
+        assertThat(brokerController.getBrokerContainer().getBrokerControllers().size()).isEqualTo(0);
+    }
+
+    private static File createBaseDir(String prefix) {
+        final File file;
+        try {
+            file = Files.createTempDirectory(prefix).toFile();
+            TMP_FILE_LIST.add(file);
+            System.out.printf("create file at %s%n", file.getAbsolutePath());
+            return file;
+        } catch (IOException e) {
+            throw new RuntimeException("Couldn't create tmp folder", e);
+        }
+    }
+
+    @Before
+    public void clear() {
+        UtilAll.deleteFile(new File(storePathRootDir));
+    }
+
+    @After
+    public void tearDown() {
+        File configFile = new File(storePathRootDir);
+        UtilAll.deleteFile(configFile);
+        UtilAll.deleteEmptyDirectory(configFile);
+        UtilAll.deleteEmptyDirectory(configFile.getParentFile());
+    }
+}
\ No newline at end of file
diff --git a/container/src/test/java/org/apache/rocketmq/container/BrokerContainerTest.java b/container/src/test/java/org/apache/rocketmq/container/BrokerContainerTest.java
new file mode 100644
index 0000000..42c934d
--- /dev/null
+++ b/container/src/test/java/org/apache/rocketmq/container/BrokerContainerTest.java
@@ -0,0 +1,339 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.rocketmq.container;
+
+import java.io.IOException;
+import java.nio.file.Files;
+import java.util.ArrayList;
+import java.util.HashSet;
+import java.util.List;
+import java.util.Random;
+import java.util.Set;
+
+import org.apache.rocketmq.broker.BrokerController;
+import org.apache.rocketmq.broker.out.BrokerOuterAPI;
+import org.apache.rocketmq.common.*;
+import org.apache.rocketmq.common.constant.PermName;
+import org.apache.rocketmq.common.protocol.body.TopicConfigSerializeWrapper;
+import org.apache.rocketmq.remoting.netty.NettyClientConfig;
+import org.apache.rocketmq.remoting.netty.NettyServerConfig;
+import org.apache.rocketmq.store.config.BrokerRole;
+import org.apache.rocketmq.store.config.MessageStoreConfig;
+import org.junit.After;
+import org.junit.Test;
+import org.mockito.ArgumentCaptor;
+import org.mockito.ArgumentMatchers;
+
+import java.io.File;
+import java.lang.reflect.Field;
+import java.util.Map;
+
+import static org.assertj.core.api.Assertions.assertThat;
+import static org.assertj.core.api.Assertions.entry;
+import static org.mockito.Mockito.*;
+
+public class BrokerContainerTest {
+    private static final List<File> TMP_FILE_LIST = new ArrayList<>();
+    private static final Random RANDOM = new Random();
+    private static final Set<Integer> PORTS_IN_USE = new HashSet<>();
+
+    /**
+     * Tests if the controller can be properly stopped and started.
+     *
+     * @throws Exception If fails.
+     */
+    @Test
+    public void testBrokerContainerRestart() throws Exception {
+        BrokerContainer brokerController = new BrokerContainer(
+            new BrokerContainerConfig(),
+            new NettyServerConfig(),
+            new NettyClientConfig());
+        assertThat(brokerController.initialize()).isTrue();
+        brokerController.start();
+        brokerController.shutdown();
+    }
+
+    @Test
+    public void testRegisterIncrementBrokerData() throws Exception {
+        BrokerController brokerController = new BrokerController(
+            new BrokerConfig(),
+            new NettyServerConfig(),
+            new NettyClientConfig(),
+            new MessageStoreConfig());
+
+        BrokerOuterAPI brokerOuterAPI = mock(BrokerOuterAPI.class);
+        Field field = BrokerController.class.getDeclaredField("brokerOuterAPI");
+        field.setAccessible(true);
+        field.set(brokerController, brokerOuterAPI);
+
+        // topic-0 doesn't have queueGroupConfig.
+        // topic-1 has queueGroupConfig.
+        List<TopicConfig> topicConfigList = new ArrayList<>(2);
+        for (int i = 0; i < 2; i++) {
+            topicConfigList.add(new TopicConfig("topic-" + i));
+        }
+        DataVersion dataVersion = new DataVersion();
+
+        // Check normal condition.
+        testRegisterIncrementBrokerDataWithPerm(brokerController, brokerOuterAPI,
+            topicConfigList, dataVersion, PermName.PERM_READ | PermName.PERM_WRITE, 1);
+        // Check unwritable broker.
+        testRegisterIncrementBrokerDataWithPerm(brokerController, brokerOuterAPI,
+            topicConfigList, dataVersion, PermName.PERM_READ, 2);
+        // Check unreadable broker.
+        testRegisterIncrementBrokerDataWithPerm(brokerController, brokerOuterAPI,
+            topicConfigList, dataVersion, PermName.PERM_WRITE, 3);
+    }
+
+    @Test
+    public void testRegisterIncrementBrokerDataPerm() throws Exception {
+        BrokerController brokerController = new BrokerController(
+            new BrokerConfig(),
+            new NettyServerConfig(),
+            new NettyClientConfig(),
+            new MessageStoreConfig());
+
+        BrokerOuterAPI brokerOuterAPI = mock(BrokerOuterAPI.class);
+        Field field = BrokerController.class.getDeclaredField("brokerOuterAPI");
+        field.setAccessible(true);
+        field.set(brokerController, brokerOuterAPI);
+
+        // topic-0 doesn't have queueGroupConfig.
+        // topic-1 has queueGroupConfig.
+        List<TopicConfig> topicConfigList = new ArrayList<>(2);
+        for (int i = 0; i < 2; i++) {
+            topicConfigList.add(new TopicConfig("topic-" + i));
+        }
+        DataVersion dataVersion = new DataVersion();
+
+        brokerController.getBrokerConfig().setBrokerPermission(4);
+
+        brokerController.registerIncrementBrokerData(topicConfigList, dataVersion);
+        // Get topicConfigSerializeWrapper created by registerIncrementBrokerData() from brokerOuterAPI.registerBrokerAll()
+        ArgumentCaptor<TopicConfigSerializeWrapper> captor = ArgumentCaptor.forClass(TopicConfigSerializeWrapper.class);
+        verify(brokerOuterAPI).registerBrokerAll(anyString(), anyString(), anyString(), anyLong(), anyString(),
+            captor.capture(), ArgumentMatchers.anyList(), anyBoolean(), anyInt(), anyBoolean(), anyBoolean(), anyBoolean());
+        TopicConfigSerializeWrapper wrapper = captor.getValue();
+        for (Map.Entry<String, TopicConfig> entry : wrapper.getTopicConfigTable().entrySet()) {
+            assertThat(entry.getValue().getPerm()).isEqualTo(brokerController.getBrokerConfig().getBrokerPermission());
+        }
+
+    }
+
+    @Test
+    public void testMasterScaleOut() throws Exception {
+        BrokerContainer brokerContainer = new BrokerContainer(
+            new BrokerContainerConfig(),
+            new NettyServerConfig(),
+            new NettyClientConfig());
+        assertThat(brokerContainer.initialize()).isTrue();
+        brokerContainer.getBrokerContainerConfig().setNamesrvAddr("127.0.0.1:9876");
+        brokerContainer.start();
+
+        BrokerConfig masterBrokerConfig = new BrokerConfig();
+
+        String baseDir = createBaseDir("unnittest-master").getAbsolutePath();
+        MessageStoreConfig messageStoreConfig = new MessageStoreConfig();
+        messageStoreConfig.setStorePathRootDir(baseDir);
+        messageStoreConfig.setStorePathCommitLog(baseDir + File.separator + "commitlog");
+        InnerBrokerController brokerController = brokerContainer.addBroker(masterBrokerConfig, messageStoreConfig);
+        assertThat(brokerController.isIsolated()).isFalse();
+
+        brokerContainer.shutdown();
+        brokerController.getMessageStore().destroy();
+    }
+
+    @Test
+    public void testAddMasterFailed() throws Exception {
+        BrokerContainer brokerContainer = new BrokerContainer(
+            new BrokerContainerConfig(),
+            new NettyServerConfig(),
+            new NettyClientConfig());
+        assertThat(brokerContainer.initialize()).isTrue();
+        brokerContainer.start();
+
+        BrokerConfig masterBrokerConfig = new BrokerConfig();
+        masterBrokerConfig.setListenPort(brokerContainer.getNettyServerConfig().getListenPort());
+        boolean exceptionCaught = false;
+        try {
+            String baseDir = createBaseDir("unnittest-master").getAbsolutePath();
+            MessageStoreConfig messageStoreConfig = new MessageStoreConfig();
+            messageStoreConfig.setStorePathRootDir(baseDir);
+            messageStoreConfig.setStorePathCommitLog(baseDir + File.separator + "commitlog");
+            brokerContainer.addBroker(masterBrokerConfig, messageStoreConfig);
+        } catch (Exception e) {
+            exceptionCaught = true;
+        } finally {
+            brokerContainer.shutdown();
+
+        }
+
+        assertThat(exceptionCaught).isTrue();
+    }
+
+    @Test
+    public void testAddSlaveFailed() throws Exception {
+        BrokerContainer sharedBrokerController = new BrokerContainer(
+            new BrokerContainerConfig(),
+            new NettyServerConfig(),
+            new NettyClientConfig());
+        assertThat(sharedBrokerController.initialize()).isTrue();
+        sharedBrokerController.start();
+
+        BrokerConfig slaveBrokerConfig = new BrokerConfig();
+        slaveBrokerConfig.setBrokerId(1);
+        slaveBrokerConfig.setListenPort(sharedBrokerController.getNettyServerConfig().getListenPort());
+        MessageStoreConfig slaveMessageStoreConfig = new MessageStoreConfig();
+        slaveMessageStoreConfig.setBrokerRole(BrokerRole.SLAVE);
+        String baseDir = createBaseDir("unnittest-slave").getAbsolutePath();
+        slaveMessageStoreConfig.setStorePathRootDir(baseDir);
+        slaveMessageStoreConfig.setStorePathCommitLog(baseDir + File.separator + "commitlog");
+        boolean exceptionCaught = false;
+        try {
+            sharedBrokerController.addBroker(slaveBrokerConfig, slaveMessageStoreConfig);
+        } catch (Exception e) {
+            exceptionCaught = true;
+        } finally {
+            sharedBrokerController.shutdown();
+        }
+
+        assertThat(exceptionCaught).isTrue();
+    }
+
+    @Test
+    public void testAddAndRemoveMaster() throws Exception {
+        BrokerContainer brokerContainer = new BrokerContainer(
+            new BrokerContainerConfig(),
+            new NettyServerConfig(),
+            new NettyClientConfig());
+        assertThat(brokerContainer.initialize()).isTrue();
+        brokerContainer.start();
+
+        BrokerConfig masterBrokerConfig = new BrokerConfig();
+        String baseDir = createBaseDir("unnittest-master").getAbsolutePath();
+        MessageStoreConfig messageStoreConfig = new MessageStoreConfig();
+        messageStoreConfig.setStorePathRootDir(baseDir);
+        messageStoreConfig.setStorePathCommitLog(baseDir + File.separator + "commitlog");
+        InnerBrokerController master = brokerContainer.addBroker(masterBrokerConfig, messageStoreConfig);
+        assertThat(master).isNotNull();
+        master.start();
+        assertThat(master.isIsolated()).isFalse();
+
+        brokerContainer.removeBroker(new BrokerIdentity(masterBrokerConfig.getBrokerClusterName(), masterBrokerConfig.getBrokerName(), masterBrokerConfig.getBrokerId()));
+        assertThat(brokerContainer.getMasterBrokers().size()).isEqualTo(0);
+
+        brokerContainer.shutdown();
+        master.getMessageStore().destroy();
+    }
+
+    @Test
+    public void testAddAndRemoveSlaveSuccess() throws Exception {
+        BrokerContainer brokerContainer = new BrokerContainer(
+            new BrokerContainerConfig(),
+            new NettyServerConfig(),
+            new NettyClientConfig());
+        assertThat(brokerContainer.initialize()).isTrue();
+        brokerContainer.start();
+
+        BrokerConfig masterBrokerConfig = new BrokerConfig();
+        String baseDir = createBaseDir("unnittest-master").getAbsolutePath();
+        MessageStoreConfig messageStoreConfig = new MessageStoreConfig();
+        messageStoreConfig.setStorePathRootDir(baseDir);
+        messageStoreConfig.setStorePathCommitLog(baseDir + File.separator + "commitlog");
+        InnerBrokerController master = brokerContainer.addBroker(masterBrokerConfig, messageStoreConfig);
+        assertThat(master).isNotNull();
+        master.start();
+        assertThat(master.isIsolated()).isFalse();
+
+        BrokerConfig slaveBrokerConfig = new BrokerConfig();
+        slaveBrokerConfig.setListenPort(generatePort(masterBrokerConfig.getListenPort(), 10000));
+        slaveBrokerConfig.setBrokerId(1);
+        MessageStoreConfig slaveMessageStoreConfig = new MessageStoreConfig();
+        slaveMessageStoreConfig.setBrokerRole(BrokerRole.SLAVE);
+        slaveMessageStoreConfig.setHaListenPort(generatePort(messageStoreConfig.getHaListenPort(), 10000));
+        baseDir = createBaseDir("unnittest-slave").getAbsolutePath();
+        slaveMessageStoreConfig.setStorePathRootDir(baseDir);
+        slaveMessageStoreConfig.setStorePathCommitLog(baseDir + File.separator + "commitlog");
+        InnerBrokerController slave = brokerContainer.addBroker(slaveBrokerConfig, slaveMessageStoreConfig);
+        assertThat(slave).isNotNull();
+        slave.start();
+        assertThat(slave.isIsolated()).isFalse();
+
+        brokerContainer.removeBroker(new BrokerIdentity(slaveBrokerConfig.getBrokerClusterName(), slaveBrokerConfig.getBrokerName(), slaveBrokerConfig.getBrokerId()));
+        assertThat(brokerContainer.getSlaveBrokers().size()).isEqualTo(0);
+
+        brokerContainer.removeBroker(new BrokerIdentity(masterBrokerConfig.getBrokerClusterName(), masterBrokerConfig.getBrokerName(), masterBrokerConfig.getBrokerId()));
+        assertThat(brokerContainer.getMasterBrokers().size()).isEqualTo(0);
+
+        brokerContainer.shutdown();
+        slave.getMessageStore().destroy();
+        master.getMessageStore().destroy();
+    }
+
+    private static File createBaseDir(String prefix) {
+        final File file;
+        try {
+            file = Files.createTempDirectory(prefix).toFile();
+            TMP_FILE_LIST.add(file);
+            return file;
+        } catch (IOException e) {
+            throw new RuntimeException("Couldn't create tmp folder", e);
+        }
+    }
+
+    public static int generatePort(int base, int range) {
+        int result = base + RANDOM.nextInt(range);
+        while (PORTS_IN_USE.contains(result) || PORTS_IN_USE.contains(result - 2)) {
+            result = base + RANDOM.nextInt(range);
+        }
+        PORTS_IN_USE.add(result);
+        PORTS_IN_USE.add(result - 2);
+        return result;
+    }
+
+    @After
+    public void destory() {
+        for (File file : TMP_FILE_LIST) {
+            UtilAll.deleteFile(file);
+        }
+    }
+
+    private void testRegisterIncrementBrokerDataWithPerm(BrokerController brokerController,
+        BrokerOuterAPI brokerOuterAPI,
+        List<TopicConfig> topicConfigList, DataVersion dataVersion, int perm, int times) {
+        brokerController.getBrokerConfig().setBrokerPermission(perm);
+
+        brokerController.registerIncrementBrokerData(topicConfigList, dataVersion);
+        // Get topicConfigSerializeWrapper created by registerIncrementBrokerData() from brokerOuterAPI.registerBrokerAll()
+        ArgumentCaptor<TopicConfigSerializeWrapper> captor = ArgumentCaptor.forClass(TopicConfigSerializeWrapper.class);
+        verify(brokerOuterAPI, times(times)).registerBrokerAll(anyString(), anyString(), anyString(), anyLong(),
+            anyString(), captor.capture(), ArgumentMatchers.anyList(), anyBoolean(), anyInt(), anyBoolean(), anyBoolean(), anyBoolean());
+        TopicConfigSerializeWrapper wrapper = captor.getValue();
+
+        for (TopicConfig topicConfig : topicConfigList) {
+            topicConfig.setPerm(perm);
+        }
+        assertThat(wrapper.getDataVersion()).isEqualTo(dataVersion);
+        assertThat(wrapper.getTopicConfigTable()).containsExactly(
+            entry("topic-0", topicConfigList.get(0)),
+            entry("topic-1", topicConfigList.get(1)));
+        for (TopicConfig topicConfig : topicConfigList) {
+            topicConfig.setPerm(PermName.PERM_READ | PermName.PERM_WRITE);
+        }
+    }
+}
diff --git a/container/src/test/java/org/apache/rocketmq/container/BrokerPreOnlineTest.java b/container/src/test/java/org/apache/rocketmq/container/BrokerPreOnlineTest.java
new file mode 100644
index 0000000..6943d28
--- /dev/null
+++ b/container/src/test/java/org/apache/rocketmq/container/BrokerPreOnlineTest.java
@@ -0,0 +1,102 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.rocketmq.container;
+
+import java.lang.reflect.Field;
+import java.time.Duration;
+import java.util.HashMap;
+import java.util.Map;
+
+import org.apache.rocketmq.broker.BrokerController;
+import org.apache.rocketmq.broker.out.BrokerOuterAPI;
+import org.apache.rocketmq.broker.transaction.TransactionalMessageCheckService;
+import org.apache.rocketmq.common.BrokerConfig;
+import org.apache.rocketmq.common.protocol.body.BrokerMemberGroup;
+import org.apache.rocketmq.store.DefaultMessageStore;
+import org.apache.rocketmq.store.config.MessageStoreConfig;
+import org.junit.Test;
+import org.junit.runner.RunWith;
+import org.mockito.Mock;
+import org.mockito.junit.MockitoJUnitRunner;
+
+import static org.awaitility.Awaitility.await;
+import static org.mockito.Mockito.mock;
+import static org.mockito.Mockito.when;
+
+@RunWith(MockitoJUnitRunner.class)
+public class BrokerPreOnlineTest {
+    @Mock
+    private BrokerContainer brokerContainer;
+
+    private InnerBrokerController innerBrokerController;
+
+    @Mock
+    private BrokerOuterAPI brokerOuterAPI;
+
+    public void init() throws Exception {
+        when(brokerContainer.getBrokerOuterAPI()).thenReturn(brokerOuterAPI);
+        when(brokerContainer.getBrokerContainerConfig()).thenReturn(new BrokerContainerConfig());
+
+        BrokerMemberGroup brokerMemberGroup1 = new BrokerMemberGroup();
+        Map<Long, String> brokerAddrMap = new HashMap<>();
+        brokerAddrMap.put(1L, "127.0.0.1:20911");
+        brokerMemberGroup1.setBrokerAddrs(brokerAddrMap);
+
+        BrokerMemberGroup brokerMemberGroup2 = new BrokerMemberGroup();
+        brokerMemberGroup2.setBrokerAddrs(new HashMap<>());
+
+//        when(brokerOuterAPI.syncBrokerMemberGroup(anyString(), anyString()))
+//            .thenReturn(brokerMemberGroup1)
+//            .thenReturn(brokerMemberGroup2);
+//        doNothing().when(brokerOuterAPI).sendBrokerHaInfo(anyString(), anyString(), anyLong(), anyString());
+
+        DefaultMessageStore defaultMessageStore = mock(DefaultMessageStore.class);
+        when(defaultMessageStore.getMessageStoreConfig()).thenReturn(new MessageStoreConfig());
+        when(defaultMessageStore.getBrokerConfig()).thenReturn(new BrokerConfig());
+
+//        HAService haService = new DefaultHAService();
+//        haService.init(defaultMessageStore);
+//        haService.start();
+//
+//        when(defaultMessageStore.getHaService()).thenReturn(haService);
+
+        innerBrokerController = new InnerBrokerController(brokerContainer,
+            defaultMessageStore.getBrokerConfig(),
+            defaultMessageStore.getMessageStoreConfig());
+
+        innerBrokerController.setTransactionalMessageCheckService(new TransactionalMessageCheckService(innerBrokerController));
+
+        Field field = InnerBrokerController.class.getDeclaredField("isIsolated");
+        field.setAccessible(true);
+        field.set(innerBrokerController, true);
+
+        field = BrokerController.class.getDeclaredField("messageStore");
+        field.setAccessible(true);
+        field.set(innerBrokerController, defaultMessageStore);
+    }
+
+    @Test
+    public void testMasterOnlineConnTimeout() throws Exception {
+        init();
+        BrokerPreOnlineService brokerPreOnlineService = new BrokerPreOnlineService(innerBrokerController);
+
+        brokerPreOnlineService.start();
+
+        await().atMost(Duration.ofSeconds(30)).until(() -> !innerBrokerController.isIsolated());
+    }
+}
diff --git a/pom.xml b/pom.xml
index 880df2d..5f6c4ed 100644
--- a/pom.xml
+++ b/pom.xml
@@ -121,6 +121,7 @@
         <module>logging</module>
         <module>acl</module>
         <module>example</module>
+        <module>container</module>
     </modules>
 
     <build>
@@ -460,6 +461,11 @@
             </dependency>
             <dependency>
                 <groupId>${project.groupId}</groupId>
+                <artifactId>rocketmq-container</artifactId>
+                <version>${project.version}</version>
+            </dependency>
+            <dependency>
+                <groupId>${project.groupId}</groupId>
                 <artifactId>rocketmq-common</artifactId>
                 <version>${project.version}</version>
             </dependency>
@@ -531,7 +537,7 @@
             <dependency>
                 <groupId>commons-cli</groupId>
                 <artifactId>commons-cli</artifactId>
-                <version>1.2</version>
+                <version>1.4</version>
             </dependency>
             <dependency>
                 <groupId>io.netty</groupId>
@@ -566,6 +572,11 @@
                 <version>3.4</version>
             </dependency>
             <dependency>
+                <groupId>commons-io</groupId>
+                <artifactId>commons-io</artifactId>
+                <version>2.6</version>
+            </dependency>
+            <dependency>
                 <groupId>com.google.guava</groupId>
                 <artifactId>guava</artifactId>
                 <version>19.0</version>
@@ -610,8 +621,11 @@
                 <artifactId>commons-validator</artifactId>
                 <version>1.7</version>
             </dependency>
-
-
+            <dependency>
+                <groupId>ch.qos.logback</groupId>
+                <artifactId>logback-core</artifactId>
+                <version>1.1.11</version>
+            </dependency>
         </dependencies>
     </dependencyManagement>
 </project>
diff --git a/remoting/src/main/java/org/apache/rocketmq/remoting/RPCHook.java b/remoting/src/main/java/org/apache/rocketmq/remoting/RPCHook.java
index 6292fc0..0f17858 100644
--- a/remoting/src/main/java/org/apache/rocketmq/remoting/RPCHook.java
+++ b/remoting/src/main/java/org/apache/rocketmq/remoting/RPCHook.java
@@ -23,5 +23,7 @@ public interface RPCHook {
     void doBeforeRequest(final String remoteAddr, final RemotingCommand request);
 
     void doAfterResponse(final String remoteAddr, final RemotingCommand request,
-        final RemotingCommand response);
+                         final RemotingCommand response);
+
+    void doAfterRpcFailure(final String remoteAddr, RemotingCommand request, Boolean remoteTimeout);
 }
diff --git a/remoting/src/main/java/org/apache/rocketmq/remoting/RemotingClient.java b/remoting/src/main/java/org/apache/rocketmq/remoting/RemotingClient.java
index c0754db..9f6933d 100644
--- a/remoting/src/main/java/org/apache/rocketmq/remoting/RemotingClient.java
+++ b/remoting/src/main/java/org/apache/rocketmq/remoting/RemotingClient.java
@@ -17,12 +17,14 @@
 package org.apache.rocketmq.remoting;
 
 import java.util.List;
+import java.util.concurrent.ConcurrentMap;
 import java.util.concurrent.ExecutorService;
 import org.apache.rocketmq.remoting.exception.RemotingConnectException;
 import org.apache.rocketmq.remoting.exception.RemotingSendRequestException;
 import org.apache.rocketmq.remoting.exception.RemotingTimeoutException;
 import org.apache.rocketmq.remoting.exception.RemotingTooMuchRequestException;
 import org.apache.rocketmq.remoting.netty.NettyRequestProcessor;
+import org.apache.rocketmq.remoting.netty.ResponseFuture;
 import org.apache.rocketmq.remoting.protocol.RemotingCommand;
 
 public interface RemotingClient extends RemotingService {
@@ -31,6 +33,8 @@ public interface RemotingClient extends RemotingService {
 
     List<String> getNameServerAddressList();
 
+    List<String> getAvailableNameSrvList();
+
     RemotingCommand invokeSync(final String addr, final RemotingCommand request,
         final long timeoutMillis) throws InterruptedException, RemotingConnectException,
         RemotingSendRequestException, RemotingTimeoutException;
@@ -48,7 +52,12 @@ public interface RemotingClient extends RemotingService {
 
     void setCallbackExecutor(final ExecutorService callbackExecutor);
 
-    ExecutorService getCallbackExecutor();
-
     boolean isChannelWritable(final String addr);
+
+    void closeChannels();
+
+    void closeChannels(final List<String> addrList);
+
+    ConcurrentMap<Integer, ResponseFuture> getResponseTable();
+
 }
diff --git a/remoting/src/main/java/org/apache/rocketmq/remoting/RemotingServer.java b/remoting/src/main/java/org/apache/rocketmq/remoting/RemotingServer.java
index a12c089..36e2035 100644
--- a/remoting/src/main/java/org/apache/rocketmq/remoting/RemotingServer.java
+++ b/remoting/src/main/java/org/apache/rocketmq/remoting/RemotingServer.java
@@ -36,6 +36,12 @@ public interface RemotingServer extends RemotingService {
 
     Pair<NettyRequestProcessor, ExecutorService> getProcessorPair(final int requestCode);
 
+    Pair<NettyRequestProcessor, ExecutorService> getDefaultProcessorPair();
+
+    RemotingServer newRemotingServer(int port);
+
+    void removeRemotingServer(int port);
+
     RemotingCommand invokeSync(final Channel channel, final RemotingCommand request,
         final long timeoutMillis) throws InterruptedException, RemotingSendRequestException,
         RemotingTimeoutException;
diff --git a/remoting/src/main/java/org/apache/rocketmq/remoting/RemotingService.java b/remoting/src/main/java/org/apache/rocketmq/remoting/RemotingService.java
index 2f88797..c718f2e 100644
--- a/remoting/src/main/java/org/apache/rocketmq/remoting/RemotingService.java
+++ b/remoting/src/main/java/org/apache/rocketmq/remoting/RemotingService.java
@@ -23,4 +23,9 @@ public interface RemotingService {
     void shutdown();
 
     void registerRPCHook(RPCHook rpcHook);
+
+    /**
+     * Remove all rpc hooks.
+     */
+    void clearRPCHook();
 }
diff --git a/remoting/src/main/java/org/apache/rocketmq/remoting/common/RemotingHelper.java b/remoting/src/main/java/org/apache/rocketmq/remoting/common/RemotingHelper.java
index 4654e49..4c8a62a 100644
--- a/remoting/src/main/java/org/apache/rocketmq/remoting/common/RemotingHelper.java
+++ b/remoting/src/main/java/org/apache/rocketmq/remoting/common/RemotingHelper.java
@@ -189,6 +189,19 @@ public class RemotingHelper {
         return "";
     }
 
+    public static String parseHostFromAddress(String address) {
+        if (address == null) {
+            return "";
+        }
+
+        String[] addressSplits = address.split(":");
+        if (addressSplits.length < 1) {
+            return "";
+        }
+
+        return addressSplits[0];
+    }
+
     public static String parseSocketAddressAddr(SocketAddress socketAddress) {
         if (socketAddress != null) {
             // Default toString of InetSocketAddress is "hostName/IP:port"
@@ -199,4 +212,29 @@ public class RemotingHelper {
         return "";
     }
 
+    public static int parseSocketAddressPort(SocketAddress socketAddress) {
+        if (socketAddress instanceof InetSocketAddress) {
+            return ((InetSocketAddress) socketAddress).getPort();
+        }
+        return -1;
+    }
+
+
+    public static int ipToInt(String ip) {
+        String[] ips = ip.split("\\.");
+        return (Integer.parseInt(ips[0]) << 24)
+            | (Integer.parseInt(ips[1]) << 16)
+            | (Integer.parseInt(ips[2]) << 8)
+            | Integer.parseInt(ips[3]);
+    }
+
+    public static boolean ipInCIDR(String ip, String cidr) {
+        int ipAddr = ipToInt(ip);
+        String[] cidrArr = cidr.split("/");
+        int netId = Integer.parseInt(cidrArr[1]);
+        int mask = 0xFFFFFFFF << (32 - netId);
+        int cidrIpAddr = ipToInt(cidrArr[0]);
+
+        return (ipAddr & mask) == (cidrIpAddr & mask);
+    }
 }
diff --git a/remoting/src/main/java/org/apache/rocketmq/remoting/common/RemotingUtil.java b/remoting/src/main/java/org/apache/rocketmq/remoting/common/RemotingUtil.java
index d5ce20b..53301e8 100644
--- a/remoting/src/main/java/org/apache/rocketmq/remoting/common/RemotingUtil.java
+++ b/remoting/src/main/java/org/apache/rocketmq/remoting/common/RemotingUtil.java
@@ -218,13 +218,17 @@ public class RemotingUtil {
 
     public static void closeChannel(Channel channel) {
         final String addrRemote = RemotingHelper.parseChannelRemoteAddr(channel);
-        channel.close().addListener(new ChannelFutureListener() {
-            @Override
-            public void operationComplete(ChannelFuture future) throws Exception {
-                log.info("closeChannel: close the connection to remote address[{}] result: {}", addrRemote,
-                    future.isSuccess());
-            }
-        });
+        if (addrRemote == "") {
+            channel.close();
+        } else {
+            channel.close().addListener(new ChannelFutureListener() {
+                @Override
+                public void operationComplete(ChannelFuture future) throws Exception {
+                    log.info("closeChannel: close the connection to remote address[{}] result: {}", addrRemote,
+                        future.isSuccess());
+                }
+            });
+        }
     }
 
 }
diff --git a/remoting/src/main/java/org/apache/rocketmq/remoting/netty/NettyClientConfig.java b/remoting/src/main/java/org/apache/rocketmq/remoting/netty/NettyClientConfig.java
index c1b9345..2f123db 100644
--- a/remoting/src/main/java/org/apache/rocketmq/remoting/netty/NettyClientConfig.java
+++ b/remoting/src/main/java/org/apache/rocketmq/remoting/netty/NettyClientConfig.java
@@ -38,11 +38,17 @@ public class NettyClientConfig {
     private boolean clientPooledByteBufAllocatorEnable = false;
     private boolean clientCloseSocketIfTimeout = NettySystemConfig.clientCloseSocketIfTimeout;
 
+    private boolean preferredDirectByteBuffer = false;
+    private boolean defaultEventExecutorGroupEnable = true;
+
     private boolean useTLS;
 
     private int writeBufferHighWaterMark = NettySystemConfig.writeBufferHighWaterMark;
     private int writeBufferLowWaterMark = NettySystemConfig.writeBufferLowWaterMark;
 
+    private boolean disableCallbackExecutor = false;
+    private boolean disableNettyWorkerGroup = false;
+
     public boolean isClientCloseSocketIfTimeout() {
         return clientCloseSocketIfTimeout;
     }
@@ -154,4 +160,36 @@ public class NettyClientConfig {
     public void setWriteBufferHighWaterMark(int writeBufferHighWaterMark) {
         this.writeBufferHighWaterMark = writeBufferHighWaterMark;
     }
+
+    public boolean isPreferredDirectByteBuffer() {
+        return preferredDirectByteBuffer;
+    }
+
+    public void setPreferredDirectByteBuffer(final boolean preferredDirectByteBuffer) {
+        this.preferredDirectByteBuffer = preferredDirectByteBuffer;
+    }
+
+    public boolean isDefaultEventExecutorGroupEnable() {
+        return defaultEventExecutorGroupEnable;
+    }
+
+    public void setDefaultEventExecutorGroupEnable(final boolean defaultEventExecutorGroupEnable) {
+        this.defaultEventExecutorGroupEnable = defaultEventExecutorGroupEnable;
+    }
+
+    public boolean isDisableCallbackExecutor() {
+        return disableCallbackExecutor;
+    }
+
+    public void setDisableCallbackExecutor(boolean disableCallbackExecutor) {
+        this.disableCallbackExecutor = disableCallbackExecutor;
+    }
+
+    public boolean isDisableNettyWorkerGroup() {
+        return disableNettyWorkerGroup;
+    }
+
+    public void setDisableNettyWorkerGroup(boolean disableNettyWorkerGroup) {
+        this.disableNettyWorkerGroup = disableNettyWorkerGroup;
+    }
 }
diff --git a/remoting/src/main/java/org/apache/rocketmq/remoting/netty/NettyRemotingAbstract.java b/remoting/src/main/java/org/apache/rocketmq/remoting/netty/NettyRemotingAbstract.java
index f02518b..c6c9178 100644
--- a/remoting/src/main/java/org/apache/rocketmq/remoting/netty/NettyRemotingAbstract.java
+++ b/remoting/src/main/java/org/apache/rocketmq/remoting/netty/NettyRemotingAbstract.java
@@ -88,7 +88,8 @@ public abstract class NettyRemotingAbstract {
     protected final NettyEventExecutor nettyEventExecutor = new NettyEventExecutor();
 
     /**
-     * The default request processor to use in case there is no exact match in {@link #processorTable} per request code.
+     * The default request processor to use in case there is no exact match in {@link #processorTable} per request
+     * code.
      */
     protected Pair<NettyRequestProcessor, ExecutorService> defaultRequestProcessor;
 
@@ -102,7 +103,6 @@ public abstract class NettyRemotingAbstract {
      */
     protected List<RPCHook> rpcHooks = new ArrayList<RPCHook>();
 
-
     static {
         NettyLogger.initNettyLogger();
     }
@@ -168,20 +168,27 @@ public abstract class NettyRemotingAbstract {
 
     protected void doBeforeRpcHooks(String addr, RemotingCommand request) {
         if (rpcHooks.size() > 0) {
-            for (RPCHook rpcHook: rpcHooks) {
+            for (RPCHook rpcHook : rpcHooks) {
                 rpcHook.doBeforeRequest(addr, request);
             }
         }
     }
 
-    protected void doAfterRpcHooks(String addr, RemotingCommand request, RemotingCommand response) {
+    public void doAfterRpcHooks(String addr, RemotingCommand request, RemotingCommand response) {
         if (rpcHooks.size() > 0) {
-            for (RPCHook rpcHook: rpcHooks) {
+            for (RPCHook rpcHook : rpcHooks) {
                 rpcHook.doAfterResponse(addr, request, response);
             }
         }
     }
 
+    public void doAfterRpcFailure(String addr, RemotingCommand request, Boolean remoteTimeout) {
+        if (rpcHooks.size() > 0) {
+            for (RPCHook rpcHook : rpcHooks) {
+                rpcHook.doAfterRpcFailure(addr, request, remoteTimeout);
+            }
+        }
+    }
 
     /**
      * Process incoming request command issued by remote peer.
@@ -198,43 +205,55 @@ public abstract class NettyRemotingAbstract {
             Runnable run = new Runnable() {
                 @Override
                 public void run() {
+                    Exception exception = null;
+                    RemotingCommand response;
+
                     try {
                         String remoteAddr = RemotingHelper.parseChannelRemoteAddr(ctx.channel());
-                        doBeforeRpcHooks(remoteAddr, cmd);
-                        final RemotingResponseCallback callback = new RemotingResponseCallback() {
-                            @Override
-                            public void callback(RemotingCommand response) {
-                                doAfterRpcHooks(remoteAddr, cmd, response);
-                                if (!cmd.isOnewayRPC()) {
-                                    if (response != null) {
-                                        response.setOpaque(opaque);
-                                        response.markResponseType();
-                                        try {
-                                            ctx.writeAndFlush(response);
-                                        } catch (Throwable e) {
-                                            log.error("process request over, but response failed", e);
-                                            log.error(cmd.toString());
-                                            log.error(response.toString());
-                                        }
-                                    } else {
-                                    }
+                        try {
+                            doBeforeRpcHooks(remoteAddr, cmd);
+                        } catch (Exception e) {
+                            exception = e;
+                        }
+
+                        if (exception == null) {
+                            response = pair.getObject1().processRequest(ctx, cmd);
+                        } else {
+                            response = RemotingCommand.createResponseCommand(null);
+                            response.setCode(RemotingSysResponseCode.SYSTEM_ERROR);
+                        }
+
+                        try {
+                            doAfterRpcHooks(remoteAddr, cmd, response);
+                        } catch (Exception e) {
+                            exception = e;
+                        }
+
+                        if (exception != null) {
+                            throw exception;
+                        }
+
+                        if (!cmd.isOnewayRPC()) {
+                            if (response != null) {
+                                response.setOpaque(opaque);
+                                response.markResponseType();
+                                try {
+                                    ctx.writeAndFlush(response);
+                                } catch (Throwable e) {
+                                    log.error("process request over, but response failed", e);
+                                    log.error(cmd.toString());
+                                    log.error(response.toString());
                                 }
+                            } else {
+
                             }
-                        };
-                        if (pair.getObject1() instanceof AsyncNettyRequestProcessor) {
-                            AsyncNettyRequestProcessor processor = (AsyncNettyRequestProcessor)pair.getObject1();
-                            processor.asyncProcessRequest(ctx, cmd, callback);
-                        } else {
-                            NettyRequestProcessor processor = pair.getObject1();
-                            RemotingCommand response = processor.processRequest(ctx, cmd);
-                            callback.callback(response);
                         }
                     } catch (Throwable e) {
                         log.error("process request exception", e);
                         log.error(cmd.toString());
 
                         if (!cmd.isOnewayRPC()) {
-                            final RemotingCommand response = RemotingCommand.createResponseCommand(RemotingSysResponseCode.SYSTEM_ERROR,
+                            response = RemotingCommand.createResponseCommand(RemotingSysResponseCode.SYSTEM_ERROR,
                                 RemotingHelper.exceptionSimpleDesc(e));
                             response.setOpaque(opaque);
                             ctx.writeAndFlush(response);
@@ -344,29 +363,24 @@ public abstract class NettyRemotingAbstract {
         }
     }
 
-
-
-    /**
-     * Custom RPC hook.
-     * Just be compatible with the previous version, use getRPCHooks instead.
-     */
-    @Deprecated
-    protected RPCHook getRPCHook() {
-        if (rpcHooks.size() > 0) {
-            return rpcHooks.get(0);
-        }
-        return null;
-    }
-
     /**
      * Custom RPC hooks.
      *
      * @return RPC hooks if specified; null otherwise.
      */
-    public List<RPCHook> getRPCHooks() {
+    public List<RPCHook> getRPCHook() {
         return rpcHooks;
     }
 
+    public void registerRPCHook(RPCHook rpcHook) {
+        if (rpcHook != null && !rpcHooks.contains(rpcHook)) {
+            rpcHooks.add(rpcHook);
+        }
+    }
+
+    public void clearRPCHook() {
+        rpcHooks.clear();
+    }
 
     /**
      * This method specifies thread pool to use while invoking callback methods.
@@ -481,6 +495,10 @@ public abstract class NettyRemotingAbstract {
                 throw new RemotingSendRequestException(RemotingHelper.parseChannelRemoteAddr(channel), e);
             }
         } else {
+            if (this instanceof NettyRemotingClient) {
+                NettyRemotingClient nettyRemotingClient = (NettyRemotingClient) this;
+                nettyRemotingClient.doAfterRpcFailure(RemotingHelper.parseChannelRemoteAddr(channel), request, false);
+            }
             if (timeoutMillis <= 0) {
                 throw new RemotingTooMuchRequestException("invokeAsyncImpl invoke too fast");
             } else {
@@ -513,13 +531,14 @@ public abstract class NettyRemotingAbstract {
 
     /**
      * mark the request of the specified channel as fail and to invoke fail callback immediately
+     *
      * @param channel the channel which is close already
      */
     protected void failFast(final Channel channel) {
         Iterator<Entry<Integer, ResponseFuture>> it = responseTable.entrySet().iterator();
         while (it.hasNext()) {
             Entry<Integer, ResponseFuture> entry = it.next();
-            if (entry.getValue().getProcessChannel() == channel) {
+            if (entry.getValue().getChannel() == channel) {
                 Integer opaque = entry.getKey();
                 if (opaque != null) {
                     requestFail(opaque);
diff --git a/remoting/src/main/java/org/apache/rocketmq/remoting/netty/NettyRemotingClient.java b/remoting/src/main/java/org/apache/rocketmq/remoting/netty/NettyRemotingClient.java
index ce83769..455d5b4 100644
--- a/remoting/src/main/java/org/apache/rocketmq/remoting/netty/NettyRemotingClient.java
+++ b/remoting/src/main/java/org/apache/rocketmq/remoting/netty/NettyRemotingClient.java
@@ -17,6 +17,7 @@
 package org.apache.rocketmq.remoting.netty;
 
 import io.netty.bootstrap.Bootstrap;
+import io.netty.buffer.PooledByteBufAllocator;
 import io.netty.channel.Channel;
 import io.netty.channel.ChannelDuplexHandler;
 import io.netty.channel.ChannelFuture;
@@ -35,20 +36,26 @@ import io.netty.handler.timeout.IdleState;
 import io.netty.handler.timeout.IdleStateEvent;
 import io.netty.handler.timeout.IdleStateHandler;
 import io.netty.util.concurrent.DefaultEventExecutorGroup;
+import io.netty.util.concurrent.EventExecutorGroup;
 import java.io.IOException;
 import java.net.SocketAddress;
 import java.security.cert.CertificateException;
+import java.util.ArrayList;
 import java.util.Collections;
+import java.util.HashSet;
 import java.util.List;
+import java.util.Set;
 import java.util.Map;
 import java.util.Random;
 import java.util.Timer;
 import java.util.TimerTask;
+import java.util.concurrent.ArrayBlockingQueue;
 import java.util.concurrent.ConcurrentHashMap;
 import java.util.concurrent.ConcurrentMap;
 import java.util.concurrent.ExecutorService;
 import java.util.concurrent.Executors;
 import java.util.concurrent.ThreadFactory;
+import java.util.concurrent.ThreadPoolExecutor;
 import java.util.concurrent.TimeUnit;
 import java.util.concurrent.atomic.AtomicInteger;
 import java.util.concurrent.atomic.AtomicReference;
@@ -58,7 +65,6 @@ import org.apache.rocketmq.logging.InternalLogger;
 import org.apache.rocketmq.logging.InternalLoggerFactory;
 import org.apache.rocketmq.remoting.ChannelEventListener;
 import org.apache.rocketmq.remoting.InvokeCallback;
-import org.apache.rocketmq.remoting.RPCHook;
 import org.apache.rocketmq.remoting.RemotingClient;
 import org.apache.rocketmq.remoting.common.Pair;
 import org.apache.rocketmq.remoting.common.RemotingHelper;
@@ -70,7 +76,7 @@ import org.apache.rocketmq.remoting.exception.RemotingTooMuchRequestException;
 import org.apache.rocketmq.remoting.protocol.RemotingCommand;
 
 public class NettyRemotingClient extends NettyRemotingAbstract implements RemotingClient {
-    private static final InternalLogger log = InternalLoggerFactory.getLogger(RemotingHelper.ROCKETMQ_REMOTING);
+    private static final InternalLogger LOGGER = InternalLoggerFactory.getLogger(RemotingHelper.ROCKETMQ_REMOTING);
 
     private static final long LOCK_TIMEOUT_MILLIS = 3000;
 
@@ -83,18 +89,20 @@ public class NettyRemotingClient extends NettyRemotingAbstract implements Remoti
     private final Timer timer = new Timer("ClientHouseKeepingService", true);
 
     private final AtomicReference<List<String>> namesrvAddrList = new AtomicReference<List<String>>();
+    private final ConcurrentMap<String, Boolean> availableNamesrvAddrMap = new ConcurrentHashMap<String, Boolean>();
     private final AtomicReference<String> namesrvAddrChoosed = new AtomicReference<String>();
     private final AtomicInteger namesrvIndex = new AtomicInteger(initValueIndex());
     private final Lock namesrvChannelLock = new ReentrantLock();
 
     private final ExecutorService publicExecutor;
+    private final ExecutorService scanExecutor;
 
     /**
      * Invoke the callback methods in this executor when process response.
      */
     private ExecutorService callbackExecutor;
     private final ChannelEventListener channelEventListener;
-    private DefaultEventExecutorGroup defaultEventExecutorGroup;
+    private EventExecutorGroup defaultEventExecutorGroup;
 
     public NettyRemotingClient(final NettyClientConfig nettyClientConfig) {
         this(nettyClientConfig, null);
@@ -102,6 +110,13 @@ public class NettyRemotingClient extends NettyRemotingAbstract implements Remoti
 
     public NettyRemotingClient(final NettyClientConfig nettyClientConfig,
         final ChannelEventListener channelEventListener) {
+        this(nettyClientConfig, channelEventListener, null, null);
+    }
+
+    public NettyRemotingClient(final NettyClientConfig nettyClientConfig,
+        final ChannelEventListener channelEventListener,
+        final EventLoopGroup eventLoopGroup,
+        final EventExecutorGroup eventExecutorGroup) {
         super(nettyClientConfig.getClientOnewaySemaphoreValue(), nettyClientConfig.getClientAsyncSemaphoreValue());
         this.nettyClientConfig = nettyClientConfig;
         this.channelEventListener = channelEventListener;
@@ -112,7 +127,7 @@ public class NettyRemotingClient extends NettyRemotingAbstract implements Remoti
         }
 
         this.publicExecutor = Executors.newFixedThreadPool(publicThreadNums, new ThreadFactory() {
-            private AtomicInteger threadIndex = new AtomicInteger(0);
+            private final AtomicInteger threadIndex = new AtomicInteger(0);
 
             @Override
             public Thread newThread(Runnable r) {
@@ -120,23 +135,39 @@ public class NettyRemotingClient extends NettyRemotingAbstract implements Remoti
             }
         });
 
-        this.eventLoopGroupWorker = new NioEventLoopGroup(1, new ThreadFactory() {
-            private AtomicInteger threadIndex = new AtomicInteger(0);
+        this.scanExecutor = new ThreadPoolExecutor(4, 10, 60, TimeUnit.SECONDS,
+            new ArrayBlockingQueue<Runnable>(32), new ThreadFactory() {
+                private final AtomicInteger threadIndex = new AtomicInteger(0);
 
-            @Override
-            public Thread newThread(Runnable r) {
-                return new Thread(r, String.format("NettyClientSelector_%d", this.threadIndex.incrementAndGet()));
+                @Override
+                public Thread newThread(Runnable r) {
+                    return new Thread(r, "NettyClientScan_thread_" + this.threadIndex.incrementAndGet());
+                }
             }
-        });
+        );
+
+        if (eventLoopGroup != null) {
+            this.eventLoopGroupWorker = eventLoopGroup;
+        } else {
+            this.eventLoopGroupWorker = new NioEventLoopGroup(1, new ThreadFactory() {
+                private final AtomicInteger threadIndex = new AtomicInteger(0);
+
+                @Override
+                public Thread newThread(Runnable r) {
+                    return new Thread(r, String.format("NettyClientSelector_%d", this.threadIndex.incrementAndGet()));
+                }
+            });
+        }
+        this.defaultEventExecutorGroup = eventExecutorGroup;
 
         if (nettyClientConfig.isUseTLS()) {
             try {
                 sslContext = TlsHelper.buildSslContext(true);
-                log.info("SSL enabled for client");
+                LOGGER.info("SSL enabled for client");
             } catch (IOException e) {
-                log.error("Failed to create SSLContext", e);
+                LOGGER.error("Failed to create SSLContext", e);
             } catch (CertificateException e) {
-                log.error("Failed to create SSLContext", e);
+                LOGGER.error("Failed to create SSLContext", e);
                 throw new RuntimeException("Failed to create SSLContext", e);
             }
         }
@@ -150,19 +181,19 @@ public class NettyRemotingClient extends NettyRemotingAbstract implements Remoti
 
     @Override
     public void start() {
-        this.defaultEventExecutorGroup = new DefaultEventExecutorGroup(
-            nettyClientConfig.getClientWorkerThreads(),
-            new ThreadFactory() {
-
-                private AtomicInteger threadIndex = new AtomicInteger(0);
-
-                @Override
-                public Thread newThread(Runnable r) {
-                    return new Thread(r, "NettyClientWorkerThread_" + this.threadIndex.incrementAndGet());
-                }
-            });
+        if (this.defaultEventExecutorGroup == null) {
+            this.defaultEventExecutorGroup = new DefaultEventExecutorGroup(
+                nettyClientConfig.getClientWorkerThreads(),
+                new ThreadFactory() {
 
+                    private AtomicInteger threadIndex = new AtomicInteger(0);
 
+                    @Override
+                    public Thread newThread(Runnable r) {
+                        return new Thread(r, "NettyClientWorkerThread_" + this.threadIndex.incrementAndGet());
+                    }
+                });
+        }
         Bootstrap handler = this.bootstrap.group(this.eventLoopGroupWorker).channel(NioSocketChannel.class)
             .option(ChannelOption.TCP_NODELAY, true)
             .option(ChannelOption.SO_KEEPALIVE, false)
@@ -174,49 +205,80 @@ public class NettyRemotingClient extends NettyRemotingAbstract implements Remoti
                     if (nettyClientConfig.isUseTLS()) {
                         if (null != sslContext) {
                             pipeline.addFirst(defaultEventExecutorGroup, "sslHandler", sslContext.newHandler(ch.alloc()));
-                            log.info("Prepend SSL handler");
+                            LOGGER.info("Prepend SSL handler");
                         } else {
-                            log.warn("Connections are insecure as SSLContext is null!");
+                            LOGGER.warn("Connections are insecure as SSLContext is null!");
                         }
                     }
-                    pipeline.addLast(
-                        defaultEventExecutorGroup,
-                        new NettyEncoder(),
-                        new NettyDecoder(),
-                        new IdleStateHandler(0, 0, nettyClientConfig.getClientChannelMaxIdleTimeSeconds()),
-                        new NettyConnectManageHandler(),
+                    if (nettyClientConfig.isDefaultEventExecutorGroupEnable() && !nettyClientConfig.isDisableNettyWorkerGroup()) {
+                        ch.pipeline().addLast(defaultEventExecutorGroup);
+                    }
+                    ch.pipeline().addLast(//
+                        new NettyEncoder(), //
+                        new NettyDecoder(), //
+                        new IdleStateHandler(0, 0, nettyClientConfig.getClientChannelMaxIdleTimeSeconds()), //
+                        new NettyConnectManageHandler(), //
                         new NettyClientHandler());
                 }
             });
         if (nettyClientConfig.getClientSocketSndBufSize() > 0) {
-            log.info("client set SO_SNDBUF to {}", nettyClientConfig.getClientSocketSndBufSize());
+            LOGGER.info("client set SO_SNDBUF to {}", nettyClientConfig.getClientSocketSndBufSize());
             handler.option(ChannelOption.SO_SNDBUF, nettyClientConfig.getClientSocketSndBufSize());
         }
         if (nettyClientConfig.getClientSocketRcvBufSize() > 0) {
-            log.info("client set SO_RCVBUF to {}", nettyClientConfig.getClientSocketRcvBufSize());
+            LOGGER.info("client set SO_RCVBUF to {}", nettyClientConfig.getClientSocketRcvBufSize());
             handler.option(ChannelOption.SO_RCVBUF, nettyClientConfig.getClientSocketRcvBufSize());
         }
         if (nettyClientConfig.getWriteBufferLowWaterMark() > 0 && nettyClientConfig.getWriteBufferHighWaterMark() > 0) {
-            log.info("client set netty WRITE_BUFFER_WATER_MARK to {},{}",
+            LOGGER.info("client set netty WRITE_BUFFER_WATER_MARK to {},{}",
                     nettyClientConfig.getWriteBufferLowWaterMark(), nettyClientConfig.getWriteBufferHighWaterMark());
             handler.option(ChannelOption.WRITE_BUFFER_WATER_MARK, new WriteBufferWaterMark(
                     nettyClientConfig.getWriteBufferLowWaterMark(), nettyClientConfig.getWriteBufferHighWaterMark()));
         }
 
+        if (nettyClientConfig.getClientSocketSndBufSize() != 0) {
+            handler.option(ChannelOption.SO_SNDBUF, nettyClientConfig.getClientSocketSndBufSize());
+        }
+        if (nettyClientConfig.getClientSocketRcvBufSize() != 0) {
+            handler.option(ChannelOption.SO_RCVBUF, nettyClientConfig.getClientSocketRcvBufSize());
+        }
+        if (nettyClientConfig.isClientPooledByteBufAllocatorEnable()) {
+            handler.option(ChannelOption.ALLOCATOR, PooledByteBufAllocator.DEFAULT);
+        }
+
         this.timer.scheduleAtFixedRate(new TimerTask() {
             @Override
             public void run() {
                 try {
                     NettyRemotingClient.this.scanResponseTable();
                 } catch (Throwable e) {
-                    log.error("scanResponseTable exception", e);
+                    LOGGER.error("scanResponseTable exception", e);
                 }
             }
         }, 1000 * 3, 1000);
 
-        if (this.channelEventListener != null) {
-            this.nettyEventExecutor.start();
-        }
+//        this.timer.scheduleAtFixedRate(new TimerTask() {
+//            @Override
+//            public void run() {
+//                try {
+//                    NettyRemotingClient.this.scanChannelTablesOfNameServer();
+//                } catch (Exception e) {
+//                    LOGGER.error("scanChannelTablesOfNameServer exception", e);
+//                }
+//            }
+//        }, 1000 * 3, 10 * 1000);
+
+        this.timer.scheduleAtFixedRate(new TimerTask() {
+            @Override
+            public void run() {
+                try {
+                    NettyRemotingClient.this.scanAvailableNameSrv();
+                } catch (Exception e) {
+                    LOGGER.error("scanAvailableNameSrv exception", e);
+                }
+            }
+        }, 1000 * 3, this.nettyClientConfig.getConnectTimeoutMillis());
+
     }
 
     @Override
@@ -224,8 +286,8 @@ public class NettyRemotingClient extends NettyRemotingAbstract implements Remoti
         try {
             this.timer.cancel();
 
-            for (ChannelWrapper cw : this.channelTables.values()) {
-                this.closeChannel(null, cw.getChannel());
+            for (String addr : this.channelTables.keySet()) {
+                this.closeChannel(addr, this.channelTables.get(addr).getChannel());
             }
 
             this.channelTables.clear();
@@ -240,21 +302,30 @@ public class NettyRemotingClient extends NettyRemotingAbstract implements Remoti
                 this.defaultEventExecutorGroup.shutdownGracefully();
             }
         } catch (Exception e) {
-            log.error("NettyRemotingClient shutdown exception, ", e);
+            LOGGER.error("NettyRemotingClient shutdown exception, ", e);
         }
 
         if (this.publicExecutor != null) {
             try {
                 this.publicExecutor.shutdown();
             } catch (Exception e) {
-                log.error("NettyRemotingServer shutdown exception, ", e);
+                LOGGER.error("NettyRemotingServer shutdown exception, ", e);
+            }
+        }
+
+        if (this.scanExecutor != null) {
+            try {
+                this.scanExecutor.shutdown();
+            } catch (Exception e) {
+                LOGGER.error("NettyRemotingServer shutdown exception, ", e);
             }
         }
     }
 
     public void closeChannel(final String addr, final Channel channel) {
-        if (null == channel)
+        if (null == channel) {
             return;
+        }
 
         final String addrRemote = null == addr ? RemotingHelper.parseChannelRemoteAddr(channel) : addr;
 
@@ -264,46 +335,40 @@ public class NettyRemotingClient extends NettyRemotingAbstract implements Remoti
                     boolean removeItemFromTable = true;
                     final ChannelWrapper prevCW = this.channelTables.get(addrRemote);
 
-                    log.info("closeChannel: begin close the channel[{}] Found: {}", addrRemote, prevCW != null);
+                    LOGGER.info("closeChannel: begin close the channel[{}] Found: {}", addrRemote, prevCW != null);
 
                     if (null == prevCW) {
-                        log.info("closeChannel: the channel[{}] has been removed from the channel table before", addrRemote);
+                        LOGGER.info("closeChannel: the channel[{}] has been removed from the channel table before", addrRemote);
                         removeItemFromTable = false;
                     } else if (prevCW.getChannel() != channel) {
-                        log.info("closeChannel: the channel[{}] has been closed before, and has been created again, nothing to do.",
+                        LOGGER.info("closeChannel: the channel[{}] has been closed before, and has been created again, nothing to do.",
                             addrRemote);
                         removeItemFromTable = false;
                     }
 
                     if (removeItemFromTable) {
                         this.channelTables.remove(addrRemote);
-                        log.info("closeChannel: the channel[{}] was removed from channel table", addrRemote);
+                        LOGGER.info("closeChannel: the channel[{}] was removed from channel table", addrRemote);
                     }
 
                     RemotingUtil.closeChannel(channel);
                 } catch (Exception e) {
-                    log.error("closeChannel: close the channel exception", e);
+                    LOGGER.error("closeChannel: close the channel exception", e);
                 } finally {
                     this.lockChannelTables.unlock();
                 }
             } else {
-                log.warn("closeChannel: try to lock channel table, but timeout, {}ms", LOCK_TIMEOUT_MILLIS);
+                LOGGER.warn("closeChannel: try to lock channel table, but timeout, {}ms", LOCK_TIMEOUT_MILLIS);
             }
         } catch (InterruptedException e) {
-            log.error("closeChannel exception", e);
-        }
-    }
-
-    @Override
-    public void registerRPCHook(RPCHook rpcHook) {
-        if (rpcHook != null && !rpcHooks.contains(rpcHook)) {
-            rpcHooks.add(rpcHook);
+            LOGGER.error("closeChannel exception", e);
         }
     }
 
     public void closeChannel(final Channel channel) {
-        if (null == channel)
+        if (null == channel) {
             return;
+        }
 
         try {
             if (this.lockChannelTables.tryLock(LOCK_TIMEOUT_MILLIS, TimeUnit.MILLISECONDS)) {
@@ -324,25 +389,25 @@ public class NettyRemotingClient extends NettyRemotingAbstract implements Remoti
                     }
 
                     if (null == prevCW) {
-                        log.info("eventCloseChannel: the channel[{}] has been removed from the channel table before", addrRemote);
+                        LOGGER.info("eventCloseChannel: the channel[{}] has been removed from the channel table before", addrRemote);
                         removeItemFromTable = false;
                     }
 
                     if (removeItemFromTable) {
                         this.channelTables.remove(addrRemote);
-                        log.info("closeChannel: the channel[{}] was removed from channel table", addrRemote);
+                        LOGGER.info("closeChannel: the channel[{}] was removed from channel table", addrRemote);
                         RemotingUtil.closeChannel(channel);
                     }
                 } catch (Exception e) {
-                    log.error("closeChannel: close the channel exception", e);
+                    LOGGER.error("closeChannel: close the channel exception", e);
                 } finally {
                     this.lockChannelTables.unlock();
                 }
             } else {
-                log.warn("closeChannel: try to lock channel table, but timeout, {}ms", LOCK_TIMEOUT_MILLIS);
+                LOGGER.warn("closeChannel: try to lock channel table, but timeout, {}ms", LOCK_TIMEOUT_MILLIS);
             }
         } catch (InterruptedException e) {
-            log.error("closeChannel exception", e);
+            LOGGER.error("closeChannel exception", e);
         }
     }
 
@@ -366,11 +431,20 @@ public class NettyRemotingClient extends NettyRemotingAbstract implements Remoti
 
             if (update) {
                 Collections.shuffle(addrs);
-                log.info("name server address updated. NEW : {} , OLD: {}", addrs, old);
+                LOGGER.info("name server address updated. NEW : {} , OLD: {}", addrs, old);
                 this.namesrvAddrList.set(addrs);
 
-                if (!addrs.contains(this.namesrvAddrChoosed.get())) {
-                    this.namesrvAddrChoosed.set(null);
+                // should close the channel if choosed addr is not exist.
+                if (this.namesrvAddrChoosed.get() != null && !addrs.contains(this.namesrvAddrChoosed.get())) {
+                    String namesrvAddr = this.namesrvAddrChoosed.get();
+                    for (String addr : this.channelTables.keySet()) {
+                        if (addr.contains(namesrvAddr)) {
+                            ChannelWrapper channelWrapper = this.channelTables.get(addr);
+                            if (channelWrapper != null) {
+                                closeChannel(channelWrapper.getChannel());
+                            }
+                        }
+                    }
                 }
             }
         }
@@ -390,17 +464,20 @@ public class NettyRemotingClient extends NettyRemotingAbstract implements Remoti
                 }
                 RemotingCommand response = this.invokeSyncImpl(channel, request, timeoutMillis - costTime);
                 doAfterRpcHooks(RemotingHelper.parseChannelRemoteAddr(channel), request, response);
+                this.updateChannelLastResponseTime(addr);
                 return response;
             } catch (RemotingSendRequestException e) {
-                log.warn("invokeSync: send request exception, so close the channel[{}]", addr);
+                LOGGER.warn("invokeSync: send request exception, so close the channel[{}]", addr);
+                doAfterRpcFailure(addr, request, false);
                 this.closeChannel(addr, channel);
                 throw e;
             } catch (RemotingTimeoutException e) {
                 if (nettyClientConfig.isClientCloseSocketIfTimeout()) {
                     this.closeChannel(addr, channel);
-                    log.warn("invokeSync: close socket because of timeout, {}ms, {}", timeoutMillis, addr);
+                    LOGGER.warn("invokeSync: close socket because of timeout, {}ms, {}", timeoutMillis, addr);
                 }
-                log.warn("invokeSync: wait response timeout exception, the channel[{}]", addr);
+                doAfterRpcFailure(addr, request, true);
+                LOGGER.warn("invokeSync: wait response timeout exception, the channel[{}]", addr);
                 throw e;
             }
         } else {
@@ -409,7 +486,54 @@ public class NettyRemotingClient extends NettyRemotingAbstract implements Remoti
         }
     }
 
-    private Channel getAndCreateChannel(final String addr) throws RemotingConnectException, InterruptedException {
+    @Override
+    public void closeChannels() {
+        closeChannels(new ArrayList<String>(this.channelTables.keySet()));
+    }
+
+    @Override
+    public void closeChannels(List<String> addrList) {
+        for (String addr : addrList) {
+            ChannelWrapper cw = this.channelTables.get(addr);
+            if (cw == null) {
+                continue;
+            }
+            this.closeChannel(addr, cw.getChannel());
+        }
+        interruptPullRequests(new HashSet<String>(addrList));
+    }
+
+    private void interruptPullRequests(Set<String> brokerAddrSet) {
+        for (ResponseFuture responseFuture : responseTable.values()) {
+            RemotingCommand cmd = responseFuture.getRequestCommand();
+            if (cmd == null) {
+                continue;
+            }
+            String remoteAddr = RemotingHelper.parseChannelRemoteAddr(responseFuture.getChannel());
+            // interrupt only pull message request
+            if (brokerAddrSet.contains(remoteAddr) && (cmd.getCode() == 11 || cmd.getCode() == 361)) {
+                LOGGER.info("interrupt {}", cmd);
+                responseFuture.interrupt();
+            }
+        }
+    }
+
+    private void updateChannelLastResponseTime(final String addr) {
+        String address = addr;
+        if (address == null) {
+            address = this.namesrvAddrChoosed.get();
+        }
+        if (address == null) {
+            LOGGER.warn("[updateChannelLastResponseTime] could not find address!!");
+            return;
+        }
+        ChannelWrapper channelWrapper = this.channelTables.get(address);
+        if (channelWrapper != null && channelWrapper.isOK()) {
+            channelWrapper.updateLastResponseTime();
+        }
+    }
+
+    private Channel getAndCreateChannel(final String addr) throws InterruptedException {
         if (null == addr) {
             return getAndCreateNameserverChannel();
         }
@@ -422,7 +546,7 @@ public class NettyRemotingClient extends NettyRemotingAbstract implements Remoti
         return this.createChannel(addr);
     }
 
-    private Channel getAndCreateNameserverChannel() throws RemotingConnectException, InterruptedException {
+    private Channel getAndCreateNameserverChannel() throws InterruptedException {
         String addr = this.namesrvAddrChoosed.get();
         if (addr != null) {
             ChannelWrapper cw = this.channelTables.get(addr);
@@ -450,7 +574,7 @@ public class NettyRemotingClient extends NettyRemotingAbstract implements Remoti
                         String newAddr = addrList.get(index);
 
                         this.namesrvAddrChoosed.set(newAddr);
-                        log.info("new name server is chosen. OLD: {} , NEW: {}. namesrvIndex = {}", addr, newAddr, namesrvIndex);
+                        LOGGER.info("new name server is chosen. OLD: {} , NEW: {}. namesrvIndex = {}", addr, newAddr, namesrvIndex);
                         Channel channelNew = this.createChannel(newAddr);
                         if (channelNew != null) {
                             return channelNew;
@@ -458,11 +582,13 @@ public class NettyRemotingClient extends NettyRemotingAbstract implements Remoti
                     }
                     throw new RemotingConnectException(addrList.toString());
                 }
+            } catch (Exception e) {
+                LOGGER.error("getAndCreateNameserverChannel: create name server channel exception", e);
             } finally {
                 this.namesrvChannelLock.unlock();
             }
         } else {
-            log.warn("getAndCreateNameserverChannel: try to lock name server, but timeout, {}ms", LOCK_TIMEOUT_MILLIS);
+            LOGGER.warn("getAndCreateNameserverChannel: try to lock name server, but timeout, {}ms", LOCK_TIMEOUT_MILLIS);
         }
 
         return null;
@@ -494,30 +620,30 @@ public class NettyRemotingClient extends NettyRemotingAbstract implements Remoti
 
                 if (createNewConnection) {
                     ChannelFuture channelFuture = this.bootstrap.connect(RemotingHelper.string2SocketAddress(addr));
-                    log.info("createChannel: begin to connect remote host[{}] asynchronously", addr);
+                    LOGGER.info("createChannel: begin to connect remote host[{}] asynchronously", addr);
                     cw = new ChannelWrapper(channelFuture);
                     this.channelTables.put(addr, cw);
                 }
             } catch (Exception e) {
-                log.error("createChannel: create channel exception", e);
+                LOGGER.error("createChannel: create channel exception", e);
             } finally {
                 this.lockChannelTables.unlock();
             }
         } else {
-            log.warn("createChannel: try to lock channel table, but timeout, {}ms", LOCK_TIMEOUT_MILLIS);
+            LOGGER.warn("createChannel: try to lock channel table, but timeout, {}ms", LOCK_TIMEOUT_MILLIS);
         }
 
         if (cw != null) {
             ChannelFuture channelFuture = cw.getChannelFuture();
             if (channelFuture.awaitUninterruptibly(this.nettyClientConfig.getConnectTimeoutMillis())) {
                 if (cw.isOK()) {
-                    log.info("createChannel: connect remote host[{}] success, {}", addr, channelFuture.toString());
+                    LOGGER.info("createChannel: connect remote host[{}] success, {}", addr, channelFuture.toString());
                     return cw.getChannel();
                 } else {
-                    log.warn("createChannel: connect remote host[" + addr + "] failed, " + channelFuture.toString(), channelFuture.cause());
+                    LOGGER.warn("createChannel: connect remote host[" + addr + "] failed, " + channelFuture.toString());
                 }
             } else {
-                log.warn("createChannel: connect remote host[{}] timeout {}ms, {}", addr, this.nettyClientConfig.getConnectTimeoutMillis(),
+                LOGGER.warn("createChannel: connect remote host[{}] timeout {}ms, {}", addr, this.nettyClientConfig.getConnectTimeoutMillis(),
                     channelFuture.toString());
             }
         }
@@ -538,9 +664,9 @@ public class NettyRemotingClient extends NettyRemotingAbstract implements Remoti
                 if (timeoutMillis < costTime) {
                     throw new RemotingTooMuchRequestException("invokeAsync call the addr[" + addr + "] timeout");
                 }
-                this.invokeAsyncImpl(channel, request, timeoutMillis - costTime, invokeCallback);
+                this.invokeAsyncImpl(channel, request, timeoutMillis - costTime, new InvokeCallbackWrapper(invokeCallback, addr));
             } catch (RemotingSendRequestException e) {
-                log.warn("invokeAsync: send request exception, so close the channel[{}]", addr);
+                LOGGER.warn("invokeAsync: send request exception, so close the channel[{}]", addr);
                 this.closeChannel(addr, channel);
                 throw e;
             }
@@ -559,7 +685,8 @@ public class NettyRemotingClient extends NettyRemotingAbstract implements Remoti
                 doBeforeRpcHooks(addr, request);
                 this.invokeOnewayImpl(channel, request, timeoutMillis);
             } catch (RemotingSendRequestException e) {
-                log.warn("invokeOneway: send request exception, so close the channel[{}]", addr);
+                LOGGER.warn("invokeOneway: send request exception, so close the channel[{}]", addr);
+                doAfterRpcFailure(addr, request, false);
                 this.closeChannel(addr, channel);
                 throw e;
             }
@@ -595,12 +722,20 @@ public class NettyRemotingClient extends NettyRemotingAbstract implements Remoti
     }
 
     @Override
+    public List<String> getAvailableNameSrvList() {
+        return new ArrayList<String>(this.availableNamesrvAddrMap.keySet());
+    }
+
+    @Override
     public ChannelEventListener getChannelEventListener() {
         return channelEventListener;
     }
 
     @Override
     public ExecutorService getCallbackExecutor() {
+        if (nettyClientConfig.isDisableCallbackExecutor()) {
+            return null;
+        }
         return callbackExecutor != null ? callbackExecutor : publicExecutor;
     }
 
@@ -609,11 +744,67 @@ public class NettyRemotingClient extends NettyRemotingAbstract implements Remoti
         this.callbackExecutor = callbackExecutor;
     }
 
+    @Override
+    public ConcurrentMap<Integer, ResponseFuture> getResponseTable() {
+        return this.responseTable;
+    }
+
+    protected void scanChannelTablesOfNameServer() {
+        List<String> nameServerList = this.namesrvAddrList.get();
+        if (nameServerList == null) {
+            LOGGER.warn("[SCAN] Addresses of name server is empty!");
+            return;
+        }
+
+        for (String addr : this.channelTables.keySet()) {
+            ChannelWrapper channelWrapper = this.channelTables.get(addr);
+            if (channelWrapper == null) {
+                continue;
+            }
+
+            if ((System.currentTimeMillis() - channelWrapper.getLastResponseTime()) > this.nettyClientConfig.getChannelNotActiveInterval()) {
+                LOGGER.warn("[SCAN] No response after {} from name server {}, so close it!", channelWrapper.getLastResponseTime(),
+                    addr);
+                closeChannel(addr, channelWrapper.getChannel());
+            }
+        }
+    }
+
+    private void scanAvailableNameSrv() {
+        List<String> nameServerList = this.namesrvAddrList.get();
+        if (nameServerList == null) {
+            LOGGER.warn("scanAvailableNameSrv Addresses of name server is empty!");
+            return;
+        }
+
+        for (final String namesrvAddr : nameServerList) {
+            scanExecutor.execute(new Runnable() {
+                @Override
+                public void run() {
+                    try {
+                        Channel channel = NettyRemotingClient.this.getAndCreateChannel(namesrvAddr);
+                        if (channel != null) {
+                            NettyRemotingClient.this.availableNamesrvAddrMap.putIfAbsent(namesrvAddr, true);
+                        } else {
+                            NettyRemotingClient.this.availableNamesrvAddrMap.remove(namesrvAddr);
+                        }
+                    } catch (Exception e) {
+                        LOGGER.error("scanAvailableNameSrv get channel of {} failed, ", namesrvAddr, e);
+                    }
+                }
+            });
+        }
+
+    }
+
     static class ChannelWrapper {
         private final ChannelFuture channelFuture;
+        // only affected by sync or async request, oneway is not included.
+        private long lastResponseTime;
 
         public ChannelWrapper(ChannelFuture channelFuture) {
             this.channelFuture = channelFuture;
+            this.lastResponseTime = System.currentTimeMillis();
         }
 
         public boolean isOK() {
@@ -631,6 +822,33 @@ public class NettyRemotingClient extends NettyRemotingAbstract implements Remoti
         public ChannelFuture getChannelFuture() {
             return channelFuture;
         }
+
+        public long getLastResponseTime() {
+            return this.lastResponseTime;
+        }
+
+        public void updateLastResponseTime() {
+            this.lastResponseTime = System.currentTimeMillis();
+        }
+    }
+
+    class InvokeCallbackWrapper implements InvokeCallback {
+
+        private final InvokeCallback invokeCallback;
+        private final String addr;
+
+        public InvokeCallbackWrapper(InvokeCallback invokeCallback, String addr) {
+            this.invokeCallback = invokeCallback;
+            this.addr = addr;
+        }
+
+        @Override
+        public void operationComplete(ResponseFuture responseFuture) {
+            if (responseFuture != null && responseFuture.isSendRequestOK() && responseFuture.getResponseCommand() != null) {
+                NettyRemotingClient.this.updateChannelLastResponseTime(addr);
+            }
+            this.invokeCallback.operationComplete(responseFuture);
+        }
     }
 
     class NettyClientHandler extends SimpleChannelInboundHandler<RemotingCommand> {
@@ -647,7 +865,7 @@ public class NettyRemotingClient extends NettyRemotingAbstract implements Remoti
             ChannelPromise promise) throws Exception {
             final String local = localAddress == null ? "UNKNOWN" : RemotingHelper.parseSocketAddressAddr(localAddress);
             final String remote = remoteAddress == null ? "UNKNOWN" : RemotingHelper.parseSocketAddressAddr(remoteAddress);
-            log.info("NETTY CLIENT PIPELINE: CONNECT  {} => {}", local, remote);
+            LOGGER.info("NETTY CLIENT PIPELINE: CONNECT  {} => {}", local, remote);
 
             super.connect(ctx, remoteAddress, localAddress, promise);
 
@@ -659,7 +877,7 @@ public class NettyRemotingClient extends NettyRemotingAbstract implements Remoti
         @Override
         public void disconnect(ChannelHandlerContext ctx, ChannelPromise promise) throws Exception {
             final String remoteAddress = RemotingHelper.parseChannelRemoteAddr(ctx.channel());
-            log.info("NETTY CLIENT PIPELINE: DISCONNECT {}", remoteAddress);
+            LOGGER.info("NETTY CLIENT PIPELINE: DISCONNECT {}", remoteAddress);
             closeChannel(ctx.channel());
             super.disconnect(ctx, promise);
 
@@ -671,7 +889,7 @@ public class NettyRemotingClient extends NettyRemotingAbstract implements Remoti
         @Override
         public void close(ChannelHandlerContext ctx, ChannelPromise promise) throws Exception {
             final String remoteAddress = RemotingHelper.parseChannelRemoteAddr(ctx.channel());
-            log.info("NETTY CLIENT PIPELINE: CLOSE {}", remoteAddress);
+            LOGGER.info("NETTY CLIENT PIPELINE: CLOSE {}", remoteAddress);
             closeChannel(ctx.channel());
             super.close(ctx, promise);
             NettyRemotingClient.this.failFast(ctx.channel());
@@ -686,7 +904,7 @@ public class NettyRemotingClient extends NettyRemotingAbstract implements Remoti
                 IdleStateEvent event = (IdleStateEvent) evt;
                 if (event.state().equals(IdleState.ALL_IDLE)) {
                     final String remoteAddress = RemotingHelper.parseChannelRemoteAddr(ctx.channel());
-                    log.warn("NETTY CLIENT PIPELINE: IDLE exception [{}]", remoteAddress);
+                    LOGGER.warn("NETTY CLIENT PIPELINE: IDLE exception [{}]", remoteAddress);
                     closeChannel(ctx.channel());
                     if (NettyRemotingClient.this.channelEventListener != null) {
                         NettyRemotingClient.this
@@ -701,8 +919,8 @@ public class NettyRemotingClient extends NettyRemotingAbstract implements Remoti
         @Override
         public void exceptionCaught(ChannelHandlerContext ctx, Throwable cause) throws Exception {
             final String remoteAddress = RemotingHelper.parseChannelRemoteAddr(ctx.channel());
-            log.warn("NETTY CLIENT PIPELINE: exceptionCaught {}", remoteAddress);
-            log.warn("NETTY CLIENT PIPELINE: exceptionCaught exception.", cause);
+            LOGGER.warn("NETTY CLIENT PIPELINE: exceptionCaught {}", remoteAddress);
+            LOGGER.warn("NETTY CLIENT PIPELINE: exceptionCaught exception.", cause);
             closeChannel(ctx.channel());
             if (NettyRemotingClient.this.channelEventListener != null) {
                 NettyRemotingClient.this.putNettyEvent(new NettyEvent(NettyEventType.EXCEPTION, remoteAddress, ctx.channel()));
diff --git a/remoting/src/main/java/org/apache/rocketmq/remoting/netty/NettyRemotingServer.java b/remoting/src/main/java/org/apache/rocketmq/remoting/netty/NettyRemotingServer.java
index 22440af..394b536 100644
--- a/remoting/src/main/java/org/apache/rocketmq/remoting/netty/NettyRemotingServer.java
+++ b/remoting/src/main/java/org/apache/rocketmq/remoting/netty/NettyRemotingServer.java
@@ -45,15 +45,17 @@ import java.security.cert.CertificateException;
 import java.util.NoSuchElementException;
 import java.util.Timer;
 import java.util.TimerTask;
+import java.util.concurrent.ConcurrentHashMap;
+import java.util.concurrent.ConcurrentMap;
 import java.util.concurrent.ExecutorService;
 import java.util.concurrent.Executors;
 import java.util.concurrent.ThreadFactory;
+import java.util.concurrent.TimeUnit;
 import java.util.concurrent.atomic.AtomicInteger;
 import org.apache.rocketmq.logging.InternalLogger;
 import org.apache.rocketmq.logging.InternalLoggerFactory;
 import org.apache.rocketmq.remoting.ChannelEventListener;
 import org.apache.rocketmq.remoting.InvokeCallback;
-import org.apache.rocketmq.remoting.RPCHook;
 import org.apache.rocketmq.remoting.RemotingServer;
 import org.apache.rocketmq.remoting.common.Pair;
 import org.apache.rocketmq.remoting.common.RemotingHelper;
@@ -77,6 +79,11 @@ public class NettyRemotingServer extends NettyRemotingAbstract implements Remoti
     private final Timer timer = new Timer("ServerHouseKeepingService", true);
     private DefaultEventExecutorGroup defaultEventExecutorGroup;
 
+    /**
+     * NettyRemotingServer may holds multiple SubRemotingServer, each server will be stored in this container wih a
+     * ListenPort key.
+     */
+    private ConcurrentMap<Integer/*Port*/, NettyRemotingAbstract> remotingServerTable = new ConcurrentHashMap<Integer, NettyRemotingAbstract>();
 
     private int port = 0;
 
@@ -156,6 +163,8 @@ public class NettyRemotingServer extends NettyRemotingAbstract implements Remoti
         }
 
         loadSslContext();
+
+        this.remotingServerTable.put(this.nettyServerConfig.getListenPort(), this);
     }
 
     public void loadSslContext() {
@@ -199,7 +208,7 @@ public class NettyRemotingServer extends NettyRemotingAbstract implements Remoti
         ServerBootstrap childHandler =
             this.serverBootstrap.group(this.eventLoopGroupBoss, this.eventLoopGroupSelector)
                 .channel(useEpoll() ? EpollServerSocketChannel.class : NioServerSocketChannel.class)
-                .option(ChannelOption.SO_BACKLOG, nettyServerConfig.getServerSocketBacklog())
+                .option(ChannelOption.SO_BACKLOG, 1024)
                 .option(ChannelOption.SO_REUSEADDR, true)
                 .option(ChannelOption.SO_KEEPALIVE, false)
                 .childOption(ChannelOption.TCP_NODELAY, true)
@@ -294,13 +303,6 @@ public class NettyRemotingServer extends NettyRemotingAbstract implements Remoti
     }
 
     @Override
-    public void registerRPCHook(RPCHook rpcHook) {
-        if (rpcHook != null && !rpcHooks.contains(rpcHook)) {
-            rpcHooks.add(rpcHook);
-        }
-    }
-
-    @Override
     public void registerProcessor(int requestCode, NettyRequestProcessor processor, ExecutorService executor) {
         ExecutorService executorThis = executor;
         if (null == executor) {
@@ -327,6 +329,28 @@ public class NettyRemotingServer extends NettyRemotingAbstract implements Remoti
     }
 
     @Override
+    public Pair<NettyRequestProcessor, ExecutorService> getDefaultProcessorPair() {
+        return defaultRequestProcessor;
+    }
+
+    @Override
+    public RemotingServer newRemotingServer(final int port) {
+        SubRemotingServer remotingServer = new SubRemotingServer(port,
+            this.nettyServerConfig.getServerOnewaySemaphoreValue(),
+            this.nettyServerConfig.getServerAsyncSemaphoreValue());
+        NettyRemotingAbstract existingServer = this.remotingServerTable.putIfAbsent(port, remotingServer);
+        if (existingServer != null) {
+            throw new RuntimeException("The port " + port + " already in use by another RemotingServer");
+        }
+        return remotingServer;
+    }
+
+    @Override
+    public void removeRemotingServer(final int port) {
+        this.remotingServerTable.remove(port);
+    }
+
+    @Override
     public RemotingCommand invokeSync(final Channel channel, final RemotingCommand request, final long timeoutMillis)
         throws InterruptedException, RemotingSendRequestException, RemotingTimeoutException {
         return this.invokeSyncImpl(channel, request, timeoutMillis);
@@ -349,7 +373,6 @@ public class NettyRemotingServer extends NettyRemotingAbstract implements Remoti
         return channelEventListener;
     }
 
-
     @Override
     public ExecutorService getCallbackExecutor() {
         return this.publicExecutor;
@@ -430,7 +453,14 @@ public class NettyRemotingServer extends NettyRemotingAbstract implements Remoti
 
         @Override
         protected void channelRead0(ChannelHandlerContext ctx, RemotingCommand msg) throws Exception {
-            processMessageReceived(ctx, msg);
+            int localPort = RemotingHelper.parseSocketAddressPort(ctx.channel().localAddress());
+            NettyRemotingAbstract remotingAbstract = NettyRemotingServer.this.remotingServerTable.get(localPort);
+            if (localPort != -1 && remotingAbstract != null) {
+                remotingAbstract.processMessageReceived(ctx, msg);
+                return;
+            }
+            // The related remoting server has been shutdown, so close the connected channel
+            RemotingUtil.closeChannel(ctx.channel());
         }
     }
 
@@ -503,4 +533,110 @@ public class NettyRemotingServer extends NettyRemotingAbstract implements Remoti
             RemotingUtil.closeChannel(ctx.channel());
         }
     }
+
+    /**
+     * The NettyRemotingServer supports bind multiple ports, each port bound by a SubRemotingServer. The
+     * SubRemotingServer will delegate all the functions to NettyRemotingServer, so the sub server can share all the
+     * resources from its parent server.
+     */
+    class SubRemotingServer extends NettyRemotingAbstract implements RemotingServer {
+        private final int listenPort;
+        private volatile Channel serverChannel;
+
+        SubRemotingServer(final int port, final int permitsOnway, final int permitsAsync) {
+            super(permitsOnway, permitsAsync);
+            listenPort = port;
+        }
+
+        @Override
+        public void registerProcessor(final int requestCode, final NettyRequestProcessor processor,
+            final ExecutorService executor) {
+            ExecutorService executorThis = executor;
+            if (null == executor) {
+                executorThis = NettyRemotingServer.this.publicExecutor;
+            }
+
+            Pair<NettyRequestProcessor, ExecutorService> pair = new Pair<NettyRequestProcessor, ExecutorService>(processor, executorThis);
+            this.processorTable.put(requestCode, pair);
+        }
+
+        @Override
+        public void registerDefaultProcessor(final NettyRequestProcessor processor, final ExecutorService executor) {
+            this.defaultRequestProcessor = new Pair<NettyRequestProcessor, ExecutorService>(processor, executor);
+        }
+
+        @Override
+        public int localListenPort() {
+            return listenPort;
+        }
+
+        @Override
+        public Pair<NettyRequestProcessor, ExecutorService> getProcessorPair(final int requestCode) {
+            return this.processorTable.get(requestCode);
+        }
+
+        @Override
+        public Pair<NettyRequestProcessor, ExecutorService> getDefaultProcessorPair() {
+            return this.defaultRequestProcessor;
+        }
+
+        @Override
+        public RemotingServer newRemotingServer(final int port) {
+            throw new UnsupportedOperationException("The SubRemotingServer of NettyRemotingServer " +
+                "doesn't support new nested RemotingServer");
+        }
+
+        @Override
+        public void removeRemotingServer(final int port) {
+            throw new UnsupportedOperationException("The SubRemotingServer of NettyRemotingServer " +
+                "doesn't support remove nested RemotingServer");
+        }
+
+        @Override
+        public RemotingCommand invokeSync(final Channel channel, final RemotingCommand request,
+            final long timeoutMillis) throws InterruptedException, RemotingSendRequestException, RemotingTimeoutException {
+            return this.invokeSyncImpl(channel, request, timeoutMillis);
+        }
+
+        @Override
+        public void invokeAsync(final Channel channel, final RemotingCommand request, final long timeoutMillis,
+            final InvokeCallback invokeCallback) throws InterruptedException, RemotingTooMuchRequestException, RemotingTimeoutException, RemotingSendRequestException {
+            this.invokeAsyncImpl(channel, request, timeoutMillis, invokeCallback);
+        }
+
+        @Override
+        public void invokeOneway(final Channel channel, final RemotingCommand request,
+            final long timeoutMillis) throws InterruptedException, RemotingTooMuchRequestException, RemotingTimeoutException, RemotingSendRequestException {
+            this.invokeOnewayImpl(channel, request, timeoutMillis);
+        }
+
+        @Override
+        public void start() {
+            try {
+                this.serverChannel = NettyRemotingServer.this.serverBootstrap.bind(listenPort).sync().channel();
+            } catch (InterruptedException e) {
+                throw new RuntimeException("this.subRemotingServer.serverBootstrap.bind().sync() InterruptedException", e);
+            }
+        }
+
+        @Override
+        public void shutdown() {
+            if (this.serverChannel != null) {
+                try {
+                    this.serverChannel.close().await(5, TimeUnit.SECONDS);
+                } catch (InterruptedException ignored) {
+                }
+            }
+        }
+
+        @Override
+        public ChannelEventListener getChannelEventListener() {
+            return NettyRemotingServer.this.getChannelEventListener();
+        }
+
+        @Override
+        public ExecutorService getCallbackExecutor() {
+            return NettyRemotingServer.this.getCallbackExecutor();
+        }
+    }
 }
diff --git a/remoting/src/main/java/org/apache/rocketmq/remoting/netty/NettyRequestProcessor.java b/remoting/src/main/java/org/apache/rocketmq/remoting/netty/NettyRequestProcessor.java
index 4800689..040f768 100644
--- a/remoting/src/main/java/org/apache/rocketmq/remoting/netty/NettyRequestProcessor.java
+++ b/remoting/src/main/java/org/apache/rocketmq/remoting/netty/NettyRequestProcessor.java
@@ -27,5 +27,4 @@ public interface NettyRequestProcessor {
         throws Exception;
 
     boolean rejectRequest();
-
 }
diff --git a/remoting/src/main/java/org/apache/rocketmq/remoting/netty/ResponseFuture.java b/remoting/src/main/java/org/apache/rocketmq/remoting/netty/ResponseFuture.java
index 5f4c8c6..19f705d 100644
--- a/remoting/src/main/java/org/apache/rocketmq/remoting/netty/ResponseFuture.java
+++ b/remoting/src/main/java/org/apache/rocketmq/remoting/netty/ResponseFuture.java
@@ -25,8 +25,9 @@ import org.apache.rocketmq.remoting.common.SemaphoreReleaseOnlyOnce;
 import org.apache.rocketmq.remoting.protocol.RemotingCommand;
 
 public class ResponseFuture {
+    private final Channel channel;
     private final int opaque;
-    private final Channel processChannel;
+    private final RemotingCommand request;
     private final long timeoutMillis;
     private final InvokeCallback invokeCallback;
     private final long beginTimestamp = System.currentTimeMillis();
@@ -38,11 +39,18 @@ public class ResponseFuture {
     private volatile RemotingCommand responseCommand;
     private volatile boolean sendRequestOK = true;
     private volatile Throwable cause;
+    private volatile boolean interrupted = false;
 
     public ResponseFuture(Channel channel, int opaque, long timeoutMillis, InvokeCallback invokeCallback,
-        SemaphoreReleaseOnlyOnce once) {
+                          SemaphoreReleaseOnlyOnce once) {
+        this(channel, opaque, null, timeoutMillis, invokeCallback, once);
+    }
+
+    public ResponseFuture(Channel channel, int opaque, RemotingCommand request, long timeoutMillis, InvokeCallback invokeCallback,
+                          SemaphoreReleaseOnlyOnce once) {
+        this.channel = channel;
         this.opaque = opaque;
-        this.processChannel = channel;
+        this.request = request;
         this.timeoutMillis = timeoutMillis;
         this.invokeCallback = invokeCallback;
         this.once = once;
@@ -56,6 +64,11 @@ public class ResponseFuture {
         }
     }
 
+    public void interrupt() {
+        interrupted = true;
+        executeInvokeCallback();
+    }
+
     public void release() {
         if (this.once != null) {
             this.once.release();
@@ -117,20 +130,23 @@ public class ResponseFuture {
         return opaque;
     }
 
-    public Channel getProcessChannel() {
-        return processChannel;
+    public RemotingCommand getRequestCommand() {
+        return request;
+    }
+
+    public Channel getChannel() {
+        return channel;
+    }
+
+    public boolean isInterrupted() {
+        return interrupted;
     }
 
     @Override
     public String toString() {
-        return "ResponseFuture [responseCommand=" + responseCommand
-            + ", sendRequestOK=" + sendRequestOK
-            + ", cause=" + cause
-            + ", opaque=" + opaque
-            + ", processChannel=" + processChannel
-            + ", timeoutMillis=" + timeoutMillis
-            + ", invokeCallback=" + invokeCallback
-            + ", beginTimestamp=" + beginTimestamp
+        return "ResponseFuture [responseCommand=" + responseCommand + ", sendRequestOK=" + sendRequestOK
+            + ", cause=" + cause + ", opaque=" + opaque + ", timeoutMillis=" + timeoutMillis
+            + ", invokeCallback=" + invokeCallback + ", beginTimestamp=" + beginTimestamp
             + ", countDownLatch=" + countDownLatch + "]";
     }
 }
diff --git a/remoting/src/main/java/org/apache/rocketmq/remoting/protocol/RemotingCommand.java b/remoting/src/main/java/org/apache/rocketmq/remoting/protocol/RemotingCommand.java
index a9e8415..7d614cf 100644
--- a/remoting/src/main/java/org/apache/rocketmq/remoting/protocol/RemotingCommand.java
+++ b/remoting/src/main/java/org/apache/rocketmq/remoting/protocol/RemotingCommand.java
@@ -123,8 +123,7 @@ public class RemotingCommand {
         return createResponseCommand(RemotingSysResponseCode.SYSTEM_ERROR, "not set any response code", classHeader);
     }
 
-    public static RemotingCommand buildErrorResponse(int code, String remark,
-        Class<? extends CommandCustomHeader> classHeader) {
+    public static RemotingCommand buildErrorResponse(int code, String remark, Class<? extends CommandCustomHeader> classHeader) {
         final RemotingCommand response = RemotingCommand.createResponseCommand(classHeader);
         response.setCode(code);
         response.setRemark(remark);
@@ -256,9 +255,8 @@ public class RemotingCommand {
         this.customHeader = customHeader;
     }
 
-    public CommandCustomHeader decodeCommandCustomHeader(
-        Class<? extends CommandCustomHeader> classHeader) throws RemotingCommandException {
-        CommandCustomHeader objectHeader;
+    public <T extends CommandCustomHeader> T decodeCommandCustomHeader(Class<T> classHeader) throws RemotingCommandException {
+        T objectHeader;
         try {
             objectHeader = classHeader.newInstance();
         } catch (InstantiationException e) {
diff --git a/remoting/src/test/java/org/apache/rocketmq/remoting/RemotingServerTest.java b/remoting/src/test/java/org/apache/rocketmq/remoting/RemotingServerTest.java
index e378a7b..a538d4b 100644
--- a/remoting/src/test/java/org/apache/rocketmq/remoting/RemotingServerTest.java
+++ b/remoting/src/test/java/org/apache/rocketmq/remoting/RemotingServerTest.java
@@ -43,15 +43,14 @@ public class RemotingServerTest {
     public static RemotingServer createRemotingServer() throws InterruptedException {
         NettyServerConfig config = new NettyServerConfig();
         RemotingServer remotingServer = new NettyRemotingServer(config);
-        remotingServer.registerProcessor(0, new AsyncNettyRequestProcessor() {
+        remotingServer.registerProcessor(0, new NettyRequestProcessor() {
             @Override
             public RemotingCommand processRequest(ChannelHandlerContext ctx, RemotingCommand request) {
                 request.setRemark("Hi " + ctx.channel().remoteAddress());
                 return request;
             }
 
-            @Override
-            public boolean rejectRequest() {
+            @Override public boolean rejectRequest() {
                 return false;
             }
         }, Executors.newCachedThreadPool());
diff --git a/remoting/src/test/java/org/apache/rocketmq/remoting/SubRemotingServerTest.java b/remoting/src/test/java/org/apache/rocketmq/remoting/SubRemotingServerTest.java
new file mode 100644
index 0000000..c3f4596
--- /dev/null
+++ b/remoting/src/test/java/org/apache/rocketmq/remoting/SubRemotingServerTest.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.remoting;
+
+import io.netty.channel.ChannelHandlerContext;
+import org.apache.rocketmq.remoting.common.RemotingHelper;
+import org.apache.rocketmq.remoting.exception.RemotingConnectException;
+import org.apache.rocketmq.remoting.exception.RemotingSendRequestException;
+import org.apache.rocketmq.remoting.exception.RemotingTimeoutException;
+import org.apache.rocketmq.remoting.netty.NettyRequestProcessor;
+import org.apache.rocketmq.remoting.protocol.RemotingCommand;
+import org.apache.rocketmq.remoting.protocol.RemotingSysResponseCode;
+import org.junit.AfterClass;
+import org.junit.BeforeClass;
+import org.junit.Test;
+
+import static org.assertj.core.api.Assertions.assertThat;
+import static org.assertj.core.api.Fail.failBecauseExceptionWasNotThrown;
+
+public class SubRemotingServerTest {
+    private static final int subServerPort = 1234;
+
+    private static RemotingServer remotingServer;
+    private static RemotingClient remotingClient;
+    private static RemotingServer subServer;
+
+    @BeforeClass
+    public static void setup() throws InterruptedException {
+        remotingServer = RemotingServerTest.createRemotingServer();
+        remotingClient = RemotingServerTest.createRemotingClient();
+        subServer = createSubRemotingServer(remotingServer);
+    }
+
+    @AfterClass
+    public static void destroy() {
+        remotingClient.shutdown();
+        remotingServer.shutdown();
+    }
+
+    public static RemotingServer createSubRemotingServer(RemotingServer parentServer) {
+        RemotingServer subServer = parentServer.newRemotingServer(subServerPort);
+        subServer.registerProcessor(1, new NettyRequestProcessor() {
+            @Override
+            public RemotingCommand processRequest(final ChannelHandlerContext ctx,
+                final RemotingCommand request) throws Exception {
+                request.setRemark(String.valueOf(RemotingHelper.parseSocketAddressPort(ctx.channel().localAddress())));
+                return request;
+            }
+
+            @Override
+            public boolean rejectRequest() {
+                return false;
+            }
+        }, null);
+        subServer.start();
+        return subServer;
+    }
+
+    @Test
+    public void testInvokeSubRemotingServer() throws InterruptedException, RemotingTimeoutException, RemotingConnectException, RemotingSendRequestException {
+        RequestHeader requestHeader = new RequestHeader();
+        requestHeader.setCount(1);
+        requestHeader.setMessageTitle("Welcome");
+
+        // Parent remoting server doesn't support RequestCode 1
+        RemotingCommand request = RemotingCommand.createRequestCommand(1, requestHeader);
+        RemotingCommand response = remotingClient.invokeSync("localhost:8888", request, 1000 * 3);
+        assertThat(response).isNotNull();
+        assertThat(response.getCode()).isEqualTo(RemotingSysResponseCode.REQUEST_CODE_NOT_SUPPORTED);
+
+        // Issue request to SubRemotingServer
+        response = remotingClient.invokeSync("localhost:1234", request, 1000 * 3);
+        assertThat(response).isNotNull();
+        assertThat(response.getExtFields()).hasSize(2);
+        assertThat(response.getRemark()).isEqualTo(String.valueOf(subServerPort));
+
+        // Issue unsupported request to SubRemotingServer
+        request.setCode(0);
+        response = remotingClient.invokeSync("localhost:1234", request, 1000 * 3);
+        assertThat(response).isNotNull();
+        assertThat(response.getCode()).isEqualTo(RemotingSysResponseCode.REQUEST_CODE_NOT_SUPPORTED);
+
+        // Issue request to a closed SubRemotingServer
+        request.setCode(1);
+        remotingServer.removeRemotingServer(subServerPort);
+        subServer.shutdown();
+        try {
+            remotingClient.invokeSync("localhost:1234", request, 1000 * 3);
+            failBecauseExceptionWasNotThrown(RemotingTimeoutException.class);
+        } catch (Exception e) {
+            assertThat(e).isInstanceOf(RemotingTimeoutException.class);
+        }
+    }
+}
diff --git a/remoting/src/test/java/org/apache/rocketmq/remoting/netty/NettyRemotingAbstractTest.java b/remoting/src/test/java/org/apache/rocketmq/remoting/netty/NettyRemotingAbstractTest.java
index a272d21..58aac7d 100644
--- a/remoting/src/test/java/org/apache/rocketmq/remoting/netty/NettyRemotingAbstractTest.java
+++ b/remoting/src/test/java/org/apache/rocketmq/remoting/netty/NettyRemotingAbstractTest.java
@@ -37,7 +37,7 @@ public class NettyRemotingAbstractTest {
     @Test
     public void testProcessResponseCommand() throws InterruptedException {
         final Semaphore semaphore = new Semaphore(0);
-        ResponseFuture responseFuture = new ResponseFuture(null,1, 3000, new InvokeCallback() {
+        ResponseFuture responseFuture = new ResponseFuture(null, 1, 3000, new InvokeCallback() {
             @Override
             public void operationComplete(final ResponseFuture responseFuture) {
                 assertThat(semaphore.availablePermits()).isEqualTo(0);
@@ -58,7 +58,7 @@ public class NettyRemotingAbstractTest {
     @Test
     public void testProcessResponseCommand_NullCallBack() throws InterruptedException {
         final Semaphore semaphore = new Semaphore(0);
-        ResponseFuture responseFuture = new ResponseFuture(null,1, 3000, null,
+        ResponseFuture responseFuture = new ResponseFuture(null, 1, 3000, null,
             new SemaphoreReleaseOnlyOnce(semaphore));
 
         remotingAbstract.responseTable.putIfAbsent(1, responseFuture);
@@ -73,7 +73,7 @@ public class NettyRemotingAbstractTest {
     @Test
     public void testProcessResponseCommand_RunCallBackInCurrentThread() throws InterruptedException {
         final Semaphore semaphore = new Semaphore(0);
-        ResponseFuture responseFuture = new ResponseFuture(null,1, 3000, new InvokeCallback() {
+        ResponseFuture responseFuture = new ResponseFuture(null, 1, 3000, new InvokeCallback() {
             @Override
             public void operationComplete(final ResponseFuture responseFuture) {
                 assertThat(semaphore.availablePermits()).isEqualTo(0);
diff --git a/remoting/src/test/java/org/apache/rocketmq/remoting/protocol/RemotingCommandTest.java b/remoting/src/test/java/org/apache/rocketmq/remoting/protocol/RemotingCommandTest.java
index ee6451d..f7ef585 100644
--- a/remoting/src/test/java/org/apache/rocketmq/remoting/protocol/RemotingCommandTest.java
+++ b/remoting/src/test/java/org/apache/rocketmq/remoting/protocol/RemotingCommandTest.java
@@ -23,6 +23,7 @@ import java.util.HashSet;
 import java.util.Set;
 
 import com.alibaba.fastjson.JSON;
+import java.util.HashSet;
 import org.apache.rocketmq.remoting.CommandCustomHeader;
 import org.apache.rocketmq.remoting.annotation.CFNotNull;
 import org.apache.rocketmq.remoting.exception.RemotingCommandException;