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:26 UTC

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

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 ...