You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@rocketmq.apache.org by do...@apache.org on 2021/12/24 06:12:34 UTC

[rocketmq] branch 5.0.0-alpha updated: support batch consume-queue.

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

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


The following commit(s) were added to refs/heads/5.0.0-alpha by this push:
     new 1afe169  support batch consume-queue.
     new 16a3195  Merge pull request #3671 from Erik1288/5.0.0-alpha-bcq
1afe169 is described below

commit 1afe169fd44a3fccb80c9d0ce66221be1472171e
Author: Erik1288 <er...@gmail.com>
AuthorDate: Thu Dec 23 18:57:43 2021 +0800

    support batch consume-queue.
---
 .../apache/rocketmq/broker/BrokerController.java   |   17 +-
 .../broker/plugin/AbstractPluginMessageStore.java  |    6 +-
 .../broker/processor/AdminBrokerProcessor.java     |   27 +-
 .../broker/processor/SendMessageProcessor.java     |   32 +-
 .../broker/pagecache/OneMessageTransferTest.java   |    8 +-
 .../broker/processor/AdminBrokerProcessorTest.java |    4 +-
 .../broker/processor/PopMessageProcessorTest.java  |    4 +-
 .../client/consumer/DefaultMQPullConsumer.java     |    7 +
 .../client/consumer/DefaultMQPushConsumer.java     |   11 +
 .../rocketmq/client/consumer/MQPullConsumer.java   |    7 +
 .../rocketmq/client/impl/MQClientAPIImpl.java      |    3 +-
 .../impl/consumer/DefaultMQPullConsumerImpl.java   |   34 +
 .../impl/consumer/DefaultMQPushConsumerImpl.java   |    1 +
 .../client/impl/consumer/PullAPIWrapper.java       |   58 +
 .../client/producer/DefaultMQProducer.java         |    1 +
 .../java/org/apache/rocketmq/common/UtilAll.java   |   13 +
 .../rocketmq/common/message/MessageConst.java      |    2 +
 .../rocketmq/common/message/MessageDecoder.java    |   49 +
 .../protocol/header/PullMessageRequestHeader.java  |   10 +
 .../protocol/header/SendMessageResponseHeader.java |    9 +
 .../rocketmq/common/sysflag/MessageSysFlag.java    |    5 +
 .../common/message/MessageDecoderTest.java         |    4 +-
 .../rocketmq/store/AllocateMappedFileService.java  |   12 +-
 .../rocketmq/store/AppendMessageCallback.java      |    2 -
 .../apache/rocketmq/store/AppendMessageResult.java |   12 +
 .../java/org/apache/rocketmq/store/CommitLog.java  |  730 ++++--
 .../org/apache/rocketmq/store/ConsumeQueue.java    |  164 +-
 .../org/apache/rocketmq/store/ConsumeQueueExt.java |    1 +
 .../apache/rocketmq/store/DefaultMessageStore.java |  541 ++--
 .../org/apache/rocketmq/store/DispatchRequest.java |   45 +
 .../apache/rocketmq/store/FileQueueSnapshot.java   |   90 +
 .../apache/rocketmq/store/GetMessageResult.java    |   23 +-
 .../org/apache/rocketmq/store/MappedFileQueue.java |   97 +-
 .../apache/rocketmq/store}/MessageExtBatch.java    |   17 +-
 .../org/apache/rocketmq/store/MessageStore.java    |  169 +-
 .../rocketmq/store/MultiPathMappedFileQueue.java   |    1 +
 .../apache/rocketmq/store/PutMessageContext.java   |   31 +
 .../rocketmq/store/SelectMappedBufferResult.java   |   11 +-
 .../org/apache/rocketmq/store/StoreCheckpoint.java |    7 +-
 .../java/org/apache/rocketmq/store/StoreUtil.java  |   48 +
 .../apache/rocketmq/store/StreamMessageStore.java  | 2573 ++++++++++++++++++++
 .../java/org/apache/rocketmq/store/Swappable.java  |   33 +-
 .../org/apache/rocketmq/store/TopicQueueLock.java  |   29 +
 .../rocketmq/store/config/MessageStoreConfig.java  |  292 ++-
 .../store/config/StorePathConfigHelper.java        |    3 +
 .../rocketmq/store/dledger/DLedgerCommitLog.java   |  217 +-
 .../org/apache/rocketmq/store/ha/HAService.java    |    8 +-
 .../org/apache/rocketmq/store/index/IndexFile.java |    5 +-
 .../apache/rocketmq/store/index/IndexService.java  |    8 +-
 .../rocketmq/store/logfile/AbstractMappedFile.java |   28 +-
 .../DefaultMappedFile.java}                        |  202 +-
 .../apache/rocketmq/store/logfile/MappedFile.java  |  331 +++
 .../rocketmq/store/queue/BatchConsumeQueue.java    |  952 ++++++++
 .../rocketmq/store/queue/BatchOffsetIndex.java     |   57 +
 .../org/apache/rocketmq/store/queue/CQType.java    |   30 +-
 .../store/queue/ConsumeQueueInterface.java         |  112 +
 .../rocketmq/store/queue/ConsumeQueueStore.java    |  176 ++
 .../org/apache/rocketmq/store/queue/CqUnit.java    |  115 +
 .../FileQueueLifeCycle.java}                       |   32 +-
 .../rocketmq/store/queue/ReferredIterator.java     |   30 +-
 .../store/schedule/ScheduleMessageService.java     |   65 +-
 .../apache/rocketmq/store/stats/BrokerStats.java   |    6 +-
 .../apache/rocketmq/store/util/PerfCounter.java    |  370 +++
 .../{StoreUtil.java => util/QueueTypeUtils.java}   |   26 +-
 .../apache/rocketmq/store/AppendCallbackTest.java  |    2 -
 .../apache/rocketmq/store/BatchPutMessageTest.java |    1 -
 .../apache/rocketmq/store/ConsumeQueueTest.java    |   63 +-
 .../store/DefaultMessageStoreCleanFilesTest.java   |    8 +-
 .../store/DefaultMessageStoreShutDownTest.java     |    4 +-
 .../rocketmq/store/DefaultMessageStoreTest.java    |   78 +-
 .../apache/rocketmq/store/MappedFileQueueTest.java |  156 +-
 .../org/apache/rocketmq/store/MappedFileTest.java  |    4 +-
 .../store/MultiPathMappedFileQueueTest.java        |    1 +
 .../org/apache/rocketmq/store/StoreTestBase.java   |   14 +-
 .../store/dledger/DLedgerCommitlogTest.java        |    6 +-
 .../store/queue/BatchConsumeMessageTest.java       |  306 +++
 .../store/queue/BatchConsumeQueueTest.java         |  315 +++
 .../rocketmq/store/queue/ConsumeQueueTest.java     |   85 +
 .../apache/rocketmq/store/queue/QueueTestBase.java |   89 +
 .../org/apache/rocketmq/test/base/BaseConf.java    |   22 +-
 .../rocketmq/test/base/IntegrationTestBase.java    |   31 +
 .../base/dledger/DLedgerProduceAndConsumeIT.java   |    2 +
 .../test/client/producer/batch/BatchSendIT.java    |  162 ++
 .../client/producer/oneway/OneWaySendWithMQIT.java |   10 -
 .../rocketmq/test/statictopic/StaticTopicIT.java   |    2 +-
 85 files changed, 8419 insertions(+), 955 deletions(-)

diff --git a/broker/src/main/java/org/apache/rocketmq/broker/BrokerController.java b/broker/src/main/java/org/apache/rocketmq/broker/BrokerController.java
index 76c8007..6f9563b 100644
--- a/broker/src/main/java/org/apache/rocketmq/broker/BrokerController.java
+++ b/broker/src/main/java/org/apache/rocketmq/broker/BrokerController.java
@@ -96,9 +96,11 @@ import org.apache.rocketmq.srvutil.FileWatchService;
 import org.apache.rocketmq.store.DefaultMessageStore;
 import org.apache.rocketmq.store.MessageArrivingListener;
 import org.apache.rocketmq.store.MessageStore;
+import org.apache.rocketmq.store.StreamMessageStore;
 import org.apache.rocketmq.store.config.BrokerRole;
 import org.apache.rocketmq.store.config.MessageStoreConfig;
 import org.apache.rocketmq.store.dledger.DLedgerCommitLog;
+import org.apache.rocketmq.store.queue.CQType;
 import org.apache.rocketmq.store.stats.BrokerStats;
 import org.apache.rocketmq.store.stats.BrokerStatsManager;
 
@@ -296,14 +298,19 @@ public class BrokerController {
 
         if (result) {
             try {
-                this.messageStore =
-                    new DefaultMessageStore(this.messageStoreConfig, this.brokerStatsManager, this.messageArrivingListener,
-                        this.brokerConfig);
+                MessageStore messageStore;
+                if (Objects.equals(CQType.BatchCQ.toString(), this.messageStoreConfig.getDefaultCQType())) {
+                    messageStore = new StreamMessageStore(this.messageStoreConfig, this.brokerStatsManager, this.messageArrivingListener, this.brokerConfig);
+                } else {
+                    messageStore = new DefaultMessageStore(this.messageStoreConfig, this.brokerStatsManager, this.messageArrivingListener, this.brokerConfig);
+                }
+
+                this.messageStore = messageStore;
                 if (messageStoreConfig.isEnableDLegerCommitLog()) {
                     DLedgerRoleChangeHandler roleChangeHandler = new DLedgerRoleChangeHandler(this, (DefaultMessageStore) messageStore);
-                    ((DLedgerCommitLog) ((DefaultMessageStore) messageStore).getCommitLog()).getdLedgerServer().getdLedgerLeaderElector().addRoleChangeHandler(roleChangeHandler);
+                    ((DLedgerCommitLog) messageStore.getCommitLog()).getdLedgerServer().getdLedgerLeaderElector().addRoleChangeHandler(roleChangeHandler);
                 }
-                this.brokerStats = new BrokerStats((DefaultMessageStore) this.messageStore);
+                this.brokerStats = new BrokerStats(this.messageStore);
                 //load plugin
                 MessageStorePluginContext context = new MessageStorePluginContext(messageStoreConfig, brokerStatsManager, messageArrivingListener, brokerConfig);
                 this.messageStore = MessageStoreFactory.build(context, this.messageStore);
diff --git a/broker/src/main/java/org/apache/rocketmq/broker/plugin/AbstractPluginMessageStore.java b/broker/src/main/java/org/apache/rocketmq/broker/plugin/AbstractPluginMessageStore.java
index d306d34..44edfe0 100644
--- a/broker/src/main/java/org/apache/rocketmq/broker/plugin/AbstractPluginMessageStore.java
+++ b/broker/src/main/java/org/apache/rocketmq/broker/plugin/AbstractPluginMessageStore.java
@@ -22,16 +22,16 @@ import java.util.LinkedList;
 import java.util.Set;
 import java.util.concurrent.CompletableFuture;
 import org.apache.rocketmq.common.message.MessageExt;
-import org.apache.rocketmq.common.message.MessageExtBatch;
 import org.apache.rocketmq.store.CommitLogDispatcher;
-import org.apache.rocketmq.store.ConsumeQueue;
 import org.apache.rocketmq.store.GetMessageResult;
+import org.apache.rocketmq.store.MessageExtBatch;
 import org.apache.rocketmq.store.MessageExtBrokerInner;
 import org.apache.rocketmq.store.MessageFilter;
 import org.apache.rocketmq.store.MessageStore;
 import org.apache.rocketmq.store.PutMessageResult;
 import org.apache.rocketmq.store.QueryMessageResult;
 import org.apache.rocketmq.store.SelectMappedBufferResult;
+import org.apache.rocketmq.store.queue.ConsumeQueueInterface;
 import org.apache.rocketmq.store.stats.BrokerStatsManager;
 
 public abstract class AbstractPluginMessageStore implements MessageStore {
@@ -261,7 +261,7 @@ public abstract class AbstractPluginMessageStore implements MessageStore {
     }
 
     @Override
-    public ConsumeQueue getConsumeQueue(String topic, int queueId) {
+    public ConsumeQueueInterface getConsumeQueue(String topic, int queueId) {
         return next.getConsumeQueue(topic, queueId);
     }
 
diff --git a/broker/src/main/java/org/apache/rocketmq/broker/processor/AdminBrokerProcessor.java b/broker/src/main/java/org/apache/rocketmq/broker/processor/AdminBrokerProcessor.java
index 0d2a59b..117b146 100644
--- a/broker/src/main/java/org/apache/rocketmq/broker/processor/AdminBrokerProcessor.java
+++ b/broker/src/main/java/org/apache/rocketmq/broker/processor/AdminBrokerProcessor.java
@@ -131,7 +131,6 @@ import org.apache.rocketmq.remoting.protocol.LanguageCode;
 import org.apache.rocketmq.remoting.protocol.RemotingCommand;
 import org.apache.rocketmq.remoting.protocol.RemotingSerializable;
 import org.apache.rocketmq.remoting.protocol.RemotingSysResponseCode;
-import org.apache.rocketmq.store.ConsumeQueue;
 import org.apache.rocketmq.store.ConsumeQueueExt;
 import org.apache.rocketmq.store.DefaultMessageStore;
 import org.apache.rocketmq.store.MessageExtBrokerInner;
@@ -140,6 +139,9 @@ import org.apache.rocketmq.store.MessageStore;
 import org.apache.rocketmq.store.PutMessageResult;
 import org.apache.rocketmq.store.PutMessageStatus;
 import org.apache.rocketmq.store.SelectMappedBufferResult;
+import org.apache.rocketmq.store.queue.ConsumeQueueInterface;
+import org.apache.rocketmq.store.queue.CqUnit;
+import org.apache.rocketmq.store.queue.ReferredIterator;
 
 import java.io.UnsupportedEncodingException;
 import java.net.UnknownHostException;
@@ -1859,7 +1861,7 @@ public class AdminBrokerProcessor extends AsyncNettyRequestProcessor implements
 
         RemotingCommand response = RemotingCommand.createResponseCommand(null);
 
-        ConsumeQueue consumeQueue = this.brokerController.getMessageStore().getConsumeQueue(requestHeader.getTopic(),
+        ConsumeQueueInterface consumeQueue = this.brokerController.getMessageStore().getConsumeQueue(requestHeader.getTopic(),
             requestHeader.getQueueId());
         if (consumeQueue == null) {
             response.setCode(ResponseCode.SYSTEM_ERROR);
@@ -1890,26 +1892,31 @@ public class AdminBrokerProcessor extends AsyncNettyRequestProcessor implements
             }
         }
 
-        SelectMappedBufferResult result = consumeQueue.getIndexBuffer(requestHeader.getIndex());
+        ReferredIterator<CqUnit> result = consumeQueue.iterateFrom(requestHeader.getIndex());
         if (result == null) {
             response.setRemark(String.format("Index %d of %d@%s is not exist!", requestHeader.getIndex(), requestHeader.getQueueId(), requestHeader.getTopic()));
             return response;
         }
         try {
             List<ConsumeQueueData> queues = new ArrayList<>();
-            for (int i = 0; i < result.getSize() && i < requestHeader.getCount() * ConsumeQueue.CQ_STORE_UNIT_SIZE; i += ConsumeQueue.CQ_STORE_UNIT_SIZE) {
+            while (result.hasNext()) {
+                CqUnit cqUnit = result.next();
+                if (cqUnit.getQueueOffset() - requestHeader.getIndex() >=  requestHeader.getCount()) {
+                    break;
+                }
+
                 ConsumeQueueData one = new ConsumeQueueData();
-                one.setPhysicOffset(result.getByteBuffer().getLong());
-                one.setPhysicSize(result.getByteBuffer().getInt());
-                one.setTagsCode(result.getByteBuffer().getLong());
+                one.setPhysicOffset(cqUnit.getPos());
+                one.setPhysicSize(cqUnit.getSize());
+                one.setTagsCode(cqUnit.getTagsCode());
 
-                if (!consumeQueue.isExtAddr(one.getTagsCode())) {
+                if (cqUnit.getCqExtUnit() == null && cqUnit.isTagsCodeValid()) {
                     queues.add(one);
                     continue;
                 }
 
-                ConsumeQueueExt.CqExtUnit cqExtUnit = consumeQueue.getExt(one.getTagsCode());
-                if (cqExtUnit != null) {
+                if (cqUnit.getCqExtUnit() != null) {
+                    ConsumeQueueExt.CqExtUnit cqExtUnit = cqUnit.getCqExtUnit();
                     one.setExtendDataJson(JSON.toJSONString(cqExtUnit));
                     if (cqExtUnit.getFilterBitMap() != null) {
                         one.setBitMap(BitsArray.create(cqExtUnit.getFilterBitMap()).toString());
diff --git a/broker/src/main/java/org/apache/rocketmq/broker/processor/SendMessageProcessor.java b/broker/src/main/java/org/apache/rocketmq/broker/processor/SendMessageProcessor.java
index 17f19cb..3e3173e 100644
--- a/broker/src/main/java/org/apache/rocketmq/broker/processor/SendMessageProcessor.java
+++ b/broker/src/main/java/org/apache/rocketmq/broker/processor/SendMessageProcessor.java
@@ -17,6 +17,8 @@
 package org.apache.rocketmq.broker.processor;
 
 import java.net.SocketAddress;
+import java.nio.ByteBuffer;
+import java.nio.ByteBuffer;
 import java.util.List;
 import java.util.Map;
 import java.util.concurrent.CompletableFuture;
@@ -39,10 +41,11 @@ import org.apache.rocketmq.common.UtilAll;
 import org.apache.rocketmq.common.constant.PermName;
 import org.apache.rocketmq.common.help.FAQUrl;
 import org.apache.rocketmq.common.message.MessageAccessor;
+import org.apache.rocketmq.common.message.MessageClientIDSetter;
+import org.apache.rocketmq.common.message.MessageClientIDSetter;
 import org.apache.rocketmq.common.message.MessageConst;
 import org.apache.rocketmq.common.message.MessageDecoder;
 import org.apache.rocketmq.common.message.MessageExt;
-import org.apache.rocketmq.common.message.MessageExtBatch;
 import org.apache.rocketmq.common.protocol.NamespaceUtil;
 import org.apache.rocketmq.common.protocol.RequestCode;
 import org.apache.rocketmq.common.protocol.ResponseCode;
@@ -57,8 +60,10 @@ import org.apache.rocketmq.remoting.exception.RemotingCommandException;
 import org.apache.rocketmq.remoting.netty.NettyRequestProcessor;
 import org.apache.rocketmq.remoting.netty.RemotingResponseCallback;
 import org.apache.rocketmq.remoting.protocol.RemotingCommand;
+import org.apache.rocketmq.store.MessageExtBatch;
 import org.apache.rocketmq.store.MessageExtBrokerInner;
 import org.apache.rocketmq.store.PutMessageResult;
+import org.apache.rocketmq.store.StoreUtil;
 import org.apache.rocketmq.store.config.MessageStoreConfig;
 import org.apache.rocketmq.store.config.StorePathConfigHelper;
 import org.apache.rocketmq.store.stats.BrokerStatsManager;
@@ -677,11 +682,29 @@ public class SendMessageProcessor extends AbstractSendMessageProcessor implement
         String clusterName = this.brokerController.getBrokerConfig().getBrokerClusterName();
         MessageAccessor.putProperty(messageExtBatch, MessageConst.PROPERTY_CLUSTER, clusterName);
 
-        CompletableFuture<PutMessageResult> putMessageResult = this.brokerController.getMessageStore().asyncPutMessages(messageExtBatch);
-        return handlePutMessageResultFuture(putMessageResult, response, request, messageExtBatch, responseHeader, mqtraceContext, ctx, queueIdInt, requestHeader, mappingContext);
-    }
+        CompletableFuture<PutMessageResult> putMessageResult;
+
+        if (StoreUtil.isStreamMode(this.brokerController.getMessageStore()) && MessageClientIDSetter.getUniqID(messageExtBatch) != null) {
+            // newly introduced inner-batch message
+            messageExtBatch.setSysFlag(messageExtBatch.getSysFlag() | MessageSysFlag.NEED_UNWRAP_FLAG);
+            messageExtBatch.setSysFlag(messageExtBatch.getSysFlag() | MessageSysFlag.INNER_BATCH_FLAG);
+            messageExtBatch.setInnerBatch(true);
+
+            int innerNum = MessageDecoder.countInnerMsgNum(ByteBuffer.wrap(messageExtBatch.getBody()));
+
+            MessageAccessor.putProperty(messageExtBatch, MessageConst.PROPERTY_INNER_NUM, String.valueOf(innerNum));
+            messageExtBatch.setPropertiesString(MessageDecoder.messageProperties2String(messageExtBatch.getProperties()));
 
+            // tell the producer that it's an inner-batch message response.
+            responseHeader.setBatchUniqId(MessageClientIDSetter.getUniqID(messageExtBatch));
+            putMessageResult = this.brokerController.getMessageStore().asyncPutMessage(messageExtBatch);
+        } else {
+            // traditional outer-batch message
+            putMessageResult = this.brokerController.getMessageStore().asyncPutMessages(messageExtBatch);
+        }
 
+        return handlePutMessageResultFuture(putMessageResult, response, request, messageExtBatch, responseHeader, mqtraceContext, ctx, queueIdInt, requestHeader, mappingContext);
+    }
 
     public boolean hasConsumeMessageHook() {
         return consumeMessageHookList != null && !this.consumeMessageHookList.isEmpty();
@@ -771,4 +794,5 @@ public class SendMessageProcessor extends AbstractSendMessageProcessor implement
 
         return response;
     }
+
 }
diff --git a/broker/src/test/java/org/apache/rocketmq/broker/pagecache/OneMessageTransferTest.java b/broker/src/test/java/org/apache/rocketmq/broker/pagecache/OneMessageTransferTest.java
index 2cd4bdc..da70584 100644
--- a/broker/src/test/java/org/apache/rocketmq/broker/pagecache/OneMessageTransferTest.java
+++ b/broker/src/test/java/org/apache/rocketmq/broker/pagecache/OneMessageTransferTest.java
@@ -18,8 +18,8 @@
 package org.apache.rocketmq.broker.pagecache;
 
 import java.nio.ByteBuffer;
-import org.apache.rocketmq.store.MappedFile;
 import org.apache.rocketmq.store.SelectMappedBufferResult;
+import org.apache.rocketmq.store.logfile.DefaultMappedFile;
 import org.junit.Assert;
 import org.junit.Test;
 
@@ -29,7 +29,7 @@ public class OneMessageTransferTest {
     public void OneMessageTransferTest(){
         ByteBuffer byteBuffer = ByteBuffer.allocate(20);
         byteBuffer.putInt(20);
-        SelectMappedBufferResult selectMappedBufferResult = new SelectMappedBufferResult(0,byteBuffer,20,new MappedFile());
+        SelectMappedBufferResult selectMappedBufferResult = new SelectMappedBufferResult(0,byteBuffer,20,new DefaultMappedFile());
         OneMessageTransfer manyMessageTransfer = new OneMessageTransfer(byteBuffer,selectMappedBufferResult);
     }
 
@@ -37,7 +37,7 @@ public class OneMessageTransferTest {
     public void OneMessageTransferCountTest(){
         ByteBuffer byteBuffer = ByteBuffer.allocate(20);
         byteBuffer.putInt(20);
-        SelectMappedBufferResult selectMappedBufferResult = new SelectMappedBufferResult(0,byteBuffer,20,new MappedFile());
+        SelectMappedBufferResult selectMappedBufferResult = new SelectMappedBufferResult(0,byteBuffer,20,new DefaultMappedFile());
         OneMessageTransfer manyMessageTransfer = new OneMessageTransfer(byteBuffer,selectMappedBufferResult);
         Assert.assertEquals(manyMessageTransfer.count(),40);
     }
@@ -46,7 +46,7 @@ public class OneMessageTransferTest {
     public void OneMessageTransferPosTest(){
         ByteBuffer byteBuffer = ByteBuffer.allocate(20);
         byteBuffer.putInt(20);
-        SelectMappedBufferResult selectMappedBufferResult = new SelectMappedBufferResult(0,byteBuffer,20,new MappedFile());
+        SelectMappedBufferResult selectMappedBufferResult = new SelectMappedBufferResult(0,byteBuffer,20,new DefaultMappedFile());
         OneMessageTransfer manyMessageTransfer = new OneMessageTransfer(byteBuffer,selectMappedBufferResult);
         Assert.assertEquals(manyMessageTransfer.position(),8);
     }
diff --git a/broker/src/test/java/org/apache/rocketmq/broker/processor/AdminBrokerProcessorTest.java b/broker/src/test/java/org/apache/rocketmq/broker/processor/AdminBrokerProcessorTest.java
index 2141f2c..acf05fb 100644
--- a/broker/src/test/java/org/apache/rocketmq/broker/processor/AdminBrokerProcessorTest.java
+++ b/broker/src/test/java/org/apache/rocketmq/broker/processor/AdminBrokerProcessorTest.java
@@ -43,13 +43,13 @@ import org.apache.rocketmq.remoting.netty.NettyServerConfig;
 import org.apache.rocketmq.remoting.protocol.RemotingCommand;
 import org.apache.rocketmq.store.AppendMessageResult;
 import org.apache.rocketmq.store.AppendMessageStatus;
-import org.apache.rocketmq.store.MappedFile;
 import org.apache.rocketmq.store.MessageExtBrokerInner;
 import org.apache.rocketmq.store.MessageStore;
 import org.apache.rocketmq.store.PutMessageResult;
 import org.apache.rocketmq.store.PutMessageStatus;
 import org.apache.rocketmq.store.SelectMappedBufferResult;
 import org.apache.rocketmq.store.config.MessageStoreConfig;
+import org.apache.rocketmq.store.logfile.DefaultMappedFile;
 import org.junit.Before;
 import org.junit.Test;
 import org.junit.runner.RunWith;
@@ -246,7 +246,7 @@ public class AdminBrokerProcessorTest {
     }
 
     private SelectMappedBufferResult createSelectMappedBufferResult() {
-        SelectMappedBufferResult result = new SelectMappedBufferResult(0, ByteBuffer.allocate(1024), 0, new MappedFile());
+        SelectMappedBufferResult result = new SelectMappedBufferResult(0, ByteBuffer.allocate(1024), 0, new DefaultMappedFile());
         return result;
     }
 
diff --git a/broker/src/test/java/org/apache/rocketmq/broker/processor/PopMessageProcessorTest.java b/broker/src/test/java/org/apache/rocketmq/broker/processor/PopMessageProcessorTest.java
index 87d96b2..55ec3e4 100644
--- a/broker/src/test/java/org/apache/rocketmq/broker/processor/PopMessageProcessorTest.java
+++ b/broker/src/test/java/org/apache/rocketmq/broker/processor/PopMessageProcessorTest.java
@@ -39,11 +39,11 @@ import org.apache.rocketmq.store.AppendMessageStatus;
 import org.apache.rocketmq.store.DefaultMessageStore;
 import org.apache.rocketmq.store.GetMessageResult;
 import org.apache.rocketmq.store.GetMessageStatus;
-import org.apache.rocketmq.store.MappedFile;
 import org.apache.rocketmq.store.PutMessageResult;
 import org.apache.rocketmq.store.PutMessageStatus;
 import org.apache.rocketmq.store.SelectMappedBufferResult;
 import org.apache.rocketmq.store.config.MessageStoreConfig;
+import org.apache.rocketmq.store.logfile.DefaultMappedFile;
 import org.apache.rocketmq.store.schedule.ScheduleMessageService;
 import org.junit.Before;
 import org.junit.Test;
@@ -184,7 +184,7 @@ public class PopMessageProcessorTest {
         for (int i = 0; i < msgCnt; i++) {
             ByteBuffer bb = ByteBuffer.allocate(64);
             bb.putLong(MessageDecoder.MESSAGE_STORE_TIMESTAMP_POSITION, System.currentTimeMillis());
-            getMessageResult.addMessage(new SelectMappedBufferResult(200, bb, 64, new MappedFile()));
+            getMessageResult.addMessage(new SelectMappedBufferResult(200, bb, 64, new DefaultMappedFile()));
         }
         return getMessageResult;
     }
diff --git a/client/src/main/java/org/apache/rocketmq/client/consumer/DefaultMQPullConsumer.java b/client/src/main/java/org/apache/rocketmq/client/consumer/DefaultMQPullConsumer.java
index 0876a94..5829f77 100644
--- a/client/src/main/java/org/apache/rocketmq/client/consumer/DefaultMQPullConsumer.java
+++ b/client/src/main/java/org/apache/rocketmq/client/consumer/DefaultMQPullConsumer.java
@@ -349,6 +349,13 @@ public class DefaultMQPullConsumer extends ClientConfig implements MQPullConsume
     }
 
     @Override
+    public void pull(MessageQueue mq, String subExpression, long offset, int maxNums, int maxSize, PullCallback pullCallback,
+        long timeout)
+        throws MQClientException, RemotingException, InterruptedException {
+        this.defaultMQPullConsumerImpl.pull(mq, subExpression, offset, maxNums, maxSize, pullCallback, timeout);
+    }
+
+    @Override
     public void pull(MessageQueue mq, MessageSelector messageSelector, long offset, int maxNums,
         PullCallback pullCallback)
         throws MQClientException, RemotingException, InterruptedException {
diff --git a/client/src/main/java/org/apache/rocketmq/client/consumer/DefaultMQPushConsumer.java b/client/src/main/java/org/apache/rocketmq/client/consumer/DefaultMQPushConsumer.java
index 8a6340b..fce6b64 100644
--- a/client/src/main/java/org/apache/rocketmq/client/consumer/DefaultMQPushConsumer.java
+++ b/client/src/main/java/org/apache/rocketmq/client/consumer/DefaultMQPushConsumer.java
@@ -231,6 +231,9 @@ public class DefaultMQPushConsumer extends ClientConfig implements MQPushConsume
      */
     private int pullBatchSize = 32;
 
+
+    private int pullBatchSizeInBytes = 256 * 1024;
+
     /**
      * Whether update subscription relationship when every pull
      */
@@ -942,6 +945,14 @@ public class DefaultMQPushConsumer extends ClientConfig implements MQPushConsume
         this.awaitTerminationMillisWhenShutdown = awaitTerminationMillisWhenShutdown;
     }
 
+    public int getPullBatchSizeInBytes() {
+        return pullBatchSizeInBytes;
+    }
+
+    public void setPullBatchSizeInBytes(int pullBatchSizeInBytes) {
+        this.pullBatchSizeInBytes = pullBatchSizeInBytes;
+    }
+
     public TraceDispatcher getTraceDispatcher() {
         return traceDispatcher;
     }
diff --git a/client/src/main/java/org/apache/rocketmq/client/consumer/MQPullConsumer.java b/client/src/main/java/org/apache/rocketmq/client/consumer/MQPullConsumer.java
index a8e9628..868ee93 100644
--- a/client/src/main/java/org/apache/rocketmq/client/consumer/MQPullConsumer.java
+++ b/client/src/main/java/org/apache/rocketmq/client/consumer/MQPullConsumer.java
@@ -114,6 +114,13 @@ public interface MQPullConsumer extends MQConsumer {
         InterruptedException;
 
     /**
+     * Pulling the messages in a async. way
+     */
+    void pull(final MessageQueue mq, final String subExpression, final long offset, final int maxNums, final int maxSize,
+        final PullCallback pullCallback, long timeout) throws MQClientException, RemotingException,
+        InterruptedException;
+
+    /**
      * Pulling the messages in a async. way. Support message selection
      */
     void pull(final MessageQueue mq, final MessageSelector selector, final long offset, final int maxNums,
diff --git a/client/src/main/java/org/apache/rocketmq/client/impl/MQClientAPIImpl.java b/client/src/main/java/org/apache/rocketmq/client/impl/MQClientAPIImpl.java
index 9823e81..20b8f41 100644
--- a/client/src/main/java/org/apache/rocketmq/client/impl/MQClientAPIImpl.java
+++ b/client/src/main/java/org/apache/rocketmq/client/impl/MQClientAPIImpl.java
@@ -727,7 +727,8 @@ public class MQClientAPIImpl {
         MessageQueue messageQueue = new MessageQueue(topic, brokerName, responseHeader.getQueueId());
 
         String uniqMsgId = MessageClientIDSetter.getUniqID(msg);
-        if (msg instanceof MessageBatch) {
+        if (msg instanceof MessageBatch && responseHeader.getBatchUniqId() == null) {
+            // This means it is not an inner batch
             StringBuilder sb = new StringBuilder();
             for (Message message : (MessageBatch) msg) {
                 sb.append(sb.length() == 0 ? "" : ",").append(MessageClientIDSetter.getUniqID(message));
diff --git a/client/src/main/java/org/apache/rocketmq/client/impl/consumer/DefaultMQPullConsumerImpl.java b/client/src/main/java/org/apache/rocketmq/client/impl/consumer/DefaultMQPullConsumerImpl.java
index 6d47573..05d3d48 100644
--- a/client/src/main/java/org/apache/rocketmq/client/impl/consumer/DefaultMQPullConsumerImpl.java
+++ b/client/src/main/java/org/apache/rocketmq/client/impl/consumer/DefaultMQPullConsumerImpl.java
@@ -447,6 +447,13 @@ public class DefaultMQPullConsumerImpl implements MQConsumerInner {
         this.pullAsyncImpl(mq, subscriptionData, offset, maxNums, pullCallback, false, timeout);
     }
 
+    public void pull(MessageQueue mq, String subExpression, long offset, int maxNums, int maxSize, PullCallback pullCallback,
+        long timeout)
+        throws MQClientException, RemotingException, InterruptedException {
+        SubscriptionData subscriptionData = getSubscriptionData(mq, subExpression);
+        this.pullAsyncImpl(mq, subscriptionData, offset, maxNums, maxSize, pullCallback, false, timeout);
+    }
+
     public void pull(MessageQueue mq, MessageSelector messageSelector, long offset, int maxNums,
         PullCallback pullCallback)
         throws MQClientException, RemotingException, InterruptedException {
@@ -466,6 +473,7 @@ public class DefaultMQPullConsumerImpl implements MQConsumerInner {
         final SubscriptionData subscriptionData,
         final long offset,
         final int maxNums,
+        final int maxSizeInBytes,
         final PullCallback pullCallback,
         final boolean block,
         final long timeout) throws MQClientException, RemotingException, InterruptedException {
@@ -483,6 +491,11 @@ public class DefaultMQPullConsumerImpl implements MQConsumerInner {
             throw new MQClientException("maxNums <= 0", null);
         }
 
+        if (maxSizeInBytes <= 0) {
+            throw new MQClientException("maxSizeInBytes <= 0", null);
+        }
+
+
         if (null == pullCallback) {
             throw new MQClientException("pullCallback is null", null);
         }
@@ -502,6 +515,7 @@ public class DefaultMQPullConsumerImpl implements MQConsumerInner {
                 isTagType ? 0L : subscriptionData.getSubVersion(),
                 offset,
                 maxNums,
+                maxSizeInBytes,
                 sysFlag,
                 0,
                 this.defaultMQPullConsumer.getBrokerSuspendMaxTimeMillis(),
@@ -526,6 +540,26 @@ public class DefaultMQPullConsumerImpl implements MQConsumerInner {
         }
     }
 
+    private void pullAsyncImpl(
+            final MessageQueue mq,
+            final SubscriptionData subscriptionData,
+            final long offset,
+            final int maxNums,
+            final PullCallback pullCallback,
+            final boolean block,
+            final long timeout) throws MQClientException, RemotingException, InterruptedException {
+        pullAsyncImpl(
+                mq,
+                subscriptionData,
+                offset,
+                maxNums,
+                Integer.MAX_VALUE,
+                pullCallback,
+                block,
+                timeout
+        );
+    }
+
     public PullResult pullBlockIfNotFound(MessageQueue mq, String subExpression, long offset, int maxNums)
         throws MQClientException, RemotingException, MQBrokerException, InterruptedException {
         SubscriptionData subscriptionData = getSubscriptionData(mq, subExpression);
diff --git a/client/src/main/java/org/apache/rocketmq/client/impl/consumer/DefaultMQPushConsumerImpl.java b/client/src/main/java/org/apache/rocketmq/client/impl/consumer/DefaultMQPushConsumerImpl.java
index 2b504cc..2fa3830 100644
--- a/client/src/main/java/org/apache/rocketmq/client/impl/consumer/DefaultMQPushConsumerImpl.java
+++ b/client/src/main/java/org/apache/rocketmq/client/impl/consumer/DefaultMQPushConsumerImpl.java
@@ -465,6 +465,7 @@ public class DefaultMQPushConsumerImpl implements MQConsumerInner {
                 subscriptionData.getSubVersion(),
                 pullRequest.getNextOffset(),
                 this.defaultMQPushConsumer.getPullBatchSize(),
+                this.defaultMQPushConsumer.getPullBatchSizeInBytes(),
                 sysFlag,
                 commitOffsetValue,
                 BROKER_SUSPEND_MAX_TIME_MILLIS,
diff --git a/client/src/main/java/org/apache/rocketmq/client/impl/consumer/PullAPIWrapper.java b/client/src/main/java/org/apache/rocketmq/client/impl/consumer/PullAPIWrapper.java
index 30e8439..6ce8e26 100644
--- a/client/src/main/java/org/apache/rocketmq/client/impl/consumer/PullAPIWrapper.java
+++ b/client/src/main/java/org/apache/rocketmq/client/impl/consumer/PullAPIWrapper.java
@@ -40,6 +40,7 @@ import org.apache.rocketmq.common.protocol.header.PopMessageRequestHeader;
 import org.apache.rocketmq.common.protocol.header.PullMessageRequestHeader;
 import org.apache.rocketmq.common.protocol.heartbeat.SubscriptionData;
 import org.apache.rocketmq.common.protocol.route.TopicRouteData;
+import org.apache.rocketmq.common.sysflag.MessageSysFlag;
 import org.apache.rocketmq.common.sysflag.PullSysFlag;
 import org.apache.rocketmq.logging.InternalLogger;
 import org.apache.rocketmq.remoting.exception.RemotingException;
@@ -79,6 +80,31 @@ public class PullAPIWrapper {
             ByteBuffer byteBuffer = ByteBuffer.wrap(pullResultExt.getMessageBinary());
             List<MessageExt> msgList = MessageDecoder.decodes(byteBuffer);
 
+            boolean needDecodeInnerMessage = false;
+            for (MessageExt messageExt: msgList) {
+                if (MessageSysFlag.check(messageExt.getSysFlag(), MessageSysFlag.INNER_BATCH_FLAG)
+                    && MessageSysFlag.check(messageExt.getSysFlag(), MessageSysFlag.NEED_UNWRAP_FLAG)) {
+                    needDecodeInnerMessage = true;
+                    break;
+                }
+            }
+            if (needDecodeInnerMessage) {
+                List<MessageExt> innerMsgList = new ArrayList<MessageExt>();
+                try {
+                    for (MessageExt messageExt: msgList) {
+                        if (MessageSysFlag.check(messageExt.getSysFlag(), MessageSysFlag.INNER_BATCH_FLAG)
+                            && MessageSysFlag.check(messageExt.getSysFlag(), MessageSysFlag.NEED_UNWRAP_FLAG)) {
+                            MessageDecoder.decodeMessage(messageExt, innerMsgList);
+                        } else {
+                            innerMsgList.add(messageExt);
+                        }
+                    }
+                    msgList = innerMsgList;
+                } catch (Throwable t) {
+                    log.error("Try to decode the inner batch failed for {}", pullResult.toString(), t);
+                }
+            }
+
             List<MessageExt> msgListFilterAgain = msgList;
             if (!subscriptionData.getTagsSet().isEmpty() && !subscriptionData.isClassFilterMode()) {
                 msgListFilterAgain = new ArrayList<MessageExt>(msgList.size());
@@ -154,6 +180,7 @@ public class PullAPIWrapper {
         final long subVersion,
         final long offset,
         final int maxNums,
+        final int maxSizeInBytes,
         final int sysFlag,
         final long commitOffset,
         final long brokerSuspendMaxTimeMillis,
@@ -198,6 +225,7 @@ public class PullAPIWrapper {
             requestHeader.setSuspendTimeoutMillis(brokerSuspendMaxTimeMillis);
             requestHeader.setSubscription(subExpression);
             requestHeader.setSubVersion(subVersion);
+            requestHeader.setMaxMsgBytes(maxSizeInBytes);
             requestHeader.setExpressionType(expressionType);
 
             String brokerAddr = findBrokerResult.getBrokerAddr();
@@ -219,6 +247,36 @@ public class PullAPIWrapper {
         throw new MQClientException("The broker[" + mq.getBrokerName() + "] not exist", null);
     }
 
+    public PullResult pullKernelImpl(
+        MessageQueue mq,
+        final String subExpression,
+        final String expressionType,
+        final long subVersion,
+        long offset,
+        final int maxNums,
+        final int sysFlag,
+        long commitOffset,
+        final long brokerSuspendMaxTimeMillis,
+        final long timeoutMillis,
+        final CommunicationMode communicationMode,
+        PullCallback pullCallback
+    ) throws MQClientException, RemotingException, MQBrokerException, InterruptedException {
+        return pullKernelImpl(
+                mq,
+                subExpression,
+                expressionType,
+                subVersion, offset,
+                maxNums,
+                Integer.MAX_VALUE,
+                sysFlag,
+                commitOffset,
+                brokerSuspendMaxTimeMillis,
+                timeoutMillis,
+                communicationMode,
+                pullCallback
+        );
+    }
+
     public long recalculatePullFromWhichNode(final MessageQueue mq) {
         if (this.isConnectBrokerByUser()) {
             return this.defaultBrokerId;
diff --git a/client/src/main/java/org/apache/rocketmq/client/producer/DefaultMQProducer.java b/client/src/main/java/org/apache/rocketmq/client/producer/DefaultMQProducer.java
index 1af416b..230785c 100644
--- a/client/src/main/java/org/apache/rocketmq/client/producer/DefaultMQProducer.java
+++ b/client/src/main/java/org/apache/rocketmq/client/producer/DefaultMQProducer.java
@@ -989,6 +989,7 @@ public class DefaultMQProducer extends ClientConfig implements MQProducer {
                 MessageClientIDSetter.setUniqID(message);
                 message.setTopic(withNamespace(message.getTopic()));
             }
+            MessageClientIDSetter.setUniqID(msgBatch);
             msgBatch.setBody(msgBatch.encode());
         } catch (Exception e) {
             throw new MQClientException("Failed to initiate the MessageBatch", e);
diff --git a/common/src/main/java/org/apache/rocketmq/common/UtilAll.java b/common/src/main/java/org/apache/rocketmq/common/UtilAll.java
index ea22aa7..d3ca237 100644
--- a/common/src/main/java/org/apache/rocketmq/common/UtilAll.java
+++ b/common/src/main/java/org/apache/rocketmq/common/UtilAll.java
@@ -196,6 +196,19 @@ public class UtilAll {
             cal.get(Calendar.SECOND));
     }
 
+    public static long getTotalSpace(final String path) {
+        if (null == path || path.isEmpty())
+            return -1;
+        try {
+            File file = new File(path);
+            if (!file.exists())
+                return -1;
+            return  file.getTotalSpace();
+        } catch (Exception e) {
+            return -1;
+        }
+    }
+
     public static boolean isPathExists(final String path) {
         File file = new File(path);
         return file.exists();
diff --git a/common/src/main/java/org/apache/rocketmq/common/message/MessageConst.java b/common/src/main/java/org/apache/rocketmq/common/message/MessageConst.java
index 628bf4e..b5c14e9 100644
--- a/common/src/main/java/org/apache/rocketmq/common/message/MessageConst.java
+++ b/common/src/main/java/org/apache/rocketmq/common/message/MessageConst.java
@@ -41,6 +41,8 @@ public class MessageConst {
     public static final String PROPERTY_UNIQ_CLIENT_MESSAGE_ID_KEYIDX = "UNIQ_KEY";
     public static final String PROPERTY_MAX_RECONSUME_TIMES = "MAX_RECONSUME_TIMES";
     public static final String PROPERTY_CONSUME_START_TIMESTAMP = "CONSUME_START_TIME";
+    public static final String PROPERTY_INNER_NUM = "INNER_NUM";
+    public static final String PROPERTY_INNER_BASE = "INNER_BASE";
     public static final String PROPERTY_TRANSACTION_PREPARED_QUEUE_OFFSET = "TRAN_PREPARED_QUEUE_OFFSET";
     public static final String PROPERTY_TRANSACTION_CHECK_TIMES = "TRANSACTION_CHECK_TIMES";
     public static final String PROPERTY_CHECK_IMMUNITY_TIME_IN_SECONDS = "CHECK_IMMUNITY_TIME_IN_SECONDS";
diff --git a/common/src/main/java/org/apache/rocketmq/common/message/MessageDecoder.java b/common/src/main/java/org/apache/rocketmq/common/message/MessageDecoder.java
index bb023fa..1e5fe44 100644
--- a/common/src/main/java/org/apache/rocketmq/common/message/MessageDecoder.java
+++ b/common/src/main/java/org/apache/rocketmq/common/message/MessageDecoder.java
@@ -395,6 +395,22 @@ public class MessageDecoder {
         return decodes(byteBuffer, true);
     }
 
+    public static List<MessageExt> decodesBatch(ByteBuffer byteBuffer,
+                                                final boolean readBody,
+                                                final boolean decompressBody,
+                                                final boolean isClient) {
+        List<MessageExt> msgExts = new ArrayList<MessageExt>();
+        while (byteBuffer.hasRemaining()) {
+            MessageExt msgExt = decode(byteBuffer, readBody, decompressBody, isClient);
+            if (null != msgExt) {
+                msgExts.add(msgExt);
+            } else {
+                break;
+            }
+        }
+        return msgExts;
+    }
+
     public static List<MessageExt> decodes(ByteBuffer byteBuffer, final boolean readBody) {
         List<MessageExt> msgExts = new ArrayList<MessageExt>();
         while (byteBuffer.hasRemaining()) {
@@ -569,4 +585,37 @@ public class MessageDecoder {
         }
         return msgs;
     }
+
+    public static void decodeMessage(MessageExt messageExt, List<MessageExt> list) throws Exception {
+        List<Message> messages = MessageDecoder.decodeMessages(ByteBuffer.wrap(messageExt.getBody()));
+        for (int i = 0; i < messages.size(); i++) {
+            Message message = messages.get(i);
+            MessageClientExt messageClientExt = new MessageClientExt();
+            messageClientExt.setTopic(messageExt.getTopic());
+            messageClientExt.setQueueOffset(messageExt.getQueueOffset() + i);
+            messageClientExt.setQueueId(messageExt.getQueueId());
+            messageClientExt.setFlag(message.getFlag());
+            MessageAccessor.setProperties(messageClientExt, message.getProperties());
+            messageClientExt.setBody(message.getBody());
+            messageClientExt.setStoreHost(messageExt.getStoreHost());
+            messageClientExt.setBornHost(messageExt.getBornHost());
+            messageClientExt.setBornTimestamp(messageExt.getBornTimestamp());
+            messageClientExt.setStoreTimestamp(messageExt.getStoreTimestamp());
+            messageClientExt.setSysFlag(messageExt.getSysFlag());
+            messageClientExt.setCommitLogOffset(messageExt.getCommitLogOffset());
+            messageClientExt.setWaitStoreMsgOK(messageExt.isWaitStoreMsgOK());
+            list.add(messageClientExt);
+        }
+    }
+
+    public static int countInnerMsgNum(ByteBuffer buffer) {
+        int count = 0;
+        while (buffer.hasRemaining()) {
+            count++;
+            int currPos = buffer.position();
+            int size = buffer.getInt();
+            buffer.position(currPos + size);
+        }
+        return count;
+    }
 }
diff --git a/common/src/main/java/org/apache/rocketmq/common/protocol/header/PullMessageRequestHeader.java b/common/src/main/java/org/apache/rocketmq/common/protocol/header/PullMessageRequestHeader.java
index e15170f..5ac1899 100644
--- a/common/src/main/java/org/apache/rocketmq/common/protocol/header/PullMessageRequestHeader.java
+++ b/common/src/main/java/org/apache/rocketmq/common/protocol/header/PullMessageRequestHeader.java
@@ -35,6 +35,8 @@ public class PullMessageRequestHeader extends TopicQueueRequestHeader {
     @CFNotNull
     private Long queueOffset;
     @CFNotNull
+    private Integer maxMsgBytes;
+    @CFNotNull
     private Integer maxMsgNums;
     @CFNotNull
     private Integer sysFlag;
@@ -143,4 +145,12 @@ public class PullMessageRequestHeader extends TopicQueueRequestHeader {
     public void setExpressionType(String expressionType) {
         this.expressionType = expressionType;
     }
+
+    public Integer getMaxMsgBytes() {
+        return maxMsgBytes;
+    }
+
+    public void setMaxMsgBytes(Integer maxMsgBytes) {
+        this.maxMsgBytes = maxMsgBytes;
+    }
 }
diff --git a/common/src/main/java/org/apache/rocketmq/common/protocol/header/SendMessageResponseHeader.java b/common/src/main/java/org/apache/rocketmq/common/protocol/header/SendMessageResponseHeader.java
index 6834881..601f720 100644
--- a/common/src/main/java/org/apache/rocketmq/common/protocol/header/SendMessageResponseHeader.java
+++ b/common/src/main/java/org/apache/rocketmq/common/protocol/header/SendMessageResponseHeader.java
@@ -32,6 +32,7 @@ public class SendMessageResponseHeader implements CommandCustomHeader {
     @CFNotNull
     private Long queueOffset;
     private String transactionId;
+    private String batchUniqId;
 
     @Override
     public void checkFields() throws RemotingCommandException {
@@ -68,4 +69,12 @@ public class SendMessageResponseHeader implements CommandCustomHeader {
     public void setTransactionId(String transactionId) {
         this.transactionId = transactionId;
     }
+
+    public String getBatchUniqId() {
+        return batchUniqId;
+    }
+
+    public void setBatchUniqId(String batchUniqId) {
+        this.batchUniqId = batchUniqId;
+    }
 }
diff --git a/common/src/main/java/org/apache/rocketmq/common/sysflag/MessageSysFlag.java b/common/src/main/java/org/apache/rocketmq/common/sysflag/MessageSysFlag.java
index d534571..d28ac62 100644
--- a/common/src/main/java/org/apache/rocketmq/common/sysflag/MessageSysFlag.java
+++ b/common/src/main/java/org/apache/rocketmq/common/sysflag/MessageSysFlag.java
@@ -25,6 +25,8 @@ public class MessageSysFlag {
     public final static int TRANSACTION_ROLLBACK_TYPE = 0x3 << 2;
     public final static int BORNHOST_V6_FLAG = 0x1 << 4;
     public final static int STOREHOSTADDRESS_V6_FLAG = 0x1 << 5;
+    public final static int NEED_UNWRAP_FLAG = 0x1 << 6;
+    public final static int INNER_BATCH_FLAG = 0x1 << 7;
 
     public static int getTransactionValue(final int flag) {
         return flag & TRANSACTION_ROLLBACK_TYPE;
@@ -38,4 +40,7 @@ public class MessageSysFlag {
         return flag & (~COMPRESSED_FLAG);
     }
 
+    public static boolean check(int flag, int expectedFlag) {
+        return (flag & expectedFlag) != 0;
+    }
 }
diff --git a/common/src/test/java/org/apache/rocketmq/common/message/MessageDecoderTest.java b/common/src/test/java/org/apache/rocketmq/common/message/MessageDecoderTest.java
index b27f246..5af7345 100644
--- a/common/src/test/java/org/apache/rocketmq/common/message/MessageDecoderTest.java
+++ b/common/src/test/java/org/apache/rocketmq/common/message/MessageDecoderTest.java
@@ -17,6 +17,7 @@
 
 package org.apache.rocketmq.common.message;
 
+import org.apache.rocketmq.common.sysflag.MessageSysFlag;
 import org.junit.Test;
 
 import java.net.InetAddress;
@@ -239,7 +240,8 @@ public class MessageDecoderTest {
         assertThat(1).isEqualTo(decodedMsg.getQueueId());
         assertThat(123456L).isEqualTo(decodedMsg.getCommitLogOffset());
         assertThat("hello!q!".getBytes()).isEqualTo(decodedMsg.getBody());
-        assertThat(48).isEqualTo(decodedMsg.getSysFlag());
+        // assertThat(48).isEqualTo(decodedMsg.getSysFlag());
+        assertThat(MessageSysFlag.check(messageExt.getSysFlag(), MessageSysFlag.STOREHOSTADDRESS_V6_FLAG)).isTrue();
 
         int msgIDLength = 16 + 4 + 8;
         ByteBuffer byteBufferMsgId = ByteBuffer.allocate(msgIDLength);
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 acb1d54..847e99d 100644
--- a/store/src/main/java/org/apache/rocketmq/store/AllocateMappedFileService.java
+++ b/store/src/main/java/org/apache/rocketmq/store/AllocateMappedFileService.java
@@ -30,6 +30,8 @@ import org.apache.rocketmq.common.constant.LoggerName;
 import org.apache.rocketmq.logging.InternalLogger;
 import org.apache.rocketmq.logging.InternalLoggerFactory;
 import org.apache.rocketmq.store.config.BrokerRole;
+import org.apache.rocketmq.store.logfile.DefaultMappedFile;
+import org.apache.rocketmq.store.logfile.MappedFile;
 
 /**
  * Create MappedFile in advance
@@ -42,9 +44,9 @@ public class AllocateMappedFileService extends ServiceThread {
     private PriorityBlockingQueue<AllocateRequest> requestQueue =
         new PriorityBlockingQueue<AllocateRequest>();
     private volatile boolean hasException = false;
-    private DefaultMessageStore messageStore;
+    private MessageStore messageStore;
 
-    public AllocateMappedFileService(DefaultMessageStore messageStore) {
+    public AllocateMappedFileService(MessageStore messageStore) {
         this.messageStore = messageStore;
     }
 
@@ -97,7 +99,9 @@ public class AllocateMappedFileService extends ServiceThread {
         AllocateRequest result = this.requestTable.get(nextFilePath);
         try {
             if (result != null) {
+                messageStore.getPerfCounter().startTick("WAIT_MAPFILE_TIME_MS");
                 boolean waitOK = result.getCountDownLatch().await(waitTimeOut, TimeUnit.MILLISECONDS);
+                messageStore.getPerfCounter().endTick("WAIT_MAPFILE_TIME_MS");
                 if (!waitOK) {
                     log.warn("create mmap timeout " + result.getFilePath() + " " + result.getFileSize());
                     return null;
@@ -170,10 +174,10 @@ public class AllocateMappedFileService extends ServiceThread {
                         mappedFile.init(req.getFilePath(), req.getFileSize(), messageStore.getTransientStorePool());
                     } catch (RuntimeException e) {
                         log.warn("Use default implementation.");
-                        mappedFile = new MappedFile(req.getFilePath(), req.getFileSize(), messageStore.getTransientStorePool());
+                        mappedFile = new DefaultMappedFile(req.getFilePath(), req.getFileSize(), messageStore.getTransientStorePool());
                     }
                 } else {
-                    mappedFile = new MappedFile(req.getFilePath(), req.getFileSize());
+                    mappedFile = new DefaultMappedFile(req.getFilePath(), req.getFileSize());
                 }
 
                 long elapsedTime = UtilAll.computeElapsedTimeMilliseconds(beginTime);
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 5499c90..accd4e2 100644
--- a/store/src/main/java/org/apache/rocketmq/store/AppendMessageCallback.java
+++ b/store/src/main/java/org/apache/rocketmq/store/AppendMessageCallback.java
@@ -17,8 +17,6 @@
 package org.apache.rocketmq.store;
 
 import java.nio.ByteBuffer;
-import org.apache.rocketmq.common.message.MessageExtBatch;
-import org.apache.rocketmq.store.CommitLog.PutMessageContext;
 
 /**
  * Write messages callback interface
diff --git a/store/src/main/java/org/apache/rocketmq/store/AppendMessageResult.java b/store/src/main/java/org/apache/rocketmq/store/AppendMessageResult.java
index de3c03b..3cfb85f 100644
--- a/store/src/main/java/org/apache/rocketmq/store/AppendMessageResult.java
+++ b/store/src/main/java/org/apache/rocketmq/store/AppendMessageResult.java
@@ -65,6 +65,18 @@ public class AppendMessageResult {
         this.pagecacheRT = pagecacheRT;
     }
 
+    public AppendMessageResult(AppendMessageStatus status, long wroteOffset, int wroteBytes, Supplier<String> msgIdSupplier,
+            long storeTimestamp, long logicsOffset, long pagecacheRT, int msgNum) {
+        this.status = status;
+        this.wroteOffset = wroteOffset;
+        this.wroteBytes = wroteBytes;
+        this.msgIdSupplier = msgIdSupplier;
+        this.storeTimestamp = storeTimestamp;
+        this.logicsOffset = logicsOffset;
+        this.pagecacheRT = pagecacheRT;
+        this.msgNum = msgNum;
+    }
+
     public long getPagecacheRT() {
         return pagecacheRT;
     }
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 604d9c2..062c269 100644
--- a/store/src/main/java/org/apache/rocketmq/store/CommitLog.java
+++ b/store/src/main/java/org/apache/rocketmq/store/CommitLog.java
@@ -22,6 +22,7 @@ import java.net.InetAddress;
 import java.net.InetSocketAddress;
 import java.net.SocketAddress;
 import java.nio.ByteBuffer;
+import java.util.ArrayList;
 import java.util.Collections;
 import java.util.HashMap;
 import java.util.LinkedList;
@@ -29,6 +30,9 @@ import java.util.List;
 import java.util.Map;
 import java.util.Set;
 import java.util.concurrent.CompletableFuture;
+import java.util.concurrent.ExecutionException;
+import java.util.concurrent.TimeUnit;
+import java.util.concurrent.TimeoutException;
 import java.util.function.Supplier;
 
 import org.apache.rocketmq.common.ServiceThread;
@@ -38,7 +42,6 @@ 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.sysflag.MessageSysFlag;
 import org.apache.rocketmq.common.topic.TopicValidator;
 import org.apache.rocketmq.logging.InternalLogger;
@@ -47,66 +50,68 @@ 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.store.queue.CQType;
 import org.apache.rocketmq.store.schedule.ScheduleMessageService;
+import org.apache.rocketmq.store.util.QueueTypeUtils;
 
 /**
  * Store all metadata downtime for recovery, data protection reliability
  */
-public class CommitLog {
+public class CommitLog implements Swappable {
     // Message's MAGIC CODE daa320a7
     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;
     protected final MappedFileQueue mappedFileQueue;
-    protected final DefaultMessageStore defaultMessageStore;
-    private final FlushCommitLogService flushCommitLogService;
+    protected final MessageStore defaultMessageStore;
 
-    //If TransientStorePool enabled, we must flush message to FileChannel at fixed periods
-    private final FlushCommitLogService commitLogService;
+    private final FlushManager flushManager;
 
     private final AppendMessageCallback appendMessageCallback;
     private final ThreadLocal<PutMessageThreadLocal> putMessageThreadLocal;
-    protected HashMap<String/* topic-queueid */, Long/* offset */> topicQueueTable = new HashMap<String, Long>(1024);
+
     protected volatile long confirmOffset = -1L;
 
     private volatile long beginTimeInLock = 0;
 
     protected final PutMessageLock putMessageLock;
 
+    protected final TopicQueueLock topicQueueLock;
+
     private volatile Set<String> fullStorePaths = Collections.emptySet();
 
-    public CommitLog(final DefaultMessageStore defaultMessageStore) {
-        String storePath = defaultMessageStore.getMessageStoreConfig().getStorePathCommitLog();
+    protected int commitLogSize;
+
+    public CommitLog(final MessageStore messageStore) {
+        String storePath = messageStore.getMessageStoreConfig().getStorePathCommitLog();
         if (storePath.contains(MessageStoreConfig.MULTI_PATH_SPLITTER)) {
-            this.mappedFileQueue = new MultiPathMappedFileQueue(defaultMessageStore.getMessageStoreConfig(),
-                    defaultMessageStore.getMessageStoreConfig().getMappedFileSizeCommitLog(),
-                    defaultMessageStore.getAllocateMappedFileService(), this::getFullStorePaths);
+            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;
 
-        if (FlushDiskType.SYNC_FLUSH == defaultMessageStore.getMessageStoreConfig().getFlushDiskType()) {
-            this.flushCommitLogService = new GroupCommitService();
-        } else {
-            this.flushCommitLogService = new FlushRealTimeService();
-        }
+        this.flushManager = new DefaultFlushManager();
 
-        this.commitLogService = new CommitRealTimeService();
-
-        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();
+
+        this.topicQueueLock = new TopicQueueLock();
 
+        this.commitLogSize = messageStore.getMessageStoreConfig().getMappedFileSizeCommitLog();
     }
 
     public void setFullStorePaths(Set<String> fullStorePaths) {
@@ -119,24 +124,19 @@ public class CommitLog {
 
     public boolean load() {
         boolean result = this.mappedFileQueue.load();
+        this.mappedFileQueue.checkSelf();
         log.info("load commit log " + (result ? "OK" : "Failed"));
         return result;
     }
 
     public void start() {
-        this.flushCommitLogService.start();
-
-        if (defaultMessageStore.getMessageStoreConfig().isTransientStorePoolEnable()) {
-            this.commitLogService.start();
-        }
+        this.flushManager.start();
+        log.info("start commitLog successfully. storeRoot: {}", this.defaultMessageStore.getMessageStoreConfig().getStorePathRootDir());
     }
 
     public void shutdown() {
-        if (defaultMessageStore.getMessageStoreConfig().isTransientStorePoolEnable()) {
-            this.commitLogService.shutdown();
-        }
-
-        this.flushCommitLogService.shutdown();
+        this.flushManager.shutdown();
+        log.info("shutdown commitLog successfully. storeRoot: {}", this.defaultMessageStore.getMessageStoreConfig().getStorePathRootDir());
     }
 
     public long flush() {
@@ -211,17 +211,21 @@ public class CommitLog {
             ByteBuffer byteBuffer = mappedFile.sliceByteBuffer();
             long processOffset = mappedFile.getFileFromOffset();
             long mappedFileOffset = 0;
+            // normal recover doesn't require dispatching
+            boolean doDispatch = false;
             while (true) {
                 DispatchRequest dispatchRequest = this.checkMessageAndReturnSize(byteBuffer, checkCRCOnRecover);
                 int size = dispatchRequest.getMsgSize();
                 // Normal data
                 if (dispatchRequest.isSuccess() && size > 0) {
                     mappedFileOffset += size;
+                    this.getMessageStore().onCommitLogDispatch(dispatchRequest, doDispatch, mappedFile, true, false);
                 }
                 // Come the end of the file, switch to the next file Since the
                 // return 0 representatives met last hole,
                 // this can not be included in truncate offset
                 else if (dispatchRequest.isSuccess() && size == 0) {
+                    this.getMessageStore().onCommitLogDispatch(dispatchRequest, doDispatch, mappedFile, true, true);
                     index++;
                     if (index >= mappedFiles.size()) {
                         // Current branch can not happen
@@ -402,26 +406,38 @@ public class CommitLog {
                 return new DispatchRequest(totalSize, false/* success */);
             }
 
-            return new DispatchRequest(
-                topic,
-                queueId,
-                physicOffset,
-                totalSize,
-                tagsCode,
-                storeTimestamp,
-                queueOffset,
-                keys,
-                uniqKey,
-                sysFlag,
-                preparedTransactionOffset,
-                propertiesMap
+            DispatchRequest dispatchRequest = new DispatchRequest(
+                    topic,
+                    queueId,
+                    physicOffset,
+                    totalSize,
+                    tagsCode,
+                    storeTimestamp,
+                    queueOffset,
+                    keys,
+                    uniqKey,
+                    sysFlag,
+                    preparedTransactionOffset,
+                    propertiesMap
             );
+
+            setBatchSizeIfNeeded(propertiesMap, dispatchRequest);
+
+            return dispatchRequest;
         } catch (Exception e) {
+            log.error("CheckMessageAndReturnSizeOld", e);
         }
 
         return new DispatchRequest(-1, false /* success */);
     }
 
+    private void setBatchSizeIfNeeded(Map<String, String> propertiesMap, DispatchRequest dispatchRequest) {
+        if (null != propertiesMap && propertiesMap.containsKey(MessageConst.PROPERTY_INNER_NUM) && propertiesMap.containsKey(MessageConst.PROPERTY_INNER_BASE)) {
+            dispatchRequest.setMsgBaseOffset(Long.valueOf(propertiesMap.get(MessageConst.PROPERTY_INNER_BASE)));
+            dispatchRequest.setBatchSize(Short.valueOf(propertiesMap.get(MessageConst.PROPERTY_INNER_NUM)));
+        }
+    }
+
     protected static int calMsgLength(int sysFlag, int bodyLength, int topicLength, int propertiesLength) {
         int bornhostLength = (sysFlag & MessageSysFlag.BORNHOST_V6_FLAG) == 0 ? 8 : 20;
         int storehostAddressLength = (sysFlag & MessageSysFlag.STOREHOSTADDRESS_V6_FLAG) == 0 ? 8 : 20;
@@ -479,6 +495,8 @@ public class CommitLog {
             ByteBuffer byteBuffer = mappedFile.sliceByteBuffer();
             long processOffset = mappedFile.getFileFromOffset();
             long mappedFileOffset = 0;
+            // abnormal recover require dispatching
+            boolean doDispatch = true;
             while (true) {
                 DispatchRequest dispatchRequest = this.checkMessageAndReturnSize(byteBuffer, checkCRCOnRecover);
                 int size = dispatchRequest.getMsgSize();
@@ -490,16 +508,17 @@ public class CommitLog {
 
                         if (this.defaultMessageStore.getMessageStoreConfig().isDuplicationEnable()) {
                             if (dispatchRequest.getCommitLogOffset() < this.defaultMessageStore.getConfirmOffset()) {
-                                this.defaultMessageStore.doDispatch(dispatchRequest);
+                                this.getMessageStore().onCommitLogDispatch(dispatchRequest, doDispatch, mappedFile, true, false);
                             }
                         } else {
-                            this.defaultMessageStore.doDispatch(dispatchRequest);
+                            this.getMessageStore().onCommitLogDispatch(dispatchRequest, doDispatch, mappedFile, true, false);
                         }
                     }
                     // Come the end of the file, switch to the next file
                     // Since the return 0 representatives met last hole, this can
                     // not be included in truncate offset
                     else if (size == 0) {
+                        this.getMessageStore().onCommitLogDispatch(dispatchRequest, doDispatch, mappedFile, true, true);
                         index++;
                         if (index >= mappedFiles.size()) {
                             // The current branch under normal circumstances should
@@ -540,6 +559,10 @@ public class CommitLog {
         }
     }
 
+    protected void onCommitLogAppend(MessageExtBrokerInner msg, AppendMessageResult result, MappedFile commitLogFile) {
+        this.getMessageStore().onCommitLogAppend(msg, result, commitLogFile);
+    }
+
     private boolean isMappedFileMatchedRecover(final MappedFile mappedFile) {
         ByteBuffer byteBuffer = mappedFile.sliceByteBuffer();
 
@@ -643,67 +666,80 @@ public class CommitLog {
         }
 
         PutMessageThreadLocal putMessageThreadLocal = this.putMessageThreadLocal.get();
-        PutMessageResult encodeResult = putMessageThreadLocal.getEncoder().encode(msg);
-        if (encodeResult != null) {
-            return CompletableFuture.completedFuture(encodeResult);
-        }
-        msg.setEncodedBuff(putMessageThreadLocal.getEncoder().encoderBuffer);
-        PutMessageContext putMessageContext = new PutMessageContext(generateKey(putMessageThreadLocal.getKeyBuilder(), msg));
-
+        String topicQueueKey = generateKey(putMessageThreadLocal.getKeyBuilder(), msg);
         long elapsedTimeInLock = 0;
         MappedFile unlockMappedFile = null;
 
-        putMessageLock.lock(); //spin or ReentrantLock ,depending on store config
+        topicQueueLock.lock(topicQueueKey);
         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);
+            defaultMessageStore.assignOffset(topicQueueKey, msg, getBatchNum(msg));
 
-            if (null == mappedFile || mappedFile.isFull()) {
-                mappedFile = this.mappedFileQueue.getLastMappedFile(0); // Mark: NewFile may be cause noise
-            }
-            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));
+            PutMessageResult encodeResult = putMessageThreadLocal.getEncoder().encode(msg);
+            if (encodeResult != null) {
+                return CompletableFuture.completedFuture(encodeResult);
             }
+            msg.setEncodedBuff(putMessageThreadLocal.getEncoder().encoderBuffer);
+            PutMessageContext putMessageContext = new PutMessageContext(topicQueueKey);
 
-            result = mappedFile.appendMessage(msg, this.appendMessageCallback, putMessageContext);
-            switch (result.getStatus()) {
-                case PUT_OK:
-                    break;
-                case END_OF_FILE:
-                    unlockMappedFile = mappedFile;
-                    // Create a new file, re-write the message
-                    mappedFile = this.mappedFileQueue.getLastMappedFile(0);
-                    if (null == mappedFile) {
-                        // 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));
-                    }
-                    result = mappedFile.appendMessage(msg, this.appendMessageCallback, putMessageContext);
-                    break;
-                case MESSAGE_SIZE_EXCEEDED:
-                case PROPERTIES_SIZE_EXCEEDED:
-                    beginTimeInLock = 0;
-                    return CompletableFuture.completedFuture(new PutMessageResult(PutMessageStatus.MESSAGE_ILLEGAL, result));
-                case UNKNOWN_ERROR:
-                    beginTimeInLock = 0;
-                    return CompletableFuture.completedFuture(new PutMessageResult(PutMessageStatus.UNKNOWN_ERROR, result));
-                default:
+            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 (null == mappedFile || mappedFile.isFull()) {
+                    mappedFile = this.mappedFileQueue.getLastMappedFile(0); // Mark: NewFile may be cause noise
+                }
+                if (null == mappedFile) {
+                    log.error("create mapped file1 error, topic: " + msg.getTopic() + " clientAddr: " + msg.getBornHostString());
                     beginTimeInLock = 0;
-                    return CompletableFuture.completedFuture(new PutMessageResult(PutMessageStatus.UNKNOWN_ERROR, result));
-            }
+                    return CompletableFuture.completedFuture(new PutMessageResult(PutMessageStatus.CREATE_MAPEDFILE_FAILED, null));
+                }
 
-            elapsedTimeInLock = this.defaultMessageStore.getSystemClock().now() - beginLockTimestamp;
-            beginTimeInLock = 0;
+                result = mappedFile.appendMessage(msg, this.appendMessageCallback, putMessageContext);
+                switch (result.getStatus()) {
+                    case PUT_OK:
+                        onCommitLogAppend(msg, result, mappedFile);
+                        break;
+                    case END_OF_FILE:
+                        onCommitLogAppend(msg, result, mappedFile);
+                        unlockMappedFile = mappedFile;
+                        // Create a new file, re-write the message
+                        mappedFile = this.mappedFileQueue.getLastMappedFile(0);
+                        if (null == mappedFile) {
+                            // 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));
+                        }
+                        result = mappedFile.appendMessage(msg, this.appendMessageCallback, putMessageContext);
+                        if (AppendMessageStatus.PUT_OK.equals(result.getStatus())) {
+                            onCommitLogAppend(msg, result, mappedFile);
+                        }
+                        break;
+                    case MESSAGE_SIZE_EXCEEDED:
+                    case PROPERTIES_SIZE_EXCEEDED:
+                        beginTimeInLock = 0;
+                        return CompletableFuture.completedFuture(new PutMessageResult(PutMessageStatus.MESSAGE_ILLEGAL, result));
+                    case UNKNOWN_ERROR:
+                        beginTimeInLock = 0;
+                        return CompletableFuture.completedFuture(new PutMessageResult(PutMessageStatus.UNKNOWN_ERROR, result));
+                    default:
+                        beginTimeInLock = 0;
+                        return CompletableFuture.completedFuture(new PutMessageResult(PutMessageStatus.UNKNOWN_ERROR, result));
+                }
+
+                elapsedTimeInLock = this.defaultMessageStore.getSystemClock().now() - beginLockTimestamp;
+                beginTimeInLock = 0;
+            } finally {
+                putMessageLock.unlock();
+            }
         } finally {
-            putMessageLock.unlock();
+            topicQueueLock.unlock(topicQueueKey);
         }
 
         if (elapsedTimeInLock > 500) {
@@ -770,57 +806,66 @@ public class CommitLog {
         PutMessageThreadLocal pmThreadLocal = this.putMessageThreadLocal.get();
         MessageExtEncoder batchEncoder = pmThreadLocal.getEncoder();
 
-        PutMessageContext putMessageContext = new PutMessageContext(generateKey(pmThreadLocal.getKeyBuilder(), messageExtBatch));
+        String topicQueueKey = generateKey(pmThreadLocal.getKeyBuilder(), messageExtBatch);
+
+        PutMessageContext putMessageContext = new PutMessageContext(topicQueueKey);
         messageExtBatch.setEncodedBuff(batchEncoder.encode(messageExtBatch, putMessageContext));
 
-        putMessageLock.lock();
+        topicQueueLock.lock(topicQueueKey);
         try {
-            long beginLockTimestamp = this.defaultMessageStore.getSystemClock().now();
-            this.beginTimeInLock = beginLockTimestamp;
+            defaultMessageStore.assignOffset(topicQueueKey, messageExtBatch, (short) putMessageContext.getBatchSize());
 
-            // Here settings are stored timestamp, in order to ensure an orderly
-            // global
-            messageExtBatch.setStoreTimestamp(beginLockTimestamp);
+            putMessageLock.lock();
+            try {
+                long beginLockTimestamp = this.defaultMessageStore.getSystemClock().now();
+                this.beginTimeInLock = beginLockTimestamp;
 
-            if (null == mappedFile || mappedFile.isFull()) {
-                mappedFile = this.mappedFileQueue.getLastMappedFile(0); // Mark: NewFile may be cause noise
-            }
-            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));
-            }
+                // Here settings are stored timestamp, in order to ensure an orderly
+                // global
+                messageExtBatch.setStoreTimestamp(beginLockTimestamp);
 
-            result = mappedFile.appendMessages(messageExtBatch, this.appendMessageCallback, putMessageContext);
-            switch (result.getStatus()) {
-                case PUT_OK:
-                    break;
-                case END_OF_FILE:
-                    unlockMappedFile = mappedFile;
-                    // Create a new file, re-write the message
-                    mappedFile = this.mappedFileQueue.getLastMappedFile(0);
-                    if (null == mappedFile) {
-                        // 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));
-                    }
-                    result = mappedFile.appendMessages(messageExtBatch, this.appendMessageCallback, putMessageContext);
-                    break;
-                case MESSAGE_SIZE_EXCEEDED:
-                case PROPERTIES_SIZE_EXCEEDED:
-                    beginTimeInLock = 0;
-                    return CompletableFuture.completedFuture(new PutMessageResult(PutMessageStatus.MESSAGE_ILLEGAL, result));
-                case UNKNOWN_ERROR:
-                default:
+                if (null == mappedFile || mappedFile.isFull()) {
+                    mappedFile = this.mappedFileQueue.getLastMappedFile(0); // Mark: NewFile may be cause noise
+                }
+                if (null == mappedFile) {
+                    log.error("Create mapped file1 error, topic: {} clientAddr: {}", messageExtBatch.getTopic(), messageExtBatch.getBornHostString());
                     beginTimeInLock = 0;
-                    return CompletableFuture.completedFuture(new PutMessageResult(PutMessageStatus.UNKNOWN_ERROR, result));
-            }
+                    return CompletableFuture.completedFuture(new PutMessageResult(PutMessageStatus.CREATE_MAPEDFILE_FAILED, null));
+                }
+
+                result = mappedFile.appendMessages(messageExtBatch, this.appendMessageCallback, putMessageContext);
+                switch (result.getStatus()) {
+                    case PUT_OK:
+                        break;
+                    case END_OF_FILE:
+                        unlockMappedFile = mappedFile;
+                        // Create a new file, re-write the message
+                        mappedFile = this.mappedFileQueue.getLastMappedFile(0);
+                        if (null == mappedFile) {
+                            // 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));
+                        }
+                        result = mappedFile.appendMessages(messageExtBatch, this.appendMessageCallback, putMessageContext);
+                        break;
+                    case MESSAGE_SIZE_EXCEEDED:
+                    case PROPERTIES_SIZE_EXCEEDED:
+                        beginTimeInLock = 0;
+                        return CompletableFuture.completedFuture(new PutMessageResult(PutMessageStatus.MESSAGE_ILLEGAL, result));
+                    case UNKNOWN_ERROR:
+                    default:
+                        beginTimeInLock = 0;
+                        return CompletableFuture.completedFuture(new PutMessageResult(PutMessageStatus.UNKNOWN_ERROR, result));
+                }
 
-            elapsedTimeInLock = this.defaultMessageStore.getSystemClock().now() - beginLockTimestamp;
-            beginTimeInLock = 0;
+                elapsedTimeInLock = this.defaultMessageStore.getSystemClock().now() - beginLockTimestamp;
+                beginTimeInLock = 0;
+            } finally {
+                putMessageLock.unlock();
+            }
         } finally {
-            putMessageLock.unlock();
+            topicQueueLock.unlock(topicQueueKey);
         }
 
         if (elapsedTimeInLock > 500) {
@@ -856,28 +901,7 @@ public class CommitLog {
     }
 
     public CompletableFuture<PutMessageStatus> submitFlushRequest(AppendMessageResult result, MessageExt messageExt) {
-        // Synchronization flush
-        if (FlushDiskType.SYNC_FLUSH == this.defaultMessageStore.getMessageStoreConfig().getFlushDiskType()) {
-            final GroupCommitService service = (GroupCommitService) this.flushCommitLogService;
-            if (messageExt.isWaitStoreMsgOK()) {
-                GroupCommitRequest request = new GroupCommitRequest(result.getWroteOffset() + result.getWroteBytes(),
-                        this.defaultMessageStore.getMessageStoreConfig().getSyncFlushTimeout());
-                service.putRequest(request);
-                return request.future();
-            } else {
-                service.wakeup();
-                return CompletableFuture.completedFuture(PutMessageStatus.PUT_OK);
-            }
-        }
-        // Asynchronous flush
-        else {
-            if (!this.defaultMessageStore.getMessageStoreConfig().isTransientStorePoolEnable()) {
-                flushCommitLogService.wakeup();
-            } else  {
-                commitLogService.wakeup();
-            }
-            return CompletableFuture.completedFuture(PutMessageStatus.PUT_OK);
-        }
+        return this.flushManager.handleDiskFlush(result, messageExt);
     }
 
     public CompletableFuture<PutMessageStatus> submitReplicaRequest(AppendMessageResult result, MessageExt messageExt) {
@@ -899,6 +923,42 @@ public class CommitLog {
         return CompletableFuture.completedFuture(PutMessageStatus.PUT_OK);
     }
 
+
+    public void handleDiskFlush(AppendMessageResult result, PutMessageResult putMessageResult, MessageExt messageExt) {
+        this.flushManager.handleDiskFlush(result, putMessageResult, messageExt);
+    }
+
+    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());
+                    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);
+                }
+            }
+        }
+
+    }
+
     /**
      * According to receive certain message or offset storage time if an error occurs, it returns -1
      */
@@ -948,14 +1008,6 @@ public class CommitLog {
         return offset + mappedFileSize - offset % mappedFileSize;
     }
 
-    public HashMap<String, Long> getTopicQueueTable() {
-        return topicQueueTable;
-    }
-
-    public void setTopicQueueTable(HashMap<String, Long> topicQueueTable) {
-        this.topicQueueTable = topicQueueTable;
-    }
-
     public void destroy() {
         this.mappedFileQueue.destroy();
     }
@@ -982,7 +1034,7 @@ public class CommitLog {
     public void removeQueueFromTopicQueueTable(final String topic, final int queueId) {
         String key = topic + "-" + queueId;
         synchronized (this) {
-            this.topicQueueTable.remove(key);
+            this.defaultMessageStore.removeOffsetTable(key);
         }
 
         log.info("removeQueueFromTopicQueueTable OK Topic: {} QueueId: {}", topic, queueId);
@@ -1006,6 +1058,21 @@ public class CommitLog {
         return diff;
     }
 
+    protected short getBatchNum(MessageExtBrokerInner msgInner) {
+        short batchNum = 1;
+        // IF inner batch, build batchQueueOffset and batchNum property.
+        CQType cqType = QueueTypeUtils.getCQType(defaultMessageStore);
+        if (MessageSysFlag.check(msgInner.getSysFlag(), MessageSysFlag.INNER_BATCH_FLAG) || CQType.BatchCQ.equals(cqType)) {
+
+            if (msgInner.getProperty(MessageConst.PROPERTY_INNER_NUM) != null) {
+                batchNum = Short.parseShort(msgInner.getProperty(MessageConst.PROPERTY_INNER_NUM));
+                batchNum = batchNum >= 1 ? batchNum : 1;
+            }
+        }
+
+        return batchNum;
+    }
+
     abstract class FlushCommitLogService extends ServiceThread {
         protected static final int RETRY_TIMES_OVER = 10;
     }
@@ -1041,10 +1108,9 @@ public class CommitLog {
                     long end = System.currentTimeMillis();
                     if (!result) {
                         this.lastCommitTimestamp = end; // result = false means some data committed.
-                        //now wake up flush thread.
-                        flushCommitLogService.wakeup();
+                        CommitLog.this.flushManager.wakeUpFlush();
                     }
-
+                    CommitLog.this.getMessageStore().getPerfCounter().flowOnce("COMMIT_DATA_TIME_MS", (int)(end - begin));
                     if (end - begin > 500) {
                         log.info("Commit data to file costs {} ms", end - begin);
                     }
@@ -1107,6 +1173,7 @@ public class CommitLog {
                         CommitLog.this.defaultMessageStore.getStoreCheckpoint().setPhysicMsgTimestamp(storeTimestamp);
                     }
                     long past = System.currentTimeMillis() - begin;
+                    CommitLog.this.getMessageStore().getPerfCounter().flowOnce("FLUSH_DATA_TIME_MS", (int)past);
                     if (past > 500) {
                         log.info("Flush data to disk costs {} ms", past);
                     }
@@ -1275,6 +1342,114 @@ public class CommitLog {
         }
     }
 
+    class GroupCheckService extends FlushCommitLogService {
+        private volatile List<GroupCommitRequest> requestsWrite = new ArrayList<GroupCommitRequest>();
+        private volatile List<GroupCommitRequest> requestsRead = new ArrayList<GroupCommitRequest>();
+
+        public boolean isAynscRequestsFull() {
+            return requestsWrite.size() > CommitLog.this.defaultMessageStore.getMessageStoreConfig().getMaxAsyncPutMessageRequests() * 2;
+        }
+
+        public synchronized boolean putRequest(final GroupCommitRequest request) {
+            synchronized (this.requestsWrite) {
+                this.requestsWrite.add(request);
+            }
+            if (hasNotified.compareAndSet(false, true)) {
+                waitPoint.countDown(); // notify
+            }
+            boolean flag = this.requestsWrite.size() >
+                    CommitLog.this.defaultMessageStore.getMessageStoreConfig().getMaxAsyncPutMessageRequests();
+            if (flag) {
+                log.info("Async requests {} exceeded the threshold {}", requestsWrite.size(),
+                        CommitLog.this.defaultMessageStore.getMessageStoreConfig().getMaxAsyncPutMessageRequests());
+            }
+
+            return flag;
+        }
+
+        private void swapRequests() {
+            List<GroupCommitRequest> tmp = this.requestsWrite;
+            this.requestsWrite = this.requestsRead;
+            this.requestsRead = tmp;
+        }
+
+        private void doCommit() {
+            synchronized (this.requestsRead) {
+                if (!this.requestsRead.isEmpty()) {
+                    for (GroupCommitRequest req : this.requestsRead) {
+                        // There may be a message in the next file, so a maximum of
+                        // two times the flush
+                        boolean flushOK = false;
+                        for (int i = 0; i < 1000; i++) {
+                            flushOK = CommitLog.this.mappedFileQueue.getFlushedWhere() >= req.getNextOffset();
+                            if (flushOK) {
+                                break;
+                            } else {
+                                try {
+                                    Thread.sleep(1);
+                                } catch (Throwable ignored) {
+
+                                }
+                            }
+                        }
+                        req.wakeupCustomer(flushOK ? PutMessageStatus.PUT_OK : PutMessageStatus.FLUSH_DISK_TIMEOUT);
+                    }
+
+                    long storeTimestamp = CommitLog.this.mappedFileQueue.getStoreTimestamp();
+                    if (storeTimestamp > 0) {
+                        CommitLog.this.defaultMessageStore.getStoreCheckpoint().setPhysicMsgTimestamp(storeTimestamp);
+                    }
+
+                    this.requestsRead.clear();
+                }
+            }
+        }
+
+        public void run() {
+            CommitLog.log.info(this.getServiceName() + " service started");
+
+            while (!this.isStopped()) {
+                try {
+                    this.waitForRunning(1);
+                    this.doCommit();
+                } catch (Exception e) {
+                    CommitLog.log.warn(this.getServiceName() + " service has exception. ", e);
+                }
+            }
+
+            // Under normal circumstances shutdown, wait for the arrival of the
+            // request, and then flush
+            try {
+                Thread.sleep(10);
+            } catch (InterruptedException e) {
+                CommitLog.log.warn("GroupCommitService Exception, ", e);
+            }
+
+            synchronized (this) {
+                this.swapRequests();
+            }
+
+            this.doCommit();
+
+            CommitLog.log.info(this.getServiceName() + " service end");
+        }
+
+        @Override
+        protected void onWaitEnd() {
+            this.swapRequests();
+        }
+
+        @Override
+        public String getServiceName() {
+            return CommitLog.GroupCheckService.class.getSimpleName();
+        }
+
+        @Override
+        public long getJointime() {
+            return 1000 * 60 * 5;
+        }
+    }
+
     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;
@@ -1310,12 +1485,10 @@ public class CommitLog {
             };
 
             // Record ConsumeQueue information
-            String key = putMessageContext.getTopicQueueTableKey();
-            Long queueOffset = CommitLog.this.topicQueueTable.get(key);
-            if (null == queueOffset) {
-                queueOffset = 0L;
-                CommitLog.this.topicQueueTable.put(key, queueOffset);
-            }
+            Long queueOffset = msgInner.getQueueOffset();
+
+            // this msg maybe a inner-batch msg.
+            short batchNum = getBatchNum(msgInner);
 
             // Transaction messages that require special handling
             final int tranType = MessageSysFlag.getTransactionValue(msgInner.getSysFlag());
@@ -1366,25 +1539,13 @@ public class CommitLog {
 
 
             final long beginTimeMills = CommitLog.this.defaultMessageStore.now();
+            CommitLog.this.getMessageStore().getPerfCounter().startTick("WRITE_MEMORY_TIME_MS");
             // Write messages to the queue buffer
             byteBuffer.put(preEncodeBuffer);
+            CommitLog.this.getMessageStore().getPerfCounter().endTick("WRITE_MEMORY_TIME_MS");
             msgInner.setEncodedBuff(null);
-            AppendMessageResult result = new AppendMessageResult(AppendMessageStatus.PUT_OK, wroteOffset, msgLen, msgIdSupplier,
-                msgInner.getStoreTimestamp(), queueOffset, CommitLog.this.defaultMessageStore.now() - beginTimeMills);
-
-            switch (tranType) {
-                case MessageSysFlag.TRANSACTION_PREPARED_TYPE:
-                case MessageSysFlag.TRANSACTION_ROLLBACK_TYPE:
-                    break;
-                case MessageSysFlag.TRANSACTION_NOT_TYPE:
-                case MessageSysFlag.TRANSACTION_COMMIT_TYPE:
-                    // The next update ConsumeQueue information
-                    CommitLog.this.topicQueueTable.put(key, ++queueOffset);
-                    break;
-                default:
-                    break;
-            }
-            return result;
+            return new AppendMessageResult(AppendMessageStatus.PUT_OK, wroteOffset, msgLen, msgIdSupplier,
+                msgInner.getStoreTimestamp(), queueOffset, CommitLog.this.defaultMessageStore.now() - beginTimeMills, batchNum);
         }
 
         public AppendMessageResult doAppend(final long fileFromOffset, final ByteBuffer byteBuffer, final int maxBlank,
@@ -1393,12 +1554,7 @@ public class CommitLog {
             //physical offset
             long wroteOffset = fileFromOffset + byteBuffer.position();
             // Record ConsumeQueue information
-            String key = putMessageContext.getTopicQueueTableKey();
-            Long queueOffset = CommitLog.this.topicQueueTable.get(key);
-            if (null == queueOffset) {
-                queueOffset = 0L;
-                CommitLog.this.topicQueueTable.put(key, queueOffset);
-            }
+            Long queueOffset = messageExtBatch.getQueueOffset();
             long beginQueueOffset = queueOffset;
             int totalMsgLen = 0;
             int msgNum = 0;
@@ -1482,7 +1638,6 @@ public class CommitLog {
             AppendMessageResult result = new AppendMessageResult(AppendMessageStatus.PUT_OK, wroteOffset, totalMsgLen, msgIdSupplier,
                 messageExtBatch.getStoreTimestamp(), beginQueueOffset, CommitLog.this.defaultMessageStore.now() - beginTimeMills);
             result.setMsgNum(msgNum);
-            CommitLog.this.topicQueueTable.put(key, queueOffset);
 
             return result;
         }
@@ -1537,6 +1692,8 @@ public class CommitLog {
 
             final int msgLen = calMsgLength(msgInner.getSysFlag(), bodyLength, topicLength, propertiesLength);
 
+            final long queueOffset = msgInner.getQueueOffset();
+
             // Exceeds the maximum message
             if (msgLen > this.maxMessageSize) {
                 CommitLog.log.warn("message size exceeded, msg total size: " + msgLen + ", msg body size: " + bodyLength
@@ -1556,8 +1713,8 @@ public class CommitLog {
             this.encoderBuffer.putInt(msgInner.getQueueId());
             // 5 FLAG
             this.encoderBuffer.putInt(msgInner.getFlag());
-            // 6 QUEUEOFFSET, need update later
-            this.encoderBuffer.putLong(0);
+            // 6 QUEUEOFFSET
+            this.encoderBuffer.putLong(queueOffset);
             // 7 PHYSICALOFFSET, need update later
             this.encoderBuffer.putLong(0);
             // 8 SYSFLAG
@@ -1706,50 +1863,153 @@ public class CommitLog {
 
     }
 
-    static class PutMessageThreadLocal {
-        private MessageExtEncoder encoder;
-        private StringBuilder keyBuilder;
-        PutMessageThreadLocal(int size) {
-            encoder = new MessageExtEncoder(size);
-            keyBuilder = new StringBuilder();
-        }
+    interface FlushManager {
+        void start();
+        void shutdown();
+        void wakeUpFlush();
+        void handleDiskFlush(AppendMessageResult result, PutMessageResult putMessageResult, MessageExt messageExt);
+        CompletableFuture<PutMessageStatus> handleDiskFlush(AppendMessageResult result, MessageExt messageExt);
+    }
 
-        public MessageExtEncoder getEncoder() {
-            return encoder;
+    class DefaultFlushManager implements FlushManager {
+
+        private final FlushCommitLogService flushCommitLogService;
+
+        //If TransientStorePool enabled, we must flush message to FileChannel at fixed periods
+        private final FlushCommitLogService commitLogService;
+
+        public DefaultFlushManager() {
+            if (FlushDiskType.SYNC_FLUSH == CommitLog.this.defaultMessageStore.getMessageStoreConfig().getFlushDiskType()) {
+                this.flushCommitLogService = new CommitLog.GroupCommitService();
+            } else {
+                this.flushCommitLogService = new CommitLog.FlushRealTimeService();
+            }
+
+            this.commitLogService = new CommitLog.CommitRealTimeService();
         }
 
-        public StringBuilder getKeyBuilder() {
-            return keyBuilder;
+        @Override
+        public void start() {
+            this.flushCommitLogService.start();
+
+            if (defaultMessageStore.getMessageStoreConfig().isTransientStorePoolEnable()) {
+                this.commitLogService.start();
+            }
         }
-    }
 
-    static class PutMessageContext {
-        private String topicQueueTableKey;
-        private long[] phyPos;
-        private int batchSize;
+        public void handleDiskFlush(AppendMessageResult result, PutMessageResult putMessageResult, MessageExt messageExt) {
+            // Synchronization flush
+            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());
+                    service.putRequest(request);
+                    CompletableFuture<PutMessageStatus> flushOkFuture = request.future();
+                    PutMessageStatus flushStatus = null;
+                    try {
+                        flushStatus = flushOkFuture.get(CommitLog.this.defaultMessageStore.getMessageStoreConfig().getSyncFlushTimeout(),
+                                TimeUnit.MILLISECONDS);
+                    } catch (InterruptedException | ExecutionException | TimeoutException e) {
+                        //flushOK=false;
+                    }
+                    if (flushStatus != PutMessageStatus.PUT_OK) {
+                        log.error("do groupcommit, wait for flush failed, topic: " + messageExt.getTopic() + " tags: " + messageExt.getTags()
+                                + " client address: " + messageExt.getBornHostString());
+                        putMessageResult.setPutMessageStatus(PutMessageStatus.FLUSH_DISK_TIMEOUT);
+                    }
+                } else {
+                    service.wakeup();
+                }
+            }
+            // Asynchronous flush
+            else {
+                if (!CommitLog.this.defaultMessageStore.getMessageStoreConfig().isTransientStorePoolEnable()) {
+                    flushCommitLogService.wakeup();
+                } else {
+                    commitLogService.wakeup();
+                }
+            }
+        }
 
-        public PutMessageContext(String topicQueueTableKey) {
-            this.topicQueueTableKey = topicQueueTableKey;
+        @Override
+        public CompletableFuture<PutMessageStatus> handleDiskFlush(AppendMessageResult result, MessageExt messageExt) {
+            // Synchronization flush
+            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());
+                    service.putRequest(request);
+                    return request.future();
+                } else {
+                    service.wakeup();
+                    return CompletableFuture.completedFuture(PutMessageStatus.PUT_OK);
+                }
+            }
+            // Asynchronous flush
+            else {
+                if (!CommitLog.this.defaultMessageStore.getMessageStoreConfig().isTransientStorePoolEnable()) {
+                    flushCommitLogService.wakeup();
+                } else  {
+                    commitLogService.wakeup();
+                }
+                return CompletableFuture.completedFuture(PutMessageStatus.PUT_OK);
+            }
         }
 
-        public String getTopicQueueTableKey() {
-            return topicQueueTableKey;
+        @Override
+        public void wakeUpFlush() {
+            // now wake up flush thread.
+            flushCommitLogService.wakeup();
         }
 
-        public long[] getPhyPos() {
-            return phyPos;
+        @Override
+        public void shutdown() {
+            if (defaultMessageStore.getMessageStoreConfig().isTransientStorePoolEnable()) {
+                this.commitLogService.shutdown();
+            }
+
+            this.flushCommitLogService.shutdown();
         }
 
-        public void setPhyPos(long[] phyPos) {
-            this.phyPos = phyPos;
+    }
+
+    public int getCommitLogSize() {
+        return commitLogSize;
+    }
+
+    public MappedFileQueue getMappedFileQueue() {
+        return mappedFileQueue;
+    }
+
+    public MessageStore getMessageStore() {
+        return defaultMessageStore;
+    }
+
+    @Override
+    public void swapMap(int reserveNum, long forceSwapIntervalMs, long normalSwapIntervalMs) {
+        this.getMappedFileQueue().swapMap(reserveNum, forceSwapIntervalMs, normalSwapIntervalMs);
+    }
+
+    @Override
+    public void cleanSwappedMap(long forceCleanSwapIntervalMs) {
+        this.getMappedFileQueue().cleanSwappedMap(forceCleanSwapIntervalMs);
+    }
+
+    static class PutMessageThreadLocal {
+        private MessageExtEncoder encoder;
+        private StringBuilder keyBuilder;
+        PutMessageThreadLocal(int size) {
+            encoder = new MessageExtEncoder(size);
+            keyBuilder = new StringBuilder();
         }
 
-        public int getBatchSize() {
-            return batchSize;
+        public MessageExtEncoder getEncoder() {
+            return encoder;
         }
 
-        public void setBatchSize(int batchSize) {
-            this.batchSize = batchSize;
+        public StringBuilder getKeyBuilder() {
+            return keyBuilder;
         }
     }
 }
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 87ff0a0..0efe74c 100644
--- a/store/src/main/java/org/apache/rocketmq/store/ConsumeQueue.java
+++ b/store/src/main/java/org/apache/rocketmq/store/ConsumeQueue.java
@@ -24,8 +24,14 @@ 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.StorePathConfigHelper;
-
-public class ConsumeQueue {
+import org.apache.rocketmq.store.logfile.MappedFile;
+import org.apache.rocketmq.store.queue.CQType;
+import org.apache.rocketmq.store.queue.ConsumeQueueInterface;
+import org.apache.rocketmq.store.queue.CqUnit;
+import org.apache.rocketmq.store.queue.FileQueueLifeCycle;
+import org.apache.rocketmq.store.queue.ReferredIterator;
+
+public class ConsumeQueue implements ConsumeQueueInterface, FileQueueLifeCycle {
     private static final InternalLogger log = InternalLoggerFactory.getLogger(LoggerName.STORE_LOGGER_NAME);
 
     public static final int CQ_STORE_UNIT_SIZE = 20;
@@ -76,6 +82,7 @@ public class ConsumeQueue {
         }
     }
 
+    @Override
     public boolean load() {
         boolean result = this.mappedFileQueue.load();
         log.info("load consume queue " + this.topic + "-" + this.queueId + " " + (result ? "OK" : "Failed"));
@@ -85,6 +92,7 @@ public class ConsumeQueue {
         return result;
     }
 
+    @Override
     public void recover() {
         final List<MappedFile> mappedFiles = this.mappedFileQueue.getMappedFiles();
         if (!mappedFiles.isEmpty()) {
@@ -152,6 +160,7 @@ public class ConsumeQueue {
         }
     }
 
+    @Override
     public long getOffsetInQueueByTime(final long timestamp) {
         MappedFile mappedFile = this.mappedFileQueue.getMappedFileByTime(timestamp);
         if (mappedFile != null) {
@@ -221,6 +230,7 @@ public class ConsumeQueue {
         return 0;
     }
 
+    @Override
     public void truncateDirtyLogicFiles(long phyOffet) {
 
         int logicFileSize = this.mappedFileSize;
@@ -291,6 +301,7 @@ public class ConsumeQueue {
         }
     }
 
+    @Override
     public long getLastOffset() {
         long lastOffset = -1;
 
@@ -321,6 +332,7 @@ public class ConsumeQueue {
         return lastOffset;
     }
 
+    @Override
     public boolean flush(final int flushLeastPages) {
         boolean result = this.mappedFileQueue.flush(flushLeastPages);
         if (isExtReadEnable()) {
@@ -330,12 +342,14 @@ public class ConsumeQueue {
         return result;
     }
 
+    @Override
     public int deleteExpiredFile(long offset) {
         int cnt = this.mappedFileQueue.deleteExpiredFileByOffset(offset, CQ_STORE_UNIT_SIZE);
         this.correctMinOffset(offset);
         return cnt;
     }
 
+    @Override
     public void correctMinOffset(long phyMinOffset) {
         MappedFile mappedFile = this.mappedFileQueue.getFirstMappedFile();
         long minExtAddr = 1;
@@ -372,10 +386,12 @@ public class ConsumeQueue {
         }
     }
 
+    @Override
     public long getMinOffsetInQueue() {
         return this.minLogicOffset / CQ_STORE_UNIT_SIZE;
     }
 
+    @Override
     public void putMessagePositionInfoWrapper(DispatchRequest request) {
         final int maxRetries = 30;
         boolean canWrite = this.defaultMessageStore.getRunningFlags().isCQWriteable();
@@ -488,7 +504,7 @@ public class ConsumeQueue {
         }
     }
 
-    public SelectMappedBufferResult getIndexBuffer(final long startIndex) {
+    private SelectMappedBufferResult getIndexBuffer(final long startIndex) {
         int mappedFileSize = this.mappedFileSize;
         long offset = startIndex * CQ_STORE_UNIT_SIZE;
         if (offset >= this.getMinLogicOffset()) {
@@ -501,6 +517,124 @@ public class ConsumeQueue {
         return null;
     }
 
+    @Override
+    public ReferredIterator<CqUnit> iterateFrom(long startOffset) {
+        SelectMappedBufferResult sbr = getIndexBuffer(startOffset);
+        if (sbr == null) {
+            return null;
+        }
+        return new ConsumeQueueIterator(sbr);
+    }
+
+    @Override
+    public CqUnit get(long offset) {
+        ReferredIterator<CqUnit> it = iterateFrom(offset);
+        if (it == null) {
+            return null;
+        }
+        return it.nextAndRelease();
+    }
+
+    @Override
+    public CqUnit getEarliestUnit() {
+        /**
+         * here maybe should not return null
+         */
+        ReferredIterator<CqUnit> it = iterateFrom(minLogicOffset / CQ_STORE_UNIT_SIZE);
+        if (it == null) {
+            return null;
+        }
+        return it.nextAndRelease();
+    }
+
+    @Override
+    public CqUnit getLatestUnit() {
+        ReferredIterator<CqUnit> it = iterateFrom((mappedFileQueue.getMaxOffset() / CQ_STORE_UNIT_SIZE) - 1);
+        if (it == null) {
+            return null;
+        }
+        return it.nextAndRelease();
+    }
+
+    @Override
+    public boolean isFirstFileAvailable() {
+        return false;
+    }
+
+    @Override
+    public boolean isFirstFileExist() {
+        return false;
+    }
+
+    private class ConsumeQueueIterator implements ReferredIterator<CqUnit> {
+        private SelectMappedBufferResult sbr;
+        private int relativePos = 0;
+
+        public ConsumeQueueIterator(SelectMappedBufferResult sbr) {
+            this.sbr =  sbr;
+            if (sbr != null && sbr.getByteBuffer() != null) {
+                relativePos = sbr.getByteBuffer().position();
+            }
+        }
+
+        @Override
+        public boolean hasNext() {
+            if (sbr == null || sbr.getByteBuffer() == null) {
+                return false;
+            }
+
+            return sbr.getByteBuffer().hasRemaining();
+        }
+
+        @Override
+        public CqUnit next() {
+            if (!hasNext()) {
+                return null;
+            }
+            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());
+
+            if (isExtAddr(cqUnit.getTagsCode())) {
+                ConsumeQueueExt.CqExtUnit cqExtUnit = new ConsumeQueueExt.CqExtUnit();
+                boolean extRet = getExt(cqUnit.getTagsCode(), cqExtUnit);
+                if (extRet) {
+                    cqUnit.setTagsCode(cqExtUnit.getTagsCode());
+                    cqUnit.setCqExtUnit(cqExtUnit);
+                } 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());
+                }
+            }
+            return cqUnit;
+        }
+
+        @Override
+        public void remove() {
+            throw new UnsupportedOperationException("remove");
+        }
+
+        @Override
+        public void release() {
+            if (sbr != null) {
+                sbr.release();
+                sbr = null;
+            }
+        }
+
+        @Override
+        public CqUnit nextAndRelease() {
+            try {
+                return next();
+            } finally {
+                release();
+            }
+        }
+    }
+
     public ConsumeQueueExt.CqExtUnit getExt(final long offset) {
         if (isExtReadEnable()) {
             return this.consumeQueueExt.get(offset);
@@ -515,6 +649,7 @@ public class ConsumeQueue {
         return false;
     }
 
+    @Override
     public long getMinLogicOffset() {
         return minLogicOffset;
     }
@@ -523,20 +658,29 @@ public class ConsumeQueue {
         this.minLogicOffset = minLogicOffset;
     }
 
+    @Override
     public long rollNextFile(final long index) {
         int mappedFileSize = this.mappedFileSize;
         int totalUnitsInFile = mappedFileSize / CQ_STORE_UNIT_SIZE;
         return index + totalUnitsInFile - index % totalUnitsInFile;
     }
 
+    @Override
     public String getTopic() {
         return topic;
     }
 
+    @Override
     public int getQueueId() {
         return queueId;
     }
 
+    @Override
+    public CQType getCQType() {
+        return CQType.SimpleCQ;
+    }
+
+    @Override
     public long getMaxPhysicOffset() {
         return maxPhysicOffset;
     }
@@ -545,6 +689,7 @@ public class ConsumeQueue {
         this.maxPhysicOffset = maxPhysicOffset;
     }
 
+    @Override
     public void destroy() {
         this.maxPhysicOffset = -1;
         this.minLogicOffset = 0;
@@ -554,14 +699,17 @@ public class ConsumeQueue {
         }
     }
 
+    @Override
     public long getMessageTotalInQueue() {
         return this.getMaxOffsetInQueue() - this.getMinOffsetInQueue();
     }
 
+    @Override
     public long getMaxOffsetInQueue() {
         return this.mappedFileQueue.getMaxOffset() / CQ_STORE_UNIT_SIZE;
     }
 
+    @Override
     public void checkSelf() {
         mappedFileQueue.checkSelf();
         if (isExtReadEnable()) {
@@ -584,4 +732,14 @@ public class ConsumeQueue {
     public boolean isExtAddr(long tagsCode) {
         return ConsumeQueueExt.isExtAddr(tagsCode);
     }
+
+    @Override
+    public void swapMap(int reserveNum, long forceSwapIntervalMs, long normalSwapIntervalMs) {
+        mappedFileQueue.swapMap(reserveNum, forceSwapIntervalMs, normalSwapIntervalMs);
+    }
+
+    @Override
+    public void cleanSwappedMap(long forceCleanSwapIntervalMs) {
+        mappedFileQueue.cleanSwappedMap(forceCleanSwapIntervalMs);
+    }
 }
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 117a70b..19c7992 100644
--- a/store/src/main/java/org/apache/rocketmq/store/ConsumeQueueExt.java
+++ b/store/src/main/java/org/apache/rocketmq/store/ConsumeQueueExt.java
@@ -26,6 +26,7 @@ import java.nio.ByteBuffer;
 import java.util.ArrayList;
 import java.util.Arrays;
 import java.util.List;
+import org.apache.rocketmq.store.logfile.MappedFile;
 
 /**
  * Extend of consume queue, to store something not important,
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 0061369..d25b9eb 100644
--- a/store/src/main/java/org/apache/rocketmq/store/DefaultMessageStore.java
+++ b/store/src/main/java/org/apache/rocketmq/store/DefaultMessageStore.java
@@ -51,30 +51,45 @@ import org.apache.rocketmq.common.UtilAll;
 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.message.MessageExtBatch;
 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.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.config.StorePathConfigHelper;
 import org.apache.rocketmq.store.dledger.DLedgerCommitLog;
 import org.apache.rocketmq.store.ha.HAService;
 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;
+
+import static java.lang.String.format;
 
 public class DefaultMessageStore implements MessageStore {
     private static final InternalLogger log = InternalLoggerFactory.getLogger(LoggerName.STORE_LOGGER_NAME);
 
+    public final PerfCounter.Ticks perfs = new PerfCounter.Ticks(log);
+
     private final MessageStoreConfig messageStoreConfig;
     // CommitLog
     private final CommitLog commitLog;
 
-    private final ConcurrentMap<String/* topic */, ConcurrentMap<Integer/* queueId */, ConsumeQueue>> consumeQueueTable;
+    private final ConsumeQueueStore consumeQueueStore;
+
+    private final ConcurrentMap<String/* topic */, ConcurrentMap<Integer/* queueId */, ConsumeQueueInterface>> consumeQueueTable;
+
+    protected HashMap<String/* topic-queueid */, Long/* offset */> topicQueueTable = new HashMap<>(1024);
 
     private final FlushConsumeQueueService flushConsumeQueueService;
 
@@ -124,6 +139,9 @@ public class DefaultMessageStore implements MessageStore {
 
     private final List<CleanFilesHook> cleanFilesHooks = new CopyOnWriteArrayList<>();
 
+    // Max pull msg size
+    private final static int MAX_PULL_MSG_SIZE = 128 * 1024 * 1024;
+
     public DefaultMessageStore(final MessageStoreConfig messageStoreConfig, final BrokerStatsManager brokerStatsManager,
         final MessageArrivingListener messageArrivingListener, final BrokerConfig brokerConfig) throws IOException {
         this.messageArrivingListener = messageArrivingListener;
@@ -137,6 +155,7 @@ public class DefaultMessageStore implements MessageStore {
             this.commitLog = new CommitLog(this);
         }
         this.consumeQueueTable = new ConcurrentHashMap<>(32);
+        this.consumeQueueStore = new ConsumeQueueStore(this, this.messageStoreConfig, this.consumeQueueTable);
 
         this.flushConsumeQueueService = new FlushConsumeQueueService();
         this.cleanCommitLogService = new CleanCommitLogService();
@@ -167,18 +186,19 @@ public class DefaultMessageStore implements MessageStore {
         this.dispatcherList.addLast(new CommitLogDispatcherBuildIndex());
 
         File file = new File(StorePathConfigHelper.getLockFile(messageStoreConfig.getStorePathRootDir()));
-        MappedFile.ensureDirOK(file.getParent());
-        MappedFile.ensureDirOK(getStorePathPhysic());
-        MappedFile.ensureDirOK(getStorePathLogic());
+        DefaultMappedFile.ensureDirOK(file.getParent());
+        DefaultMappedFile.ensureDirOK(getStorePathPhysic());
+        DefaultMappedFile.ensureDirOK(getStorePathLogic());
         lockFile = new RandomAccessFile(file, "rw");
     }
 
+    @Override
     public void truncateDirtyLogicFiles(long phyOffset) {
-        ConcurrentMap<String, ConcurrentMap<Integer, ConsumeQueue>> tables = DefaultMessageStore.this.consumeQueueTable;
+        ConcurrentMap<String, ConcurrentMap<Integer, ConsumeQueueInterface>> tables = DefaultMessageStore.this.consumeQueueTable;
 
-        for (ConcurrentMap<Integer, ConsumeQueue> maps : tables.values()) {
-            for (ConsumeQueue logic : maps.values()) {
-                logic.truncateDirtyLogicFiles(phyOffset);
+        for (ConcurrentMap<Integer, ConsumeQueueInterface> maps : tables.values()) {
+            for (ConsumeQueueInterface logic : maps.values()) {
+                this.consumeQueueStore.truncateDirtyLogicFiles(logic, phyOffset);
             }
         }
     }
@@ -186,12 +206,14 @@ public class DefaultMessageStore implements MessageStore {
     /**
      * @throws IOException
      */
+    @Override
     public boolean load() {
         boolean result = true;
 
         try {
+            long start = System.currentTimeMillis();
             boolean lastExitOK = !this.isTempFileExist();
-            log.info("last shutdown {}", lastExitOK ? "normally" : "abnormally");
+            log.info("last shutdown {}, root dir: {}", lastExitOK ? "normally" : "abnormally", messageStoreConfig.getStorePathRootDir());
 
             // load Commit Log
             result = result && this.commitLog.load();
@@ -229,6 +251,7 @@ public class DefaultMessageStore implements MessageStore {
     /**
      * @throws Exception
      */
+    @Override
     public void start() throws Exception {
 
         lock = lockFile.getChannel().tryLock(0, 1, false);
@@ -246,8 +269,8 @@ public class DefaultMessageStore implements MessageStore {
              * 4. Make sure the fall-behind messages to be dispatched before starting the commitlog, especially when the broker role are automatically changed.
              */
             long maxPhysicalPosInLogicQueue = commitLog.getMinOffset();
-            for (ConcurrentMap<Integer, ConsumeQueue> maps : this.consumeQueueTable.values()) {
-                for (ConsumeQueue logic : maps.values()) {
+            for (ConcurrentMap<Integer, ConsumeQueueInterface> maps : this.consumeQueueTable.values()) {
+                for (ConsumeQueueInterface logic : maps.values()) {
                     if (logic.getMaxPhysicOffset() > maxPhysicalPosInLogicQueue) {
                         maxPhysicalPosInLogicQueue = logic.getMaxPhysicOffset();
                     }
@@ -298,9 +321,11 @@ public class DefaultMessageStore implements MessageStore {
 
         this.createTempFile();
         this.addScheduleTask();
+        this.perfs.start();
         this.shutdown = false;
     }
 
+    @Override
     public void shutdown() {
         if (!this.shutdown) {
             this.shutdown = true;
@@ -330,6 +355,8 @@ public class DefaultMessageStore implements MessageStore {
             this.storeCheckpoint.flush();
             this.storeCheckpoint.shutdown();
 
+            this.perfs.shutdown();
+
             if (this.runningFlags.isWriteable() && dispatchBehindBytes() == 0) {
                 this.deleteFile(StorePathConfigHelper.getAbortFile(this.messageStoreConfig.getStorePathRootDir()));
                 shutDownNormal = true;
@@ -349,6 +376,7 @@ public class DefaultMessageStore implements MessageStore {
         }
     }
 
+    @Override
     public void destroy() {
         this.destroyLogics();
         this.commitLog.destroy();
@@ -357,10 +385,11 @@ public class DefaultMessageStore implements MessageStore {
         this.deleteFile(StorePathConfigHelper.getStoreCheckpoint(this.messageStoreConfig.getStorePathRootDir()));
     }
 
+    @Override
     public void destroyLogics() {
-        for (ConcurrentMap<Integer, ConsumeQueue> maps : this.consumeQueueTable.values()) {
-            for (ConsumeQueue logic : maps.values()) {
-                logic.destroy();
+        for (ConcurrentMap<Integer, ConsumeQueueInterface> maps : this.consumeQueueTable.values()) {
+            for (ConsumeQueueInterface logic : maps.values()) {
+                this.consumeQueueStore.destroy(logic);
             }
         }
     }
@@ -453,6 +482,7 @@ public class DefaultMessageStore implements MessageStore {
         return putResultFuture;
     }
 
+    @Override
     public CompletableFuture<PutMessageResult> asyncPutMessages(MessageExtBatch messageExtBatch) {
         PutMessageStatus checkStoreStatus = this.checkStoreStatus();
         if (checkStoreStatus != PutMessageStatus.PUT_OK) {
@@ -515,16 +545,27 @@ public class DefaultMessageStore implements MessageStore {
         return this.commitLog.lockTimeMills();
     }
 
+    @Override
     public SystemClock getSystemClock() {
         return systemClock;
     }
 
+    @Override
     public CommitLog getCommitLog() {
         return commitLog;
     }
 
+    @Override
+    public GetMessageResult getMessage(final String group, final String topic, final int queueId, final long offset,
+        final int maxMsgNums,
+        final MessageFilter messageFilter) {
+        return getMessage(group, topic, queueId, offset, maxMsgNums, MAX_PULL_MSG_SIZE, messageFilter);
+    }
+
+    @Override
     public GetMessageResult getMessage(final String group, final String topic, final int queueId, final long offset,
         final int maxMsgNums,
+        final int maxTotalMsgSize,
         final MessageFilter messageFilter) {
         if (this.shutdown) {
             log.warn("message store has shutdown, so getMessage is forbidden");
@@ -543,12 +584,11 @@ public class DefaultMessageStore implements MessageStore {
         long minOffset = 0;
         long maxOffset = 0;
 
-        // lazy init when find msg.
-        GetMessageResult getResult = null;
+        GetMessageResult getResult = new GetMessageResult();
 
         final long maxOffsetPy = this.commitLog.getMaxOffset();
 
-        ConsumeQueue consumeQueue = findConsumeQueue(topic, queueId);
+        ConsumeQueueInterface consumeQueue = findConsumeQueue(topic, queueId);
         if (consumeQueue != null) {
             minOffset = consumeQueue.getMinOffsetInQueue();
             maxOffset = consumeQueue.getMaxOffsetInQueue();
@@ -570,55 +610,67 @@ public class DefaultMessageStore implements MessageStore {
                     nextBeginOffset = nextOffsetCorrection(offset, maxOffset);
                 }
             } else {
-                SelectMappedBufferResult bufferConsumeQueue = consumeQueue.getIndexBuffer(offset);
-                if (bufferConsumeQueue != null) {
-                    try {
-                        status = GetMessageStatus.NO_MATCHED_MESSAGE;
-
-                        long nextPhyFileStartOffset = Long.MIN_VALUE;
-                        long maxPhyOffsetPulling = 0;
-
-                        int i = 0;
-                        final int maxFilterMessageCount = Math.max(16000, maxMsgNums * ConsumeQueue.CQ_STORE_UNIT_SIZE);
-                        final boolean diskFallRecorded = this.messageStoreConfig.isDiskFallRecorded();
+                final int maxFilterMessageCount = Math.max(16000, maxMsgNums * ConsumeQueue.CQ_STORE_UNIT_SIZE);
+                final boolean diskFallRecorded = this.messageStoreConfig.isDiskFallRecorded();
 
-                        getResult = new GetMessageResult(maxMsgNums);
+                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);
+                    maxPullSize = MAX_PULL_MSG_SIZE;
+                }
+                status = GetMessageStatus.NO_MATCHED_MESSAGE;
+                long maxPhyOffsetPulling = 0;
+                int cqFileNum = 0;
+
+                while (getResult.getBufferTotalSize() <= 0
+                        && 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);
+                        break;
+                    }
 
-                        ConsumeQueueExt.CqExtUnit cqExtUnit = new ConsumeQueueExt.CqExtUnit();
-                        for (; i < bufferConsumeQueue.getSize() && i < maxFilterMessageCount; i += ConsumeQueue.CQ_STORE_UNIT_SIZE) {
-                            long offsetPy = bufferConsumeQueue.getByteBuffer().getLong();
-                            int sizePy = bufferConsumeQueue.getByteBuffer().getInt();
-                            long tagsCode = bufferConsumeQueue.getByteBuffer().getLong();
+                    try {
+                        long nextPhyFileStartOffset = Long.MIN_VALUE;
+                        while (bufferConsumeQueue.hasNext()
+                                && nextBeginOffset < maxOffset) {
+                            CqUnit cqUnit = bufferConsumeQueue.next();
+                            long offsetPy = cqUnit.getPos();
+                            int sizePy = cqUnit.getSize();
 
-                            maxPhyOffsetPulling = offsetPy;
+                            boolean isInDisk = checkInDiskByCommitOffset(offsetPy, maxOffsetPy);
 
-                            if (nextPhyFileStartOffset != Long.MIN_VALUE) {
-                                if (offsetPy < nextPhyFileStartOffset)
-                                    continue;
+                            if (cqUnit.getQueueOffset() - offset > maxFilterMessageCount) {
+                                break;
                             }
 
-                            boolean isInDisk = checkInDiskByCommitOffset(offsetPy, maxOffsetPy);
+                            if (this.isTheBatchFull(sizePy, maxMsgNums, maxPullSize, getResult.getBufferTotalSize(), getResult.getMessageCount(),
+                                    isInDisk)) {
+                                break;
+                            }
 
-                            if (this.isTheBatchFull(sizePy, maxMsgNums, getResult.getBufferTotalSize(), getResult.getMessageCount(),
-                                isInDisk)) {
+                            if (getResult.getBufferTotalSize() >= maxPullSize) {
                                 break;
                             }
 
-                            boolean extRet = false, isTagsCodeLegal = true;
-                            if (consumeQueue.isExtAddr(tagsCode)) {
-                                extRet = consumeQueue.getExt(tagsCode, cqExtUnit);
-                                if (extRet) {
-                                    tagsCode = cqExtUnit.getTagsCode();
-                                } 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={}, group={}",
-                                        tagsCode, offsetPy, sizePy, topic, group);
-                                    isTagsCodeLegal = false;
+                            maxPhyOffsetPulling = offsetPy;
+
+                            //Be careful, here should before the isTheBatchFull
+                            nextBeginOffset = cqUnit.getQueueOffset() + cqUnit.getBatchNum();
+
+                            if (nextPhyFileStartOffset != Long.MIN_VALUE) {
+                                if (offsetPy < nextPhyFileStartOffset) {
+                                    continue;
                                 }
                             }
 
                             if (messageFilter != null
-                                && !messageFilter.isMatchedByConsumeQueue(isTagsCodeLegal ? tagsCode : null, extRet ? cqExtUnit : null)) {
+                                    && !messageFilter.isMatchedByConsumeQueue(cqUnit.getValidTagsCodeAsLong(), cqUnit.getCqExtUnit())) {
                                 if (getResult.getBufferTotalSize() == 0) {
                                     status = GetMessageStatus.NO_MATCHED_MESSAGE;
                                 }
@@ -637,7 +689,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;
                                 }
@@ -647,32 +699,24 @@ public class DefaultMessageStore implements MessageStore {
                             }
 
                             this.storeStatsService.getGetMessageTransferedMsgCount().add(1);
-                            getResult.addMessage(selectResult, offset + (i / ConsumeQueue.CQ_STORE_UNIT_SIZE));
+                            getResult.addMessage(selectResult, cqUnit.getQueueOffset(), cqUnit.getBatchNum());
                             status = GetMessageStatus.FOUND;
                             nextPhyFileStartOffset = Long.MIN_VALUE;
                         }
-
-                        if (diskFallRecorded) {
-                            long fallBehind = maxOffsetPy - maxPhyOffsetPulling;
-                            brokerStatsManager.recordDiskFallBehindSize(group, topic, queueId, fallBehind);
-                        }
-
-                        nextBeginOffset = offset + (i / ConsumeQueue.CQ_STORE_UNIT_SIZE);
-
-                        long diff = maxOffsetPy - maxPhyOffsetPulling;
-                        long memory = (long) (StoreUtil.TOTAL_PHYSICAL_MEMORY_SIZE
-                            * (this.messageStoreConfig.getAccessMessageInMemoryMaxRatio() / 100.0));
-                        getResult.setSuggestPullingFromSlave(diff > memory);
                     } finally {
-
                         bufferConsumeQueue.release();
                     }
-                } else {
-                    status = GetMessageStatus.OFFSET_FOUND_NULL;
-                    nextBeginOffset = nextOffsetCorrection(offset, consumeQueue.rollNextFile(offset));
-                    log.warn("consumer request topic: " + topic + "offset: " + offset + " minOffset: " + minOffset + " maxOffset: "
-                        + maxOffset + ", but access logic queue failed.");
                 }
+
+                if (diskFallRecorded) {
+                    long fallBehind = maxOffsetPy - maxPhyOffsetPulling;
+                    brokerStatsManager.recordDiskFallBehindSize(group, topic, queueId, fallBehind);
+                }
+
+                long diff = maxOffsetPy - maxPhyOffsetPulling;
+                long memory = (long) (StoreUtil.TOTAL_PHYSICAL_MEMORY_SIZE
+                        * (this.messageStoreConfig.getAccessMessageInMemoryMaxRatio() / 100.0));
+                getResult.setSuggestPullingFromSlave(diff > memory);
             }
         } else {
             status = GetMessageStatus.NO_MATCHED_LOGIC_QUEUE;
@@ -699,18 +743,20 @@ public class DefaultMessageStore implements MessageStore {
         return getResult;
     }
 
+    @Override
     public long getMaxOffsetInQueue(String topic, int queueId) {
         return getMaxOffsetInQueue(topic, queueId, true);
     }
 
+    @Override
     public long getMaxOffsetInQueue(String topic, int queueId, boolean committed) {
         if (committed) {
-            ConsumeQueue logic = this.findConsumeQueue(topic, queueId);
+            ConsumeQueueInterface logic = this.findConsumeQueue(topic, queueId);
             if (logic != null) {
                 return logic.getMaxOffsetInQueue();
             }
         } else {
-            Long offset = this.commitLog.getTopicQueueTable().get(topic + "-" + queueId);
+            Long offset = this.topicQueueTable.get(topic + "-" + queueId);
             if (offset != null) {
                 return offset;
             }
@@ -719,8 +765,9 @@ public class DefaultMessageStore implements MessageStore {
         return 0;
     }
 
+    @Override
     public long getMinOffsetInQueue(String topic, int queueId) {
-        ConsumeQueue logic = this.findConsumeQueue(topic, queueId);
+        ConsumeQueueInterface logic = this.findConsumeQueue(topic, queueId);
         if (logic != null) {
             return logic.getMinOffsetInQueue();
         }
@@ -730,13 +777,16 @@ public class DefaultMessageStore implements MessageStore {
 
     @Override
     public long getCommitLogOffsetInQueue(String topic, int queueId, long consumeQueueOffset) {
-        ConsumeQueue consumeQueue = findConsumeQueue(topic, queueId);
+        ConsumeQueueInterface consumeQueue = findConsumeQueue(topic, queueId);
         if (consumeQueue != null) {
-            SelectMappedBufferResult bufferConsumeQueue = consumeQueue.getIndexBuffer(consumeQueueOffset);
+
+            ReferredIterator<CqUnit> bufferConsumeQueue = consumeQueue.iterateFrom(consumeQueueOffset);
             if (bufferConsumeQueue != null) {
                 try {
-                    long offsetPy = bufferConsumeQueue.getByteBuffer().getLong();
-                    return offsetPy;
+                    if (bufferConsumeQueue.hasNext()) {
+                        long offsetPy = bufferConsumeQueue.next().getPos();
+                        return offsetPy;
+                    }
                 } finally {
                     bufferConsumeQueue.release();
                 }
@@ -746,8 +796,9 @@ public class DefaultMessageStore implements MessageStore {
         return 0;
     }
 
+    @Override
     public long getOffsetInQueueByTime(String topic, int queueId, long timestamp) {
-        ConsumeQueue logic = this.findConsumeQueue(topic, queueId);
+        ConsumeQueueInterface logic = this.findConsumeQueue(topic, queueId);
         if (logic != null) {
             return logic.getOffsetInQueueByTime(timestamp);
         }
@@ -755,6 +806,7 @@ public class DefaultMessageStore implements MessageStore {
         return 0;
     }
 
+    @Override
     public MessageExt lookMessageByOffset(long commitLogOffset) {
         SelectMappedBufferResult sbr = this.commitLog.getMessage(commitLogOffset, 4);
         if (null != sbr) {
@@ -791,6 +843,7 @@ public class DefaultMessageStore implements MessageStore {
         return this.commitLog.getMessage(commitLogOffset, msgSize);
     }
 
+    @Override
     public String getRunningDataInfo() {
         return this.storeStatsService.toString();
     }
@@ -855,27 +908,22 @@ public class DefaultMessageStore implements MessageStore {
 
     @Override
     public long getEarliestMessageTime(String topic, int queueId) {
-        ConsumeQueue logicQueue = this.findConsumeQueue(topic, queueId);
+        ConsumeQueueInterface logicQueue = this.findConsumeQueue(topic, queueId);
         if (logicQueue != null) {
-            long minLogicOffset = logicQueue.getMinLogicOffset();
-
-            SelectMappedBufferResult result = logicQueue.getIndexBuffer(minLogicOffset / ConsumeQueue.CQ_STORE_UNIT_SIZE);
-            return getStoreTime(result);
+            return getStoreTime(logicQueue.getEarliestUnit());
         }
 
         return -1;
     }
 
-    private long getStoreTime(SelectMappedBufferResult result) {
+    protected long getStoreTime(CqUnit result) {
         if (result != null) {
             try {
-                final long phyOffset = result.getByteBuffer().getLong();
-                final int size = result.getByteBuffer().getInt();
+                final long phyOffset = result.getPos();
+                final int size = result.getSize();
                 long storeTime = this.getCommitLog().pickupStoreTimestamp(phyOffset, size);
                 return storeTime;
             } catch (Exception e) {
-            } finally {
-                result.release();
             }
         }
         return -1;
@@ -890,10 +938,9 @@ public class DefaultMessageStore implements MessageStore {
 
     @Override
     public long getMessageStoreTimeStamp(String topic, int queueId, long consumeQueueOffset) {
-        ConsumeQueue logicQueue = this.findConsumeQueue(topic, queueId);
+        ConsumeQueueInterface logicQueue = this.findConsumeQueue(topic, queueId);
         if (logicQueue != null) {
-            SelectMappedBufferResult result = logicQueue.getIndexBuffer(consumeQueueOffset);
-            return getStoreTime(result);
+            return getStoreTime(logicQueue.get(consumeQueueOffset));
         }
 
         return -1;
@@ -901,7 +948,7 @@ public class DefaultMessageStore implements MessageStore {
 
     @Override
     public long getMessageTotalInQueue(String topic, int queueId) {
-        ConsumeQueue logicQueue = this.findConsumeQueue(topic, queueId);
+        ConsumeQueueInterface logicQueue = this.findConsumeQueue(topic, queueId);
         if (logicQueue != null) {
             return logicQueue.getMessageTotalInQueue();
         }
@@ -1024,16 +1071,16 @@ public class DefaultMessageStore implements MessageStore {
 
     @Override
     public int cleanUnusedTopic(Set<String> topics) {
-        Iterator<Entry<String, ConcurrentMap<Integer, ConsumeQueue>>> it = this.consumeQueueTable.entrySet().iterator();
+        Iterator<Entry<String, ConcurrentMap<Integer, ConsumeQueueInterface>>> it = this.consumeQueueTable.entrySet().iterator();
         while (it.hasNext()) {
-            Entry<String, ConcurrentMap<Integer, ConsumeQueue>> next = it.next();
+            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)) {
-                ConcurrentMap<Integer, ConsumeQueue> queueTable = next.getValue();
-                for (ConsumeQueue cq : queueTable.values()) {
-                    cq.destroy();
+                ConcurrentMap<Integer, ConsumeQueueInterface> queueTable = next.getValue();
+                for (ConsumeQueueInterface cq : queueTable.values()) {
+                    this.consumeQueueStore.destroy(cq);
                     log.info("cleanUnusedTopic: {} {} ConsumeQueue cleaned",
                         cq.getTopic(),
                         cq.getQueueId()
@@ -1054,18 +1101,19 @@ public class DefaultMessageStore implements MessageStore {
         return 0;
     }
 
+    @Override
     public void cleanExpiredConsumerQueue() {
         long minCommitLogOffset = this.commitLog.getMinOffset();
 
-        Iterator<Entry<String, ConcurrentMap<Integer, ConsumeQueue>>> it = this.consumeQueueTable.entrySet().iterator();
+        Iterator<Entry<String, ConcurrentMap<Integer, ConsumeQueueInterface>>> it = this.consumeQueueTable.entrySet().iterator();
         while (it.hasNext()) {
-            Entry<String, ConcurrentMap<Integer, ConsumeQueue>> next = it.next();
+            Entry<String, ConcurrentMap<Integer, ConsumeQueueInterface>> next = it.next();
             String topic = next.getKey();
             if (!topic.equals(TopicValidator.RMQ_SYS_SCHEDULE_TOPIC)) {
-                ConcurrentMap<Integer, ConsumeQueue> queueTable = next.getValue();
-                Iterator<Entry<Integer, ConsumeQueue>> itQT = queueTable.entrySet().iterator();
+                ConcurrentMap<Integer, ConsumeQueueInterface> queueTable = next.getValue();
+                Iterator<Entry<Integer, ConsumeQueueInterface>> itQT = queueTable.entrySet().iterator();
                 while (itQT.hasNext()) {
-                    Entry<Integer, ConsumeQueue> nextQT = itQT.next();
+                    Entry<Integer, ConsumeQueueInterface> nextQT = itQT.next();
                     long maxCLOffsetInConsumeQueue = nextQT.getValue().getLastOffset();
 
                     if (maxCLOffsetInConsumeQueue == -1) {
@@ -1085,7 +1133,7 @@ public class DefaultMessageStore implements MessageStore {
                         DefaultMessageStore.this.commitLog.removeQueueFromTopicQueueTable(nextQT.getValue().getTopic(),
                             nextQT.getValue().getQueueId());
 
-                        nextQT.getValue().destroy();
+                        this.consumeQueueStore.destroy(nextQT.getValue());
                         itQT.remove();
                     }
                 }
@@ -1105,7 +1153,7 @@ public class DefaultMessageStore implements MessageStore {
             return messageIds;
         }
 
-        ConsumeQueue consumeQueue = findConsumeQueue(topic, queueId);
+        ConsumeQueueInterface consumeQueue = findConsumeQueue(topic, queueId);
         if (consumeQueue != null) {
             minOffset = Math.max(minOffset, consumeQueue.getMinOffsetInQueue());
             maxOffset = Math.min(maxOffset, consumeQueue.getMaxOffsetInQueue());
@@ -1116,28 +1164,30 @@ public class DefaultMessageStore implements MessageStore {
 
             long nextOffset = minOffset;
             while (nextOffset < maxOffset) {
-                SelectMappedBufferResult bufferConsumeQueue = consumeQueue.getIndexBuffer(nextOffset);
-                if (bufferConsumeQueue != null) {
-                    try {
-                        int i = 0;
-                        for (; i < bufferConsumeQueue.getSize(); i += ConsumeQueue.CQ_STORE_UNIT_SIZE) {
-                            long offsetPy = bufferConsumeQueue.getByteBuffer().getLong();
+                ReferredIterator<CqUnit> bufferConsumeQueue = consumeQueue.iterateFrom(nextOffset);
+                try {
+                    if (bufferConsumeQueue != null && bufferConsumeQueue.hasNext()) {
+                        while (bufferConsumeQueue.hasNext()) {
+                            CqUnit cqUnit = bufferConsumeQueue.next();
+                            long offsetPy = cqUnit.getPos();
                             InetSocketAddress inetSocketAddress = (InetSocketAddress) storeHost;
                             int msgIdLength = (inetSocketAddress.getAddress() instanceof Inet6Address) ? 16 + 4 + 8 : 4 + 4 + 8;
                             final ByteBuffer msgIdMemory = ByteBuffer.allocate(msgIdLength);
                             String msgId =
-                                MessageDecoder.createMessageId(msgIdMemory, MessageExt.socketAddress2ByteBuffer(storeHost), offsetPy);
-                            messageIds.put(msgId, nextOffset++);
-                            if (nextOffset > maxOffset) {
+                                    MessageDecoder.createMessageId(msgIdMemory, MessageExt.socketAddress2ByteBuffer(storeHost), offsetPy);
+                            messageIds.put(msgId, cqUnit.getQueueOffset());
+                            nextOffset = cqUnit.getQueueOffset() + cqUnit.getBatchNum();
+                            if (nextOffset >= maxOffset) {
                                 return messageIds;
                             }
                         }
-                    } finally {
-
+                    } else {
+                        return messageIds;
+                    }
+                } finally {
+                    if (bufferConsumeQueue != null) {
                         bufferConsumeQueue.release();
                     }
-                } else {
-                    return messageIds;
                 }
             }
         }
@@ -1149,20 +1199,13 @@ public class DefaultMessageStore implements MessageStore {
 
         final long maxOffsetPy = this.commitLog.getMaxOffset();
 
-        ConsumeQueue consumeQueue = findConsumeQueue(topic, queueId);
+        ConsumeQueueInterface consumeQueue = findConsumeQueue(topic, queueId);
         if (consumeQueue != null) {
-            SelectMappedBufferResult bufferConsumeQueue = consumeQueue.getIndexBuffer(consumeOffset);
-            if (bufferConsumeQueue != null) {
-                try {
-                    for (int i = 0; i < bufferConsumeQueue.getSize(); ) {
-                        i += ConsumeQueue.CQ_STORE_UNIT_SIZE;
-                        long offsetPy = bufferConsumeQueue.getByteBuffer().getLong();
-                        return checkInDiskByCommitOffset(offsetPy, maxOffsetPy);
-                    }
-                } finally {
+            CqUnit cqUnit = consumeQueue.get(consumeOffset);
 
-                    bufferConsumeQueue.release();
-                }
+            if (cqUnit != null) {
+                long offsetPy = cqUnit.getPos();
+                return checkInDiskByCommitOffset(offsetPy, maxOffsetPy);
             } else {
                 return false;
             }
@@ -1195,6 +1238,7 @@ public class DefaultMessageStore implements MessageStore {
         this.commitLog.setConfirmOffset(phyOffset);
     }
 
+    @Override
     public MessageExt lookMessageByOffset(long commitLogOffset, int size) {
         SelectMappedBufferResult sbr = this.commitLog.getMessage(commitLogOffset, size);
         if (null != sbr) {
@@ -1208,35 +1252,8 @@ public class DefaultMessageStore implements MessageStore {
         return null;
     }
 
-    public ConsumeQueue findConsumeQueue(String topic, int queueId) {
-        ConcurrentMap<Integer, ConsumeQueue> map = consumeQueueTable.get(topic);
-        if (null == map) {
-            ConcurrentMap<Integer, ConsumeQueue> newMap = new ConcurrentHashMap<Integer, ConsumeQueue>(128);
-            ConcurrentMap<Integer, ConsumeQueue> oldMap = consumeQueueTable.putIfAbsent(topic, newMap);
-            if (oldMap != null) {
-                map = oldMap;
-            } else {
-                map = newMap;
-            }
-        }
-
-        ConsumeQueue logic = map.get(queueId);
-        if (null == logic) {
-            ConsumeQueue newLogic = new ConsumeQueue(
-                topic,
-                queueId,
-                StorePathConfigHelper.getStorePathConsumeQueue(this.messageStoreConfig.getStorePathRootDir()),
-                this.getMessageStoreConfig().getMappedFileSizeConsumeQueue(),
-                this);
-            ConsumeQueue oldLogic = map.putIfAbsent(queueId, newLogic);
-            if (oldLogic != null) {
-                logic = oldLogic;
-            } else {
-                logic = newLogic;
-            }
-        }
-
-        return logic;
+    public ConsumeQueueInterface findConsumeQueue(String topic, int queueId) {
+        return this.consumeQueueStore.findOrCreateConsumeQueue(topic, queueId);
     }
 
     private long nextOffsetCorrection(long oldOffset, long newOffset) {
@@ -1252,7 +1269,7 @@ public class DefaultMessageStore implements MessageStore {
         return (maxOffsetPy - offsetPy) > memory;
     }
 
-    private boolean isTheBatchFull(int sizePy, int maxMsgNums, int bufferTotal, int messageTotal, boolean isInDisk) {
+    private boolean isTheBatchFull(int sizePy, int maxMsgNums, long maxMsgSize, int bufferTotal, int messageTotal, boolean isInDisk) {
 
         if (0 == bufferTotal || 0 == messageTotal) {
             return false;
@@ -1262,6 +1279,10 @@ public class DefaultMessageStore implements MessageStore {
             return true;
         }
 
+        if (bufferTotal + sizePy > maxMsgSize) {
+            return true;
+        }
+
         if (isInDisk) {
             if ((bufferTotal + sizePy) > this.messageStoreConfig.getMaxTransferBytesOnMessageInDisk()) {
                 return true;
@@ -1295,11 +1316,12 @@ public class DefaultMessageStore implements MessageStore {
     private void createTempFile() throws IOException {
         String fileName = StorePathConfigHelper.getAbortFile(this.messageStoreConfig.getStorePathRootDir());
         File file = new File(fileName);
-        MappedFile.ensureDirOK(file.getParent());
+        DefaultMappedFile.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);
     }
@@ -1371,13 +1393,13 @@ public class DefaultMessageStore implements MessageStore {
     private void checkSelf() {
         this.commitLog.checkSelf();
 
-        Iterator<Entry<String, ConcurrentMap<Integer, ConsumeQueue>>> it = this.consumeQueueTable.entrySet().iterator();
+        Iterator<Entry<String, ConcurrentMap<Integer, ConsumeQueueInterface>>> it = this.consumeQueueTable.entrySet().iterator();
         while (it.hasNext()) {
-            Entry<String, ConcurrentMap<Integer, ConsumeQueue>> next = it.next();
-            Iterator<Entry<Integer, ConsumeQueue>> itNext = next.getValue().entrySet().iterator();
+            Entry<String, ConcurrentMap<Integer, ConsumeQueueInterface>> next = it.next();
+            Iterator<Entry<Integer, ConsumeQueueInterface>> itNext = next.getValue().entrySet().iterator();
             while (itNext.hasNext()) {
-                Entry<Integer, ConsumeQueue> cq = itNext.next();
-                cq.getValue().checkSelf();
+                Entry<Integer, ConsumeQueueInterface> cq = itNext.next();
+                this.consumeQueueStore.checkSelf(cq.getValue());
             }
         }
     }
@@ -1389,6 +1411,8 @@ public class DefaultMessageStore implements MessageStore {
     }
 
     private boolean loadConsumeQueue() {
+        checkOtherConsumeQueue();
+
         File dirLogic = new File(StorePathConfigHelper.getStorePathConsumeQueue(this.messageStoreConfig.getStorePathRootDir()));
         File[] fileTopicList = dirLogic.listFiles();
         if (fileTopicList != null) {
@@ -1405,14 +1429,14 @@ public class DefaultMessageStore implements MessageStore {
                         } catch (NumberFormatException e) {
                             continue;
                         }
-                        ConsumeQueue logic = new ConsumeQueue(
+                        ConsumeQueueInterface logic = new ConsumeQueue(
                             topic,
                             queueId,
                             StorePathConfigHelper.getStorePathConsumeQueue(this.messageStoreConfig.getStorePathRootDir()),
                             this.getMessageStoreConfig().getMappedFileSizeConsumeQueue(),
                             this);
                         this.putConsumeQueue(topic, queueId, logic);
-                        if (!logic.load()) {
+                        if (!this.consumeQueueStore.load(logic)) {
                             return false;
                         }
                     }
@@ -1425,30 +1449,46 @@ public class DefaultMessageStore implements MessageStore {
         return true;
     }
 
+    private void checkOtherConsumeQueue() {
+        File dirLogic = new File(StorePathConfigHelper.getStorePathBatchConsumeQueue(this.messageStoreConfig.getStorePathRootDir()));
+        if (dirLogic.exists()) {
+            throw new RuntimeException(format("Batch consume queue directory: [%s] exist. Can not load consume queue while batch consume queue exists.",
+                    StorePathConfigHelper.getStorePathBatchConsumeQueue(this.messageStoreConfig.getStorePathRootDir())));
+        }
+    }
+
     private void recover(final boolean lastExitOK) {
+        long recoverCqStart = System.currentTimeMillis();
         long maxPhyOffsetOfConsumeQueue = this.recoverConsumeQueue();
+        long recoverCqEnd = System.currentTimeMillis();
 
         if (lastExitOK) {
             this.commitLog.recoverNormally(maxPhyOffsetOfConsumeQueue);
         } else {
             this.commitLog.recoverAbnormally(maxPhyOffsetOfConsumeQueue);
         }
-
+        long recoverClogEnd = System.currentTimeMillis();
         this.recoverTopicQueueTable();
+        long recoverOffsetEnd = System.currentTimeMillis();
+
+        log.info("Recover end total:{} recoverCq:{} recoverClog:{} recoverOffset:{}",
+                recoverOffsetEnd - recoverCqStart, recoverCqEnd - recoverCqStart, recoverClogEnd - recoverCqEnd, recoverOffsetEnd - recoverClogEnd);
     }
 
+    @Override
     public MessageStoreConfig getMessageStoreConfig() {
         return messageStoreConfig;
     }
 
+    @Override
     public TransientStorePool getTransientStorePool() {
         return transientStorePool;
     }
 
-    private void putConsumeQueue(final String topic, final int queueId, final ConsumeQueue consumeQueue) {
-        ConcurrentMap<Integer/* queueId */, ConsumeQueue> map = this.consumeQueueTable.get(topic);
+    private void putConsumeQueue(final String topic, final int queueId, final ConsumeQueueInterface consumeQueue) {
+        ConcurrentMap<Integer/* queueId */, ConsumeQueueInterface> map = this.consumeQueueTable.get(topic);
         if (null == map) {
-            map = new ConcurrentHashMap<Integer/* queueId */, ConsumeQueue>();
+            map = new ConcurrentHashMap<Integer/* queueId */, ConsumeQueueInterface>();
             map.put(queueId, consumeQueue);
             this.consumeQueueTable.put(topic, map);
         } else {
@@ -1458,9 +1498,9 @@ public class DefaultMessageStore implements MessageStore {
 
     private long recoverConsumeQueue() {
         long maxPhysicOffset = -1;
-        for (ConcurrentMap<Integer, ConsumeQueue> maps : this.consumeQueueTable.values()) {
-            for (ConsumeQueue logic : maps.values()) {
-                logic.recover();
+        for (ConcurrentMap<Integer, ConsumeQueueInterface> maps : this.consumeQueueTable.values()) {
+            for (ConsumeQueueInterface logic : maps.values()) {
+                this.consumeQueueStore.recover(logic);
                 if (logic.getMaxPhysicOffset() > maxPhysicOffset) {
                     maxPhysicOffset = logic.getMaxPhysicOffset();
                 }
@@ -1473,21 +1513,23 @@ public class DefaultMessageStore implements MessageStore {
     public void recoverTopicQueueTable() {
         HashMap<String/* topic-queueid */, Long/* offset */> table = new HashMap<String, Long>(1024);
         long minPhyOffset = this.commitLog.getMinOffset();
-        for (ConcurrentMap<Integer, ConsumeQueue> maps : this.consumeQueueTable.values()) {
-            for (ConsumeQueue logic : maps.values()) {
+        for (ConcurrentMap<Integer, ConsumeQueueInterface> maps : this.consumeQueueTable.values()) {
+            for (ConsumeQueueInterface logic : maps.values()) {
                 String key = logic.getTopic() + "-" + logic.getQueueId();
                 table.put(key, logic.getMaxOffsetInQueue());
-                logic.correctMinOffset(minPhyOffset);
+                this.consumeQueueStore.correctMinOffset(logic, minPhyOffset);
             }
         }
 
-        this.commitLog.setTopicQueueTable(table);
+        this.topicQueueTable = table;
     }
 
+    @Override
     public AllocateMappedFileService getAllocateMappedFileService() {
         return allocateMappedFileService;
     }
 
+    @Override
     public StoreStatsService getStoreStatsService() {
         return storeStatsService;
     }
@@ -1496,14 +1538,16 @@ public class DefaultMessageStore implements MessageStore {
         return runningFlags;
     }
 
-    public ConcurrentMap<String, ConcurrentMap<Integer, ConsumeQueue>> getConsumeQueueTable() {
+    public ConcurrentMap<String, ConcurrentMap<Integer, ConsumeQueueInterface>> getConsumeQueueTable() {
         return consumeQueueTable;
     }
 
+    @Override
     public StoreCheckpoint getStoreCheckpoint() {
         return storeCheckpoint;
     }
 
+    @Override
     public HAService getHaService() {
         return haService;
     }
@@ -1513,6 +1557,7 @@ public class DefaultMessageStore implements MessageStore {
         return scheduleMessageService;
     }
 
+    @Override
     public RunningFlags getRunningFlags() {
         return runningFlags;
     }
@@ -1524,8 +1569,8 @@ public class DefaultMessageStore implements MessageStore {
     }
 
     public void putMessagePositionInfo(DispatchRequest dispatchRequest) {
-        ConsumeQueue cq = this.findConsumeQueue(dispatchRequest.getTopic(), dispatchRequest.getQueueId());
-        cq.putMessagePositionInfoWrapper(dispatchRequest);
+        ConsumeQueueInterface cq = this.findConsumeQueue(dispatchRequest.getTopic(), dispatchRequest.getQueueId());
+        this.consumeQueueStore.putMessagePositionInfoWrapper(cq, dispatchRequest);
     }
 
     @Override
@@ -1560,14 +1605,15 @@ public class DefaultMessageStore implements MessageStore {
     }
 
     @Override
-    public ConsumeQueue getConsumeQueue(String topic, int queueId) {
-        ConcurrentMap<Integer, ConsumeQueue> map = consumeQueueTable.get(topic);
+    public ConsumeQueueInterface getConsumeQueue(String topic, int queueId) {
+        ConcurrentMap<Integer, ConsumeQueueInterface> map = consumeQueueTable.get(topic);
         if (map == null) {
             return null;
         }
         return map.get(queueId);
     }
 
+    @Override
     public void unlockMappedFile(final MappedFile mappedFile) {
         this.scheduledExecutorService.schedule(new Runnable() {
             @Override
@@ -1577,6 +1623,54 @@ public class DefaultMessageStore implements MessageStore {
         }, 6, TimeUnit.SECONDS);
     }
 
+    @Override
+    public PerfCounter.Ticks getPerfCounter() {
+        return perfs;
+    }
+
+    @Override
+    public ConsumeQueueStore getQueueStore() {
+        return consumeQueueStore;
+    }
+
+    @Override
+    public void onCommitLogAppend(MessageExtBrokerInner msg, AppendMessageResult result, MappedFile commitLogFile) {
+        // empty
+    }
+
+    @Override
+    public void onCommitLogDispatch(DispatchRequest dispatchRequest, boolean doDispatch, MappedFile commitLogFile, boolean isRecover, boolean isFileEnd) {
+        if (doDispatch && !isFileEnd) {
+            this.doDispatch(dispatchRequest);
+        }
+    }
+
+    @Override
+    public boolean isSyncDiskFlush() {
+        return FlushDiskType.SYNC_FLUSH == this.getMessageStoreConfig().getFlushDiskType();
+    }
+
+    @Override
+    public boolean isSyncMaster() {
+        return BrokerRole.SYNC_MASTER == this.getMessageStoreConfig().getBrokerRole();
+    }
+
+    @Override
+    public void assignOffset(String topicQueueKey, MessageExtBrokerInner msg, short batchNum) {
+        final int tranType = MessageSysFlag.getTransactionValue(msg.getSysFlag());
+
+        if (tranType == MessageSysFlag.TRANSACTION_NOT_TYPE || tranType == MessageSysFlag.TRANSACTION_COMMIT_TYPE) {
+            long topicOffset = this.topicQueueTable.computeIfAbsent(topicQueueKey, k -> 0L);
+            msg.setQueueOffset(topicOffset);
+            this.topicQueueTable.put(topicQueueKey, topicOffset + batchNum);
+        }
+    }
+
+    @Override
+    public void removeOffsetTable(String topicQueueKey) {
+        this.topicQueueTable.remove(topicQueueKey);
+    }
+
     class CommitLogDispatcherBuildConsumeQueue implements CommitLogDispatcher {
 
         @Override
@@ -1607,17 +1701,53 @@ public class DefaultMessageStore implements MessageStore {
     class CleanCommitLogService {
 
         private final static int MAX_MANUAL_DELETE_FILE_TIMES = 20;
-        private final double diskSpaceWarningLevelRatio =
-            Double.parseDouble(System.getProperty("rocketmq.broker.diskSpaceWarningLevelRatio", "0.90"));
+        private final String diskSpaceWarningLevelRatio =
+                System.getProperty("rocketmq.broker.diskSpaceWarningLevelRatio", "");
 
-        private final double diskSpaceCleanForciblyRatio =
-            Double.parseDouble(System.getProperty("rocketmq.broker.diskSpaceCleanForciblyRatio", "0.85"));
+        private final String diskSpaceCleanForciblyRatio =
+                System.getProperty("rocketmq.broker.diskSpaceCleanForciblyRatio", "");
         private long lastRedeleteTimestamp = 0;
 
         private volatile int manualDeleteFileSeveralTimes = 0;
 
         private volatile boolean cleanImmediately = false;
 
+        double getDiskSpaceWarningLevelRatio() {
+            double finalDiskSpaceWarningLevelRatio;
+            if ("".equals(diskSpaceWarningLevelRatio)) {
+                finalDiskSpaceWarningLevelRatio = DefaultMessageStore.this.getMessageStoreConfig().getDiskSpaceWarningLevelRatio() / 100.0;
+            } else {
+                finalDiskSpaceWarningLevelRatio = Double.parseDouble(diskSpaceWarningLevelRatio);
+            }
+
+            if (finalDiskSpaceWarningLevelRatio > 0.90) {
+                finalDiskSpaceWarningLevelRatio = 0.90;
+            }
+            if (finalDiskSpaceWarningLevelRatio < 0.35) {
+                finalDiskSpaceWarningLevelRatio = 0.35;
+            }
+
+            return finalDiskSpaceWarningLevelRatio;
+        }
+
+        double getDiskSpaceCleanForciblyRatio() {
+            double finalDiskSpaceCleanForciblyRatio;
+            if ("".equals(diskSpaceCleanForciblyRatio)) {
+                finalDiskSpaceCleanForciblyRatio = DefaultMessageStore.this.getMessageStoreConfig().getDiskSpaceCleanForciblyRatio() / 100.0;
+            } else {
+                finalDiskSpaceCleanForciblyRatio = Double.parseDouble(diskSpaceCleanForciblyRatio);
+            }
+
+            if (finalDiskSpaceCleanForciblyRatio > 0.85) {
+                finalDiskSpaceCleanForciblyRatio = 0.85;
+            }
+            if (finalDiskSpaceCleanForciblyRatio < 0.30) {
+                finalDiskSpaceCleanForciblyRatio = 0.30;
+            }
+
+            return finalDiskSpaceCleanForciblyRatio;
+        }
+
         public void excuteDeleteFilesManualy() {
             this.manualDeleteFileSeveralTimes = MAX_MANUAL_DELETE_FILE_TIMES;
             DefaultMessageStore.log.info("executeDeleteFilesManually was invoked");
@@ -1714,12 +1844,12 @@ public class DefaultMessageStore implements MessageStore {
                         minPhysicRatio =  physicRatio;
                         minStorePath = storePathPhysic;
                     }
-                    if (physicRatio > diskSpaceCleanForciblyRatio) {
+                    if (physicRatio > getDiskSpaceCleanForciblyRatio()) {
                         fullStorePath.add(storePathPhysic);
                     }
                 }
                 DefaultMessageStore.this.commitLog.setFullStorePaths(fullStorePath);
-                if (minPhysicRatio > diskSpaceWarningLevelRatio) {
+                if (minPhysicRatio > getDiskSpaceWarningLevelRatio()) {
                     boolean diskok = DefaultMessageStore.this.runningFlags.getAndMakeDiskFull();
                     if (diskok) {
                         DefaultMessageStore.log.error("physic disk maybe full soon " + minPhysicRatio +
@@ -1727,7 +1857,7 @@ public class DefaultMessageStore implements MessageStore {
                     }
 
                     cleanImmediately = true;
-                } else if (minPhysicRatio > diskSpaceCleanForciblyRatio) {
+                } else if (minPhysicRatio > getDiskSpaceCleanForciblyRatio()) {
                     cleanImmediately = true;
                 } else {
                     boolean diskok = DefaultMessageStore.this.runningFlags.getAndMakeDiskOK();
@@ -1748,14 +1878,14 @@ public class DefaultMessageStore implements MessageStore {
                 String storePathLogics = StorePathConfigHelper
                     .getStorePathConsumeQueue(DefaultMessageStore.this.getMessageStoreConfig().getStorePathRootDir());
                 double logicsRatio = UtilAll.getDiskPartitionSpaceUsedPercent(storePathLogics);
-                if (logicsRatio > diskSpaceWarningLevelRatio) {
+                if (logicsRatio > getDiskSpaceWarningLevelRatio()) {
                     boolean diskok = DefaultMessageStore.this.runningFlags.getAndMakeDiskFull();
                     if (diskok) {
                         DefaultMessageStore.log.error("logics disk maybe full soon " + logicsRatio + ", so mark disk full");
                     }
 
                     cleanImmediately = true;
-                } else if (logicsRatio > diskSpaceCleanForciblyRatio) {
+                } else if (logicsRatio > getDiskSpaceCleanForciblyRatio()) {
                     cleanImmediately = true;
                 } else {
                     boolean diskok = DefaultMessageStore.this.runningFlags.getAndMakeDiskOK();
@@ -1790,7 +1920,7 @@ public class DefaultMessageStore implements MessageStore {
                 double physicRatio = UtilAll.isPathExists(path) ?
                         UtilAll.getDiskPartitionSpaceUsedPercent(path) : -1;
                 minPhysicRatio = Math.min(minPhysicRatio, physicRatio);
-                if (physicRatio > diskSpaceCleanForciblyRatio) {
+                if (physicRatio > getDiskSpaceCleanForciblyRatio()) {
                     fullStorePath.add(path);
                 }
             }
@@ -1805,7 +1935,7 @@ public class DefaultMessageStore implements MessageStore {
             if (physicRatio > ratio) {
                 DefaultMessageStore.log.info("physic disk of commitLog used: " + physicRatio);
             }
-            if (physicRatio > this.diskSpaceWarningLevelRatio) {
+            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");
@@ -1845,11 +1975,11 @@ public class DefaultMessageStore implements MessageStore {
             if (minOffset > this.lastPhysicalMinOffset) {
                 this.lastPhysicalMinOffset = minOffset;
 
-                ConcurrentMap<String, ConcurrentMap<Integer, ConsumeQueue>> tables = DefaultMessageStore.this.consumeQueueTable;
+                ConcurrentMap<String, ConcurrentMap<Integer, ConsumeQueueInterface>> tables = DefaultMessageStore.this.consumeQueueTable;
 
-                for (ConcurrentMap<Integer, ConsumeQueue> maps : tables.values()) {
-                    for (ConsumeQueue logic : maps.values()) {
-                        int deleteCount = logic.deleteExpiredFile(minOffset);
+                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 {
@@ -1891,13 +2021,13 @@ public class DefaultMessageStore implements MessageStore {
                 logicsMsgTimestamp = DefaultMessageStore.this.getStoreCheckpoint().getLogicsMsgTimestamp();
             }
 
-            ConcurrentMap<String, ConcurrentMap<Integer, ConsumeQueue>> tables = DefaultMessageStore.this.consumeQueueTable;
+            ConcurrentMap<String, ConcurrentMap<Integer, ConsumeQueueInterface>> tables = DefaultMessageStore.this.consumeQueueTable;
 
-            for (ConcurrentMap<Integer, ConsumeQueue> maps : tables.values()) {
-                for (ConsumeQueue cq : maps.values()) {
+            for (ConcurrentMap<Integer, ConsumeQueueInterface> maps : tables.values()) {
+                for (ConsumeQueueInterface cq : maps.values()) {
                     boolean result = false;
                     for (int i = 0; i < retryTimes && !result; i++) {
-                        result = cq.flush(flushConsumeQueueLeastPages);
+                        result = DefaultMessageStore.this.consumeQueueStore.flush(cq, flushConsumeQueueLeastPages);
                     }
                 }
             }
@@ -1910,6 +2040,7 @@ public class DefaultMessageStore implements MessageStore {
             }
         }
 
+        @Override
         public void run() {
             DefaultMessageStore.log.info(this.getServiceName() + " service started");
 
diff --git a/store/src/main/java/org/apache/rocketmq/store/DispatchRequest.java b/store/src/main/java/org/apache/rocketmq/store/DispatchRequest.java
index 89d47ce..bfe54ba 100644
--- a/store/src/main/java/org/apache/rocketmq/store/DispatchRequest.java
+++ b/store/src/main/java/org/apache/rocketmq/store/DispatchRequest.java
@@ -37,6 +37,12 @@ public class DispatchRequest {
 
     private int bufferSize = -1;//the buffer size maybe larger than the msg size if the message is wrapped by something
 
+    // for batch consume queue
+    private long  msgBaseOffset = -1;
+    private short batchSize = 1;
+
+    private long nextReputFromOffset = -1;
+
     public DispatchRequest(
         final String topic,
         final int queueId,
@@ -58,6 +64,7 @@ public class DispatchRequest {
         this.tagsCode = tagsCode;
         this.storeTimestamp = storeTimestamp;
         this.consumeQueueOffset = consumeQueueOffset;
+        this.msgBaseOffset = consumeQueueOffset;
         this.keys = keys;
         this.uniqKey = uniqKey;
 
@@ -159,10 +166,26 @@ public class DispatchRequest {
         this.bitMap = bitMap;
     }
 
+    public short getBatchSize() {
+        return batchSize;
+    }
+
+    public void setBatchSize(short batchSize) {
+        this.batchSize = batchSize;
+    }
+
     public void setMsgSize(int msgSize) {
         this.msgSize = msgSize;
     }
 
+    public long getMsgBaseOffset() {
+        return msgBaseOffset;
+    }
+
+    public void setMsgBaseOffset(long msgBaseOffset) {
+        this.msgBaseOffset = msgBaseOffset;
+    }
+
     public int getBufferSize() {
         return bufferSize;
     }
@@ -170,4 +193,26 @@ public class DispatchRequest {
     public void setBufferSize(int bufferSize) {
         this.bufferSize = bufferSize;
     }
+
+    public long getNextReputFromOffset() {
+        return nextReputFromOffset;
+    }
+
+    public void setNextReputFromOffset(long nextReputFromOffset) {
+        this.nextReputFromOffset = nextReputFromOffset;
+    }
+
+    @Override
+    public String toString() {
+        return "DispatchRequest{" +
+                "topic='" + topic + '\'' +
+                ", queueId=" + queueId +
+                ", commitLogOffset=" + commitLogOffset +
+                ", msgSize=" + msgSize +
+                ", success=" + success +
+                ", msgBaseOffset=" + msgBaseOffset +
+                ", batchSize=" + batchSize +
+                ", nextReputFromOffset=" + nextReputFromOffset +
+            '}';
+    }
 }
diff --git a/store/src/main/java/org/apache/rocketmq/store/FileQueueSnapshot.java b/store/src/main/java/org/apache/rocketmq/store/FileQueueSnapshot.java
new file mode 100644
index 0000000..6521a76
--- /dev/null
+++ b/store/src/main/java/org/apache/rocketmq/store/FileQueueSnapshot.java
@@ -0,0 +1,90 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.rocketmq.store;
+
+import org.apache.rocketmq.store.logfile.MappedFile;
+
+public class FileQueueSnapshot {
+    private MappedFile firstFile;
+    private long firstFileIndex;
+    private MappedFile lastFile;
+    private long lastFileIndex;
+    private long currentFile;
+    private long currentFileIndex;
+    private long behindCount;
+    private boolean exist;
+
+    public FileQueueSnapshot() {
+    }
+
+    public FileQueueSnapshot(MappedFile firstFile, long firstFileIndex, MappedFile lastFile, long lastFileIndex, long currentFile, long currentFileIndex, long behindCount, boolean exist) {
+        this.firstFile = firstFile;
+        this.firstFileIndex = firstFileIndex;
+        this.lastFile = lastFile;
+        this.lastFileIndex = lastFileIndex;
+        this.currentFile = currentFile;
+        this.currentFileIndex = currentFileIndex;
+        this.behindCount = behindCount;
+        this.exist = exist;
+    }
+
+    public MappedFile getFirstFile() {
+        return firstFile;
+    }
+
+    public long getFirstFileIndex() {
+        return firstFileIndex;
+    }
+
+    public MappedFile getLastFile() {
+        return lastFile;
+    }
+
+    public long getLastFileIndex() {
+        return lastFileIndex;
+    }
+
+    public long getCurrentFile() {
+        return currentFile;
+    }
+
+    public long getCurrentFileIndex() {
+        return currentFileIndex;
+    }
+
+    public long getBehindCount() {
+        return behindCount;
+    }
+
+    public boolean isExist() {
+        return exist;
+    }
+
+    @Override
+    public String toString() {
+        return "FileQueueSnapshot{" +
+                "firstFile=" + firstFile +
+                ", firstFileIndex=" + firstFileIndex +
+                ", lastFile=" + lastFile +
+                ", lastFileIndex=" + lastFileIndex +
+                ", currentFile=" + currentFile +
+                ", currentFileIndex=" + currentFileIndex +
+                ", behindCount=" + behindCount +
+                ", exist=" + exist +
+                '}';
+    }
+}
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 fafff6b..b027914 100644
--- a/store/src/main/java/org/apache/rocketmq/store/GetMessageResult.java
+++ b/store/src/main/java/org/apache/rocketmq/store/GetMessageResult.java
@@ -34,6 +34,8 @@ public class GetMessageResult {
 
     private int bufferTotalSize = 0;
 
+    private int messageCount = 0;
+
     private boolean suggestPullingFromSlave = false;
 
     private int msgCount4Commercial = 0;
@@ -99,10 +101,20 @@ public class GetMessageResult {
     }
 
     public void addMessage(final SelectMappedBufferResult mapedBuffer, final long queueOffset) {
-        addMessage(mapedBuffer);
+        this.messageMapedList.add(mapedBuffer);
+        this.messageBufferList.add(mapedBuffer.getByteBuffer());
+        this.bufferTotalSize += mapedBuffer.getSize();
+        this.msgCount4Commercial += (int) Math.ceil(
+            mapedBuffer.getSize() / BrokerStatsManager.SIZE_PER_COUNT);
         this.messageQueueOffset.add(queueOffset);
     }
 
+
+    public void addMessage(final SelectMappedBufferResult mapedBuffer, final long queueOffset, final int batchNum) {
+        addMessage(mapedBuffer, queueOffset);
+        messageCount += batchNum;
+    }
+
     public void release() {
         for (SelectMappedBufferResult select : this.messageMapedList) {
             select.release();
@@ -113,12 +125,8 @@ public class GetMessageResult {
         return bufferTotalSize;
     }
 
-    public void setBufferTotalSize(int bufferTotalSize) {
-        this.bufferTotalSize = bufferTotalSize;
-    }
-
     public int getMessageCount() {
-        return this.messageMapedList.size();
+        return messageCount;
     }
 
     public boolean isSuggestPullingFromSlave() {
@@ -144,8 +152,7 @@ public class GetMessageResult {
     @Override
     public String toString() {
         return "GetMessageResult [status=" + status + ", nextBeginOffset=" + nextBeginOffset + ", minOffset="
-            + minOffset + ", maxOffset=" + maxOffset + ", bufferTotalSize=" + bufferTotalSize
+            + minOffset + ", maxOffset=" + maxOffset + ", bufferTotalSize=" + bufferTotalSize + ", messageCount=" + messageCount
             + ", suggestPullingFromSlave=" + suggestPullingFromSlave + "]";
     }
-
 }
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 1aa9ef8..a7d5083 100644
--- a/store/src/main/java/org/apache/rocketmq/store/MappedFileQueue.java
+++ b/store/src/main/java/org/apache/rocketmq/store/MappedFileQueue.java
@@ -25,29 +25,34 @@ import java.util.Iterator;
 import java.util.List;
 import java.util.ListIterator;
 import java.util.concurrent.CopyOnWriteArrayList;
+import java.util.stream.Stream;
+
+import com.google.common.collect.Lists;
 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.store.logfile.DefaultMappedFile;
+import org.apache.rocketmq.store.logfile.MappedFile;
 
-public class MappedFileQueue {
+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;
 
-    private final String storePath;
+    protected final String storePath;
 
     protected final int mappedFileSize;
 
     protected final CopyOnWriteArrayList<MappedFile> mappedFiles = new CopyOnWriteArrayList<MappedFile>();
 
-    private final AllocateMappedFileService allocateMappedFileService;
+    protected final AllocateMappedFileService allocateMappedFileService;
 
     protected long flushedWhere = 0;
-    private long committedWhere = 0;
+    protected long committedWhere = 0;
 
-    private volatile long storeTimestamp = 0;
+    protected volatile long storeTimestamp = 0;
 
     public MappedFileQueue(final String storePath, int mappedFileSize,
         AllocateMappedFileService allocateMappedFileService) {
@@ -91,7 +96,7 @@ public class MappedFileQueue {
         return (MappedFile) mfs[mfs.length - 1];
     }
 
-    private Object[] copyMappedFiles(final int reservedMappedFiles) {
+    protected Object[] copyMappedFiles(final int reservedMappedFiles) {
         Object[] mfs;
 
         if (this.mappedFiles.size() <= reservedMappedFiles) {
@@ -166,8 +171,8 @@ public class MappedFileQueue {
                 return true;
             }
 
-            try {
-                MappedFile mappedFile = new MappedFile(file.getPath(), mappedFileSize);
+                try {
+                    MappedFile mappedFile = new DefaultMappedFile(file.getPath(), mappedFileSize);
 
                 mappedFile.setWrotePosition(this.mappedFileSize);
                 mappedFile.setFlushedPosition(this.mappedFileSize);
@@ -229,13 +234,13 @@ public class MappedFileQueue {
         if (this.allocateMappedFileService != null) {
             mappedFile = this.allocateMappedFileService.putRequestAndReturnMappedFile(nextFilePath,
                     nextNextFilePath, this.mappedFileSize);
-        } else {
-            try {
-                mappedFile = new MappedFile(nextFilePath, this.mappedFileSize);
-            } catch (IOException e) {
-                log.error("create mappedFile exception", e);
+            } else {
+                try {
+                    mappedFile = new DefaultMappedFile(nextFilePath, this.mappedFileSize);
+                } catch (IOException e) {
+                    log.error("create mappedFile exception", e);
+                }
             }
-        }
 
         if (mappedFile != null) {
             if (this.mappedFiles.isEmpty()) {
@@ -585,6 +590,70 @@ public class MappedFileQueue {
         }
     }
 
+    @Override
+    public void swapMap(int reserveNum, long forceSwapIntervalMs, long normalSwapIntervalMs) {
+
+        if (mappedFiles.isEmpty()) {
+            return;
+        }
+
+        if (reserveNum < 3) {
+            reserveNum = 3;
+        }
+
+        Object[] mfs = this.copyMappedFiles(0);
+        if (null == mfs) {
+            return;
+        }
+
+        for (int i = mfs.length - reserveNum - 1; i >= 0; i--) {
+            MappedFile mappedFile = (MappedFile) mfs[i];
+            if (System.currentTimeMillis() - mappedFile.getRecentSwapMapTime() > forceSwapIntervalMs) {
+                mappedFile.swapMap();
+                continue;
+            }
+            if (System.currentTimeMillis() - mappedFile.getRecentSwapMapTime() > normalSwapIntervalMs
+                    && mappedFile.getMappedByteBufferAccessCountSinceLastSwap() > 0) {
+                mappedFile.swapMap();
+                continue;
+            }
+        }
+    }
+
+    @Override
+    public void cleanSwappedMap(long forceCleanSwapIntervalMs) {
+
+        if (mappedFiles.isEmpty()) {
+            return;
+        }
+
+        int reserveNum = 3;
+        Object[] mfs = this.copyMappedFiles(0);
+        if (null == mfs) {
+            return;
+        }
+
+        for (int i = mfs.length - reserveNum - 1; i >= 0; i--) {
+            MappedFile mappedFile = (MappedFile) mfs[i];
+            if (System.currentTimeMillis() - mappedFile.getRecentSwapMapTime() > forceCleanSwapIntervalMs) {
+                mappedFile.cleanSwapedMap(false);
+            }
+        }
+    }
+
+    public Object[] snapshot() {
+        // return a safe copy
+        return this.mappedFiles.toArray();
+    }
+
+    public Stream<MappedFile> stream() {
+        return this.mappedFiles.stream();
+    }
+
+    public Stream<MappedFile> reversedStream() {
+        return Lists.reverse(this.mappedFiles).stream();
+    }
+
     public long getFlushedWhere() {
         return flushedWhere;
     }
diff --git a/common/src/main/java/org/apache/rocketmq/common/message/MessageExtBatch.java b/store/src/main/java/org/apache/rocketmq/store/MessageExtBatch.java
similarity index 76%
copy from common/src/main/java/org/apache/rocketmq/common/message/MessageExtBatch.java
copy to store/src/main/java/org/apache/rocketmq/store/MessageExtBatch.java
index a2713cb..e62dfb4 100644
--- a/common/src/main/java/org/apache/rocketmq/common/message/MessageExtBatch.java
+++ b/store/src/main/java/org/apache/rocketmq/store/MessageExtBatch.java
@@ -15,19 +15,30 @@
  * limitations under the License.
  */
 
-package org.apache.rocketmq.common.message;
+package org.apache.rocketmq.store;
 
 import java.nio.ByteBuffer;
 
-public class MessageExtBatch extends MessageExt {
+public class MessageExtBatch extends MessageExtBrokerInner {
 
     private static final long serialVersionUID = -2353110995348498537L;
-
+    /**
+     * Inner batch means the batch dose not need to be unwrapped
+     */
+    private boolean isInnerBatch = false;
     public ByteBuffer wrap() {
         assert getBody() != null;
         return ByteBuffer.wrap(getBody(), 0, getBody().length);
     }
 
+    public boolean isInnerBatch() {
+        return isInnerBatch;
+    }
+
+    public void setInnerBatch(boolean innerBatch) {
+        isInnerBatch = innerBatch;
+    }
+
     private ByteBuffer encodedBuff;
 
     public ByteBuffer getEncodedBuff() {
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 4917c25..51d8a24 100644
--- a/store/src/main/java/org/apache/rocketmq/store/MessageStore.java
+++ b/store/src/main/java/org/apache/rocketmq/store/MessageStore.java
@@ -20,11 +20,18 @@ import java.util.HashMap;
 import java.util.LinkedList;
 import java.util.Set;
 import java.util.concurrent.CompletableFuture;
+
+import org.apache.rocketmq.common.SystemClock;
 import org.apache.rocketmq.common.message.MessageExt;
-import org.apache.rocketmq.common.message.MessageExtBatch;
 import org.apache.rocketmq.store.config.BrokerRole;
+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.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;
 
 /**
  * This class defines contracting interfaces to implement, allowing third-party vendor to use customized message store.
@@ -107,6 +114,22 @@ public interface MessageStore {
         final long offset, final int maxMsgNums, final MessageFilter messageFilter);
 
     /**
+     * Query at most <code>maxMsgNums</code> messages belonging to <code>topic</code> at <code>queueId</code> starting
+     * from given <code>offset</code>. Resulting messages will further be screened using provided message filter.
+     *
+     * @param group Consumer group that launches this query.
+     * @param topic Topic to query.
+     * @param queueId Queue ID to query.
+     * @param offset Logical offset to start from.
+     * @param maxMsgNums Maximum count of messages to query.
+     * @param maxTotalMsgSize Maxisum total msg size of the messages
+     * @param messageFilter Message filter used to screen desired messages.
+     * @return Matched messages.
+     */
+    GetMessageResult getMessage(final String group, final String topic, final int queueId,
+        final long offset, final int maxMsgNums, final int maxTotalMsgSize, final MessageFilter messageFilter);
+
+    /**
      * Get maximum offset of the topic queue.
      *
      * @param topic Topic name.
@@ -163,6 +186,15 @@ public interface MessageStore {
     MessageExt lookMessageByOffset(final long commitLogOffset);
 
     /**
+     * Look up the message by given commit log offset and size.
+     *
+     * @param commitLogOffset physical offset.
+     * @param size message size
+     * @return Message whose physical offset is as specified.
+     */
+    MessageExt lookMessageByOffset(long commitLogOffset, int size);
+
+    /**
      * Get one message from the specified commit log offset.
      *
      * @param commitLogOffset commit log offset.
@@ -393,7 +425,7 @@ public interface MessageStore {
      * @param queueId Queue ID.
      * @return Consume queue.
      */
-    ConsumeQueue getConsumeQueue(String topic, int queueId);
+    ConsumeQueueInterface getConsumeQueue(String topic, int queueId);
 
     ScheduleMessageService getScheduleMessageService();
 
@@ -409,4 +441,137 @@ public interface MessageStore {
      * @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
+     * @param commitLogFile commit log file
+     */
+    void onCommitLogAppend(MessageExtBrokerInner msg, AppendMessageResult result, MappedFile commitLogFile);
+
+    /**
+     * Will be triggered when a new dispatch request is sent to message store.
+     * @param dispatchRequest dispatch request
+     * @param doDispatch do dispatch if true
+     * @param commitLogFile commit log file
+     * @param isRecover is from recover process
+     * @param isFileEnd if the dispatch request represents 'file end'
+     */
+    void onCommitLogDispatch(DispatchRequest dispatchRequest, boolean doDispatch, MappedFile commitLogFile, boolean isRecover, boolean isFileEnd);
+
+    /**
+     * Get the message store config
+     * @return the message store config
+     */
+    MessageStoreConfig getMessageStoreConfig();
+
+    /**
+     * Get the statistics service
+     * @return the statistics service
+     */
+    StoreStatsService getStoreStatsService();
+
+    /**
+     * Get the store checkpoint component
+     * @return the checkpoint component
+     */
+    StoreCheckpoint getStoreCheckpoint();
+
+    /**
+     * Get the system clock
+     * @return the system clock
+     */
+    SystemClock getSystemClock();
+
+    /**
+     * Get the commit log
+     * @return the commit log
+     */
+    CommitLog getCommitLog();
+
+    /**
+     * Get running flags
+     * @return running flags
+     */
+    RunningFlags getRunningFlags();
+
+    /**
+     * Get the transient store pool
+     * @return the transient store pool
+     */
+    TransientStorePool getTransientStorePool();
+
+    /**
+     * Get the HA service
+     * @return the HA service
+     */
+    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
+     */
+    AllocateMappedFileService getAllocateMappedFileService();
+
+    /**
+     * Truncate dirty logic files
+     * @param phyOffset physical offset
+     */
+    void truncateDirtyLogicFiles(long phyOffset);
+
+    /**
+     * Destroy logics files
+     */
+    void destroyLogics();
+
+    /**
+     * Unlock mappedFile
+     * @param unlockMappedFile the file that needs to be unlocked
+     */
+    void unlockMappedFile(MappedFile unlockMappedFile);
+
+    /**
+     * Get the perf counter component
+     * @return the perf counter component
+     */
+    PerfCounter.Ticks getPerfCounter();
+
+    /**
+     * Get the queue store
+     * @return the queue store
+     */
+    ConsumeQueueStore getQueueStore();
+
+    /**
+     * If 'sync disk flush' is configured in this message store
+     * @return yes if true, no if false
+     */
+    boolean isSyncDiskFlush();
+
+    /**
+     * If this message store is sync master role
+     * @return yes if true, no if false
+     */
+    boolean isSyncMaster();
+
+    /**
+     * assign an queue offset and increase it.
+     * @param topicQueueKey topic-queue key
+     * @param msg message
+     * @param batchNum batch num
+     */
+    void assignOffset(String topicQueueKey, MessageExtBrokerInner msg, short batchNum);
+
+    /**
+     * remove offset table
+     * @param topicQueueKey topic-queue key
+     */
+    void removeOffsetTable(String topicQueueKey);
 }
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 669698f..c5b2316 100644
--- a/store/src/main/java/org/apache/rocketmq/store/MultiPathMappedFileQueue.java
+++ b/store/src/main/java/org/apache/rocketmq/store/MultiPathMappedFileQueue.java
@@ -24,6 +24,7 @@ import java.util.function.Supplier;
 import org.apache.commons.lang3.StringUtils;
 import org.apache.rocketmq.common.UtilAll;
 import org.apache.rocketmq.store.config.MessageStoreConfig;
+import org.apache.rocketmq.store.logfile.MappedFile;
 
 import java.io.File;
 import java.util.ArrayList;
diff --git a/store/src/main/java/org/apache/rocketmq/store/PutMessageContext.java b/store/src/main/java/org/apache/rocketmq/store/PutMessageContext.java
new file mode 100644
index 0000000..c3dd89b
--- /dev/null
+++ b/store/src/main/java/org/apache/rocketmq/store/PutMessageContext.java
@@ -0,0 +1,31 @@
+package org.apache.rocketmq.store;
+
+public class PutMessageContext {
+        private String topicQueueTableKey;
+        private long[] phyPos;
+        private int batchSize;
+
+        public PutMessageContext(String topicQueueTableKey) {
+            this.topicQueueTableKey = topicQueueTableKey;
+        }
+
+        public String getTopicQueueTableKey() {
+            return topicQueueTableKey;
+        }
+
+        public long[] getPhyPos() {
+            return phyPos;
+        }
+
+        public void setPhyPos(long[] phyPos) {
+            this.phyPos = phyPos;
+        }
+
+        public int getBatchSize() {
+            return batchSize;
+        }
+
+        public void setBatchSize(int batchSize) {
+            this.batchSize = batchSize;
+        }
+    }
\ No newline at end of file
diff --git a/store/src/main/java/org/apache/rocketmq/store/SelectMappedBufferResult.java b/store/src/main/java/org/apache/rocketmq/store/SelectMappedBufferResult.java
index 03061e6..76919f3 100644
--- a/store/src/main/java/org/apache/rocketmq/store/SelectMappedBufferResult.java
+++ b/store/src/main/java/org/apache/rocketmq/store/SelectMappedBufferResult.java
@@ -16,6 +16,8 @@
  */
 package org.apache.rocketmq.store;
 
+import org.apache.rocketmq.store.logfile.MappedFile;
+
 import java.nio.ByteBuffer;
 
 public class SelectMappedBufferResult {
@@ -26,7 +28,7 @@ public class SelectMappedBufferResult {
 
     private int size;
 
-    private MappedFile mappedFile;
+    protected MappedFile mappedFile;
 
     public SelectMappedBufferResult(long startOffset, ByteBuffer byteBuffer, int size, MappedFile mappedFile) {
         this.startOffset = startOffset;
@@ -48,12 +50,19 @@ public class SelectMappedBufferResult {
         this.byteBuffer.limit(this.size);
     }
 
+    public MappedFile getMappedFile() {
+        return mappedFile;
+    }
+
     public synchronized void release() {
         if (this.mappedFile != null) {
             this.mappedFile.release();
             this.mappedFile = null;
         }
     }
+    public synchronized boolean hasReleased() {
+        return this.mappedFile == null;
+    }
 
     public long getStartOffset() {
         return startOffset;
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 7e6c706..f209ed2 100644
--- a/store/src/main/java/org/apache/rocketmq/store/StoreCheckpoint.java
+++ b/store/src/main/java/org/apache/rocketmq/store/StoreCheckpoint.java
@@ -26,6 +26,7 @@ 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.store.logfile.DefaultMappedFile;
 
 public class StoreCheckpoint {
     private static final InternalLogger log = InternalLoggerFactory.getLogger(LoggerName.STORE_LOGGER_NAME);
@@ -38,12 +39,12 @@ public class StoreCheckpoint {
 
     public StoreCheckpoint(final String scpPath) throws IOException {
         File file = new File(scpPath);
-        MappedFile.ensureDirOK(file.getParent());
+        DefaultMappedFile.ensureDirOK(file.getParent());
         boolean fileExists = file.exists();
 
         this.randomAccessFile = new RandomAccessFile(file, "rw");
         this.fileChannel = this.randomAccessFile.getChannel();
-        this.mappedByteBuffer = fileChannel.map(MapMode.READ_WRITE, 0, MappedFile.OS_PAGE_SIZE);
+        this.mappedByteBuffer = fileChannel.map(MapMode.READ_WRITE, 0, DefaultMappedFile.OS_PAGE_SIZE);
 
         if (fileExists) {
             log.info("store checkpoint file exists, " + scpPath);
@@ -66,7 +67,7 @@ public class StoreCheckpoint {
         this.flush();
 
         // unmap mappedByteBuffer
-        MappedFile.clean(this.mappedByteBuffer);
+        DefaultMappedFile.clean(this.mappedByteBuffer);
 
         try {
             this.fileChannel.close();
diff --git a/store/src/main/java/org/apache/rocketmq/store/StoreUtil.java b/store/src/main/java/org/apache/rocketmq/store/StoreUtil.java
index f63efd6..81b407e 100644
--- a/store/src/main/java/org/apache/rocketmq/store/StoreUtil.java
+++ b/store/src/main/java/org/apache/rocketmq/store/StoreUtil.java
@@ -16,10 +16,21 @@
  */
 package org.apache.rocketmq.store;
 
+import com.google.common.base.Preconditions;
+import org.apache.rocketmq.common.constant.LoggerName;
+import org.apache.rocketmq.logging.InternalLogger;
+import org.apache.rocketmq.logging.InternalLoggerFactory;
+import org.apache.rocketmq.store.logfile.MappedFile;
+
 import java.lang.management.ManagementFactory;
 import java.lang.management.OperatingSystemMXBean;
+import java.nio.ByteBuffer;
+
+import static java.lang.String.format;
 
 public class StoreUtil {
+    private static final InternalLogger log = InternalLoggerFactory.getLogger(LoggerName.STORE_LOGGER_NAME);
+
     public static final long TOTAL_PHYSICAL_MEMORY_SIZE = getTotalPhysicalMemorySize();
 
     @SuppressWarnings("restriction")
@@ -32,4 +43,41 @@ public class StoreUtil {
 
         return physicalTotal;
     }
+
+    public static boolean isStreamMode(MessageStore messageStore) {
+        return messageStore instanceof StreamMessageStore;
+    }
+
+    public static void fileAppend(MappedFile file, ByteBuffer data) {
+        boolean success = file.appendMessage(data);
+        if (!success) {
+            throw new RuntimeException(format("fileAppend failed for file: %s and data remaining: %d", file, data.remaining()));
+        }
+    }
+
+    public static FileQueueSnapshot getFileQueueSnapshot(MappedFileQueue mappedFileQueue) {
+        return getFileQueueSnapshot(mappedFileQueue, mappedFileQueue.getLastMappedFile().getFileFromOffset());
+    }
+
+    public static FileQueueSnapshot getFileQueueSnapshot(MappedFileQueue mappedFileQueue, final long currentFile) {
+        try {
+            Preconditions.checkNotNull(mappedFileQueue, "file queue shouldn't be null");
+            MappedFile firstFile = mappedFileQueue.getFirstMappedFile();
+            MappedFile lastFile = mappedFileQueue.getLastMappedFile();
+            int mappedFileSize = mappedFileQueue.getMappedFileSize();
+            if (firstFile == null || lastFile == null) {
+                return new FileQueueSnapshot(firstFile, -1, lastFile, -1, currentFile, -1, 0, false);
+            }
+
+            long firstFileIndex = 0;
+            long lastFileIndex = (lastFile.getFileFromOffset() - firstFile.getFileFromOffset()) / mappedFileSize;
+            long currentFileIndex = (currentFile - firstFile.getFileFromOffset()) / mappedFileSize;
+            long behind = (lastFile.getFileFromOffset() - currentFile) / mappedFileSize;
+            boolean exist = firstFile.getFileFromOffset() <= currentFile && currentFile <= lastFile.getFileFromOffset();
+            return new FileQueueSnapshot(firstFile, firstFileIndex, lastFile, lastFileIndex, currentFile, currentFileIndex, behind, exist);
+        } catch (Exception e) {
+            log.error("[BUG] get file queue snapshot failed. fileQueue: {}, currentFile: {}", mappedFileQueue, currentFile, e);
+        }
+        return new FileQueueSnapshot();
+    }
 }
diff --git a/store/src/main/java/org/apache/rocketmq/store/StreamMessageStore.java b/store/src/main/java/org/apache/rocketmq/store/StreamMessageStore.java
new file mode 100644
index 0000000..d9277ff
--- /dev/null
+++ b/store/src/main/java/org/apache/rocketmq/store/StreamMessageStore.java
@@ -0,0 +1,2573 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.rocketmq.store;
+
+import org.apache.rocketmq.common.BrokerConfig;
+import org.apache.rocketmq.common.MixAll;
+import org.apache.rocketmq.common.ServiceThread;
+import org.apache.rocketmq.common.SystemClock;
+import org.apache.rocketmq.common.ThreadFactoryImpl;
+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.running.RunningStats;
+import org.apache.rocketmq.common.sysflag.MessageSysFlag;
+import org.apache.rocketmq.common.topic.TopicValidator;
+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.config.StorePathConfigHelper;
+import org.apache.rocketmq.store.ha.HAService;
+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.BatchConsumeQueue;
+import org.apache.rocketmq.store.queue.CQType;
+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;
+import org.apache.rocketmq.store.util.QueueTypeUtils;
+
+import java.io.File;
+import java.io.IOException;
+import java.io.RandomAccessFile;
+import java.nio.ByteBuffer;
+import java.nio.channels.FileLock;
+import java.util.Collections;
+import java.util.HashMap;
+import java.util.Iterator;
+import java.util.LinkedList;
+import java.util.List;
+import java.util.Map;
+import java.util.Map.Entry;
+import java.util.Set;
+import java.util.concurrent.BlockingQueue;
+import java.util.concurrent.CompletableFuture;
+import java.util.concurrent.ConcurrentHashMap;
+import java.util.concurrent.ConcurrentMap;
+import java.util.concurrent.Executors;
+import java.util.concurrent.Future;
+import java.util.concurrent.LinkedBlockingDeque;
+import java.util.concurrent.RejectedExecutionException;
+import java.util.concurrent.ScheduledExecutorService;
+import java.util.concurrent.ThreadFactory;
+import java.util.concurrent.ThreadPoolExecutor;
+import java.util.concurrent.TimeUnit;
+import java.util.concurrent.atomic.AtomicLong;
+
+import static java.lang.String.format;
+
+public class StreamMessageStore implements MessageStore {
+    private static final InternalLogger log = InternalLoggerFactory.getLogger(LoggerName.STORE_LOGGER_NAME);
+
+    public final PerfCounter.Ticks perfs = new PerfCounter.Ticks(log);
+
+    private final MessageStoreConfig messageStoreConfig;
+    // CommitLog
+    private final CommitLog commitLog;
+
+    private final ConsumeQueueStore consumeQueueStore;
+
+    private final ConcurrentMap<String/* topic */, ConcurrentMap<Integer/* queueId */, ConsumeQueueInterface>> consumeQueueTable;
+
+    protected HashMap<String/* topic-queueid */, Long/* offset */> topicQueueTable = new HashMap<String, Long>(1024);
+
+    protected HashMap<String/* topic-queueid */, Long/* offset */> batchTopicQueueTable = new HashMap<String, Long>(1024);
+
+    private final FlushConsumeQueueService flushConsumeQueueService;
+
+    private final CleanCommitLogService cleanCommitLogService;
+
+    private final CleanConsumeQueueService cleanConsumeQueueService;
+
+    private final CorrectLogicOffsetService correctLogicOffsetService;
+
+    private final IndexService indexService;
+
+    private final AllocateMappedFileService allocateMappedFileService;
+
+    private final ReputMessageService reputMessageService;
+
+    private final HAService haService;
+
+    private final ScheduleMessageService scheduleMessageService;
+
+    private final StoreStatsService storeStatsService;
+
+    private final TransientStorePool transientStorePool;
+
+    private final RunningFlags runningFlags = new RunningFlags();
+    private final SystemClock systemClock = new SystemClock();
+
+    private final ScheduledExecutorService scheduledExecutorService =
+        Executors.newSingleThreadScheduledExecutor(new ThreadFactoryImpl("StoreScheduledThread"));
+    private final BrokerStatsManager brokerStatsManager;
+    private final MessageArrivingListener messageArrivingListener;
+    private final BrokerConfig brokerConfig;
+
+    private volatile boolean shutdown = true;
+
+    private StoreCheckpoint storeCheckpoint;
+
+    private AtomicLong printTimes = new AtomicLong(0);
+
+    private final LinkedList<CommitLogDispatcher> dispatcherList;
+
+    private RandomAccessFile lockFile;
+
+    private FileLock lock;
+
+    boolean shutDownNormal = false;
+
+    //polish for reput
+    private ThreadPoolExecutor[] reputExecutors;
+
+    private BlockingQueue<Runnable>[] reputQueues;
+
+    private boolean isDispatchFromSenderThread;
+
+    private static final Future EMPTY_FUTURE = new Future() {
+        @Override
+        public boolean cancel(final boolean mayInterruptIfRunning) {
+            return false;
+        }
+
+        @Override
+        public boolean isCancelled() {
+            return false;
+        }
+
+        @Override
+        public boolean isDone() {
+            return true;
+        }
+
+        @Override
+        public Object get() {
+            return null;
+        }
+
+        @Override
+        public Object get(final long timeout, final TimeUnit unit) {
+            return null;
+        }
+    };
+
+    // Max pull msg size
+    private final static int MAX_PULL_MSG_SIZE = 128 * 1024 * 1024;
+
+    public StreamMessageStore(final MessageStoreConfig messageStoreConfig, final BrokerStatsManager brokerStatsManager,
+                               final MessageArrivingListener messageArrivingListener, final BrokerConfig brokerConfig) throws IOException {
+        this.messageArrivingListener = messageArrivingListener;
+        this.brokerConfig = brokerConfig;
+        this.messageStoreConfig = messageStoreConfig;
+        this.brokerStatsManager = brokerStatsManager;
+        this.allocateMappedFileService = new AllocateMappedFileService(this);
+        if (messageStoreConfig.isEnableDLegerCommitLog()) {
+            throw new RuntimeException("dleger is not supported in this message store.");
+        }
+        this.isDispatchFromSenderThread = messageStoreConfig.isDispatchFromSenderThread();
+        this.commitLog = new CommitLog(this);
+        this.consumeQueueTable = new ConcurrentHashMap<>(32);
+        this.consumeQueueStore = new ConsumeQueueStore(this, this.messageStoreConfig, this.consumeQueueTable);
+
+        this.flushConsumeQueueService = new FlushConsumeQueueService();
+        this.cleanCommitLogService = new CleanCommitLogService();
+        this.cleanConsumeQueueService = new CleanConsumeQueueService();
+        this.correctLogicOffsetService = new CorrectLogicOffsetService();
+        this.storeStatsService = new StoreStatsService();
+        this.indexService = new IndexService(this);
+        if (!messageStoreConfig.isEnableDLegerCommitLog()) {
+            this.haService = new HAService(this);
+        } else {
+            this.haService = null;
+        }
+        if (isDispatchFromSenderThread) {
+            this.reputMessageService = new SyncReputMessageService();
+        } else {
+            this.reputMessageService = new ReputMessageService();
+        }
+
+        this.scheduleMessageService = new ScheduleMessageService(this);
+
+        this.transientStorePool = new TransientStorePool(messageStoreConfig);
+
+        if (messageStoreConfig.isTransientStorePoolEnable()) {
+            this.transientStorePool.init();
+        }
+
+        this.allocateMappedFileService.start();
+
+        this.indexService.start();
+
+        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());
+        lockFile = new RandomAccessFile(file, "rw");
+        initAsyncReputThreads(messageStoreConfig.getDispatchCqThreads(), messageStoreConfig.getDispatchCqCacheNum());
+    }
+
+    /**
+     * @throws IOException
+     */
+    @Override
+    public boolean load() {
+        boolean result = true;
+
+        try {
+            long start = System.currentTimeMillis();
+            boolean lastExitOK = !this.isTempFileExist();
+            log.info("last shutdown {}, root dir: {}", lastExitOK ? "normally" : "abnormally", messageStoreConfig.getStorePathRootDir());
+
+            if (null != scheduleMessageService) {
+                result = result && this.scheduleMessageService.load();
+            }
+
+            // load Commit Log
+            result = result && this.commitLog.load();
+
+            // load Batch Consume Queue
+            result = result && this.loadBatchConsumeQueue();
+
+            if (result) {
+                this.storeCheckpoint =
+                    new StoreCheckpoint(StorePathConfigHelper.getStoreCheckpoint(this.messageStoreConfig.getStorePathRootDir()));
+
+                this.indexService.load(lastExitOK);
+
+                this.recover(lastExitOK);
+
+                log.info("load over, and the max phy offset = {} cost = {}", this.getMaxPhyOffset(), System.currentTimeMillis() - start);
+            }
+        } catch (Exception e) {
+            log.error("load exception", e);
+            result = false;
+        }
+
+        if (!result) {
+            this.allocateMappedFileService.shutdown();
+        }
+
+        return result;
+    }
+
+    /**
+     * @throws Exception
+     */
+    @Override
+    public void start() throws Exception {
+
+        lock = lockFile.getChannel().tryLock(0, 1, false);
+        if (lock == null || lock.isShared() || !lock.isValid()) {
+            throw new RuntimeException("Lock failed,MQ already started");
+        }
+
+        lockFile.getChannel().write(ByteBuffer.wrap("lock".getBytes()));
+        lockFile.getChannel().force(true);
+
+        if (this.getMessageStoreConfig().isDuplicationEnable()) {
+            this.reputMessageService.setReputFromOffset(this.commitLog.getConfirmOffset());
+        } else {
+            this.reputMessageService.setReputFromOffset(this.commitLog.getMaxOffset());
+        }
+        this.reputMessageService.start();
+
+        if (!messageStoreConfig.isEnableDLegerCommitLog()) {
+            this.haService.start();
+            this.handleScheduleMessageService(messageStoreConfig.getBrokerRole());
+        }
+
+        this.flushConsumeQueueService.start();
+        this.commitLog.start();
+        this.storeStatsService.start();
+
+        this.createTempFile();
+        this.addScheduleTask();
+        this.perfs.start();
+        this.shutdown = false;
+    }
+
+    @Override
+    public void shutdown() {
+        if (!this.shutdown) {
+            this.shutdown = true;
+
+            this.scheduledExecutorService.shutdown();
+
+            try {
+
+                Thread.sleep(1000 * 3);
+            } catch (InterruptedException e) {
+                log.error("shutdown Exception, ", e);
+            }
+
+            if (this.scheduleMessageService != null) {
+                this.scheduleMessageService.shutdown();
+            }
+            if (this.haService != null) {
+                this.haService.shutdown();
+            }
+
+            this.storeStatsService.shutdown();
+            this.indexService.shutdown();
+            this.commitLog.shutdown();
+            this.reputMessageService.shutdown();
+            this.flushConsumeQueueService.shutdown();
+            this.allocateMappedFileService.shutdown();
+            this.storeCheckpoint.flush();
+            this.storeCheckpoint.shutdown();
+
+            this.perfs.shutdown();
+
+            if (this.runningFlags.isWriteable() && dispatchBehindBytes() == 0) {
+                this.deleteFile(StorePathConfigHelper.getAbortFile(this.messageStoreConfig.getStorePathRootDir()));
+                shutDownNormal = true;
+            } else {
+                log.warn("the store may be wrong, so shutdown abnormally, and keep abort file. writable: {}, dispatchBehindBytes: {}, abort file: {}",
+                        this.runningFlags.isWriteable(), dispatchBehindBytes(), StorePathConfigHelper.getAbortFile(this.messageStoreConfig.getStorePathRootDir()));
+            }
+        }
+
+        this.transientStorePool.destroy();
+
+        if (lockFile != null && lock != null) {
+            try {
+                lock.release();
+                lockFile.close();
+            } catch (IOException e) {
+            }
+        }
+    }
+
+    @Override
+    public void destroy() {
+        this.destroyLogics();
+        this.commitLog.destroy();
+        this.indexService.destroy();
+        this.deleteFile(StorePathConfigHelper.getAbortFile(this.messageStoreConfig.getStorePathRootDir()));
+        this.deleteFile(StorePathConfigHelper.getStoreCheckpoint(this.messageStoreConfig.getStorePathRootDir()));
+    }
+
+    @Override
+    public void destroyLogics() {
+        for (ConcurrentMap<Integer, ConsumeQueueInterface> maps : this.consumeQueueTable.values()) {
+            for (ConsumeQueueInterface logic : maps.values()) {
+                this.consumeQueueStore.destroy(logic);
+            }
+        }
+    }
+
+    @Override
+    public CompletableFuture<PutMessageResult> asyncPutMessage(final MessageExtBrokerInner msg) {
+        if (this.shutdown) {
+            log.warn("message store has shutdown, so putMessage is forbidden");
+            return CompletableFuture.completedFuture(new PutMessageResult(PutMessageStatus.SERVICE_NOT_AVAILABLE, null));
+        }
+
+        if (BrokerRole.SLAVE == this.messageStoreConfig.getBrokerRole()) {
+            long value = this.printTimes.getAndIncrement();
+            if ((value % 50000) == 0) {
+                log.warn("message store is slave mode, so putMessage is forbidden ");
+            }
+
+            return CompletableFuture.completedFuture(new PutMessageResult(PutMessageStatus.SERVICE_NOT_AVAILABLE, null));
+        }
+
+        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);
+            return CompletableFuture.completedFuture(new PutMessageResult(PutMessageStatus.MESSAGE_ILLEGAL, null));
+        }
+
+        if (MessageSysFlag.check(msg.getSysFlag(), MessageSysFlag.INNER_BATCH_FLAG)) {
+            CQType cqType = QueueTypeUtils.getCQType(this);
+
+            if (!CQType.BatchCQ.equals(cqType)) {
+                log.warn("[BUG]The message is an inner batch but cq type is not batch consume queue");
+                return CompletableFuture.completedFuture(new PutMessageResult(PutMessageStatus.MESSAGE_ILLEGAL, null));
+            }
+        }
+
+        if (!this.runningFlags.isWriteable()) {
+            long value = this.printTimes.getAndIncrement();
+            if ((value % 50000) == 0) {
+                log.warn("message store is not writeable, so putMessage is forbidden " + this.runningFlags.getFlagBits());
+            }
+
+            return CompletableFuture.completedFuture(new PutMessageResult(PutMessageStatus.SERVICE_NOT_AVAILABLE, null));
+        } else {
+            this.printTimes.set(0);
+        }
+
+        int topicLen = msg.getTopic().length();
+        if (topicLen > this.messageStoreConfig.getMaxTopicLength()) {
+            log.warn("putMessage message topic[{}] length too long {}", msg.getTopic(), topicLen);
+            return CompletableFuture.completedFuture(new PutMessageResult(PutMessageStatus.MESSAGE_ILLEGAL, null));
+        }
+
+        if (topicLen > Byte.MAX_VALUE) {
+            log.warn("putMessage message topic[{}] length too long {}, but it is not supported by broker",
+                    msg.getTopic(), topicLen);
+            return CompletableFuture.completedFuture(new PutMessageResult(PutMessageStatus.MESSAGE_ILLEGAL, null));
+        }
+
+        if (msg.getPropertiesString() != null && msg.getPropertiesString().length() > Short.MAX_VALUE) {
+            log.warn("putMessage message properties length too long " + msg.getPropertiesString().length());
+            return CompletableFuture.completedFuture(new PutMessageResult(PutMessageStatus.PROPERTIES_SIZE_EXCEEDED, null));
+        }
+
+        if (this.isOSPageCacheBusy()) {
+            return CompletableFuture.completedFuture(new PutMessageResult(PutMessageStatus.OS_PAGECACHE_BUSY, null));
+        }
+
+        long beginTime = this.getSystemClock().now();
+        perfs.startTick("PUT_MESSAGE_TIME_MS");
+        CompletableFuture<PutMessageResult> result = this.commitLog.asyncPutMessage(msg);
+        perfs.endTick("PUT_MESSAGE_TIME_MS");
+
+        long eclipseTime = this.getSystemClock().now() - beginTime;
+        if (eclipseTime > 500) {
+            log.warn("putMessage not in lock eclipse time(ms)={}, bodyLength={}", eclipseTime, msg.getBody().length);
+        }
+
+        return result;
+    }
+
+    @Override
+    public PutMessageResult putMessage(MessageExtBrokerInner msg) {
+        CompletableFuture<PutMessageResult> future = asyncPutMessage(msg);
+        try {
+            return future.get(3, TimeUnit.SECONDS);
+        } catch (Throwable t) {
+            log.error("Get async put result failed", t);
+            return new PutMessageResult(PutMessageStatus.OS_PAGECACHE_BUSY, null);
+        }
+    }
+
+    @Override
+    public PutMessageResult putMessages(MessageExtBatch messageExtBatch) {
+        CompletableFuture<PutMessageResult> future = asyncPutMessages(messageExtBatch);
+        try {
+            return future.get(3, TimeUnit.SECONDS);
+        } catch (Throwable t) {
+            log.error("Get async put result failed", t);
+            return new PutMessageResult(PutMessageStatus.OS_PAGECACHE_BUSY, null);
+        }
+    }
+
+    @Override
+    public CompletableFuture<PutMessageResult> asyncPutMessages(MessageExtBatch messageExtBatch) {
+        if (this.shutdown) {
+            log.warn("StreamMessageStore has shutdown, so putMessages is forbidden");
+            return CompletableFuture.completedFuture(new PutMessageResult(PutMessageStatus.SERVICE_NOT_AVAILABLE, null));
+        }
+
+        if (BrokerRole.SLAVE == this.messageStoreConfig.getBrokerRole()) {
+            long value = this.printTimes.getAndIncrement();
+            if ((value % 50000) == 0) {
+                log.warn("StreamMessageStore is in slave mode, so putMessages is forbidden ");
+            }
+
+            return CompletableFuture.completedFuture(new PutMessageResult(PutMessageStatus.SERVICE_NOT_AVAILABLE, null));
+        }
+
+        if (!this.runningFlags.isWriteable()) {
+            long value = this.printTimes.getAndIncrement();
+            if ((value % 50000) == 0) {
+                log.warn("StreamMessageStore is not writable, so putMessages is forbidden " + this.runningFlags.getFlagBits());
+            }
+
+            return CompletableFuture.completedFuture(new PutMessageResult(PutMessageStatus.SERVICE_NOT_AVAILABLE, null));
+        } else {
+            this.printTimes.set(0);
+        }
+
+        int topicLen = messageExtBatch.getTopic().length();
+        if (topicLen > this.messageStoreConfig.getMaxTopicLength()) {
+            log.warn("putMessage batch message topic[{}] length too long {}", messageExtBatch.getTopic(), topicLen);
+            return CompletableFuture.completedFuture(new PutMessageResult(PutMessageStatus.MESSAGE_ILLEGAL, null));
+        }
+
+        if (topicLen > Byte.MAX_VALUE) {
+            log.warn("putMessage batch message topic[{}] length too long {}, but it is not supported by broker",
+                    messageExtBatch.getTopic(), topicLen);
+            return CompletableFuture.completedFuture(new PutMessageResult(PutMessageStatus.MESSAGE_ILLEGAL, null));
+        }
+
+        if (messageExtBatch.getBody().length > messageStoreConfig.getMaxMessageSize()) {
+            log.warn("PutMessages body length too long " + messageExtBatch.getBody().length);
+            return CompletableFuture.completedFuture(new PutMessageResult(PutMessageStatus.MESSAGE_ILLEGAL, null));
+        }
+
+        if (this.isOSPageCacheBusy()) {
+            return CompletableFuture.completedFuture(new PutMessageResult(PutMessageStatus.OS_PAGECACHE_BUSY, null));
+        }
+
+        long beginTime = this.getSystemClock().now();
+        CompletableFuture<PutMessageResult> result = this.commitLog.asyncPutMessages(messageExtBatch);
+
+        long eclipseTime = this.getSystemClock().now() - beginTime;
+        if (eclipseTime > 500) {
+            log.warn("not in lock eclipse time(ms)={}, bodyLength={}", eclipseTime, messageExtBatch.getBody().length);
+        }
+        this.storeStatsService.setPutMessageEntireTimeMax(eclipseTime);
+
+        return result;
+    }
+
+    @Override
+    public boolean isOSPageCacheBusy() {
+        long begin = this.getCommitLog().getBeginTimeInLock();
+        long diff = this.systemClock.now() - begin;
+
+        return diff < 10000000
+            && diff > this.messageStoreConfig.getOsPageCacheBusyTimeOutMills();
+    }
+
+    @Override
+    public long lockTimeMills() {
+        return this.commitLog.lockTimeMills();
+    }
+
+    @Override
+    public SystemClock getSystemClock() {
+        return systemClock;
+    }
+
+    @Override
+    public CommitLog getCommitLog() {
+        return commitLog;
+    }
+
+    public boolean isDispatchFromSenderThread() {
+        return isDispatchFromSenderThread;
+    }
+
+    @Override
+    public GetMessageResult getMessage(final String group, final String topic, final int queueId, final long offset,
+        final int maxMsgNums,
+        final MessageFilter messageFilter) {
+        return getMessage(group, topic, queueId, offset, maxMsgNums, MAX_PULL_MSG_SIZE, messageFilter);
+    }
+
+    @Override
+    public GetMessageResult getMessage(final String group, final String topic, final int queueId, final long offset,
+        final int maxMsgNums,
+        final int maxTotalMsgSize,
+        final MessageFilter messageFilter) {
+        if (this.shutdown) {
+            log.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());
+            return null;
+        }
+
+        long beginTime = this.getSystemClock().now();
+
+        GetMessageStatus status = GetMessageStatus.NO_MESSAGE_IN_QUEUE;
+        long nextBeginOffset = offset;
+        long minOffset = 0;
+        long maxOffset = 0;
+
+        GetMessageResult getResult = new GetMessageResult();
+
+        final long maxOffsetPy = this.commitLog.getMaxOffset();
+
+        ConsumeQueueInterface consumeQueue = getConsumeQueue(topic, queueId);
+        if (consumeQueue != null) {
+            minOffset = consumeQueue.getMinOffsetInQueue();
+            maxOffset = consumeQueue.getMaxOffsetInQueue();
+
+            if (maxOffset == 0) {
+                status = GetMessageStatus.NO_MESSAGE_IN_QUEUE;
+                nextBeginOffset = nextOffsetCorrection(offset, 0);
+            } else if (offset < minOffset) {
+                status = GetMessageStatus.OFFSET_TOO_SMALL;
+                nextBeginOffset = nextOffsetCorrection(offset, minOffset);
+            } else if (offset == maxOffset) {
+                status = GetMessageStatus.OFFSET_OVERFLOW_ONE;
+                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);
+                }
+            } else {
+                final int maxFilterMessageCount = Math.max(messageStoreConfig.getPullBatchMaxMessageCount(), maxMsgNums);
+                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);
+                    maxPullSize = MAX_PULL_MSG_SIZE;
+                }
+                status = GetMessageStatus.NO_MATCHED_MESSAGE;
+                long maxPhyOffsetPulling = 0;
+                int cqFileNum = 0;
+
+                while (getResult.getBufferTotalSize() <= 0
+                        && 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);
+                        break;
+                    }
+
+                    try {
+                        long nextPhyFileStartOffset = Long.MIN_VALUE;
+                        while (bufferConsumeQueue.hasNext()
+                                && nextBeginOffset < maxOffset) {
+                            CqUnit cqUnit = bufferConsumeQueue.next();
+                            long offsetPy = cqUnit.getPos();
+                            int sizePy = cqUnit.getSize();
+
+                            boolean isInDisk = checkInDiskByCommitOffset(offsetPy, maxOffsetPy);
+
+                            if (cqUnit.getQueueOffset() - offset > maxFilterMessageCount) {
+                                break;
+                            }
+
+                            if (this.isTheBatchFull(sizePy, cqUnit.getBatchNum(), maxMsgNums, maxPullSize, getResult.getBufferTotalSize(), getResult.getMessageCount(),
+                                    isInDisk)) {
+                                break;
+                            }
+
+                            if (getResult.getBufferTotalSize() >= maxPullSize) {
+                                break;
+                            }
+
+                            maxPhyOffsetPulling = offsetPy;
+
+                            //Be careful, here should before the isTheBatchFull
+                            nextBeginOffset = cqUnit.getQueueOffset() + cqUnit.getBatchNum();
+
+                            if (nextPhyFileStartOffset != Long.MIN_VALUE) {
+                                if (offsetPy < nextPhyFileStartOffset) {
+                                    continue;
+                                }
+                            }
+
+                            if (messageFilter != null
+                                    && !messageFilter.isMatchedByConsumeQueue(cqUnit.getValidTagsCodeAsLong(), cqUnit.getCqExtUnit())) {
+                                if (getResult.getBufferTotalSize() == 0) {
+                                    status = GetMessageStatus.NO_MATCHED_MESSAGE;
+                                }
+
+                                continue;
+                            }
+
+                            SelectMappedBufferResult selectResult = this.commitLog.getMessage(offsetPy, sizePy);
+                            if (null == selectResult) {
+                                if (getResult.getBufferTotalSize() == 0) {
+                                    status = GetMessageStatus.MESSAGE_WAS_REMOVING;
+                                }
+
+                                nextPhyFileStartOffset = this.commitLog.rollNextFile(offsetPy);
+                                continue;
+                            }
+
+                            if (messageFilter != null
+                                    && !messageFilter.isMatchedByCommitLog(selectResult.getByteBuffer().slice(), null)) {
+                                if (getResult.getBufferTotalSize() == 0) {
+                                    status = GetMessageStatus.NO_MATCHED_MESSAGE;
+                                }
+                                // release...
+                                selectResult.release();
+                                continue;
+                            }
+
+                            this.storeStatsService.getGetMessageTransferedMsgCount().add(1);
+                            getResult.addMessage(selectResult, cqUnit.getQueueOffset(), cqUnit.getBatchNum());
+                            status = GetMessageStatus.FOUND;
+                            nextPhyFileStartOffset = Long.MIN_VALUE;
+                        }
+                    } finally {
+                        bufferConsumeQueue.release();
+                    }
+                }
+
+                if (diskFallRecorded) {
+                    long fallBehind = maxOffsetPy - maxPhyOffsetPulling;
+                    brokerStatsManager.recordDiskFallBehindSize(group, topic, queueId, fallBehind);
+                }
+
+                long diff = maxOffsetPy - maxPhyOffsetPulling;
+                long memory = (long) (StoreUtil.TOTAL_PHYSICAL_MEMORY_SIZE
+                        * (this.messageStoreConfig.getAccessMessageInMemoryMaxRatio() / 100.0));
+                getResult.setSuggestPullingFromSlave(diff > memory);
+            }
+        } else {
+            status = GetMessageStatus.NO_MATCHED_LOGIC_QUEUE;
+            nextBeginOffset = nextOffsetCorrection(offset, 0);
+        }
+
+        if (GetMessageStatus.FOUND == status) {
+            this.storeStatsService.getGetMessageTimesTotalFound().add(1);
+        } else {
+            this.storeStatsService.getGetMessageTimesTotalMiss().add(1);
+        }
+        long elapsedTime = this.getSystemClock().now() - beginTime;
+        this.storeStatsService.setGetMessageEntireTimeMax(elapsedTime);
+
+        getResult.setStatus(status);
+        getResult.setNextBeginOffset(nextBeginOffset);
+        getResult.setMaxOffset(maxOffset);
+        getResult.setMinOffset(minOffset);
+        return getResult;
+    }
+
+    @Override
+    public long getMaxOffsetInQueue(String topic, int queueId) {
+        ConsumeQueueInterface logic = this.getConsumeQueue(topic, queueId);
+        if (logic != null) {
+            return logic.getMaxOffsetInQueue();
+        }
+
+        return 0;
+    }
+
+    @Override
+    public long getMaxOffsetInQueue(String topic, int queueId, boolean committed) {
+        if (committed) {
+            ConsumeQueueInterface logic = this.getConsumeQueue(topic, queueId);
+            if (logic != null) {
+                return logic.getMaxOffsetInQueue();
+            }
+        } else {
+            Long offset = this.batchTopicQueueTable.get(topic + "-" + queueId);
+            if (offset != null) {
+                return offset;
+            }
+        }
+
+        return 0;
+    }
+
+    @Override
+    public long getMinOffsetInQueue(String topic, int queueId) {
+        ConsumeQueueInterface logic = this.getConsumeQueue(topic, queueId);
+        if (logic != null) {
+            return logic.getMinOffsetInQueue();
+        }
+
+        return -1;
+    }
+
+    @Override
+    public long getCommitLogOffsetInQueue(String topic, int queueId, long consumeQueueOffset) {
+        ConsumeQueueInterface consumeQueue = getConsumeQueue(topic, queueId);
+        if (consumeQueue != null) {
+
+            ReferredIterator<CqUnit> bufferConsumeQueue = consumeQueue.iterateFrom(consumeQueueOffset);
+            if (bufferConsumeQueue != null) {
+                try {
+                    if (bufferConsumeQueue.hasNext()) {
+                        long offsetPy = bufferConsumeQueue.next().getPos();
+                        return offsetPy;
+                    }
+                } finally {
+                    bufferConsumeQueue.release();
+                }
+            }
+        }
+
+        return 0;
+    }
+
+    @Override
+    public long getOffsetInQueueByTime(String topic, int queueId, long timestamp) {
+        ConsumeQueueInterface logic = getConsumeQueue(topic, queueId);
+        if (logic != null) {
+            long resultOffset = logic.getOffsetInQueueByTime(timestamp);
+            // -1 means no msg found.
+            if (resultOffset == -1) {
+                return -1;
+            }
+            // Make sure the result offset should in valid range.
+            resultOffset = Math.max(resultOffset, logic.getMinOffsetInQueue());
+            resultOffset = Math.min(resultOffset, logic.getMaxOffsetInQueue());
+            return resultOffset;
+        }
+
+        // logic is null means there is no message in this queue, return -1.
+        return -1;
+    }
+
+    @Override
+    public MessageExt lookMessageByOffset(long commitLogOffset) {
+        SelectMappedBufferResult sbr = this.commitLog.getMessage(commitLogOffset, 4);
+        if (null != sbr) {
+            try {
+                // 1 TOTALSIZE
+                int size = sbr.getByteBuffer().getInt();
+                return lookMessageByOffset(commitLogOffset, size);
+            } finally {
+                sbr.release();
+            }
+        }
+
+        return null;
+    }
+
+    @Override
+    public SelectMappedBufferResult selectOneMessageByOffset(long commitLogOffset) {
+        SelectMappedBufferResult sbr = this.commitLog.getMessage(commitLogOffset, 4);
+        if (null != sbr) {
+            try {
+                // 1 TOTALSIZE
+                int size = sbr.getByteBuffer().getInt();
+                return this.commitLog.getMessage(commitLogOffset, size);
+            } finally {
+                sbr.release();
+            }
+        }
+
+        return null;
+    }
+
+    @Override
+    public SelectMappedBufferResult selectOneMessageByOffset(long commitLogOffset, int msgSize) {
+        return this.commitLog.getMessage(commitLogOffset, msgSize);
+    }
+
+    @Override
+    public String getRunningDataInfo() {
+        return this.storeStatsService.toString();
+    }
+
+    private String getStorePathPhysic() {
+        String storePathPhysic = StreamMessageStore.this.getMessageStoreConfig().getStorePathCommitLog();
+        return storePathPhysic;
+    }
+
+    @Override
+    public HashMap<String, String> getRuntimeInfo() {
+        HashMap<String, String> result = this.storeStatsService.getRuntimeInfo();
+
+        {
+            String storePathPhysic = this.getMessageStoreConfig().getStorePathCommitLog();
+            double physicRatio = UtilAll.getDiskPartitionSpaceUsedPercent(storePathPhysic);
+            result.put(RunningStats.commitLogDiskRatio.name(), String.valueOf(physicRatio));
+
+        }
+
+        {
+
+            String storePathLogics = StorePathConfigHelper.getStorePathConsumeQueue(this.messageStoreConfig.getStorePathRootDir());
+            double logicsRatio = UtilAll.getDiskPartitionSpaceUsedPercent(storePathLogics);
+            result.put(RunningStats.consumeQueueDiskRatio.name(), String.valueOf(logicsRatio));
+        }
+
+        {
+            if (this.scheduleMessageService != null) {
+                this.scheduleMessageService.buildRunningStats(result);
+            }
+        }
+
+        result.put(RunningStats.commitLogMinOffset.name(), String.valueOf(this.getMinPhyOffset()));
+        result.put(RunningStats.commitLogMaxOffset.name(), String.valueOf(this.getMaxPhyOffset()));
+
+        return result;
+    }
+
+    @Override
+    public long getMaxPhyOffset() {
+        return this.commitLog.getMaxOffset();
+    }
+
+    @Override
+    public long getMinPhyOffset() {
+        return this.commitLog.getMinOffset();
+    }
+
+    @Override
+    public long getEarliestMessageTime(String topic, int queueId) {
+        ConsumeQueueInterface logicQueue = this.getConsumeQueue(topic, queueId);
+        if (logicQueue != null) {
+            return getStoreTime(logicQueue.getEarliestUnit());
+        }
+
+        return -1;
+    }
+
+    private long getStoreTime(CqUnit result) {
+        if (result != null) {
+            try {
+                final long phyOffset = result.getPos();
+                final int size = result.getSize();
+                long storeTime = this.getCommitLog().pickupStoreTimestamp(phyOffset, size);
+                return storeTime;
+            } catch (Exception e) {
+            }
+        }
+        return -1;
+    }
+
+    @Override
+    public long getEarliestMessageTime() {
+        final long minPhyOffset = this.getMinPhyOffset();
+        final int size = this.messageStoreConfig.getMaxMessageSize() * 2;
+        return this.getCommitLog().pickupStoreTimestamp(minPhyOffset, size);
+    }
+
+    @Override
+    public long getMessageStoreTimeStamp(String topic, int queueId, long consumeQueueOffset) {
+        ConsumeQueueInterface logicQueue = this.getConsumeQueue(topic, queueId);
+        if (logicQueue != null) {
+            return getStoreTime(logicQueue.get(consumeQueueOffset));
+        }
+
+        return -1;
+    }
+
+    @Override
+    public long getMessageTotalInQueue(String topic, int queueId) {
+        ConsumeQueueInterface logicQueue = this.getConsumeQueue(topic, queueId);
+        if (logicQueue != null) {
+            return logicQueue.getMessageTotalInQueue();
+        }
+
+        return -1;
+    }
+
+    @Override
+    public SelectMappedBufferResult getCommitLogData(final long offset) {
+        if (this.shutdown) {
+            log.warn("message store has shutdown, so getPhyQueueData is forbidden");
+            return null;
+        }
+
+        return this.commitLog.getData(offset);
+    }
+
+    @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");
+            return false;
+        }
+
+        boolean result = this.commitLog.appendData(startOffset, data, dataStart, dataLength);
+        if (result) {
+            this.reputMessageService.wakeup();
+        } else {
+            log.error("appendToPhyQueue failed " + startOffset + " " + data.length);
+        }
+
+        return result;
+    }
+
+    @Override
+    public void executeDeleteFilesManually() {
+        this.cleanCommitLogService.excuteDeleteFilesManualy();
+    }
+
+    @Override
+    public QueryMessageResult queryMessage(String topic, String key, int maxNum, long begin, long end) {
+        QueryMessageResult queryMessageResult = new QueryMessageResult();
+
+        long lastQueryMsgTime = end;
+
+        for (int i = 0; i < 3; i++) {
+            QueryOffsetResult queryOffsetResult = this.indexService.queryOffset(topic, key, maxNum, begin, lastQueryMsgTime);
+            if (queryOffsetResult.getPhyOffsets().isEmpty()) {
+                break;
+            }
+
+            Collections.sort(queryOffsetResult.getPhyOffsets());
+
+            queryMessageResult.setIndexLastUpdatePhyoffset(queryOffsetResult.getIndexLastUpdatePhyoffset());
+            queryMessageResult.setIndexLastUpdateTimestamp(queryOffsetResult.getIndexLastUpdateTimestamp());
+
+            for (int m = 0; m < queryOffsetResult.getPhyOffsets().size(); m++) {
+                long offset = queryOffsetResult.getPhyOffsets().get(m);
+
+                try {
+
+                    boolean match = true;
+                    MessageExt msg = this.lookMessageByOffset(offset);
+                    if (0 == m) {
+                        lastQueryMsgTime = msg.getStoreTimestamp();
+                    }
+
+//                    String[] keyArray = msg.getKeys().split(MessageConst.KEY_SEPARATOR);
+//                    if (topic.equals(msg.getTopic())) {
+//                        for (String k : keyArray) {
+//                            if (k.equals(key)) {
+//                                match = true;
+//                                break;
+//                            }
+//                        }
+//                    }
+
+                    if (match) {
+                        SelectMappedBufferResult result = this.commitLog.getData(offset, false);
+                        if (result != null) {
+                            int size = result.getByteBuffer().getInt(0);
+                            result.getByteBuffer().limit(size);
+                            result.setSize(size);
+                            queryMessageResult.addMessage(result);
+                        }
+                    } else {
+                        log.warn("queryMessage hash duplicate, {} {}", topic, key);
+                    }
+                } catch (Exception e) {
+                    log.error("queryMessage exception", e);
+                }
+            }
+
+            if (queryMessageResult.getBufferTotalSize() > 0) {
+                break;
+            }
+
+            if (lastQueryMsgTime < begin) {
+                break;
+            }
+        }
+
+        return queryMessageResult;
+    }
+
+    @Override
+    public void updateHaMasterAddress(String newAddr) {
+        this.haService.updateMasterAddress(newAddr);
+    }
+
+    @Override
+    public long slaveFallBehindMuch() {
+        return this.commitLog.getMaxOffset() - this.haService.getPush2SlaveMaxOffset().get();
+    }
+
+    @Override
+    public long now() {
+        return this.systemClock.now();
+    }
+
+    @Override
+    public int cleanUnusedTopic(Set<String> topics) {
+        Iterator<Map.Entry<String, ConcurrentMap<Integer, ConsumeQueueInterface>>> it = consumeQueueTable.entrySet().iterator();
+        while (it.hasNext()) {
+            Map.Entry<String, ConcurrentMap<Integer, ConsumeQueueInterface>> next = it.next();
+            String topic = next.getKey();
+
+            if (!topics.contains(topic) && !topic.equals(TopicValidator.RMQ_SYS_SCHEDULE_TOPIC)) {
+                ConcurrentMap<Integer, ConsumeQueueInterface> queueTable = next.getValue();
+                for (ConsumeQueueInterface cq : queueTable.values()) {
+                    this.consumeQueueStore.destroy(cq);
+                    log.info("cleanUnusedTopic: {} {} ConsumeQueue cleaned",
+                            cq.getTopic(),
+                            cq.getQueueId()
+                    );
+
+                    this.commitLog.removeQueueFromTopicQueueTable(cq.getTopic(), cq.getQueueId());
+                }
+                it.remove();
+                log.info("cleanUnusedTopic: {},topic consumeQueue destroyed", topic);
+            }
+        }
+        return 0;
+    }
+
+    @Override
+    public void cleanExpiredConsumerQueue() {
+        long minCommitLogOffset = this.commitLog.getMinOffset();
+
+        Iterator<Map.Entry<String, ConcurrentMap<Integer, ConsumeQueueInterface>>> it = this.consumeQueueTable.entrySet().iterator();
+        while (it.hasNext()) {
+            Map.Entry<String, ConcurrentMap<Integer, ConsumeQueueInterface>> next = it.next();
+            String topic = next.getKey();
+            if (!topic.equals(TopicValidator.RMQ_SYS_SCHEDULE_TOPIC)) {
+                ConcurrentMap<Integer, ConsumeQueueInterface> queueTable = next.getValue();
+                Iterator<Map.Entry<Integer, ConsumeQueueInterface>> itQT = queueTable.entrySet().iterator();
+                while (itQT.hasNext()) {
+                    Map.Entry<Integer, ConsumeQueueInterface> nextQT = itQT.next();
+                    long maxCLOffsetInConsumeQueue = nextQT.getValue().getMaxPhysicOffset();
+
+                    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());
+                    } else if (maxCLOffsetInConsumeQueue < minCommitLogOffset) {
+                        log.info(
+                                "cleanExpiredConsumerQueue: {} {} consumer queue destroyed, minCommitLogOffset: {} maxCLOffsetInConsumeQueue: {}",
+                                topic,
+                                nextQT.getKey(),
+                                minCommitLogOffset,
+                                maxCLOffsetInConsumeQueue);
+
+                        this.commitLog.removeQueueFromTopicQueueTable(nextQT.getValue().getTopic(),
+                                nextQT.getValue().getQueueId());
+
+                        this.consumeQueueStore.destroy(nextQT.getValue());
+                        itQT.remove();
+                    }
+                }
+
+                if (queueTable.isEmpty()) {
+                    log.info("cleanExpiredConsumerQueue: {},topic destroyed", topic);
+                    it.remove();
+                }
+            }
+        }
+    }
+
+    public double getDiskSpaceWarningLevelRatio() {
+        return cleanCommitLogService.getDiskSpaceWarningLevelRatio();
+    }
+
+    @Override
+    public boolean checkInDiskByConsumeOffset(final String topic, final int queueId, long consumeOffset) {
+
+        final long maxOffsetPy = this.commitLog.getMaxOffset();
+
+        ConsumeQueueInterface consumeQueue = getConsumeQueue(topic, queueId);
+        if (consumeQueue != null) {
+            CqUnit cqUnit = consumeQueue.get(consumeOffset);
+
+            if (cqUnit != null) {
+                long offsetPy = cqUnit.getPos();
+                return checkInDiskByCommitOffset(offsetPy, maxOffsetPy);
+            } else {
+                return false;
+            }
+        }
+        return false;
+    }
+
+    @Override
+    public long dispatchBehindBytes() {
+        return this.reputMessageService.behind();
+    }
+
+    @Override
+    public long flush() {
+        return this.commitLog.flush();
+    }
+
+    @Override
+    public boolean resetWriteOffset(long phyOffset) {
+        return this.commitLog.resetOffset(phyOffset);
+    }
+
+    @Override
+    public long getConfirmOffset() {
+        return this.commitLog.getConfirmOffset();
+    }
+
+    @Override
+    public void setConfirmOffset(long phyOffset) {
+        this.commitLog.setConfirmOffset(phyOffset);
+    }
+
+    @Override
+    public MessageExt lookMessageByOffset(long commitLogOffset, int size) {
+        SelectMappedBufferResult sbr = this.commitLog.getMessage(commitLogOffset, size);
+        if (null != sbr) {
+            try {
+                return MessageDecoder.decode(sbr.getByteBuffer(), true, false);
+            } finally {
+                sbr.release();
+            }
+        }
+
+        return null;
+    }
+
+    private long nextOffsetCorrection(long oldOffset, long newOffset) {
+        long nextOffset = oldOffset;
+        if (this.getMessageStoreConfig().getBrokerRole() != BrokerRole.SLAVE || this.getMessageStoreConfig().isOffsetCheckInSlave()) {
+            nextOffset = newOffset;
+        }
+        return nextOffset;
+    }
+
+    private boolean checkInDiskByCommitOffset(long offsetPy, long maxOffsetPy) {
+        long memory = (long) (StoreUtil.TOTAL_PHYSICAL_MEMORY_SIZE * (this.messageStoreConfig.getAccessMessageInMemoryMaxRatio() / 100.0));
+        return (maxOffsetPy - offsetPy) > memory;
+    }
+
+    private boolean isTheBatchFull(int sizePy, int unitBatchNum, int maxMsgNums, long maxMsgSize, int bufferTotal,
+                                   int messageTotal, boolean isInDisk) {
+
+        //At least has one message(batch)
+        if (0 == bufferTotal || 0 == messageTotal) {
+            return false;
+        }
+
+        if (messageTotal + unitBatchNum > maxMsgNums) {
+            return true;
+        }
+
+        if (bufferTotal + sizePy > maxMsgSize) {
+            return true;
+        }
+
+        if (isInDisk) {
+            if ((bufferTotal + sizePy) > this.messageStoreConfig.getMaxTransferBytesOnMessageInDisk()) {
+                return true;
+            }
+            if (messageTotal + unitBatchNum > this.messageStoreConfig.getMaxTransferCountOnMessageInDisk()) {
+                return true;
+            }
+        } else {
+            if ((bufferTotal + sizePy) > this.messageStoreConfig.getMaxTransferBytesOnMessageInMemory()) {
+                return true;
+            }
+
+            if (messageTotal + unitBatchNum > this.messageStoreConfig.getMaxTransferCountOnMessageInMemory()) {
+                return true;
+            }
+        }
+
+        return false;
+    }
+
+    private void deleteFile(final String fileName) {
+        File file = new File(fileName);
+        boolean result = file.delete();
+        log.info(fileName + (result ? " delete OK" : " delete Failed"));
+    }
+
+    /**
+     * @throws IOException
+     */
+    private void createTempFile() throws IOException {
+        String fileName = StorePathConfigHelper.getAbortFile(this.messageStoreConfig.getStorePathRootDir());
+        File file = new File(fileName);
+        DefaultMappedFile.ensureDirOK(file.getParent());
+        boolean result = file.createNewFile();
+        log.info(fileName + (result ? " create OK" : " already exists"));
+    }
+
+    private void addScheduleTask() {
+
+        this.scheduledExecutorService.scheduleAtFixedRate(new Runnable() {
+            @Override
+            public void run() {
+                StreamMessageStore.this.cleanFilesPeriodically();
+            }
+        }, 1000 * 60, this.messageStoreConfig.getCleanResourceInterval(), TimeUnit.MILLISECONDS);
+
+        this.scheduledExecutorService.scheduleAtFixedRate(new Runnable() {
+            @Override
+            public void run() {
+                StreamMessageStore.this.checkSelf();
+            }
+        }, 1, 10, TimeUnit.MINUTES);
+
+        this.scheduledExecutorService.scheduleAtFixedRate(new Runnable() {
+            @Override
+            public void run() {
+                try {
+                    if (!getMessageStoreConfig().isMappedFileSwapEnable()) {
+                        log.warn("Swap is not enabled.");
+                        return ;
+                    }
+                    StreamMessageStore.this.commitLog.swapMap(getMessageStoreConfig().getCommitLogSwapMapReserveFileNum(),
+                            getMessageStoreConfig().getCommitLogForceSwapMapInterval(), getMessageStoreConfig().getCommitLogSwapMapInterval());
+                    for (ConcurrentMap<Integer, ConsumeQueueInterface> maps : StreamMessageStore.this.consumeQueueTable.values()) {
+                        for (ConsumeQueueInterface logic : maps.values()) {
+                            StreamMessageStore.this.consumeQueueStore.swapMap(logic, getMessageStoreConfig().getLogicQueueSwapMapReserveFileNum(),
+                                    getMessageStoreConfig().getLogicQueueForceSwapMapInterval(), getMessageStoreConfig().getLogicQueueSwapMapInterval());
+                        }
+                    }
+                } catch (Exception e) {
+                    log.error("swap map exception", e);
+                }
+            }
+        }, 1, 5, TimeUnit.MINUTES);
+
+        this.scheduledExecutorService.scheduleAtFixedRate(new Runnable() {
+            @Override
+            public void run() {
+                try {
+                    StreamMessageStore.this.commitLog.cleanSwappedMap(getMessageStoreConfig().getCleanSwapedMapInterval());
+                    for (ConcurrentMap<Integer, ConsumeQueueInterface> maps : StreamMessageStore.this.consumeQueueTable.values()) {
+                        for (ConsumeQueueInterface logic : maps.values()) {
+                            StreamMessageStore.this.consumeQueueStore.cleanSwappedMap(logic, getMessageStoreConfig().getCleanSwapedMapInterval());
+                        }
+                    }
+                } catch (Exception e) {
+                    log.error("clean swap map exception", e);
+                }
+            }
+        }, 1, 5, TimeUnit.MINUTES);
+
+        this.scheduledExecutorService.scheduleAtFixedRate(new Runnable() {
+            @Override
+            public void run() {
+                if (StreamMessageStore.this.getMessageStoreConfig().isDebugLockEnable()) {
+                    try {
+                        if (StreamMessageStore.this.commitLog.getBeginTimeInLock() != 0) {
+                            long lockTime = System.currentTimeMillis() - StreamMessageStore.this.commitLog.getBeginTimeInLock();
+                            if (lockTime > 1000 && lockTime < 10000000) {
+
+                                String stack = UtilAll.jstack();
+                                final String fileName = System.getProperty("user.home") + File.separator + "debug/lock/stack-"
+                                        + StreamMessageStore.this.commitLog.getBeginTimeInLock() + "-" + lockTime;
+                                MixAll.string2FileNotSafe(stack, fileName);
+                            }
+                        }
+                    } catch (Exception e) {
+                    }
+                }
+            }
+        }, 1, 1, TimeUnit.SECONDS);
+
+        // this.scheduledExecutorService.scheduleAtFixedRate(new Runnable() {
+        // @Override
+        // public void run() {
+        // StreamMessageStore.this.cleanExpiredConsumerQueue();
+        // }
+        // }, 1, 1, TimeUnit.HOURS);
+    }
+
+    private void cleanFilesPeriodically() {
+        this.cleanCommitLogService.run();
+        this.cleanConsumeQueueService.run();
+        this.correctLogicOffsetService.run();
+    }
+
+    private void checkSelf() {
+        this.commitLog.checkSelf();
+
+        Iterator<Entry<String, ConcurrentMap<Integer, ConsumeQueueInterface>>> it = this.consumeQueueTable.entrySet().iterator();
+        while (it.hasNext()) {
+            Entry<String, ConcurrentMap<Integer, ConsumeQueueInterface>> next = it.next();
+            Iterator<Entry<Integer, ConsumeQueueInterface>> itNext = next.getValue().entrySet().iterator();
+            while (itNext.hasNext()) {
+                Entry<Integer, ConsumeQueueInterface> cq = itNext.next();
+                this.consumeQueueStore.checkSelf(cq.getValue());
+            }
+        }
+    }
+
+    private boolean isTempFileExist() {
+        String fileName = StorePathConfigHelper.getAbortFile(this.messageStoreConfig.getStorePathRootDir());
+        File file = new File(fileName);
+        return file.exists();
+    }
+
+    protected boolean loadBatchConsumeQueue() {
+        checkOtherConsumeQueue();
+
+        File dirLogic = new File(StorePathConfigHelper.getStorePathBatchConsumeQueue(this.messageStoreConfig.getStorePathRootDir()));
+        File[] fileTopicList = dirLogic.listFiles();
+        if (fileTopicList != null) {
+
+            for (File fileTopic : fileTopicList) {
+                String topic = fileTopic.getName();
+
+                File[] fileQueueIdList = fileTopic.listFiles();
+                if (fileQueueIdList != null) {
+                    for (File fileQueueId : fileQueueIdList) {
+                        int queueId;
+                        try {
+                            queueId = Integer.parseInt(fileQueueId.getName());
+                        } catch (NumberFormatException e) {
+                            continue;
+                        }
+                        ConsumeQueueInterface logic = new BatchConsumeQueue(
+                                topic,
+                                queueId,
+                                StorePathConfigHelper.getStorePathBatchConsumeQueue(this.messageStoreConfig.getStorePathRootDir()),
+                                this.getMessageStoreConfig().getMapperFileSizeBatchConsumeQueue(),
+                                this);
+                        this.putConsumeQueue(topic, queueId, logic);
+                        if (!this.consumeQueueStore.load(logic)) {
+                            return false;
+                        }
+                    }
+                }
+            }
+        }
+
+        log.info("load logics queue all over, OK");
+
+        return true;
+    }
+
+    private void checkOtherConsumeQueue() {
+        File dirLogic = new File(StorePathConfigHelper.getStorePathConsumeQueue(this.messageStoreConfig.getStorePathRootDir()));
+        if (dirLogic.exists()) {
+            throw new RuntimeException(format("Consume queue directory: [%s] exist. Can not load batch consume queue while consume queue exists.",
+                    StorePathConfigHelper.getStorePathConsumeQueue(this.messageStoreConfig.getStorePathRootDir())));
+        }
+    }
+
+    private void recover(final boolean lastExitOK) {
+        long recoverCqStart = System.currentTimeMillis();
+        long maxPhyOffsetOfConsumeQueue = this.recoverConsumeQueue();
+        long recoverCqEnd = System.currentTimeMillis();
+
+        if (lastExitOK) {
+            this.commitLog.recoverNormally(maxPhyOffsetOfConsumeQueue);
+        } else {
+            this.commitLog.recoverAbnormally(maxPhyOffsetOfConsumeQueue);
+        }
+        long recoverClogEnd = System.currentTimeMillis();
+        this.recoverTopicQueueTable();
+        long recoverOffsetEnd = System.currentTimeMillis();
+
+        log.info("Recover end total:{} recoverCq:{} recoverClog:{} recoverOffset:{}",
+                recoverOffsetEnd - recoverCqStart, recoverCqEnd - recoverCqStart, recoverClogEnd - recoverCqEnd, recoverOffsetEnd - recoverClogEnd);
+    }
+
+    @Override
+    public MessageStoreConfig getMessageStoreConfig() {
+        return messageStoreConfig;
+    }
+
+    @Override
+    public TransientStorePool getTransientStorePool() {
+        return transientStorePool;
+    }
+
+    private void putConsumeQueue(final String topic, final int queueId, final ConsumeQueueInterface consumeQueue) {
+        ConcurrentMap<Integer/* queueId */, ConsumeQueueInterface> map = this.consumeQueueTable.get(topic);
+        if (null == map) {
+            map = new ConcurrentHashMap<Integer/* queueId */, ConsumeQueueInterface>();
+            map.put(queueId, consumeQueue);
+            this.consumeQueueTable.put(topic, map);
+        } else {
+            map.put(queueId, consumeQueue);
+        }
+    }
+
+    private long recoverConsumeQueue() {
+        long maxPhysicOffset = -1;
+        for (ConcurrentMap<Integer, ConsumeQueueInterface> maps : this.consumeQueueTable.values()) {
+            for (ConsumeQueueInterface logic : maps.values()) {
+                this.consumeQueueStore.recover(logic);
+                if (logic.getMaxPhysicOffset() > maxPhysicOffset) {
+                    maxPhysicOffset = logic.getMaxPhysicOffset();
+                }
+            }
+        }
+
+        return maxPhysicOffset;
+    }
+
+    public void recoverTopicQueueTable() {
+        HashMap<String/* topic-queueid */, Long/* offset */> table = new HashMap<String, Long>(1024);
+        long minPhyOffset = this.commitLog.getMinOffset();
+        for (ConcurrentMap<Integer, ConsumeQueueInterface> maps : this.consumeQueueTable.values()) {
+            for (ConsumeQueueInterface logic : maps.values()) {
+                String key = logic.getTopic() + "-" + logic.getQueueId();
+                table.put(key, logic.getMaxOffsetInQueue());
+                this.consumeQueueStore.correctMinOffset(logic, minPhyOffset);
+            }
+        }
+
+        this.batchTopicQueueTable = table;
+    }
+
+    @Override
+    public AllocateMappedFileService getAllocateMappedFileService() {
+        return allocateMappedFileService;
+    }
+
+    @Override
+    public void truncateDirtyLogicFiles(long phyOffset) {
+        ConcurrentMap<String, ConcurrentMap<Integer, ConsumeQueueInterface>> tables = StreamMessageStore.this.consumeQueueTable;
+
+        for (ConcurrentMap<Integer, ConsumeQueueInterface> maps : tables.values()) {
+            for (ConsumeQueueInterface logic : maps.values()) {
+                this.consumeQueueStore.truncateDirtyLogicFiles(logic, phyOffset);
+            }
+        }
+    }
+
+    @Override
+    public StoreStatsService getStoreStatsService() {
+        return storeStatsService;
+    }
+
+    public ConcurrentMap<String, ConcurrentMap<Integer, ConsumeQueueInterface>> getConsumeQueueTable() {
+        return consumeQueueTable;
+    }
+
+    @Override
+    public StoreCheckpoint getStoreCheckpoint() {
+        return storeCheckpoint;
+    }
+
+    @Override
+    public HAService getHaService() {
+        return haService;
+    }
+
+    @Override
+    public void registerCleanFileHook(CleanFilesHook logicalQueueCleanHook) {
+
+    }
+
+    @Override
+    public ScheduleMessageService getScheduleMessageService() {
+        return scheduleMessageService;
+    }
+
+    @Override
+    public RunningFlags getRunningFlags() {
+        return runningFlags;
+    }
+
+    public void initAsyncReputThreads(int tsNum, int cacheNum) {
+        if (tsNum <= 0) {
+            tsNum = 1;
+        }
+        if (cacheNum < 512) {
+            cacheNum = 512;
+        }
+        reputExecutors = new ThreadPoolExecutor[tsNum];
+        reputQueues = new BlockingQueue[tsNum];
+
+        for (int i = 0; i < tsNum; i++) {
+            final int tmp = i;
+            reputQueues[i] = new LinkedBlockingDeque<>(cacheNum);
+            //Each executor can only have one thread, otherwise the cq index will get wrong
+            reputExecutors[i] = new ThreadPoolExecutor(1, 1, 0L, TimeUnit.MILLISECONDS,
+                    reputQueues[i],
+                    new ThreadFactory() {
+                        @Override
+                        public Thread newThread(Runnable r) {
+                            return new Thread(r, "MQDispatchThread-" + tmp);
+                        }
+                    });
+        }
+        for (ThreadPoolExecutor executorService: reputExecutors) {
+            if (executorService.getMaximumPoolSize() != 1 ||
+                    executorService.getCorePoolSize() != 1) {
+                throw new RuntimeException("The MQDispatchThreadPoll can only have one thread");
+            }
+        }
+
+    }
+
+    public Future doDispatch(final DispatchRequest request) {
+        return doDispatch(request, false);
+    }
+
+    public Future doDispatch(final DispatchRequest request, boolean async) {
+        Runnable task = new Runnable() {
+            @Override
+            public void run() {
+                for (CommitLogDispatcher dispatcher : StreamMessageStore.this.dispatcherList) {
+                    dispatcher.dispatch(request);
+                }
+            }
+        };
+        if (!async) {
+            task.run();
+            return EMPTY_FUTURE;
+        }
+        int hash = Math.abs((request.getTopic() + request.getQueueId()).hashCode());
+        int slot = hash % reputExecutors.length;
+        try {
+            return reputExecutors[slot].submit(task);
+        } catch (RejectedExecutionException ignored) {
+            int tryNum = 0;
+            while (tryNum++ < Integer.MAX_VALUE) {
+                try {
+                    Thread.sleep(1);
+                } catch (Throwable ignored2) {
+
+                }
+                try {
+                    return reputExecutors[slot].submit(task);
+                } catch (RejectedExecutionException e) {
+                    log.warn("DispatchReject topic:{} queue:{} pyOffset:{} tryNum:{}", request.getTopic(), request.getQueueId(), request.getCommitLogOffset(), tryNum,  e);
+                }
+            }
+        }
+        return EMPTY_FUTURE;
+    }
+
+    public void syncProcessDispatchRequest(DispatchRequest request, boolean isRecover) throws InterruptedException {
+        if (!isDispatchFromSenderThread) {
+            log.error("addDispatchRequestQueue operation not supported while isCreateDispatchRequestAsync is true");
+        } else {
+            if (isRecover) {
+                ((SyncReputMessageService) this.reputMessageService).processDispatchRequestForRecover(request);
+            } else {
+                ((SyncReputMessageService) this.reputMessageService).processDispatchRequest(request);
+            }
+        }
+    }
+
+    public boolean dispatched(long physicalOffset) {
+        return reputMessageService.dispatched(physicalOffset);
+    }
+
+    public void putMessagePositionInfo(DispatchRequest dispatchRequest) {
+        ConsumeQueueInterface cq = this.findOrCreateConsumeQueue(dispatchRequest.getTopic(), dispatchRequest.getQueueId());
+        this.consumeQueueStore.putMessagePositionInfoWrapper(cq, dispatchRequest);
+    }
+
+    private ConsumeQueueInterface findOrCreateConsumeQueue(String topic, int queueId) {
+        return this.consumeQueueStore.findOrCreateConsumeQueue(topic, queueId);
+    }
+
+    @Override
+    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 int remainTransientStoreBufferNumbs() {
+        return this.transientStorePool.availableBufferNums();
+    }
+
+    @Override
+    public boolean isTransientStorePoolDeficient() {
+        return remainTransientStoreBufferNumbs() == 0;
+    }
+
+    @Override
+    public LinkedList<CommitLogDispatcher> getDispatcherList() {
+        return this.dispatcherList;
+    }
+
+    @Override
+    public ConsumeQueueInterface getConsumeQueue(String topic, int queueId) {
+        ConcurrentMap<Integer, ConsumeQueueInterface> map = consumeQueueTable.get(topic);
+        if (map == null) {
+            return null;
+        }
+        return map.get(queueId);
+    }
+
+    @Override
+    public void unlockMappedFile(final MappedFile mappedFile) {
+        this.scheduledExecutorService.schedule(new Runnable() {
+            @Override
+            public void run() {
+                mappedFile.munlock();
+            }
+        }, 6, TimeUnit.SECONDS);
+    }
+
+    @Override
+    public PerfCounter.Ticks getPerfCounter() {
+        return perfs;
+    }
+
+    @Override
+    public ConsumeQueueStore getQueueStore() {
+        return consumeQueueStore;
+    }
+
+    @Override
+    public boolean isSyncDiskFlush() {
+        return FlushDiskType.SYNC_FLUSH == this.getMessageStoreConfig().getFlushDiskType();
+    }
+
+    @Override
+    public boolean isSyncMaster() {
+        return BrokerRole.SYNC_MASTER == this.getMessageStoreConfig().getBrokerRole();
+    }
+
+    @Override
+    public void assignOffset(String topicQueueKey, MessageExtBrokerInner msg, short batchNum) {
+        final int tranType = MessageSysFlag.getTransactionValue(msg.getSysFlag());
+
+        if (tranType == MessageSysFlag.TRANSACTION_NOT_TYPE || tranType == MessageSysFlag.TRANSACTION_COMMIT_TYPE) {
+            assignOffsetForCq(topicQueueKey, msg);
+            assignOffsetForBcq(topicQueueKey, msg, batchNum);
+        }
+    }
+
+    private void assignOffsetForCq(String topicQueueKey, MessageExtBrokerInner msg) {
+        // not supported yet
+    }
+
+    private void assignOffsetForBcq(String topicQueueKey, MessageExtBrokerInner msg, short batchNum) {
+        Long batchTopicOffset = this.batchTopicQueueTable.computeIfAbsent(topicQueueKey, k -> 0L);
+        CQType cqType = QueueTypeUtils.getCQType(this);
+        if (MessageSysFlag.check(msg.getSysFlag(), MessageSysFlag.INNER_BATCH_FLAG) || CQType.BatchCQ.equals(cqType)) {
+            MessageAccessor.putProperty(msg, MessageConst.PROPERTY_INNER_BASE, String.valueOf(batchTopicOffset));
+            msg.setPropertiesString(MessageDecoder.messageProperties2String(msg.getProperties()));
+        }
+        msg.setQueueOffset(batchTopicOffset);
+        this.batchTopicQueueTable.put(topicQueueKey, batchTopicOffset + batchNum);
+    }
+
+    @Override
+    public void removeOffsetTable(String topicQueueKey) {
+        this.topicQueueTable.remove(topicQueueKey);
+        this.batchTopicQueueTable.remove(topicQueueKey);
+    }
+
+    @Override
+    public void onCommitLogAppend(MessageExtBrokerInner msg, AppendMessageResult result, MappedFile commitLogFile) {
+        DispatchRequest dispatchRequest;
+        switch (result.getStatus()) {
+            case PUT_OK:
+                dispatchRequest = constructDispatchRequest(msg, result);
+                onCommitLogDispatch(dispatchRequest, this.isDispatchFromSenderThread(), commitLogFile, false, false);
+                break;
+            case END_OF_FILE:
+                dispatchRequest = new DispatchRequest(0, true);
+                onCommitLogDispatch(dispatchRequest, this.isDispatchFromSenderThread(), commitLogFile, false, true);
+                break;
+            default:
+                throw new RuntimeException("");
+        }
+    }
+
+    @Override
+    public void onCommitLogDispatch(DispatchRequest dispatchRequest, boolean doDispatch, MappedFile commitLogFile, boolean isRecover, boolean isFileEnd) {
+        if (isFileEnd) {
+            if (doDispatch) {
+                long nextReputFromOffset = this.getCommitLog().rollNextFile(commitLogFile.getFileFromOffset());
+                dispatchRequest.setNextReputFromOffset(nextReputFromOffset);
+                syncDispatch(dispatchRequest, isRecover);
+            }
+        } else {
+            if (doDispatch) {
+                dispatchRequest.setNextReputFromOffset(dispatchRequest.getCommitLogOffset() + dispatchRequest.getMsgSize());
+                syncDispatch(dispatchRequest, isRecover);
+            }
+        }
+    }
+
+    private DispatchRequest constructDispatchRequest(MessageExtBrokerInner msg, AppendMessageResult appendResult) {
+        long tagsCode = 0;
+        String keys = "";
+        String uniqKey = null;
+        int sysFlag = msg.getSysFlag();
+        String topic = msg.getTopic();
+        long storeTimestamp = msg.getStoreTimestamp();
+        int queueId = msg.getQueueId();
+        long preparedTransactionOffset = msg.getPreparedTransactionOffset();
+        Map<String, String> propertiesMap = msg.getProperties();
+
+        if (msg.getProperties() != null && msg.getProperties().size() > 0) {
+
+            keys = propertiesMap.get(MessageConst.PROPERTY_KEYS);
+
+            uniqKey = propertiesMap.get(MessageConst.PROPERTY_UNIQ_CLIENT_MESSAGE_ID_KEYIDX);
+
+            String tags = propertiesMap.get(MessageConst.PROPERTY_TAGS);
+            if (tags != null && tags.length() > 0) {
+                tagsCode = MessageExtBrokerInner.tagsString2tagsCode(MessageExt.parseTopicFilterType(sysFlag), tags);
+            }
+
+            // Timing message processing
+            {
+                String t = propertiesMap.get(MessageConst.PROPERTY_DELAY_TIME_LEVEL);
+                if (TopicValidator.RMQ_SYS_SCHEDULE_TOPIC.equals(topic) && t != null) {
+                    int delayLevel = Integer.parseInt(t);
+
+                    if (delayLevel > this.getScheduleMessageService().getMaxDelayLevel()) {
+                        delayLevel = this.getScheduleMessageService().getMaxDelayLevel();
+                    }
+
+                    if (delayLevel > 0) {
+                        tagsCode = this.getScheduleMessageService().computeDeliverTimestamp(delayLevel,
+                                storeTimestamp);
+                    }
+                }
+            }
+        }
+
+        DispatchRequest dispatchRequest = new DispatchRequest(
+                topic,
+                queueId,
+                appendResult.getWroteOffset(),
+                appendResult.getWroteBytes(),
+                tagsCode,
+                storeTimestamp,
+                appendResult.getLogicsOffset(),
+                keys,
+                uniqKey,
+                sysFlag,
+                preparedTransactionOffset,
+                propertiesMap
+        );
+
+        if (null != propertiesMap && propertiesMap.containsKey(MessageConst.PROPERTY_INNER_NUM) && propertiesMap.containsKey(MessageConst.PROPERTY_INNER_BASE)) {
+            dispatchRequest.setMsgBaseOffset(Long.parseLong(propertiesMap.get(MessageConst.PROPERTY_INNER_BASE)));
+            dispatchRequest.setBatchSize(Short.parseShort(propertiesMap.get(MessageConst.PROPERTY_INNER_NUM)));
+        }
+        return dispatchRequest;
+    }
+
+    private void syncDispatch(DispatchRequest dispatchRequest, boolean isRecover) {
+        try {
+            this.syncProcessDispatchRequest(dispatchRequest, isRecover);
+        } catch (InterruptedException e) {
+            log.error("OnCommitlogAppend sync dispatch failed, addDispatchRequestQueue interrupted. DispatchRequest:{}", dispatchRequest);
+        }
+    }
+
+    class CommitLogDispatcherBuildConsumeQueue implements CommitLogDispatcher {
+
+        @Override
+        public void dispatch(DispatchRequest dispatchRequest) {
+            final int tranType = MessageSysFlag.getTransactionValue(dispatchRequest.getSysFlag());
+            switch (tranType) {
+                case MessageSysFlag.TRANSACTION_NOT_TYPE:
+                case MessageSysFlag.TRANSACTION_COMMIT_TYPE:
+                    StreamMessageStore.this.putMessagePositionInfo(dispatchRequest);
+                    if (BrokerRole.SLAVE != StreamMessageStore.this.getMessageStoreConfig().getBrokerRole()
+                            && StreamMessageStore.this.brokerConfig.isLongPollingEnable()) {
+                        StreamMessageStore.this.messageArrivingListener.arriving(dispatchRequest.getTopic(),
+                                dispatchRequest.getQueueId(), dispatchRequest.getConsumeQueueOffset() + 1,
+                                dispatchRequest.getTagsCode(), dispatchRequest.getStoreTimestamp(),
+                                dispatchRequest.getBitMap(), dispatchRequest.getPropertiesMap());
+                    }
+                    if (StreamMessageStore.this.getMessageStoreConfig().getBrokerRole() == BrokerRole.SLAVE) {
+                        StreamMessageStore.this.storeStatsService
+                                .getSinglePutMessageTopicTimesTotal(dispatchRequest.getTopic()).add(1);
+                        StreamMessageStore.this.storeStatsService
+                                .getSinglePutMessageTopicSizeTotal(dispatchRequest.getTopic())
+                                .add(dispatchRequest.getMsgSize());
+                    }
+                    break;
+                case MessageSysFlag.TRANSACTION_PREPARED_TYPE:
+                case MessageSysFlag.TRANSACTION_ROLLBACK_TYPE:
+                    break;
+            }
+        }
+    }
+
+    class CommitLogDispatcherBuildIndex implements CommitLogDispatcher {
+
+        @Override
+        public void dispatch(DispatchRequest request) {
+            if (StreamMessageStore.this.messageStoreConfig.isMessageIndexEnable()) {
+                StreamMessageStore.this.indexService.buildIndex(request);
+            }
+        }
+    }
+
+    class CleanCommitLogService {
+
+        private final static int MAX_MANUAL_DELETE_FILE_TIMES = 20;
+        private final String diskSpaceWarningLevelRatio =
+                System.getProperty("rocketmq.broker.diskSpaceWarningLevelRatio", "");
+
+        private final String diskSpaceCleanForciblyRatio =
+                System.getProperty("rocketmq.broker.diskSpaceCleanForciblyRatio", "");
+        private long lastRedeleteTimestamp = 0;
+
+        private volatile int manualDeleteFileSeveralTimes = 0;
+
+        private volatile boolean cleanImmediately = false;
+
+        double getDiskSpaceWarningLevelRatio() {
+            double finalDiskSpaceWarningLevelRatio;
+            if ("".equals(diskSpaceWarningLevelRatio)) {
+                finalDiskSpaceWarningLevelRatio = StreamMessageStore.this.getMessageStoreConfig().getDiskSpaceWarningLevelRatio() / 100.0;
+            } else {
+                finalDiskSpaceWarningLevelRatio = Double.parseDouble(diskSpaceWarningLevelRatio);
+            }
+
+            if (finalDiskSpaceWarningLevelRatio > 0.90) {
+                finalDiskSpaceWarningLevelRatio = 0.90;
+            }
+            if (finalDiskSpaceWarningLevelRatio < 0.35) {
+                finalDiskSpaceWarningLevelRatio = 0.35;
+            }
+
+            return finalDiskSpaceWarningLevelRatio;
+        }
+
+        double getDiskSpaceCleanForciblyRatio() {
+            double finalDiskSpaceCleanForciblyRatio;
+            if ("".equals(diskSpaceCleanForciblyRatio)) {
+                finalDiskSpaceCleanForciblyRatio = StreamMessageStore.this.getMessageStoreConfig().getDiskSpaceCleanForciblyRatio() / 100.0;
+            } else {
+                finalDiskSpaceCleanForciblyRatio = Double.parseDouble(diskSpaceCleanForciblyRatio);
+            }
+
+            if (finalDiskSpaceCleanForciblyRatio > 0.85) {
+                finalDiskSpaceCleanForciblyRatio = 0.85;
+            }
+            if (finalDiskSpaceCleanForciblyRatio < 0.30) {
+                finalDiskSpaceCleanForciblyRatio = 0.30;
+            }
+
+            return finalDiskSpaceCleanForciblyRatio;
+        }
+
+        public void excuteDeleteFilesManualy() {
+            this.manualDeleteFileSeveralTimes = MAX_MANUAL_DELETE_FILE_TIMES;
+            StreamMessageStore.log.info("executeDeleteFilesManually was invoked");
+        }
+
+        public long run() {
+            int deleteCount = 0;
+            try {
+                deleteCount = this.deleteExpiredFiles();
+
+                this.redeleteHangedFile();
+            } catch (Throwable e) {
+                StreamMessageStore.log.warn(this.getServiceName() + " service has exception. ", e);
+            }
+            return deleteCount;
+        }
+
+        private int deleteExpiredFiles() {
+            int deleteCount = 0;
+            long fileReservedTime = StreamMessageStore.this.getMessageStoreConfig().getFileReservedTime();
+            int deletePhysicFilesInterval = StreamMessageStore.this.getMessageStoreConfig().getDeleteCommitLogFilesInterval();
+            int destroyMapedFileIntervalForcibly = StreamMessageStore.this.getMessageStoreConfig().getDestroyMapedFileIntervalForcibly();
+            int maxBatchDeleteFilesNum = StreamMessageStore.this.getMessageStoreConfig().getMaxBatchDeleteFilesNum();
+            if (maxBatchDeleteFilesNum < 10) {
+                maxBatchDeleteFilesNum = 10;
+            }
+
+            boolean timeup = this.isTimeToDelete();
+            boolean spacefull = this.isSpaceToDelete();
+            boolean manualDelete = this.manualDeleteFileSeveralTimes > 0;
+
+            boolean needDelete = timeup || spacefull || manualDelete;
+
+            if (needDelete) {
+
+                if (manualDelete)
+                    this.manualDeleteFileSeveralTimes--;
+
+                boolean cleanAtOnce = StreamMessageStore.this.getMessageStoreConfig().isCleanFileForciblyEnable() && this.cleanImmediately;
+
+                String storePathPhysic = StreamMessageStore.this.getMessageStoreConfig().getStorePathCommitLog();
+                double physicRatio = UtilAll.getDiskPartitionSpaceUsedPercent(storePathPhysic);
+                long totalSpace = UtilAll.getTotalSpace(storePathPhysic);
+                double realRatio = (StreamMessageStore.this.commitLog.getMaxOffset() - StreamMessageStore.this.commitLog.getMinOffset()) / (totalSpace + 0.001);
+
+                cleanAtOnce = cleanAtOnce && (realRatio > 0.3);
+
+                log.info("begin to delete before {} hours file. timeup:{} spacefull:{} manualDeleteFileSeveralTimes:{} cleanAtOnce:{} maxBatchDeleteFilesNum:{} physicRatio:{}",
+                        fileReservedTime,
+                        timeup,
+                        spacefull,
+                        manualDeleteFileSeveralTimes,
+                        cleanAtOnce,
+                        maxBatchDeleteFilesNum,
+                        physicRatio);
+
+                fileReservedTime *= 60 * 60 * 1000;
+
+                deleteCount = StreamMessageStore.this.commitLog.deleteExpiredFile(fileReservedTime, deletePhysicFilesInterval,
+                        destroyMapedFileIntervalForcibly, cleanAtOnce);
+                if (deleteCount > 0) {
+                } else if (spacefull) {
+                    log.warn("disk space will be full soon, but delete file failed. timeup:{} manualDelete:{} cleanAtOnce:{} physicRatio:{}",
+                            timeup,
+                            manualDelete,
+                            cleanAtOnce,
+                            physicRatio);
+                }
+            }
+            return deleteCount;
+        }
+
+        private void redeleteHangedFile() {
+            int interval = StreamMessageStore.this.getMessageStoreConfig().getRedeleteHangedFileInterval();
+            long currentTimestamp = System.currentTimeMillis();
+            if ((currentTimestamp - this.lastRedeleteTimestamp) > interval) {
+                this.lastRedeleteTimestamp = currentTimestamp;
+                int destroyMapedFileIntervalForcibly =
+                        StreamMessageStore.this.getMessageStoreConfig().getDestroyMapedFileIntervalForcibly();
+                if (StreamMessageStore.this.commitLog.retryDeleteFirstFile(destroyMapedFileIntervalForcibly)) {
+                }
+            }
+        }
+
+        public String getServiceName() {
+            return CleanCommitLogService.class.getSimpleName();
+        }
+
+        private boolean isTimeToDelete() {
+            String when = StreamMessageStore.this.getMessageStoreConfig().getDeleteWhen();
+            if (UtilAll.isItTimeToDo(when)) {
+                StreamMessageStore.log.info("it's time to reclaim disk space, " + when);
+                return true;
+            }
+
+            return false;
+        }
+
+        private boolean isSpaceToDelete() {
+            double ratio = StreamMessageStore.this.getMessageStoreConfig().getDiskMaxUsedSpaceRatio() / 100.0;
+
+            cleanImmediately = false;
+
+            {
+                double physicRatio = UtilAll.getDiskPartitionSpaceUsedPercent(getStorePathPhysic());
+                if (physicRatio > getDiskSpaceWarningLevelRatio()) {
+                    boolean diskok = StreamMessageStore.this.runningFlags.getAndMakeDiskFull();
+                    if (diskok) {
+                        StreamMessageStore.log.error("physic disk maybe full soon " + physicRatio + ", so mark disk full");
+                    }
+
+                    cleanImmediately = true;
+                } else if (physicRatio > getDiskSpaceCleanForciblyRatio()) {
+                    cleanImmediately = true;
+                } else {
+                    boolean diskok = StreamMessageStore.this.runningFlags.getAndMakeDiskOK();
+                    if (!diskok) {
+                        StreamMessageStore.log.info("physic disk space OK " + physicRatio + ", so mark disk ok");
+                    }
+                }
+
+                if (physicRatio < 0 || physicRatio > ratio) {
+                    StreamMessageStore.log.info("physic disk maybe full soon, so reclaim space, {}, cleanImmediately {}", physicRatio, cleanImmediately);
+                    return true;
+                }
+            }
+
+            {
+                String storePathLogics = StorePathConfigHelper
+                        .getStorePathConsumeQueue(StreamMessageStore.this.getMessageStoreConfig().getStorePathRootDir());
+                double logicsRatio = UtilAll.getDiskPartitionSpaceUsedPercent(storePathLogics);
+                if (logicsRatio > getDiskSpaceWarningLevelRatio()) {
+                    boolean diskok = StreamMessageStore.this.runningFlags.getAndMakeDiskFull();
+                    if (diskok) {
+                        StreamMessageStore.log.error("logics disk maybe full soon " + logicsRatio + ", so mark disk full");
+                    }
+
+                    cleanImmediately = true;
+                } else if (logicsRatio > getDiskSpaceCleanForciblyRatio()) {
+                    cleanImmediately = true;
+                } else {
+                    boolean diskok = StreamMessageStore.this.runningFlags.getAndMakeDiskOK();
+                    if (!diskok) {
+                        StreamMessageStore.log.info("logics disk space OK " + logicsRatio + ", so mark disk ok");
+                    }
+                }
+
+                if (logicsRatio < 0 || logicsRatio > ratio) {
+                    StreamMessageStore.log.info("logics disk maybe full soon, so reclaim space, " + logicsRatio);
+                    return true;
+                }
+            }
+
+            return false;
+        }
+
+        public int getManualDeleteFileSeveralTimes() {
+            return manualDeleteFileSeveralTimes;
+        }
+
+        public void setManualDeleteFileSeveralTimes(int manualDeleteFileSeveralTimes) {
+            this.manualDeleteFileSeveralTimes = manualDeleteFileSeveralTimes;
+        }
+    }
+
+    class CleanConsumeQueueService {
+        private long lastPhysicalMinOffset = 0;
+
+        public void run() {
+            try {
+                this.deleteExpiredFiles();
+            } catch (Throwable e) {
+                StreamMessageStore.log.warn(this.getServiceName() + " service has exception. ", e);
+            }
+        }
+
+        private void deleteExpiredFiles() {
+            int deleteLogicsFilesInterval = StreamMessageStore.this.getMessageStoreConfig().getDeleteConsumeQueueFilesInterval();
+
+            long minOffset = StreamMessageStore.this.commitLog.getMinOffset();
+            if (minOffset > this.lastPhysicalMinOffset) {
+                this.lastPhysicalMinOffset = minOffset;
+
+                ConcurrentMap<String, ConcurrentMap<Integer, ConsumeQueueInterface>> tables = StreamMessageStore.this.consumeQueueTable;
+
+                for (ConcurrentMap<Integer, ConsumeQueueInterface> maps : tables.values()) {
+                    for (ConsumeQueueInterface logic : maps.values()) {
+                        int deleteCount = StreamMessageStore.this.consumeQueueStore.deleteExpiredFile(logic, minOffset);
+
+                        if (deleteCount > 0 && deleteLogicsFilesInterval > 0) {
+                            try {
+                                Thread.sleep(deleteLogicsFilesInterval);
+                            } catch (InterruptedException ignored) {
+                            }
+                        }
+                    }
+                }
+
+                StreamMessageStore.this.indexService.deleteExpiredFile(minOffset);
+            }
+        }
+
+        public String getServiceName() {
+            return CleanConsumeQueueService.class.getSimpleName();
+        }
+    }
+
+    class CorrectLogicOffsetService {
+
+        private long lastForceCorrectTime = -1L;
+
+        public void run() {
+            try {
+                this.correctLogicMinOffset();
+            } catch (Throwable e) {
+                StreamMessageStore.log.warn(this.getServiceName() + " service has exception. ", e);
+            }
+        }
+
+        private boolean needCorrect(ConsumeQueueInterface logic, long minPhyOffset, long lastForeCorrectTimeCurRun) {
+            if (logic == null) {
+                return false;
+            }
+            // If first exist and not available, it means first file may destroy failed, delete it.
+            if (StreamMessageStore.this.consumeQueueStore.isFirstFileExist(logic) && !StreamMessageStore.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());
+                return true;
+            }
+
+            // logic.getMaxPhysicOffset() or minPhyOffset = -1
+            // means there is no message in current queue, so no need to correct.
+            if (logic.getMaxPhysicOffset() == -1 || minPhyOffset == -1) {
+                return false;
+            }
+
+            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());
+                    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());
+                    return false;
+                }
+            }
+            //the logic.getMaxPhysicOffset() >= minPhyOffset
+            int forceCorrectInterval = StreamMessageStore.this.getMessageStoreConfig().getCorrectLogicMinOffsetForceInterval();
+            if ((System.currentTimeMillis() - lastForeCorrectTimeCurRun) > forceCorrectInterval) {
+                lastForceCorrectTime = System.currentTimeMillis();
+                CqUnit cqUnit = logic.getEarliestUnit();
+                if (cqUnit == null) {
+                    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());
+                        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());
+                    return true;
+                }
+
+                if (cqUnit.getPos() >= minPhyOffset) {
+
+                    // Normal case, do not need correct.
+                    return false;
+                }
+            }
+
+            return false;
+        }
+
+        private void correctLogicMinOffset() {
+
+            long lastForeCorrectTimeCurRun = lastForceCorrectTime;
+            long minPhyOffset = getMinPhyOffset();
+            ConcurrentMap<String, ConcurrentMap<Integer, ConsumeQueueInterface>> tables = StreamMessageStore.this.consumeQueueTable;
+            for (ConcurrentMap<Integer, ConsumeQueueInterface> maps : tables.values()) {
+                for (ConsumeQueueInterface logic : maps.values()) {
+                    if (needCorrect(logic, minPhyOffset, lastForeCorrectTimeCurRun)) {
+                        doCorrect(logic, minPhyOffset);
+                    }
+                }
+            }
+        }
+
+        private void doCorrect(ConsumeQueueInterface logic, long minPhyOffset) {
+            StreamMessageStore.this.consumeQueueStore.deleteExpiredFile(logic, minPhyOffset);
+            int sleepIntervalWhenCorrectMinOffset = StreamMessageStore.this.getMessageStoreConfig().getCorrectLogicMinOffsetSleepInterval();
+            if (sleepIntervalWhenCorrectMinOffset > 0) {
+                try {
+                    Thread.sleep(sleepIntervalWhenCorrectMinOffset);
+                } catch (InterruptedException ignored) {
+                }
+            }
+        }
+
+        public String getServiceName() {
+            return CorrectLogicOffsetService.class.getSimpleName();
+        }
+    }
+
+    class FlushConsumeQueueService extends ServiceThread {
+        private static final int RETRY_TIMES_OVER = 3;
+        private long lastFlushTimestamp = 0;
+
+        private void doFlush(int retryTimes) {
+            int flushConsumeQueueLeastPages = StreamMessageStore.this.getMessageStoreConfig().getFlushConsumeQueueLeastPages();
+
+            if (retryTimes == RETRY_TIMES_OVER) {
+                flushConsumeQueueLeastPages = 0;
+            }
+
+            long logicsMsgTimestamp = 0;
+
+            int flushConsumeQueueThoroughInterval = StreamMessageStore.this.getMessageStoreConfig().getFlushConsumeQueueThoroughInterval();
+            long currentTimeMillis = System.currentTimeMillis();
+            if (currentTimeMillis >= (this.lastFlushTimestamp + flushConsumeQueueThoroughInterval)) {
+                this.lastFlushTimestamp = currentTimeMillis;
+                flushConsumeQueueLeastPages = 0;
+                logicsMsgTimestamp = StreamMessageStore.this.getStoreCheckpoint().getLogicsMsgTimestamp();
+            }
+
+            ConcurrentMap<String, ConcurrentMap<Integer, ConsumeQueueInterface>> tables = StreamMessageStore.this.consumeQueueTable;
+
+            for (ConcurrentMap<Integer, ConsumeQueueInterface> maps : tables.values()) {
+                for (ConsumeQueueInterface cq : maps.values()) {
+                    boolean result = false;
+                    for (int i = 0; i < retryTimes && !result; i++) {
+                        result = StreamMessageStore.this.consumeQueueStore.flush(cq, flushConsumeQueueLeastPages);
+                    }
+                }
+            }
+
+            if (0 == flushConsumeQueueLeastPages) {
+                if (logicsMsgTimestamp > 0) {
+                    StreamMessageStore.this.getStoreCheckpoint().setLogicsMsgTimestamp(logicsMsgTimestamp);
+                }
+                StreamMessageStore.this.getStoreCheckpoint().flush();
+            }
+        }
+
+        @Override
+        public void run() {
+            StreamMessageStore.log.info(this.getServiceName() + " service started");
+
+            while (!this.isStopped()) {
+                try {
+                    int interval = StreamMessageStore.this.getMessageStoreConfig().getFlushIntervalConsumeQueue();
+                    this.waitForRunning(interval);
+                    this.doFlush(1);
+                } catch (Exception e) {
+                    StreamMessageStore.log.warn(this.getServiceName() + " service has exception. ", e);
+                }
+            }
+
+            this.doFlush(RETRY_TIMES_OVER);
+
+            StreamMessageStore.log.info(this.getServiceName() + " service end");
+        }
+
+        @Override
+        public String getServiceName() {
+            return FlushConsumeQueueService.class.getSimpleName();
+        }
+
+        @Override
+        public long getJointime() {
+            return 1000 * 60;
+        }
+    }
+
+    private class SyncReputMessageServiceFutureItem {
+        private Future future;
+        private long nextReputFromOffset;
+
+        public SyncReputMessageServiceFutureItem(Future future, long nextReputFromOffset) {
+            this.future = future;
+            this.nextReputFromOffset = nextReputFromOffset;
+        }
+
+        public Future getFuture() {
+            return future;
+        }
+
+        public void setFuture(Future future) {
+            this.future = future;
+        }
+
+        public long getNextReputFromOffset() {
+            return nextReputFromOffset;
+        }
+
+        public void setNextReputFromOffset(long nextReputFromOffset) {
+            this.nextReputFromOffset = nextReputFromOffset;
+        }
+    }
+
+    class SyncReputMessageService extends ReputMessageService {
+
+        private BlockingQueue<SyncReputMessageServiceFutureItem> reputResultQueue;
+
+        SyncReputMessageService() {
+            reputResultQueue = new LinkedBlockingDeque<>(messageStoreConfig.getDispatchCqThreads() * messageStoreConfig.getDispatchCqCacheNum());
+        }
+
+        void processDispatchRequestForRecover(DispatchRequest dispatchRequest) {
+            int size = dispatchRequest.getBufferSize() == -1 ? dispatchRequest.getMsgSize() : dispatchRequest.getBufferSize();
+            if (dispatchRequest.isSuccess() && size > 0) {
+                StreamMessageStore.this.doDispatch(dispatchRequest);
+                this.reputFromOffset = Math.max(dispatchRequest.getNextReputFromOffset(), this.reputFromOffset);
+            }
+        }
+
+        void processDispatchRequest(DispatchRequest dispatchRequest) throws InterruptedException {
+            int size = dispatchRequest.getBufferSize() == -1 ? dispatchRequest.getMsgSize() : dispatchRequest.getBufferSize();
+            if (dispatchRequest.isSuccess() && size > 0) {
+                Future future = StreamMessageStore.this.doDispatch(dispatchRequest, messageStoreConfig.isEnableAsyncReput());
+                reputResultQueue.put(new SyncReputMessageServiceFutureItem(future, dispatchRequest.getNextReputFromOffset()));
+            }
+        }
+
+        @Override
+        public void shutdown() {
+            for (int i = 0; i < 30 && this.reputResultQueue.size() > 0; i++) {
+                try {
+                    Thread.sleep(1000);
+                } catch (InterruptedException ignored) {
+                }
+            }
+
+            if (this.reputResultQueue.size() > 0) {
+                log.warn("shutdown SyncReputMessageService, but reputResultQueue not all processed, CLMaxOffset: {} reputFromOffset: {}",
+                        StreamMessageStore.this.commitLog.getMaxOffset(), this.reputFromOffset);
+            }
+
+            super.shutdown();
+        }
+        @Override
+        public void run() {
+            StreamMessageStore.log.info(this.getServiceName() + " service started");
+            while (!this.isStopped()) {
+                try {
+                    SyncReputMessageServiceFutureItem item = reputResultQueue.poll(1, TimeUnit.SECONDS);
+                    if (null == item) {
+                        continue;
+                    }
+                    item.getFuture().get();
+                    this.reputFromOffset = Math.max(item.getNextReputFromOffset(), this.reputFromOffset);
+
+                } catch (Exception e) {
+                    StreamMessageStore.log.warn(this.getServiceName() + " service has exception. ", e);
+                    waitForRunning(100);
+                }
+            }
+
+            StreamMessageStore.log.info(this.getServiceName() + " service end");
+        }
+
+        @Override
+        public String getServiceName() {
+            return SyncReputMessageService.class.getSimpleName();
+        }
+    }
+
+    class ReputMessageService extends ServiceThread {
+
+        protected volatile long reputFromOffset = 0;
+
+        public long getReputFromOffset() {
+            return reputFromOffset;
+        }
+
+        public void setReputFromOffset(long reputFromOffset) {
+            this.reputFromOffset = reputFromOffset;
+        }
+
+        public boolean dispatched(long physicalOffset) {
+            return this.reputFromOffset > physicalOffset;
+        }
+
+        @Override
+        public void shutdown() {
+            for (int i = 0; i < 30 && this.isCommitLogAvailable(); i++) {
+                try {
+                    Thread.sleep(1000);
+                } catch (InterruptedException ignored) {
+                }
+            }
+
+            if (this.isCommitLogAvailable()) {
+                log.warn("shutdown ReputMessageService, but commitlog have not finish to be dispatched, CL: {} reputFromOffset: {}",
+                        StreamMessageStore.this.commitLog.getMaxOffset(), this.reputFromOffset);
+            }
+
+            super.shutdown();
+        }
+
+        public long behind() {
+            return StreamMessageStore.this.commitLog.getMaxOffset() - this.reputFromOffset;
+        }
+
+        protected boolean isCommitLogAvailable() {
+            return this.reputFromOffset < StreamMessageStore.this.commitLog.getMaxOffset();
+        }
+
+        private void doReput() throws Exception {
+            if (this.reputFromOffset < StreamMessageStore.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.",
+                        this.reputFromOffset, StreamMessageStore.this.commitLog.getMinOffset());
+                this.reputFromOffset = StreamMessageStore.this.commitLog.getMinOffset();
+            }
+            for (boolean doNext = true; this.isCommitLogAvailable() && doNext; ) {
+
+                if (StreamMessageStore.this.getMessageStoreConfig().isDuplicationEnable()
+                        && this.reputFromOffset >= StreamMessageStore.this.getConfirmOffset()) {
+                    break;
+                }
+
+                SelectMappedBufferResult result = StreamMessageStore.this.commitLog.getData(reputFromOffset);
+                boolean reputAsync = messageStoreConfig.isEnableAsyncReput();
+                if (result != null) {
+                    long cacheReputFromOffset = this.reputFromOffset;
+                    List<Future> futures = new LinkedList<>();
+                    try {
+                        this.reputFromOffset = result.getStartOffset();
+                        cacheReputFromOffset = this.reputFromOffset;
+
+                        for (int readSize = 0; readSize < result.getSize() && doNext; ) {
+                            DispatchRequest dispatchRequest =
+                                    StreamMessageStore.this.commitLog.checkMessageAndReturnSize(result.getByteBuffer(), false, false);
+                            int size = dispatchRequest.getBufferSize() == -1 ? dispatchRequest.getMsgSize() : dispatchRequest.getBufferSize();
+
+                            if (dispatchRequest.isSuccess()) {
+                                if (size > 0) {
+                                    futures.add(StreamMessageStore.this.doDispatch(dispatchRequest, reputAsync));
+
+                                    if (BrokerRole.SLAVE != StreamMessageStore.this.getMessageStoreConfig().getBrokerRole()
+                                            && StreamMessageStore.this.brokerConfig.isLongPollingEnable()) {
+                                        StreamMessageStore.this.messageArrivingListener.arriving(dispatchRequest.getTopic(),
+                                                dispatchRequest.getQueueId(), dispatchRequest.getConsumeQueueOffset() + 1,
+                                                dispatchRequest.getTagsCode(), dispatchRequest.getStoreTimestamp(),
+                                                dispatchRequest.getBitMap(), dispatchRequest.getPropertiesMap());
+                                    }
+
+                                    cacheReputFromOffset += size;
+                                    readSize += size;
+                                    if (StreamMessageStore.this.getMessageStoreConfig().getBrokerRole() == BrokerRole.SLAVE) {
+                                        StreamMessageStore.this.storeStatsService
+                                                .getSinglePutMessageTopicTimesTotal(dispatchRequest.getTopic()).add(1);
+                                        StreamMessageStore.this.storeStatsService
+                                                .getSinglePutMessageTopicSizeTotal(dispatchRequest.getTopic())
+                                                .add(dispatchRequest.getMsgSize());
+                                    }
+                                } else if (size == 0) {
+                                    cacheReputFromOffset = StreamMessageStore.this.commitLog.rollNextFile(cacheReputFromOffset);
+                                    readSize = result.getSize();
+                                }
+                            } else if (!dispatchRequest.isSuccess()) {
+
+                                if (size > 0) {
+                                    log.error("[BUG]read total count not equals msg total size. reputFromOffset={}", cacheReputFromOffset);
+                                    cacheReputFromOffset += size;
+                                } else {
+                                    doNext = false;
+                                    if (StreamMessageStore.this.brokerConfig.getBrokerId() == MixAll.MASTER_ID) {
+                                        log.error("[BUG]the master dispatch message to consume queue error, COMMITLOG OFFSET: {}",
+                                                cacheReputFromOffset);
+                                        cacheReputFromOffset += result.getSize() - readSize;
+                                    }
+                                }
+                            }
+                        }
+                        for (Future future: futures) {
+                            future.get();
+                        }
+                        futures.clear();
+                        this.reputFromOffset = cacheReputFromOffset;
+                    } finally {
+                        result.release();
+                    }
+                } else {
+                    doNext = false;
+                }
+            }
+        }
+
+        @Override
+        public void run() {
+            StreamMessageStore.log.info(this.getServiceName() + " service started");
+
+            while (!this.isStopped()) {
+                try {
+                    Thread.sleep(1);
+                    this.doReput();
+                } catch (Exception e) {
+                    StreamMessageStore.log.warn(this.getServiceName() + " service has exception. ", e);
+                }
+            }
+
+            StreamMessageStore.log.info(this.getServiceName() + " service end");
+        }
+
+        @Override
+        public String getServiceName() {
+            return ReputMessageService.class.getSimpleName();
+        }
+
+    }
+}
diff --git a/common/src/main/java/org/apache/rocketmq/common/message/MessageExtBatch.java b/store/src/main/java/org/apache/rocketmq/store/Swappable.java
similarity index 55%
copy from common/src/main/java/org/apache/rocketmq/common/message/MessageExtBatch.java
copy to store/src/main/java/org/apache/rocketmq/store/Swappable.java
index a2713cb..cb8dee5 100644
--- a/common/src/main/java/org/apache/rocketmq/common/message/MessageExtBatch.java
+++ b/store/src/main/java/org/apache/rocketmq/store/Swappable.java
@@ -1,4 +1,4 @@
-/**
+/*
  * Licensed to the Apache Software Foundation (ASF) under one or more
  * contributor license agreements.  See the NOTICE file distributed with
  * this work for additional information regarding copyright ownership.
@@ -6,7 +6,7 @@
  * (the "License"); you may not use this file except in compliance with
  * the License.  You may obtain a copy of the License at
  *
- * http://www.apache.org/licenses/LICENSE-2.0
+ *     http://www.apache.org/licenses/LICENSE-2.0
  *
  * Unless required by applicable law or agreed to in writing, software
  * distributed under the License is distributed on an "AS IS" BASIS,
@@ -14,27 +14,12 @@
  * See the License for the specific language governing permissions and
  * limitations under the License.
  */
+package org.apache.rocketmq.store;
 
-package org.apache.rocketmq.common.message;
-
-import java.nio.ByteBuffer;
-
-public class MessageExtBatch extends MessageExt {
-
-    private static final long serialVersionUID = -2353110995348498537L;
-
-    public ByteBuffer wrap() {
-        assert getBody() != null;
-        return ByteBuffer.wrap(getBody(), 0, getBody().length);
-    }
-
-    private ByteBuffer encodedBuff;
-
-    public ByteBuffer getEncodedBuff() {
-        return encodedBuff;
-    }
-
-    public void setEncodedBuff(ByteBuffer encodedBuff) {
-        this.encodedBuff = encodedBuff;
-    }
+/**
+ * Clean up page-table on super large disk
+ */
+public interface Swappable {
+    void swapMap(int reserveNum, long forceSwapIntervalMs, long normalSwapIntervalMs);
+    void cleanSwappedMap(long forceCleanSwapIntervalMs);
 }
diff --git a/store/src/main/java/org/apache/rocketmq/store/TopicQueueLock.java b/store/src/main/java/org/apache/rocketmq/store/TopicQueueLock.java
new file mode 100644
index 0000000..0338cf7
--- /dev/null
+++ b/store/src/main/java/org/apache/rocketmq/store/TopicQueueLock.java
@@ -0,0 +1,29 @@
+package org.apache.rocketmq.store;
+
+import java.util.ArrayList;
+import java.util.List;
+import java.util.concurrent.locks.Lock;
+import java.util.concurrent.locks.ReentrantLock;
+
+public class TopicQueueLock {
+    private final int size;
+    private final List<Lock> lockList;
+
+    public TopicQueueLock() {
+        this.size = 32;
+        this.lockList = new ArrayList<>(32);
+        for (int i = 0; i < this.size; i++) {
+            this.lockList.add(new ReentrantLock());
+        }
+    }
+
+    public void lock(String topicQueueKey) {
+        Lock lock = this.lockList.get((topicQueueKey.hashCode() & 0x7fffffff) % this.size);
+        lock.lock();
+    }
+
+    public void unlock(String topicQueueKey) {
+        Lock lock = this.lockList.get((topicQueueKey.hashCode() & 0x7fffffff) % this.size);
+        lock.unlock();
+    }
+}
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 e1439a0..7a41c27 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
@@ -16,10 +16,12 @@
  */
 package org.apache.rocketmq.store.config;
 
-import java.io.File;
-
 import org.apache.rocketmq.common.annotation.ImportantField;
 import org.apache.rocketmq.store.ConsumeQueue;
+import org.apache.rocketmq.store.queue.BatchConsumeQueue;
+import org.apache.rocketmq.store.queue.CQType;
+
+import java.io.File;
 
 public class MessageStoreConfig {
 
@@ -44,6 +46,7 @@ public class MessageStoreConfig {
     private boolean enableConsumeQueueExt = false;
     // ConsumeQueue extend file size, 48M
     private int mappedFileSizeConsumeQueueExt = 48 * 1024 * 1024;
+    private int mapperFileSizeBatchConsumeQueue = 300000 * BatchConsumeQueue.CQ_STORE_UNIT_SIZE;
     // Bit count of filter bit map.
     // this will be set by pipe of calculate filter bit map.
     private int bitMapLengthConsumeQueueExt = 64;
@@ -58,6 +61,12 @@ public class MessageStoreConfig {
     @ImportantField
     private int commitIntervalCommitLog = 200;
 
+    private int maxRecoveryCommitlogFiles = 30;
+
+    private int diskSpaceWarningLevelRatio = 90;
+
+    private int diskSpaceCleanForciblyRatio = 85;
+
     /**
      * introduced since 4.0.x. Determine whether to use mutex reentrantLock when putting message.<br/>
      */
@@ -159,6 +168,51 @@ public class MessageStoreConfig {
 
     private boolean enableScheduleMessageStats = true;
 
+    private int maxBatchDeleteFilesNum = 50;
+    //Polish dispatch
+    private int dispatchCqThreads = 10;
+    private int dispatchCqCacheNum = 1024 * 4;
+    private boolean enableAsyncReput = true;
+    //For recheck the reput
+    private boolean recheckReputOffsetFromCq = false;
+
+    // Maximum length of topic
+    private int maxTopicLength = 1000;
+    private int travelCqFileNumWhenGetMessage = 1;
+    // Sleep interval between to corrections
+    private int correctLogicMinOffsetSleepInterval = 1;
+    // Force correct min offset interval
+    private int correctLogicMinOffsetForceInterval = 5 * 60 * 1000;
+    // swap
+    private boolean mappedFileSwapEnable = true;
+    private long commitLogForceSwapMapInterval = 12L * 60 * 60 * 1000;
+    private long commitLogSwapMapInterval = 1L * 60 * 60 * 1000;
+    private int commitLogSwapMapReserveFileNum = 100;
+    private long logicQueueForceSwapMapInterval = 12L * 60 * 60 * 1000;
+    private long logicQueueSwapMapInterval = 1L * 60 * 60 * 1000;
+    private long cleanSwapedMapInterval = 5L * 60 * 1000;
+    private int logicQueueSwapMapReserveFileNum = 20;
+
+    private boolean searchBcqByCacheEnable = true;
+
+    @ImportantField
+    private boolean dispatchFromSenderThread = false;
+
+    @ImportantField
+    private boolean wakeCommitWhenPutMessage = true;
+    @ImportantField
+    private boolean wakeFlushWhenPutMessage = false;
+
+    @ImportantField
+    private boolean enableCleanExpiredOffset = false;
+
+    @ImportantField
+    private String defaultCQType = CQType.SimpleCQ.toString();
+
+    private int maxAsyncPutMessageRequests = 5000;
+
+    private int pullBatchMaxMessageCount = 160;
+
     public boolean isDebugLockEnable() {
         return debugLockEnable;
     }
@@ -281,6 +335,38 @@ public class MessageStoreConfig {
         this.maxMessageSize = maxMessageSize;
     }
 
+    public int getMaxTopicLength() {
+        return maxTopicLength;
+    }
+
+    public void setMaxTopicLength(int maxTopicLength) {
+        this.maxTopicLength = maxTopicLength;
+    }
+
+    public int getTravelCqFileNumWhenGetMessage() {
+        return travelCqFileNumWhenGetMessage;
+    }
+
+    public void setTravelCqFileNumWhenGetMessage(int travelCqFileNumWhenGetMessage) {
+        this.travelCqFileNumWhenGetMessage = travelCqFileNumWhenGetMessage;
+    }
+
+    public int getCorrectLogicMinOffsetSleepInterval() {
+        return correctLogicMinOffsetSleepInterval;
+    }
+
+    public void setCorrectLogicMinOffsetSleepInterval(int correctLogicMinOffsetSleepInterval) {
+        this.correctLogicMinOffsetSleepInterval = correctLogicMinOffsetSleepInterval;
+    }
+
+    public int getCorrectLogicMinOffsetForceInterval() {
+        return correctLogicMinOffsetForceInterval;
+    }
+
+    public void setCorrectLogicMinOffsetForceInterval(int correctLogicMinOffsetForceInterval) {
+        this.correctLogicMinOffsetForceInterval = correctLogicMinOffsetForceInterval;
+    }
+
     public boolean isCheckCRCOnRecover() {
         return checkCRCOnRecover;
     }
@@ -626,8 +712,7 @@ public class MessageStoreConfig {
      * @return <tt>true</tt> or <tt>false</tt>
      */
     public boolean isTransientStorePoolEnable() {
-        return transientStorePoolEnable && FlushDiskType.ASYNC_FLUSH == getFlushDiskType()
-            && BrokerRole.SLAVE != getBrokerRole();
+        return transientStorePoolEnable && BrokerRole.SLAVE != getBrokerRole();
     }
 
     public void setTransientStorePoolEnable(final boolean transientStorePoolEnable) {
@@ -682,6 +767,45 @@ public class MessageStoreConfig {
         this.commitCommitLogThoroughInterval = commitCommitLogThoroughInterval;
     }
 
+    public boolean isWakeCommitWhenPutMessage() {
+        return wakeCommitWhenPutMessage;
+    }
+
+    public void setWakeCommitWhenPutMessage(boolean wakeCommitWhenPutMessage) {
+        this.wakeCommitWhenPutMessage = wakeCommitWhenPutMessage;
+    }
+
+    public boolean isWakeFlushWhenPutMessage() {
+        return wakeFlushWhenPutMessage;
+    }
+
+    public void setWakeFlushWhenPutMessage(boolean wakeFlushWhenPutMessage) {
+        this.wakeFlushWhenPutMessage = wakeFlushWhenPutMessage;
+    }
+
+    public int getMapperFileSizeBatchConsumeQueue() {
+        return mapperFileSizeBatchConsumeQueue;
+    }
+
+    public void setMapperFileSizeBatchConsumeQueue(int mapperFileSizeBatchConsumeQueue) {
+        this.mapperFileSizeBatchConsumeQueue = mapperFileSizeBatchConsumeQueue;
+    }
+
+    public boolean isEnableCleanExpiredOffset() {
+        return enableCleanExpiredOffset;
+    }
+
+    public void setEnableCleanExpiredOffset(boolean enableCleanExpiredOffset) {
+        this.enableCleanExpiredOffset = enableCleanExpiredOffset;
+    }
+
+    public String getDefaultCQType() {
+        return defaultCQType;
+    }
+
+    public void setDefaultCQType(String defaultCQType) {
+        this.defaultCQType = defaultCQType;
+    }
     public String getReadOnlyCommitLogStorePaths() {
         return readOnlyCommitLogStorePaths;
     }
@@ -744,4 +868,164 @@ public class MessageStoreConfig {
     public void setEnableScheduleMessageStats(boolean enableScheduleMessageStats) {
         this.enableScheduleMessageStats = enableScheduleMessageStats;
     }
+
+    public int getMaxAsyncPutMessageRequests() {
+        return maxAsyncPutMessageRequests;
+    }
+
+    public void setMaxAsyncPutMessageRequests(int maxAsyncPutMessageRequests) {
+        this.maxAsyncPutMessageRequests = maxAsyncPutMessageRequests;
+    }
+
+    public int getMaxRecoveryCommitlogFiles() {
+        return maxRecoveryCommitlogFiles;
+    }
+
+    public void setMaxRecoveryCommitlogFiles(final int maxRecoveryCommitlogFiles) {
+        this.maxRecoveryCommitlogFiles = maxRecoveryCommitlogFiles;
+    }
+
+    public boolean isDispatchFromSenderThread() {
+        return dispatchFromSenderThread;
+    }
+
+    public void setDispatchFromSenderThread(boolean dispatchFromSenderThread) {
+        this.dispatchFromSenderThread = dispatchFromSenderThread;
+    }
+
+    public int getDispatchCqThreads() {
+        return dispatchCqThreads;
+    }
+
+    public void setDispatchCqThreads(final int dispatchCqThreads) {
+        this.dispatchCqThreads = dispatchCqThreads;
+    }
+
+    public int getDispatchCqCacheNum() {
+        return dispatchCqCacheNum;
+    }
+
+    public void setDispatchCqCacheNum(final int dispatchCqCacheNum) {
+        this.dispatchCqCacheNum = dispatchCqCacheNum;
+    }
+
+    public boolean isEnableAsyncReput() {
+        return enableAsyncReput;
+    }
+
+    public void setEnableAsyncReput(final boolean enableAsyncReput) {
+        this.enableAsyncReput = enableAsyncReput;
+    }
+
+    public boolean isRecheckReputOffsetFromCq() {
+        return recheckReputOffsetFromCq;
+    }
+
+    public void setRecheckReputOffsetFromCq(final boolean recheckReputOffsetFromCq) {
+        this.recheckReputOffsetFromCq = recheckReputOffsetFromCq;
+    }
+
+    public long getCommitLogForceSwapMapInterval() {
+        return commitLogForceSwapMapInterval;
+    }
+
+    public void setCommitLogForceSwapMapInterval(long commitLogForceSwapMapInterval) {
+        this.commitLogForceSwapMapInterval = commitLogForceSwapMapInterval;
+    }
+
+    public int getCommitLogSwapMapReserveFileNum() {
+        return commitLogSwapMapReserveFileNum;
+    }
+
+    public void setCommitLogSwapMapReserveFileNum(int commitLogSwapMapReserveFileNum) {
+        this.commitLogSwapMapReserveFileNum = commitLogSwapMapReserveFileNum;
+    }
+
+    public long getLogicQueueForceSwapMapInterval() {
+        return logicQueueForceSwapMapInterval;
+    }
+
+    public void setLogicQueueForceSwapMapInterval(long logicQueueForceSwapMapInterval) {
+        this.logicQueueForceSwapMapInterval = logicQueueForceSwapMapInterval;
+    }
+
+    public int getLogicQueueSwapMapReserveFileNum() {
+        return logicQueueSwapMapReserveFileNum;
+    }
+
+    public void setLogicQueueSwapMapReserveFileNum(int logicQueueSwapMapReserveFileNum) {
+        this.logicQueueSwapMapReserveFileNum = logicQueueSwapMapReserveFileNum;
+    }
+
+    public long getCleanSwapedMapInterval() {
+        return cleanSwapedMapInterval;
+    }
+
+    public void setCleanSwapedMapInterval(long cleanSwapedMapInterval) {
+        this.cleanSwapedMapInterval = cleanSwapedMapInterval;
+    }
+
+    public long getCommitLogSwapMapInterval() {
+        return commitLogSwapMapInterval;
+    }
+
+    public void setCommitLogSwapMapInterval(long commitLogSwapMapInterval) {
+        this.commitLogSwapMapInterval = commitLogSwapMapInterval;
+    }
+
+    public long getLogicQueueSwapMapInterval() {
+        return logicQueueSwapMapInterval;
+    }
+
+    public void setLogicQueueSwapMapInterval(long logicQueueSwapMapInterval) {
+        this.logicQueueSwapMapInterval = logicQueueSwapMapInterval;
+    }
+
+    public int getMaxBatchDeleteFilesNum() {
+        return maxBatchDeleteFilesNum;
+    }
+
+    public void setMaxBatchDeleteFilesNum(int maxBatchDeleteFilesNum) {
+        this.maxBatchDeleteFilesNum = maxBatchDeleteFilesNum;
+    }
+
+    public boolean isSearchBcqByCacheEnable() {
+        return searchBcqByCacheEnable;
+    }
+
+    public void setSearchBcqByCacheEnable(boolean searchBcqByCacheEnable) {
+        this.searchBcqByCacheEnable = searchBcqByCacheEnable;
+    }
+
+    public int getDiskSpaceWarningLevelRatio() {
+        return diskSpaceWarningLevelRatio;
+    }
+
+    public void setDiskSpaceWarningLevelRatio(int diskSpaceWarningLevelRatio) {
+        this.diskSpaceWarningLevelRatio = diskSpaceWarningLevelRatio;
+    }
+
+    public int getDiskSpaceCleanForciblyRatio() {
+        return diskSpaceCleanForciblyRatio;
+    }
+
+    public void setDiskSpaceCleanForciblyRatio(int diskSpaceCleanForciblyRatio) {
+        this.diskSpaceCleanForciblyRatio = diskSpaceCleanForciblyRatio;
+    }
+
+    public boolean isMappedFileSwapEnable() {
+        return mappedFileSwapEnable;
+    }
+
+    public void setMappedFileSwapEnable(boolean mappedFileSwapEnable) {
+        this.mappedFileSwapEnable = mappedFileSwapEnable;
+    }
+
+    public int getPullBatchMaxMessageCount() {
+        return pullBatchMaxMessageCount;
+    }
+
+    public void setPullBatchMaxMessageCount(int pullBatchMaxMessageCount) {
+        this.pullBatchMaxMessageCount = pullBatchMaxMessageCount;
+    }
 }
diff --git a/store/src/main/java/org/apache/rocketmq/store/config/StorePathConfigHelper.java b/store/src/main/java/org/apache/rocketmq/store/config/StorePathConfigHelper.java
index ccd76c4..2f34e7d 100644
--- a/store/src/main/java/org/apache/rocketmq/store/config/StorePathConfigHelper.java
+++ b/store/src/main/java/org/apache/rocketmq/store/config/StorePathConfigHelper.java
@@ -27,6 +27,9 @@ public class StorePathConfigHelper {
     public static String getStorePathConsumeQueueExt(final String rootDir) {
         return rootDir + File.separator + "consumequeue_ext";
     }
+    public static String getStorePathBatchConsumeQueue(final String rootDir) {
+        return rootDir + File.separator + "batchconsumequeue";
+    }
 
     public static String getStorePathIndex(final String rootDir) {
         return rootDir + File.separator + "index";
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 4939175..50078c9 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
@@ -41,7 +41,6 @@ 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;
@@ -49,12 +48,13 @@ 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.MappedFile;
+import org.apache.rocketmq.store.MessageExtBatch;
 import org.apache.rocketmq.store.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;
 
 /**
@@ -428,55 +428,51 @@ public class DLedgerCommitLog extends CommitLog {
         AppendFuture<AppendEntryResponse> dledgerFuture;
         EncodeResult encodeResult;
 
-        encodeResult = this.messageSerializer.serialize(msg);
-        if (encodeResult.status != AppendMessageStatus.PUT_OK) {
-            return CompletableFuture.completedFuture(new PutMessageResult(PutMessageStatus.MESSAGE_ILLEGAL, new AppendMessageResult(encodeResult.status)));
-        }
-        putMessageLock.lock(); //spin or ReentrantLock ,depending on store config
-        long elapsedTimeInLock;
-        long queueOffset;
+        String topicQueueKey = msg.getTopic() + "-" + msg.getQueueId();
+        topicQueueLock.lock(topicQueueKey);
         try {
-            beginTimeInDledgerLock = this.defaultMessageStore.getSystemClock().now();
-            queueOffset = getQueueOffsetByKey(encodeResult.queueOffsetKey, tranType);
-            encodeResult.setQueueOffsetKey(queueOffset, false);
-            AppendEntryRequest request = new AppendEntryRequest();
-            request.setGroup(dLedgerConfig.getGroup());
-            request.setRemoteId(dLedgerServer.getMemberState().getSelfId());
-            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)));
-            }
-            long wroteOffset = dledgerFuture.getPos() + DLedgerEntry.BODY_OFFSET;
+            defaultMessageStore.assignOffset(topicQueueKey, msg, getBatchNum(msg));
 
-            int msgIdLength = (msg.getSysFlag() & MessageSysFlag.STOREHOSTADDRESS_V6_FLAG) == 0 ? 4 + 4 + 8 : 16 + 4 + 8;
-            ByteBuffer buffer = ByteBuffer.allocate(msgIdLength);
+            encodeResult = this.messageSerializer.serialize(msg);
+            if (encodeResult.status != AppendMessageStatus.PUT_OK) {
+                return CompletableFuture.completedFuture(new PutMessageResult(PutMessageStatus.MESSAGE_ILLEGAL, new AppendMessageResult(encodeResult.status)));
+            }
+            putMessageLock.lock(); //spin or ReentrantLock ,depending on store config
+            long elapsedTimeInLock;
+            long queueOffset;
+            try {
+                beginTimeInDledgerLock = this.defaultMessageStore.getSystemClock().now();
+                queueOffset = getQueueOffsetByKey(msg, tranType);
+                encodeResult.setQueueOffsetKey(queueOffset, false);
+                AppendEntryRequest request = new AppendEntryRequest();
+                request.setGroup(dLedgerConfig.getGroup());
+                request.setRemoteId(dLedgerServer.getMemberState().getSelfId());
+                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)));
+                }
+                long wroteOffset = dledgerFuture.getPos() + DLedgerEntry.BODY_OFFSET;
+
+                int msgIdLength = (msg.getSysFlag() & MessageSysFlag.STOREHOSTADDRESS_V6_FLAG) == 0 ? 4 + 4 + 8 : 16 + 4 + 8;
+                ByteBuffer buffer = ByteBuffer.allocate(msgIdLength);
+
+                String msgId = MessageDecoder.createMessageId(buffer, msg.getStoreHostBytes(), wroteOffset);
+                elapsedTimeInLock = this.defaultMessageStore.getSystemClock().now() - beginTimeInDledgerLock;
+                appendResult = new AppendMessageResult(AppendMessageStatus.PUT_OK, wroteOffset, encodeResult.getData().length, msgId, System.currentTimeMillis(), queueOffset, elapsedTimeInLock);
+            } catch (Exception e) {
+                log.error("Put message error", e);
+                return CompletableFuture.completedFuture(new PutMessageResult(PutMessageStatus.UNKNOWN_ERROR, new AppendMessageResult(AppendMessageStatus.UNKNOWN_ERROR)));
+            } finally {
+                beginTimeInDledgerLock = 0;
+                putMessageLock.unlock();
+            }
 
-            String msgId = MessageDecoder.createMessageId(buffer, msg.getStoreHostBytes(), wroteOffset);
-            elapsedTimeInLock = this.defaultMessageStore.getSystemClock().now() - beginTimeInDledgerLock;
-            appendResult = new AppendMessageResult(AppendMessageStatus.PUT_OK, wroteOffset, encodeResult.getData().length, msgId, System.currentTimeMillis(), queueOffset, elapsedTimeInLock);
-            switch (tranType) {
-                case MessageSysFlag.TRANSACTION_PREPARED_TYPE:
-                case MessageSysFlag.TRANSACTION_ROLLBACK_TYPE:
-                    break;
-                case MessageSysFlag.TRANSACTION_NOT_TYPE:
-                case MessageSysFlag.TRANSACTION_COMMIT_TYPE:
-                    // The next update ConsumeQueue information
-                    DLedgerCommitLog.this.topicQueueTable.put(encodeResult.queueOffsetKey, queueOffset + 1);
-                    break;
-                default:
-                    break;
+            if (elapsedTimeInLock > 500) {
+                log.warn("[NOTIFYME]putMessage in lock cost time(ms)={}, bodyLength={} AppendMessageResult={}", elapsedTimeInLock, msg.getBody().length, appendResult);
             }
-        } catch (Exception e) {
-            log.error("Put message error", e);
-            return CompletableFuture.completedFuture(new PutMessageResult(PutMessageStatus.UNKNOWN_ERROR, new AppendMessageResult(AppendMessageStatus.UNKNOWN_ERROR)));
         } finally {
-            beginTimeInDledgerLock = 0;
-            putMessageLock.unlock();
-        }
-
-        if (elapsedTimeInLock > 500) {
-            log.warn("[NOTIFYME]putMessage in lock cost time(ms)={}, bodyLength={} AppendMessageResult={}", elapsedTimeInLock, msg.getBody().length, appendResult);
+            topicQueueLock.unlock(topicQueueKey);
         }
 
         return dledgerFuture.thenApply(appendEntryResponse -> {
@@ -546,64 +542,71 @@ public class DLedgerCommitLog extends CommitLog {
                     .status)));
         }
 
-        putMessageLock.lock(); //spin or ReentrantLock ,depending on store config
-        msgIdBuilder.setLength(0);
-        long elapsedTimeInLock;
-        long queueOffset;
-        int msgNum = 0;
+        int batchNum = encodeResult.batchData.size();
+        topicQueueLock.lock(encodeResult.queueOffsetKey);
         try {
-            beginTimeInDledgerLock = this.defaultMessageStore.getSystemClock().now();
-            queueOffset = getQueueOffsetByKey(encodeResult.queueOffsetKey, tranType);
-            encodeResult.setQueueOffsetKey(queueOffset, true);
-            BatchAppendEntryRequest request = new BatchAppendEntryRequest();
-            request.setGroup(dLedgerConfig.getGroup());
-            request.setRemoteId(dLedgerServer.getMemberState().getSelfId());
-            request.setBatchMsgs(encodeResult.batchData);
-            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)));
-            }
-            dledgerFuture = (BatchAppendFuture<AppendEntryResponse>) appendFuture;
-
-            long wroteOffset = 0;
-
-            int msgIdLength = (messageExtBatch.getSysFlag() & MessageSysFlag.STOREHOSTADDRESS_V6_FLAG) == 0 ? 4 + 4 + 8 : 16 + 4 + 8;
-            ByteBuffer buffer = ByteBuffer.allocate(msgIdLength);
-
-            boolean isFirstOffset = true;
-            long firstWroteOffset = 0;
-            for (long pos : dledgerFuture.getPositions()) {
-                wroteOffset = pos + DLedgerEntry.BODY_OFFSET;
-                if (isFirstOffset) {
-                    firstWroteOffset = wroteOffset;
-                    isFirstOffset = false;
+            defaultMessageStore.assignOffset(encodeResult.queueOffsetKey, messageExtBatch, (short) batchNum);
+
+            putMessageLock.lock(); //spin or ReentrantLock ,depending on store config
+            msgIdBuilder.setLength(0);
+            long elapsedTimeInLock;
+            long queueOffset;
+            int msgNum = 0;
+            try {
+                beginTimeInDledgerLock = this.defaultMessageStore.getSystemClock().now();
+                queueOffset = getQueueOffsetByKey(messageExtBatch, tranType);
+                encodeResult.setQueueOffsetKey(queueOffset, true);
+                BatchAppendEntryRequest request = new BatchAppendEntryRequest();
+                request.setGroup(dLedgerConfig.getGroup());
+                request.setRemoteId(dLedgerServer.getMemberState().getSelfId());
+                request.setBatchMsgs(encodeResult.batchData);
+                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)));
                 }
-                String msgId = MessageDecoder.createMessageId(buffer, messageExtBatch.getStoreHostBytes(), wroteOffset);
-                if (msgIdBuilder.length() > 0) {
-                    msgIdBuilder.append(',').append(msgId);
-                } else {
-                    msgIdBuilder.append(msgId);
+                dledgerFuture = (BatchAppendFuture<AppendEntryResponse>) appendFuture;
+
+                long wroteOffset = 0;
+
+                int msgIdLength = (messageExtBatch.getSysFlag() & MessageSysFlag.STOREHOSTADDRESS_V6_FLAG) == 0 ? 4 + 4 + 8 : 16 + 4 + 8;
+                ByteBuffer buffer = ByteBuffer.allocate(msgIdLength);
+
+                boolean isFirstOffset = true;
+                long firstWroteOffset = 0;
+                for (long pos : dledgerFuture.getPositions()) {
+                    wroteOffset = pos + DLedgerEntry.BODY_OFFSET;
+                    if (isFirstOffset) {
+                        firstWroteOffset = wroteOffset;
+                        isFirstOffset = false;
+                    }
+                    String msgId = MessageDecoder.createMessageId(buffer, messageExtBatch.getStoreHostBytes(), wroteOffset);
+                    if (msgIdBuilder.length() > 0) {
+                        msgIdBuilder.append(',').append(msgId);
+                    } else {
+                        msgIdBuilder.append(msgId);
+                    }
+                    msgNum++;
                 }
-                msgNum++;
+
+                elapsedTimeInLock = this.defaultMessageStore.getSystemClock().now() - beginTimeInDledgerLock;
+                appendResult = new AppendMessageResult(AppendMessageStatus.PUT_OK, firstWroteOffset, encodeResult.totalMsgLen,
+                        msgIdBuilder.toString(), System.currentTimeMillis(), queueOffset, elapsedTimeInLock);
+                appendResult.setMsgNum(msgNum);
+            } catch (Exception e) {
+                log.error("Put message error", e);
+                return CompletableFuture.completedFuture(new PutMessageResult(PutMessageStatus.UNKNOWN_ERROR, new AppendMessageResult(AppendMessageStatus.UNKNOWN_ERROR)));
+            } finally {
+                beginTimeInDledgerLock = 0;
+                putMessageLock.unlock();
             }
 
-            elapsedTimeInLock = this.defaultMessageStore.getSystemClock().now() - beginTimeInDledgerLock;
-            appendResult = new AppendMessageResult(AppendMessageStatus.PUT_OK, firstWroteOffset, encodeResult.totalMsgLen,
-                    msgIdBuilder.toString(), System.currentTimeMillis(), queueOffset, elapsedTimeInLock);
-            appendResult.setMsgNum(msgNum);
-            DLedgerCommitLog.this.topicQueueTable.put(encodeResult.queueOffsetKey, queueOffset + msgNum);
-        } catch (Exception e) {
-            log.error("Put message error", e);
-            return CompletableFuture.completedFuture(new PutMessageResult(PutMessageStatus.UNKNOWN_ERROR, new AppendMessageResult(AppendMessageStatus.UNKNOWN_ERROR)));
+            if (elapsedTimeInLock > 500) {
+                log.warn("[NOTIFYME]putMessage in lock cost time(ms)={}, bodyLength={} AppendMessageResult={}",
+                        elapsedTimeInLock, messageExtBatch.getBody().length, appendResult);
+            }
         } finally {
-            beginTimeInDledgerLock = 0;
-            putMessageLock.unlock();
-        }
-
-        if (elapsedTimeInLock > 500) {
-            log.warn("[NOTIFYME]putMessage in lock cost time(ms)={}, bodyLength={} AppendMessageResult={}",
-                    elapsedTimeInLock, messageExtBatch.getBody().length, appendResult);
+            topicQueueLock.unlock(encodeResult.queueOffsetKey);
         }
 
         return dledgerFuture.thenApply(appendEntryResponse -> {
@@ -657,16 +660,6 @@ public class DLedgerCommitLog extends CommitLog {
     }
 
     @Override
-    public HashMap<String, Long> getTopicQueueTable() {
-        return topicQueueTable;
-    }
-
-    @Override
-    public void setTopicQueueTable(HashMap<String, Long> topicQueueTable) {
-        this.topicQueueTable = topicQueueTable;
-    }
-
-    @Override
     public void destroy() {
         super.destroy();
         dLedgerFileList.destroy();
@@ -698,12 +691,8 @@ public class DLedgerCommitLog extends CommitLog {
         return diff;
     }
 
-    private long getQueueOffsetByKey(String key, int tranType) {
-        Long queueOffset = DLedgerCommitLog.this.topicQueueTable.get(key);
-        if (null == queueOffset) {
-            queueOffset = 0L;
-            DLedgerCommitLog.this.topicQueueTable.put(key, queueOffset);
-        }
+    private long getQueueOffsetByKey(MessageExtBrokerInner msg, int tranType) {
+        Long queueOffset = msg.getQueueOffset();
 
         // Transaction messages that require special handling
         switch (tranType) {
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 845935b..deeb69f 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
@@ -37,7 +37,7 @@ 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.MessageStore;
 import org.apache.rocketmq.store.PutMessageSpinLock;
 import org.apache.rocketmq.store.PutMessageStatus;
 
@@ -50,7 +50,7 @@ public class HAService {
 
     private final AcceptSocketService acceptSocketService;
 
-    private final DefaultMessageStore defaultMessageStore;
+    private final MessageStore defaultMessageStore;
 
     private final WaitNotifyObject waitNotifyObject = new WaitNotifyObject();
     private final AtomicLong push2SlaveMaxOffset = new AtomicLong(0);
@@ -59,7 +59,7 @@ public class HAService {
 
     private final HAClient haClient;
 
-    public HAService(final DefaultMessageStore defaultMessageStore) throws IOException {
+    public HAService(final MessageStore defaultMessageStore) throws IOException {
         this.defaultMessageStore = defaultMessageStore;
         this.acceptSocketService =
             new AcceptSocketService(defaultMessageStore.getMessageStoreConfig().getHaListenPort());
@@ -142,7 +142,7 @@ public class HAService {
         }
     }
 
-    public DefaultMessageStore getDefaultMessageStore() {
+    public MessageStore getDefaultMessageStore() {
         return defaultMessageStore;
     }
 
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 e513edf..e920c84 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
@@ -25,7 +25,8 @@ import java.util.List;
 import org.apache.rocketmq.common.constant.LoggerName;
 import org.apache.rocketmq.logging.InternalLogger;
 import org.apache.rocketmq.logging.InternalLoggerFactory;
-import org.apache.rocketmq.store.MappedFile;
+import org.apache.rocketmq.store.logfile.DefaultMappedFile;
+import org.apache.rocketmq.store.logfile.MappedFile;
 
 public class IndexFile {
     private static final InternalLogger log = InternalLoggerFactory.getLogger(LoggerName.STORE_LOGGER_NAME);
@@ -43,7 +44,7 @@ public class IndexFile {
         final long endPhyOffset, final long endTimestamp) throws IOException {
         int fileTotalSize =
             IndexHeader.INDEX_HEADER_SIZE + (hashSlotNum * hashSlotSize) + (indexNum * indexSize);
-        this.mappedFile = new MappedFile(fileName, fileTotalSize);
+        this.mappedFile = new DefaultMappedFile(fileName, fileTotalSize);
         this.fileChannel = this.mappedFile.getFileChannel();
         this.mappedByteBuffer = this.mappedFile.getMappedByteBuffer();
         this.hashSlotNum = hashSlotNum;
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 bf17ecf..f2c5616 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
@@ -29,8 +29,8 @@ 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 {
@@ -39,14 +39,14 @@ public class IndexService {
      * Maximum times to attempt index file creation.
      */
     private static final int MAX_TRY_IDX_CREATE = 3;
-    private final DefaultMessageStore defaultMessageStore;
+    private final MessageStore 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 DefaultMessageStore store) {
+    public IndexService(final MessageStore store) {
         this.defaultMessageStore = store;
         this.hashSlotNum = store.getMessageStoreConfig().getMaxHashSlotNum();
         this.indexNum = store.getMessageStoreConfig().getMaxIndexNum();
@@ -282,7 +282,7 @@ public class IndexService {
         }
 
         if (null == indexFile) {
-            this.defaultMessageStore.getAccessRights().makeIndexFileError();
+            this.defaultMessageStore.getRunningFlags().makeIndexFileError();
             log.error("Mark index file cannot build flag");
         }
 
diff --git a/common/src/main/java/org/apache/rocketmq/common/message/MessageExtBatch.java b/store/src/main/java/org/apache/rocketmq/store/logfile/AbstractMappedFile.java
similarity index 55%
copy from common/src/main/java/org/apache/rocketmq/common/message/MessageExtBatch.java
copy to store/src/main/java/org/apache/rocketmq/store/logfile/AbstractMappedFile.java
index a2713cb..28d443c 100644
--- a/common/src/main/java/org/apache/rocketmq/common/message/MessageExtBatch.java
+++ b/store/src/main/java/org/apache/rocketmq/store/logfile/AbstractMappedFile.java
@@ -1,4 +1,4 @@
-/**
+/*
  * Licensed to the Apache Software Foundation (ASF) under one or more
  * contributor license agreements.  See the NOTICE file distributed with
  * this work for additional information regarding copyright ownership.
@@ -6,7 +6,7 @@
  * (the "License"); you may not use this file except in compliance with
  * the License.  You may obtain a copy of the License at
  *
- * http://www.apache.org/licenses/LICENSE-2.0
+ *     http://www.apache.org/licenses/LICENSE-2.0
  *
  * Unless required by applicable law or agreed to in writing, software
  * distributed under the License is distributed on an "AS IS" BASIS,
@@ -14,27 +14,9 @@
  * See the License for the specific language governing permissions and
  * limitations under the License.
  */
+package org.apache.rocketmq.store.logfile;
 
-package org.apache.rocketmq.common.message;
+import org.apache.rocketmq.store.ReferenceResource;
 
-import java.nio.ByteBuffer;
-
-public class MessageExtBatch extends MessageExt {
-
-    private static final long serialVersionUID = -2353110995348498537L;
-
-    public ByteBuffer wrap() {
-        assert getBody() != null;
-        return ByteBuffer.wrap(getBody(), 0, getBody().length);
-    }
-
-    private ByteBuffer encodedBuff;
-
-    public ByteBuffer getEncodedBuff() {
-        return encodedBuff;
-    }
-
-    public void setEncodedBuff(ByteBuffer encodedBuff) {
-        this.encodedBuff = encodedBuff;
-    }
+public abstract class AbstractMappedFile extends ReferenceResource implements MappedFile {
 }
diff --git a/store/src/main/java/org/apache/rocketmq/store/MappedFile.java b/store/src/main/java/org/apache/rocketmq/store/logfile/DefaultMappedFile.java
similarity index 75%
rename from store/src/main/java/org/apache/rocketmq/store/MappedFile.java
rename to store/src/main/java/org/apache/rocketmq/store/logfile/DefaultMappedFile.java
index 297271d..1ee8a88 100644
--- a/store/src/main/java/org/apache/rocketmq/store/MappedFile.java
+++ b/store/src/main/java/org/apache/rocketmq/store/logfile/DefaultMappedFile.java
@@ -14,7 +14,7 @@
  * See the License for the specific language governing permissions and
  * limitations under the License.
  */
-package org.apache.rocketmq.store;
+package org.apache.rocketmq.store.logfile;
 
 import com.sun.jna.NativeLong;
 import com.sun.jna.Pointer;
@@ -36,22 +36,28 @@ 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.MessageExt;
-import org.apache.rocketmq.common.message.MessageExtBatch;
-import org.apache.rocketmq.store.CommitLog.PutMessageContext;
+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.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.util.LibC;
 import sun.nio.ch.DirectBuffer;
 
-public class MappedFile extends ReferenceResource {
+public class DefaultMappedFile extends AbstractMappedFile {
     public static final int OS_PAGE_SIZE = 1024 * 4;
     protected static final InternalLogger log = InternalLoggerFactory.getLogger(LoggerName.STORE_LOGGER_NAME);
 
-    private static final AtomicLong TOTAL_MAPPED_VIRTUAL_MEMORY = new AtomicLong(0);
+    protected static final AtomicLong TOTAL_MAPPED_VIRTUAL_MEMORY = new AtomicLong(0);
 
-    private static final AtomicInteger TOTAL_MAPPED_FILES = new AtomicInteger(0);
+    protected static final AtomicInteger TOTAL_MAPPED_FILES = new AtomicInteger(0);
     protected final AtomicInteger wrotePosition = new AtomicInteger(0);
     protected final AtomicInteger committedPosition = new AtomicInteger(0);
-    private final AtomicInteger flushedPosition = new AtomicInteger(0);
+    protected final AtomicInteger flushedPosition = new AtomicInteger(0);
     protected int fileSize;
     protected FileChannel fileChannel;
     /**
@@ -59,21 +65,26 @@ public class MappedFile extends ReferenceResource {
      */
     protected ByteBuffer writeBuffer = null;
     protected TransientStorePool transientStorePool = null;
-    private String fileName;
-    private long fileFromOffset;
-    private File file;
-    private MappedByteBuffer mappedByteBuffer;
-    private volatile long storeTimestamp = 0;
-    private boolean firstCreateInQueue = false;
+    protected String fileName;
+    protected long fileFromOffset;
+    protected File file;
+    protected MappedByteBuffer mappedByteBuffer;
+    protected volatile long storeTimestamp = 0;
+    protected boolean firstCreateInQueue = false;
+    private long lastFlushTime = -1L;
 
-    public MappedFile() {
+    protected MappedByteBuffer mappedByteBufferWaitToClean = null;
+    protected long swapMapTime = 0L;
+    protected long mappedByteBufferAccessCountSinceLastSwap = 0L;
+
+    public DefaultMappedFile() {
     }
 
-    public MappedFile(final String fileName, final int fileSize) throws IOException {
+    public DefaultMappedFile(final String fileName, final int fileSize) throws IOException {
         init(fileName, fileSize);
     }
 
-    public MappedFile(final String fileName, final int fileSize,
+    public DefaultMappedFile(final String fileName, final int fileSize,
         final TransientStorePool transientStorePool) throws IOException {
         init(fileName, fileSize, transientStorePool);
     }
@@ -96,6 +107,7 @@ public class MappedFile extends ReferenceResource {
 
     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);
@@ -142,8 +154,9 @@ public class MappedFile extends ReferenceResource {
         return TOTAL_MAPPED_VIRTUAL_MEMORY.get();
     }
 
+    @Override
     public void init(final String fileName, final int fileSize,
-        final TransientStorePool transientStorePool) throws IOException {
+                     final TransientStorePool transientStorePool) throws IOException {
         init(fileName, fileSize);
         this.writeBuffer = transientStorePool.borrowBuffer();
         this.transientStorePool = transientStorePool;
@@ -177,23 +190,28 @@ public class MappedFile extends ReferenceResource {
         }
     }
 
+    @Override
     public long getLastModifiedTimestamp() {
         return this.file.lastModified();
     }
 
+    @Override
     public int getFileSize() {
         return fileSize;
     }
 
+    @Override
     public FileChannel getFileChannel() {
         return fileChannel;
     }
 
+    @Override
     public AppendMessageResult appendMessage(final MessageExtBrokerInner msg, final AppendMessageCallback cb,
             PutMessageContext putMessageContext) {
         return appendMessagesInner(msg, cb, putMessageContext);
     }
 
+    @Override
     public AppendMessageResult appendMessages(final MessageExtBatch messageExtBatch, final AppendMessageCallback cb,
             PutMessageContext putMessageContext) {
         return appendMessagesInner(messageExtBatch, cb, putMessageContext);
@@ -207,15 +225,17 @@ public class MappedFile extends ReferenceResource {
         int currentPos = this.wrotePosition.get();
 
         if (currentPos < this.fileSize) {
-            ByteBuffer byteBuffer = writeBuffer != null ? writeBuffer.slice() : this.mappedByteBuffer.slice();
+            ByteBuffer byteBuffer = appendMessageBuffer().slice();
             byteBuffer.position(currentPos);
             AppendMessageResult result;
-            if (messageExt instanceof MessageExtBrokerInner) {
-                result = cb.doAppend(this.getFileFromOffset(), byteBuffer, this.fileSize - currentPos,
-                        (MessageExtBrokerInner) messageExt, putMessageContext);
-            } else if (messageExt instanceof MessageExtBatch) {
+            if (messageExt instanceof MessageExtBatch && !((MessageExtBatch) messageExt).isInnerBatch()) {
+                // traditional batch message
                 result = cb.doAppend(this.getFileFromOffset(), byteBuffer, this.fileSize - currentPos,
                         (MessageExtBatch) messageExt, putMessageContext);
+            } else if (messageExt instanceof MessageExtBrokerInner) {
+                // traditional single message or newly introduced inner-batch message
+                result = cb.doAppend(this.getFileFromOffset(), byteBuffer, this.fileSize - currentPos,
+                        (MessageExtBrokerInner) messageExt, putMessageContext);
             } else {
                 return new AppendMessageResult(AppendMessageStatus.UNKNOWN_ERROR);
             }
@@ -227,24 +247,38 @@ public class MappedFile extends ReferenceResource {
         return new AppendMessageResult(AppendMessageStatus.UNKNOWN_ERROR);
     }
 
+    protected ByteBuffer appendMessageBuffer() {
+        this.mappedByteBufferAccessCountSinceLastSwap++;
+        return writeBuffer != null ? writeBuffer : this.mappedByteBuffer;
+    }
+
+    @Override
     public long getFileFromOffset() {
         return this.fileFromOffset;
     }
 
+    @Override
     public boolean appendMessage(final byte[] data) {
+        return appendMessage(data, 0, data.length);
+    }
+
+    @Override
+    public boolean appendMessage(ByteBuffer data) {
         int currentPos = this.wrotePosition.get();
+        int remaining = data.remaining();
 
-        if ((currentPos + data.length) <= this.fileSize) {
+        if ((currentPos + remaining) <= this.fileSize) {
             try {
                 this.fileChannel.position(currentPos);
-                this.fileChannel.write(ByteBuffer.wrap(data));
+                while (data.hasRemaining()) {
+                    this.fileChannel.write(data);
+                }
             } catch (Throwable e) {
                 log.error("Error occurred when append message to mappedFile.", e);
             }
-            this.wrotePosition.addAndGet(data.length);
+            this.wrotePosition.addAndGet(remaining);
             return true;
         }
-
         return false;
     }
 
@@ -254,13 +288,17 @@ public class MappedFile extends ReferenceResource {
      * @param offset The offset of the subarray to be used.
      * @param length The length of the subarray to be used.
      */
+    @Override
     public boolean appendMessage(final byte[] data, final int offset, final int length) {
         int currentPos = this.wrotePosition.get();
 
         if ((currentPos + length) <= this.fileSize) {
             try {
                 this.fileChannel.position(currentPos);
-                this.fileChannel.write(ByteBuffer.wrap(data, offset, length));
+                ByteBuffer buffer = ByteBuffer.wrap(data, offset, length);
+                while (buffer.hasRemaining()) {
+                    this.fileChannel.write(buffer);
+                }
             } catch (Throwable e) {
                 log.error("Error occurred when append message to mappedFile.", e);
             }
@@ -274,18 +312,22 @@ public class MappedFile extends ReferenceResource {
     /**
      * @return The current flushed position
      */
+    @Override
     public int flush(final int flushLeastPages) {
         if (this.isAbleToFlush(flushLeastPages)) {
             if (this.hold()) {
                 int value = getReadPosition();
 
                 try {
+                    this.mappedByteBufferAccessCountSinceLastSwap++;
+
                     //We only append data to fileChannel or mappedByteBuffer, never both.
                     if (writeBuffer != null || this.fileChannel.position() != 0) {
                         this.fileChannel.force(false);
                     } else {
                         this.mappedByteBuffer.force();
                     }
+                    this.lastFlushTime = System.currentTimeMillis();
                 } catch (Throwable e) {
                     log.error("Error occurred when force data to disk.", e);
                 }
@@ -300,6 +342,7 @@ public class MappedFile extends ReferenceResource {
         return this.getFlushedPosition();
     }
 
+    @Override
     public int commit(final int commitLeastPages) {
         if (writeBuffer == null) {
             //no need to commit data to file channel, so just regard wrotePosition as committedPosition.
@@ -371,22 +414,28 @@ public class MappedFile extends ReferenceResource {
         return write > flush;
     }
 
+    @Override
     public int getFlushedPosition() {
         return flushedPosition.get();
     }
 
+    @Override
     public void setFlushedPosition(int pos) {
         this.flushedPosition.set(pos);
     }
 
+    @Override
     public boolean isFull() {
         return this.fileSize == this.wrotePosition.get();
     }
 
+    @Override
     public SelectMappedBufferResult selectMappedBuffer(int pos, int size) {
         int readPosition = getReadPosition();
         if ((pos + size) <= readPosition) {
             if (this.hold()) {
+                this.mappedByteBufferAccessCountSinceLastSwap++;
+
                 ByteBuffer byteBuffer = this.mappedByteBuffer.slice();
                 byteBuffer.position(pos);
                 ByteBuffer byteBufferNew = byteBuffer.slice();
@@ -404,10 +453,12 @@ public class MappedFile extends ReferenceResource {
         return null;
     }
 
+    @Override
     public SelectMappedBufferResult selectMappedBuffer(int pos) {
         int readPosition = getReadPosition();
         if (pos < readPosition && pos >= 0) {
             if (this.hold()) {
+                this.mappedByteBufferAccessCountSinceLastSwap++;
                 ByteBuffer byteBuffer = this.mappedByteBuffer.slice();
                 byteBuffer.position(pos);
                 int size = readPosition - pos;
@@ -435,26 +486,31 @@ public class MappedFile extends ReferenceResource {
         }
 
         clean(this.mappedByteBuffer);
+        clean(this.mappedByteBufferWaitToClean);
+        this.mappedByteBufferWaitToClean = null;
         TOTAL_MAPPED_VIRTUAL_MEMORY.addAndGet(this.fileSize * (-1));
         TOTAL_MAPPED_FILES.decrementAndGet();
         log.info("unmap file[REF:" + currentRef + "] " + this.fileName + " OK");
         return true;
     }
 
+    @Override
     public boolean destroy(final long intervalForcibly) {
         this.shutdown(intervalForcibly);
 
         if (this.isCleanupOver()) {
             try {
+                long lastModified = getLastModifiedTimestamp();
                 this.fileChannel.close();
                 log.info("close file channel " + this.fileName + " OK");
 
                 long beginTime = System.currentTimeMillis();
                 boolean result = this.file.delete();
                 log.info("delete file[REF:" + this.getRefCount() + "] " + this.fileName
-                    + (result ? " OK, " : " Failed, ") + "W:" + this.getWrotePosition() + " M:"
-                    + this.getFlushedPosition() + ", "
-                    + UtilAll.computeElapsedTimeMilliseconds(beginTime));
+                        + (result ? " OK, " : " Failed, ") + "W:" + this.getWrotePosition() + " M:"
+                        + this.getFlushedPosition() + ", "
+                        + UtilAll.computeElapsedTimeMilliseconds(beginTime)
+                        + "," + (System.currentTimeMillis() - lastModified));
             } catch (Exception e) {
                 log.warn("close file channel " + this.fileName + " Failed. ", e);
             }
@@ -462,16 +518,18 @@ public class MappedFile extends ReferenceResource {
             return true;
         } else {
             log.warn("destroy mapped file[REF:" + this.getRefCount() + "] " + this.fileName
-                + " Failed. cleanupOver: " + this.cleanupOver);
+                    + " Failed. cleanupOver: " + this.cleanupOver);
         }
 
         return false;
     }
 
+    @Override
     public int getWrotePosition() {
         return wrotePosition.get();
     }
 
+    @Override
     public void setWrotePosition(int pos) {
         this.wrotePosition.set(pos);
     }
@@ -479,20 +537,25 @@ public class MappedFile extends ReferenceResource {
     /**
      * @return The max position which have valid data
      */
+    @Override
     public int getReadPosition() {
         return this.writeBuffer == null ? this.wrotePosition.get() : this.committedPosition.get();
     }
 
+    @Override
     public void setCommittedPosition(int pos) {
         this.committedPosition.set(pos);
     }
 
+    @Override
     public void warmMappedFile(FlushDiskType type, int pages) {
+        this.mappedByteBufferAccessCountSinceLastSwap++;
+
         long beginTime = System.currentTimeMillis();
         ByteBuffer byteBuffer = this.mappedByteBuffer.slice();
         int flush = 0;
         long time = System.currentTimeMillis();
-        for (int i = 0, j = 0; i < this.fileSize; i += MappedFile.OS_PAGE_SIZE, j++) {
+        for (int i = 0, j = 0; i < this.fileSize; i += DefaultMappedFile.OS_PAGE_SIZE, j++) {
             byteBuffer.put(i, (byte) 0);
             // force flush when flush disk type is sync
             if (type == FlushDiskType.SYNC_FLUSH) {
@@ -526,30 +589,99 @@ public class MappedFile extends ReferenceResource {
         this.mlock();
     }
 
+    @Override
+    public boolean swapMap() {
+        if (getRefCount() == 1 && this.mappedByteBufferWaitToClean == null) {
+
+            if (!hold()) {
+                log.warn("in swapMap, hold failed, fileName: " + this.fileName);
+                return false;
+            }
+            try {
+                this.mappedByteBufferWaitToClean = this.mappedByteBuffer;
+                this.mappedByteBuffer = this.fileChannel.map(FileChannel.MapMode.READ_WRITE, 0, fileSize);
+                this.mappedByteBufferAccessCountSinceLastSwap = 0L;
+                this.swapMapTime = System.currentTimeMillis();
+                log.info("swap file " + this.fileName + " success.");
+                return true;
+            } catch (Exception e) {
+                log.error("swapMap file " + this.fileName + " Failed. ", e);
+            } finally {
+                this.release();
+            }
+        } else {
+            log.info("Will not swap file: " + this.fileName + ", ref=" + getRefCount());
+        }
+        return false;
+    }
+
+    @Override
+    public void cleanSwapedMap(boolean force) {
+        try {
+            if (this.mappedByteBufferWaitToClean == null) {
+                return;
+            }
+            long minGapTime = 120 * 1000L;
+            long gapTime = System.currentTimeMillis() - this.swapMapTime;
+            if (!force && gapTime < minGapTime) {
+                Thread.sleep(minGapTime - gapTime);
+            }
+            clean(this.mappedByteBufferWaitToClean);
+            mappedByteBufferWaitToClean = null;
+            log.info("cleanSwapedMap file " + this.fileName + " success.");
+        } catch (Exception e) {
+            log.error("cleanSwapedMap file " + this.fileName + " Failed. ", e);
+        }
+    }
+
+    @Override
+    public long getRecentSwapMapTime() {
+        return 0;
+    }
+
+    @Override
+    public long getMappedByteBufferAccessCountSinceLastSwap() {
+        return this.mappedByteBufferAccessCountSinceLastSwap;
+    }
+
+    @Override
+    public long getLastFlushTime() {
+        return this.lastFlushTime;
+    }
+
+    @Override
     public String getFileName() {
         return fileName;
     }
 
+    @Override
     public MappedByteBuffer getMappedByteBuffer() {
+        this.mappedByteBufferAccessCountSinceLastSwap++;
         return mappedByteBuffer;
     }
 
+    @Override
     public ByteBuffer sliceByteBuffer() {
+        this.mappedByteBufferAccessCountSinceLastSwap++;
         return this.mappedByteBuffer.slice();
     }
 
+    @Override
     public long getStoreTimestamp() {
         return storeTimestamp;
     }
 
+    @Override
     public boolean isFirstCreateInQueue() {
         return firstCreateInQueue;
     }
 
+    @Override
     public void setFirstCreateInQueue(boolean firstCreateInQueue) {
         this.firstCreateInQueue = firstCreateInQueue;
     }
 
+    @Override
     public void mlock() {
         final long beginTime = System.currentTimeMillis();
         final long address = ((DirectBuffer) (this.mappedByteBuffer)).address();
@@ -565,6 +697,7 @@ public class MappedFile extends ReferenceResource {
         }
     }
 
+    @Override
     public void munlock() {
         final long beginTime = System.currentTimeMillis();
         final long address = ((DirectBuffer) (this.mappedByteBuffer)).address();
@@ -573,8 +706,8 @@ public class MappedFile extends ReferenceResource {
         log.info("munlock {} {} {} ret = {} time consuming = {}", address, this.fileName, this.fileSize, ret, System.currentTimeMillis() - beginTime);
     }
 
-    //testable
-    File getFile() {
+    @Override
+    public File getFile() {
         return this.file;
     }
 
@@ -582,4 +715,5 @@ public class MappedFile extends ReferenceResource {
     public String toString() {
         return this.fileName;
     }
+
 }
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
new file mode 100644
index 0000000..0b6a9dd
--- /dev/null
+++ b/store/src/main/java/org/apache/rocketmq/store/logfile/MappedFile.java
@@ -0,0 +1,331 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.rocketmq.store.logfile;
+
+import org.apache.rocketmq.store.AppendMessageCallback;
+import org.apache.rocketmq.store.AppendMessageResult;
+import org.apache.rocketmq.store.CommitLog;
+import org.apache.rocketmq.store.MessageExtBatch;
+import org.apache.rocketmq.store.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 java.io.File;
+import java.io.IOException;
+import java.nio.ByteBuffer;
+import java.nio.MappedByteBuffer;
+import java.nio.channels.FileChannel;
+
+public interface MappedFile {
+    /**
+     * Returns the file name of the {@code MappedFile}.
+     *
+     * @return the file name
+     */
+    String getFileName();
+
+    /**
+     * Returns the file size of the {@code MappedFile}.
+     *
+     * @return the file size
+     */
+    int getFileSize();
+
+    /**
+     * Returns the {@code FileChannel} behind the {@code MappedFile}.
+     *
+     * @return the file channel
+     */
+    FileChannel getFileChannel();
+
+    /**
+     * Returns true if this {@code MappedFile} is full and no new messages can be added.
+     *
+     * @return true if the file is full
+     */
+    boolean isFull();
+
+    /**
+     * Returns true if this {@code MappedFile} is available.
+     * <p>
+     * The mapped file will be not available if it's shutdown or destroyed.
+     *
+     * @return true if the file is available
+     */
+    boolean isAvailable();
+
+    /**
+     * Appends a message object to the current {@code MappedFile} with a specific call back.
+     *
+     * @param message a message to append
+     * @param messageCallback the specific call back to execute the real append action
+     * @param putMessageContext
+     * @return the append result
+     */
+    AppendMessageResult appendMessage(MessageExtBrokerInner message, AppendMessageCallback messageCallback, PutMessageContext putMessageContext);
+
+    /**
+     * Appends a batch message object to the current {@code MappedFile} with a specific call back.
+     *
+     * @param message a message to append
+     * @param messageCallback the specific call back to execute the real append action
+     * @param putMessageContext
+     * @return the append result
+     */
+    AppendMessageResult appendMessages(MessageExtBatch message, AppendMessageCallback messageCallback, PutMessageContext putMessageContext);
+
+    /**
+     * Appends a raw message data represents by a byte array to the current {@code MappedFile}.
+     *
+     * @param data the byte array to append
+     * @return true if success; false otherwise.
+     */
+    boolean appendMessage(byte[] data);
+
+    /**
+     * Appends a raw message data represents by a byte array to the current {@code MappedFile}.
+     *
+     * @param data the byte buffer to append
+     * @return true if success; false otherwise.
+     */
+    boolean appendMessage(ByteBuffer data);
+
+    /**
+     * Appends a raw message data represents by a byte array to the current {@code MappedFile},
+     * starting at the given offset in the array.
+     *
+     * @param data the byte array to append
+     * @param offset the offset within the array of the first byte to be read
+     * @param length the number of bytes to be read from the given array
+     * @return true if success; false otherwise.
+     */
+    boolean appendMessage(byte[] data, int offset, int length);
+
+    /**
+     * Returns the global offset of the current {code MappedFile}, it's a long value of the file name.
+     *
+     * @return the offset of this file
+     */
+    long getFileFromOffset();
+
+    /**
+     * Flushes the data in cache to disk immediately.
+     *
+     * @param flushLeastPages the least pages to flush
+     * @return the flushed position after the method call
+     */
+    int flush(int flushLeastPages);
+
+    /**
+     * Flushes the data in the secondary cache to page cache or disk immediately.
+     *
+     * @param commitLeastPages the least pages to commit
+     * @return the committed position after the method call
+     */
+    int commit(int commitLeastPages);
+
+    /**
+     * Selects a slice of the mapped byte buffer's sub-region behind the mapped file,
+     * starting at the given position.
+     *
+     * @param pos the given position
+     * @param size the size of the returned sub-region
+     * @return a {@code SelectMappedBufferResult} instance contains the selected slice
+     */
+    SelectMappedBufferResult selectMappedBuffer(int pos, int size);
+
+    /**
+     * Selects a slice of the mapped byte buffer's sub-region behind the mapped file,
+     * starting at the given position.
+     *
+     * @param pos the given position
+     * @return a {@code SelectMappedBufferResult} instance contains the selected slice
+     */
+    SelectMappedBufferResult selectMappedBuffer(int pos);
+
+    /**
+     * Returns the mapped byte buffer behind the mapped file.
+     *
+     * @return the mapped byte buffer
+     */
+    MappedByteBuffer getMappedByteBuffer();
+
+    /**
+     * Returns a slice of the mapped byte buffer behind the mapped file.
+     *
+     * @return the slice of the mapped byte buffer
+     */
+    ByteBuffer sliceByteBuffer();
+
+    /**
+     * Returns the store timestamp of the last message.
+     *
+     * @return the store timestamp
+     */
+    long getStoreTimestamp();
+
+    /**
+     * Returns the last modified timestamp of the file.
+     *
+     * @return the last modified timestamp
+     */
+    long getLastModifiedTimestamp();
+
+    /**
+     * 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
+     * @return true if success; false otherwise.
+     */
+    boolean destroy(long intervalForcibly);
+
+    /**
+     * Shutdowns the file and mark it unavailable.
+     *
+     * @param intervalForcibly If {@code true} then this method will shutdown the file forcibly and ignore the reference
+     */
+    void shutdown(long intervalForcibly);
+
+    /**
+     * Decreases the reference count by {@code 1} and clean up the mapped file if the reference count reaches at
+     * {@code 0}.
+     */
+    void release();
+
+    /**
+     * Increases the reference count by {@code 1}.
+     *
+     * @return true if success; false otherwise.
+     */
+    boolean hold();
+
+    /**
+     * Returns true if the current file is first mapped file of some consume queue.
+     *
+     * @return true or false
+     */
+    boolean isFirstCreateInQueue();
+
+    /**
+     * Sets the flag whether the current file is first mapped file of some consume queue.
+     *
+     * @param firstCreateInQueue true or false
+     */
+    void setFirstCreateInQueue(boolean firstCreateInQueue);
+
+    /**
+     * Returns the flushed position of this mapped file.
+     *
+     * @return the flushed posotion
+     */
+    int getFlushedPosition();
+
+    /**
+     * Sets the flushed position of this mapped file.
+     *
+     * @param flushedPosition the specific flushed position
+     */
+    void setFlushedPosition(int flushedPosition);
+
+    /**
+     * Returns the wrote position of this mapped file.
+     *
+     * @return the wrote position
+     */
+    int getWrotePosition();
+
+    /**
+     * Sets the wrote position of this mapped file.
+     *
+     * @param wrotePosition the specific wrote position
+     */
+    void setWrotePosition(int wrotePosition);
+
+    /**
+     * Returns the current max readable position of this mapped file.
+     *
+     * @return the max readable position
+     */
+    int getReadPosition();
+
+    /**
+     * Sets the committed position of this mapped file.
+     *
+     * @param committedPosition the specific committed position
+     */
+    void setCommittedPosition(int committedPosition);
+
+    /**
+     * Lock the mapped bytebuffer
+     */
+    void mlock();
+
+    /**
+     * Unlock the mapped bytebuffer
+     */
+    void munlock();
+
+    /**
+     * Warm up the mapped bytebuffer
+     * @param type
+     * @param pages
+     */
+    void warmMappedFile(FlushDiskType type, int pages);
+
+    /**
+     * Swap map
+     */
+    boolean swapMap();
+
+    /**
+     * Clean pageTable
+     */
+    void cleanSwapedMap(boolean force);
+
+    /**
+     * Get recent swap map time
+     */
+    long getRecentSwapMapTime();
+
+    /**
+     * Get recent MappedByteBuffer access count since last swap
+     */
+    long getMappedByteBufferAccessCountSinceLastSwap();
+
+    /**
+     * Get the underlying file
+     * @return
+     */
+    File getFile();
+
+    /**
+     * Get the last flush time
+     * @return
+     */
+    long getLastFlushTime();
+
+    /**
+     * Init mapped file
+     * @param fileName file name
+     * @param fileSize file size
+     * @param transientStorePool transient store pool
+     * @throws IOException
+     */
+    void init(String fileName, int fileSize, TransientStorePool transientStorePool) throws IOException;
+}
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
new file mode 100644
index 0000000..de0e636
--- /dev/null
+++ b/store/src/main/java/org/apache/rocketmq/store/queue/BatchConsumeQueue.java
@@ -0,0 +1,952 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF 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.queue;
+
+import org.apache.rocketmq.common.constant.LoggerName;
+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.MessageStore;
+import org.apache.rocketmq.store.SelectMappedBufferResult;
+import org.apache.rocketmq.store.config.BrokerRole;
+import org.apache.rocketmq.store.logfile.MappedFile;
+
+import java.io.File;
+import java.nio.ByteBuffer;
+import java.util.List;
+import java.util.Map;
+import java.util.concurrent.ConcurrentSkipListMap;
+
+public class BatchConsumeQueue implements ConsumeQueueInterface, FileQueueLifeCycle {
+    private static final InternalLogger log = InternalLoggerFactory.getLogger(LoggerName.STORE_LOGGER_NAME);
+
+    //position 8, size 4, tagscode 8, storetime 8, msgBaseOffset 8, batchSize 2, compactedOffset 4, reserved 4
+    public static final int CQ_STORE_UNIT_SIZE = 46;
+    public static final int MSG_STORE_TIME_OFFSET_INDEX = 20;
+    public static final int MSG_BASE_OFFSET_INDEX = 28;
+    public static final int MSG_BATCH_SIZE_INDEX = 36;
+    public static final int MSG_COMPACT_OFFSET_INDEX = 38;
+    private static final int MSG_COMPACT_OFFSET_LENGTH = 4;
+    public static final int INVALID_POS = -1;
+    final MappedFileQueue mappedFileQueue;
+    private MessageStore defaultMessageStore;
+    private final String topic;
+    private final int queueId;
+    private final ByteBuffer byteBufferItem;
+
+    private final String storePath;
+    private final int mappedFileSize;
+    private volatile long maxMsgPhyOffsetInCommitLog = -1;
+
+    private volatile long minLogicOffset = 0;
+
+    private volatile long maxOffsetInQueue = 0;
+    private volatile long minOffsetInQueue = -1;
+    private final int commitLogSize;
+
+    private ConcurrentSkipListMap<Long, MappedFile> offsetCache = new ConcurrentSkipListMap<>();
+    private ConcurrentSkipListMap<Long, MappedFile> timeCache = new ConcurrentSkipListMap<>();
+
+    public BatchConsumeQueue(
+        final String topic,
+        final int queueId,
+        final String storePath,
+        final int mappedFileSize,
+        final MessageStore defaultMessageStore) {
+        this.storePath = storePath;
+        this.mappedFileSize = mappedFileSize;
+        this.defaultMessageStore = defaultMessageStore;
+        this.commitLogSize = defaultMessageStore.getCommitLog().getCommitLogSize();
+
+        this.topic = topic;
+        this.queueId = queueId;
+
+        String queueDir = this.storePath
+            + File.separator + topic
+            + File.separator + queueId;
+
+        this.mappedFileQueue = new MappedFileQueue(queueDir, mappedFileSize, null);
+
+        this.byteBufferItem = ByteBuffer.allocate(CQ_STORE_UNIT_SIZE);
+    }
+
+    @Override
+    public boolean load() {
+        boolean result = this.mappedFileQueue.load();
+        log.info("Load batch consume queue {}-{} {} {}", topic, queueId, result ? "OK" : "Failed", mappedFileQueue.getMappedFiles().size());
+        return result;
+    }
+
+    private void refreshCache() {
+        if (!this.defaultMessageStore.getMessageStoreConfig().isSearchBcqByCacheEnable()) {
+            return ;
+        }
+        ConcurrentSkipListMap<Long, MappedFile> newOffsetCache = new ConcurrentSkipListMap<>();
+        ConcurrentSkipListMap<Long, MappedFile> newTimeCache = new ConcurrentSkipListMap<>();
+
+        List<MappedFile> mappedFiles = mappedFileQueue.getMappedFiles();
+        // iterate all BCQ files
+        for (int i = 0; i < mappedFiles.size(); i++) {
+            MappedFile bcq = mappedFiles.get(i);
+            if (isNewFile(bcq)) {
+                continue;
+            }
+
+            BatchOffsetIndex min = getMinMsgOffset(bcq, false, true);
+            newOffsetCache.put(min.getMsgOffset(), min.getMappedFile());
+            newTimeCache.put(min.getStoreTimestamp(), min.getMappedFile());
+        }
+
+        this.offsetCache = newOffsetCache;
+        this.timeCache = newTimeCache;
+
+        log.info("refreshCache for BCQ [Topic: {}, QueueId: {}]." +
+                        "offsetCacheSize: {}, minCachedMsgOffset: {}, maxCachedMsgOffset: {}, " +
+                        "timeCacheSize: {}, minCachedTime: {}, maxCachedTime: {}", this.topic, this.queueId,
+                this.offsetCache.size(), this.offsetCache.firstEntry(), this.offsetCache.lastEntry(),
+                this.timeCache.size(), this.timeCache.firstEntry(), this.timeCache.lastEntry());
+    }
+
+    private void destroyCache() {
+        this.offsetCache.clear();
+        this.timeCache.clear();
+
+        log.info("BCQ [Topic: {}, QueueId: {}]. Cache destroyed", this.topic, this.queueId);
+    }
+
+    private void cacheBcq(MappedFile bcq) {
+        try {
+            BatchOffsetIndex min = getMinMsgOffset(bcq, false, true);
+            this.offsetCache.put(min.getMsgOffset(), min.getMappedFile());
+            this.timeCache.put(min.getStoreTimestamp(), min.getMappedFile());
+        } catch (Exception e) {
+            log.error("Failed caching offset and time on BCQ [Topic: {}, QueueId: {}, File: {}]", this.topic, this.queueId, bcq);
+        }
+    }
+
+    private boolean isNewFile(MappedFile mappedFile) {
+        return mappedFile.getReadPosition() < CQ_STORE_UNIT_SIZE;
+    }
+
+    private MappedFile searchOffsetFromCache(long msgOffset) {
+        Map.Entry<Long, MappedFile> floorEntry = this.offsetCache.floorEntry(msgOffset);
+        if (floorEntry == null) {
+            // the offset is too small.
+            return null;
+        } else {
+            return floorEntry.getValue();
+        }
+    }
+
+    private MappedFile searchTimeFromCache(long time) {
+        Map.Entry<Long, MappedFile> floorEntry = this.timeCache.floorEntry(time);
+        if (floorEntry == null) {
+            // the timestamp is too small. so we decide to result first BCQ file.
+            return this.mappedFileQueue.getFirstMappedFile();
+        } else {
+            return floorEntry.getValue();
+        }
+    }
+
+    @Override
+    public void recover() {
+        final List<MappedFile> mappedFiles = this.mappedFileQueue.getMappedFiles();
+        if (!mappedFiles.isEmpty()) {
+            int index = mappedFiles.size() - 3;
+            if (index < 0)
+                index = 0;
+
+            int mappedFileSizeLogics = this.mappedFileSize;
+            MappedFile mappedFile = mappedFiles.get(index);
+            ByteBuffer byteBuffer = mappedFile.sliceByteBuffer();
+            long processOffset = mappedFile.getFileFromOffset();
+            long mappedFileOffset = 0;
+            while (true) {
+                for (int i = 0; i < mappedFileSizeLogics; i += CQ_STORE_UNIT_SIZE) {
+                    byteBuffer.position(i);
+                    long offset = byteBuffer.getLong();
+                    int size = byteBuffer.getInt();
+                    byteBuffer.getLong();//tagscode
+                    byteBuffer.getLong();//timestamp
+                    long msgBaseOffset = byteBuffer.getLong();
+                    short batchSize = byteBuffer.getShort();
+                    if (offset >= 0 && size > 0 && msgBaseOffset >= 0 && batchSize > 0) {
+                        mappedFileOffset = i + CQ_STORE_UNIT_SIZE;
+                        this.maxMsgPhyOffsetInCommitLog = offset;
+                    } else {
+                        log.info("Recover current batch consume queue file over, file:{} offset:{} size:{} msgBaseOffset:{} batchSize:{} mappedFileOffset:{}",
+                            mappedFile.getFileName(), offset, size, msgBaseOffset, batchSize, mappedFileOffset);
+                        break;
+                    }
+                }
+
+                if (mappedFileOffset == mappedFileSizeLogics) {
+                    index++;
+                    if (index >= mappedFiles.size()) {
+                        log.info("Recover last batch consume queue file over, last mapped file:{} ", mappedFile.getFileName());
+                        break;
+                    } else {
+                        mappedFile = mappedFiles.get(index);
+                        byteBuffer = mappedFile.sliceByteBuffer();
+                        processOffset = mappedFile.getFileFromOffset();
+                        mappedFileOffset = 0;
+                        log.info("Recover next batch consume queue file: " + mappedFile.getFileName());
+                    }
+                } else {
+                    log.info("Recover current batch consume queue file over:{} processOffset:{}", mappedFile.getFileName(), processOffset + mappedFileOffset);
+                    break;
+                }
+            }
+            processOffset += mappedFileOffset;
+            this.mappedFileQueue.setFlushedWhere(processOffset);
+            this.mappedFileQueue.setCommittedWhere(processOffset);
+            this.mappedFileQueue.truncateDirtyFiles(processOffset);
+            reviseMaxAndMinOffsetInQueue();
+        }
+    }
+
+    void reviseMinOffsetInQueue() {
+        MappedFile firstMappedFile = this.mappedFileQueue.getFirstMappedFile();
+        if (null == firstMappedFile) {
+            maxOffsetInQueue = 0;
+            minOffsetInQueue = -1;
+            minLogicOffset = -1;
+            log.info("reviseMinOffsetInQueue found firstMappedFile null, topic:{} queue:{}", topic, queueId);
+            return;
+        }
+        minLogicOffset = firstMappedFile.getFileFromOffset();
+        BatchOffsetIndex min = getMinMsgOffset(firstMappedFile, false, false);
+        minOffsetInQueue = null == min ? -1 : min.getMsgOffset();
+    }
+
+    void reviseMaxOffsetInQueue() {
+        MappedFile lastMappedFile = this.mappedFileQueue.getLastMappedFile();
+        BatchOffsetIndex max = getMaxMsgOffset(lastMappedFile, true, false);
+        if (null == max && this.mappedFileQueue.getMappedFiles().size() >= 2) {
+            MappedFile lastTwoMappedFile = this.mappedFileQueue.getMappedFiles().get(this.mappedFileQueue.getMappedFiles().size() - 2);
+            max = getMaxMsgOffset(lastTwoMappedFile, true, false);
+        }
+        maxOffsetInQueue = (null == max) ? 0 : max.getMsgOffset() + max.getBatchSize();
+    }
+
+    void reviseMaxAndMinOffsetInQueue() {
+        reviseMinOffsetInQueue();
+        reviseMaxOffsetInQueue();
+    }
+
+    @Override
+    public long getMaxPhysicOffset() {
+        return maxMsgPhyOffsetInCommitLog;
+    }
+
+    @Override
+    public long getMinLogicOffset() {
+        return minLogicOffset;
+    }
+
+    @Override
+    public ReferredIterator<CqUnit> iterateFrom(long startOffset) {
+        SelectMappedBufferResult sbr = getBatchMsgIndexBuffer(startOffset);
+        if (sbr == null) {
+            return null;
+        }
+        return new BatchConsumeQueueIterator(sbr);
+    }
+
+    @Override
+    public CqUnit get(long offset) {
+        ReferredIterator<CqUnit> it = iterateFrom(offset);
+        if (it == null) {
+            return null;
+        }
+        return it.nextAndRelease();
+    }
+
+    @Override
+    public CqUnit getEarliestUnit() {
+        return get(minOffsetInQueue);
+    }
+
+    @Override
+    public CqUnit getLatestUnit() {
+        return get(maxOffsetInQueue - 1);
+    }
+
+    @Override
+    public long getLastOffset() {
+        CqUnit latestUnit = getLatestUnit();
+        return latestUnit.getPos() + latestUnit.getSize();
+    }
+
+    @Override
+    public boolean isFirstFileAvailable() {
+        MappedFile mappedFile = this.mappedFileQueue.getFirstMappedFile();
+        if (mappedFile != null) {
+            return mappedFile.isAvailable();
+        }
+        return false;
+    }
+
+    @Override
+    public boolean isFirstFileExist() {
+        MappedFile mappedFile = this.mappedFileQueue.getFirstMappedFile();
+        return mappedFile != null;
+    }
+
+    @Override
+    public void truncateDirtyLogicFiles(long phyOffset) {
+
+        long oldMinOffset =  minOffsetInQueue;
+        long oldMaxOffset = maxOffsetInQueue;
+
+        int logicFileSize = this.mappedFileSize;
+
+        this.maxMsgPhyOffsetInCommitLog = phyOffset - 1;
+        boolean stop = false;
+        while (!stop) {
+            MappedFile mappedFile = this.mappedFileQueue.getLastMappedFile();
+            if (mappedFile != null) {
+                ByteBuffer byteBuffer = mappedFile.sliceByteBuffer();
+
+                mappedFile.setWrotePosition(0);
+                mappedFile.setCommittedPosition(0);
+                mappedFile.setFlushedPosition(0);
+
+                for (int i = 0; i < logicFileSize; i += CQ_STORE_UNIT_SIZE) {
+                    byteBuffer.position(i);
+                    long offset = byteBuffer.getLong();
+                    int size = byteBuffer.getInt();
+                    byteBuffer.getLong();//tagscode
+                    byteBuffer.getLong();//timestamp
+                    long msgBaseOffset = byteBuffer.getLong();
+                    short batchSize = byteBuffer.getShort();
+
+                    if (0 == i) {
+                        if (offset >= phyOffset) {
+                            this.mappedFileQueue.deleteLastMappedFile();
+                            break;
+                        } else {
+                            int pos = i + CQ_STORE_UNIT_SIZE;
+                            mappedFile.setWrotePosition(pos);
+                            mappedFile.setCommittedPosition(pos);
+                            mappedFile.setFlushedPosition(pos);
+                            this.maxMsgPhyOffsetInCommitLog = offset;
+                        }
+                    } else {
+                        if (offset >= 0 && size > 0 && msgBaseOffset >= 0 && batchSize > 0) {
+                            if (offset >= phyOffset) {
+                                stop = true;
+                                break;
+                            }
+
+                            int pos = i + CQ_STORE_UNIT_SIZE;
+                            mappedFile.setWrotePosition(pos);
+                            mappedFile.setCommittedPosition(pos);
+                            mappedFile.setFlushedPosition(pos);
+                            this.maxMsgPhyOffsetInCommitLog = offset;
+                            if (pos == logicFileSize) {
+                                stop = true;
+                                break;
+                            }
+                        } else {
+                            stop = true;
+                            break;
+                        }
+                    }
+                }
+            } else {
+                break;
+            }
+        }
+        reviseMaxAndMinOffsetInQueue();
+        log.info("Truncate batch logic file topic={} queue={} oldMinOffset={} oldMaxOffset={} minOffset={} maxOffset={} maxPhyOffsetHere={} maxPhyOffsetThere={}",
+            topic, queueId, oldMinOffset, oldMaxOffset, minOffsetInQueue, maxOffsetInQueue, maxMsgPhyOffsetInCommitLog, phyOffset);
+    }
+
+    @Override
+    public boolean flush(final int flushLeastPages) {
+        boolean result = this.mappedFileQueue.flush(flushLeastPages);
+        return result;
+    }
+
+    @Override
+    public int deleteExpiredFile(long offset) {
+        int cnt = this.mappedFileQueue.deleteExpiredFileByOffset(offset, CQ_STORE_UNIT_SIZE);
+        this.correctMinOffset(offset);
+        return cnt;
+    }
+
+    @Override
+    public void correctMinOffset(long phyMinOffset) {
+        reviseMinOffsetInQueue();
+        refreshCache();
+        long oldMinOffset = minOffsetInQueue;
+        MappedFile mappedFile = this.mappedFileQueue.getFirstMappedFile();
+        if (mappedFile != null) {
+            SelectMappedBufferResult result = mappedFile.selectMappedBuffer(0);
+            if (result != null) {
+                try {
+                    int startPos = result.getByteBuffer().position();
+                    for (int i = 0; i < result.getSize(); i += BatchConsumeQueue.CQ_STORE_UNIT_SIZE) {
+                        result.getByteBuffer().position(startPos + i);
+                        long offsetPy = result.getByteBuffer().getLong();
+                        result.getByteBuffer().getInt(); //size
+                        result.getByteBuffer().getLong();//tagscode
+                        result.getByteBuffer().getLong();//timestamp
+                        long msgBaseOffset = result.getByteBuffer().getLong();
+                        short batchSize = result.getByteBuffer().getShort();
+
+                        if (offsetPy < phyMinOffset) {
+                            this.minOffsetInQueue = msgBaseOffset + batchSize;
+                        } else {
+                            break;
+                        }
+                    }
+                } catch (Exception e) {
+                    log.error("Exception thrown when correctMinOffset", e);
+                } finally {
+                    result.release();
+                }
+            } else {
+                /**
+                 *  It will go to here under two conditions:
+                 1. the files number is 1, and it has no data
+                 2. the pull process hold the cq reference, and release it just the moment
+                 */
+                log.warn("Correct min offset found null cq file topic:{} queue:{} files:{} minOffset:{} maxOffset:{}",
+                    topic, queueId, this.mappedFileQueue.getMappedFiles().size(), minOffsetInQueue, maxOffsetInQueue);
+            }
+        }
+        if (oldMinOffset != this.minOffsetInQueue) {
+            log.info("BatchCQ Compute new minOffset:{} oldMinOffset{} topic:{} queue:{}", minOffsetInQueue, oldMinOffset, topic, queueId);
+        }
+    }
+
+    @Override
+    public void putMessagePositionInfoWrapper(DispatchRequest request) {
+        final int maxRetries = 30;
+        boolean canWrite = this.defaultMessageStore.getRunningFlags().isCQWriteable();
+        if (request.getMsgBaseOffset() < 0 || request.getBatchSize() < 0) {
+            log.warn("[NOTIFYME]unexpected dispacth request in batch consume queue topic:{} queue:{} offset:{}", topic, queueId, request.getCommitLogOffset());
+            return;
+        }
+        for (int i = 0; i < maxRetries && canWrite; i++) {
+            boolean result = this.putBatchMessagePositionInfo(request.getCommitLogOffset(),
+                request.getMsgSize(), request.getTagsCode(),
+                request.getStoreTimestamp(), request.getMsgBaseOffset(), request.getBatchSize());
+            if (result) {
+                if (BrokerRole.SLAVE == this.defaultMessageStore.getMessageStoreConfig().getBrokerRole()) {
+                    this.defaultMessageStore.getStoreCheckpoint().setPhysicMsgTimestamp(request.getStoreTimestamp());
+                }
+                this.defaultMessageStore.getStoreCheckpoint().setLogicsMsgTimestamp(request.getStoreTimestamp());
+                return;
+            } else {
+                // XXX: warn and notify me
+                log.warn("[NOTIFYME]put commit log position info to batch consume queue " + topic + ":" + queueId + " " + request.getCommitLogOffset()
+                    + " failed, retry " + i + " times");
+
+                try {
+                    Thread.sleep(1000);
+                } catch (InterruptedException e) {
+                    log.warn("", e);
+                }
+            }
+        }
+        // XXX: warn and notify me
+        log.error("[NOTIFYME]batch consume queue can not write, {} {}", this.topic, this.queueId);
+        this.defaultMessageStore.getRunningFlags().makeLogicsQueueError();
+    }
+
+    boolean putBatchMessagePositionInfo(final long offset, final int size, final long tagsCode, final long storeTime,
+        final long msgBaseOffset, final short batchSize) {
+
+        if (offset <= this.maxMsgPhyOffsetInCommitLog) {
+            if (System.currentTimeMillis() % 1000 == 0) {
+                log.warn("Build batch consume queue repeatedly, maxMsgPhyOffsetInCommitLog:{} offset:{} Topic: {} QID: {}",
+                    maxMsgPhyOffsetInCommitLog, offset, this.topic, this.queueId);
+            }
+            return true;
+        }
+
+        long behind = System.currentTimeMillis() - storeTime;
+        if (behind > 10000 && (System.currentTimeMillis() % 10000 == 0)) {
+            String flag = "LEVEL" + (behind / 10000);
+            log.warn("Reput behind {} topic:{} queue:{} offset:{} behind:{}", flag, topic, queueId, offset, behind);
+        }
+
+        this.byteBufferItem.flip();
+        this.byteBufferItem.limit(CQ_STORE_UNIT_SIZE);
+        this.byteBufferItem.putLong(offset);
+        this.byteBufferItem.putInt(size);
+        this.byteBufferItem.putLong(tagsCode);
+        this.byteBufferItem.putLong(storeTime);
+        this.byteBufferItem.putLong(msgBaseOffset);
+        this.byteBufferItem.putShort(batchSize);
+        this.byteBufferItem.putInt(INVALID_POS);
+        this.byteBufferItem.putInt(0); // 4 bytes reserved
+
+        MappedFile mappedFile = this.mappedFileQueue.getLastMappedFile(this.mappedFileQueue.getMaxOffset());
+        if (mappedFile != null) {
+            boolean isNewFile = isNewFile(mappedFile);
+            boolean appendRes = mappedFile.appendMessage(this.byteBufferItem.array());
+            if (appendRes) {
+                maxMsgPhyOffsetInCommitLog = offset;
+                maxOffsetInQueue = msgBaseOffset + batchSize;
+                //only the first time need to correct the minOffsetInQueue
+                //the other correctness is done in correctLogicMinoffsetService
+                if (mappedFile.isFirstCreateInQueue() && minOffsetInQueue == -1) {
+                    reviseMinOffsetInQueue();
+                }
+                if (isNewFile) {
+                    // cache new file
+                    this.cacheBcq(mappedFile);
+                }
+            }
+            return appendRes;
+        }
+        return false;
+    }
+
+    private BatchOffsetIndex getMinMsgOffset(MappedFile mappedFile, boolean getBatchSize, boolean getStoreTime) {
+        if (mappedFile.getReadPosition() < CQ_STORE_UNIT_SIZE) {
+            return null;
+        }
+        return getBatchOffsetIndexByPos(mappedFile, 0, getBatchSize, getStoreTime);
+    }
+
+    private BatchOffsetIndex getBatchOffsetIndexByPos(MappedFile mappedFile, int pos, boolean getBatchSize,
+        boolean getStoreTime) {
+        SelectMappedBufferResult sbr = mappedFile.selectMappedBuffer(pos);
+        try {
+            return new BatchOffsetIndex(mappedFile, pos, sbr.getByteBuffer().getLong(MSG_BASE_OFFSET_INDEX),
+                getBatchSize ? sbr.getByteBuffer().getShort(MSG_BATCH_SIZE_INDEX) : 0,
+                getStoreTime ? sbr.getByteBuffer().getLong(MSG_STORE_TIME_OFFSET_INDEX) : 0);
+        } finally {
+            sbr.release();
+        }
+    }
+
+    private BatchOffsetIndex getMaxMsgOffset(MappedFile mappedFile, boolean getBatchSize, boolean getStoreTime) {
+        if (mappedFile == null || mappedFile.getReadPosition() < CQ_STORE_UNIT_SIZE) {
+            return null;
+        }
+        int pos = mappedFile.getReadPosition() - CQ_STORE_UNIT_SIZE;
+        return getBatchOffsetIndexByPos(mappedFile, pos, getBatchSize, getStoreTime);
+    }
+
+    private static int ceil(int pos) {
+        return (pos / CQ_STORE_UNIT_SIZE) * CQ_STORE_UNIT_SIZE;
+    }
+
+    /**
+     * Gets SelectMappedBufferResult by batch-message offset
+     * Node: the caller is responsible for the release of SelectMappedBufferResult
+     * @param msgOffset
+     * @return SelectMappedBufferResult
+     */
+    public SelectMappedBufferResult getBatchMsgIndexBuffer(final long msgOffset) {
+        if (msgOffset >= maxOffsetInQueue) {
+            return null;
+        }
+        MappedFile targetBcq;
+        BatchOffsetIndex targetMinOffset;
+
+        // first check the last bcq file
+        MappedFile lastBcq = mappedFileQueue.getLastMappedFile();
+        BatchOffsetIndex minForLastBcq = getMinMsgOffset(lastBcq, false, false);
+        if (null != minForLastBcq && minForLastBcq.getMsgOffset() <= msgOffset) {
+            // found, it's the last bcq.
+            targetBcq = lastBcq;
+            targetMinOffset = minForLastBcq;
+        } else {
+            boolean searchBcqByCacheEnable = this.defaultMessageStore.getMessageStoreConfig().isSearchBcqByCacheEnable();
+            if (searchBcqByCacheEnable) {
+                // it's not the last BCQ file, so search it through cache.
+                targetBcq = this.searchOffsetFromCache(msgOffset);
+                // not found in cache
+                if (targetBcq == null) {
+                    MappedFile firstBcq = mappedFileQueue.getFirstMappedFile();
+                    BatchOffsetIndex minForFirstBcq = getMinMsgOffset(firstBcq, false, false);
+                    if (minForFirstBcq != null && minForFirstBcq.getMsgOffset() <= msgOffset && msgOffset < minForLastBcq.getMsgOffset()) {
+                        // old search logic
+                        targetBcq = this.searchOffsetFromFiles(msgOffset);
+                    }
+                    log.warn("cache is not working on BCQ [Topic: {}, QueueId: {}] for msgOffset: {}, targetBcq: {}", this.topic, this.queueId, msgOffset, targetBcq);
+                }
+            } else {
+                // old search logic
+                targetBcq = this.searchOffsetFromFiles(msgOffset);
+            }
+
+            if (targetBcq == null) {
+                return null;
+            }
+
+            targetMinOffset = getMinMsgOffset(targetBcq, false, false);
+        }
+
+        BatchOffsetIndex targetMaxOffset = getMaxMsgOffset(targetBcq, false, false);
+        if (null == targetMinOffset || null == targetMaxOffset) {
+            return null;
+        }
+
+        // then use binary search to find the indexed position
+        SelectMappedBufferResult sbr = targetMinOffset.getMappedFile().selectMappedBuffer(0);
+        try {
+            ByteBuffer byteBuffer = sbr.getByteBuffer();
+            int left = targetMinOffset.getIndexPos(), right = targetMaxOffset.getIndexPos();
+            int mid = binarySearch(byteBuffer, left, right, CQ_STORE_UNIT_SIZE, MSG_BASE_OFFSET_INDEX, msgOffset);
+            if (mid != -1) {
+                // return a buffer that needs to be released manually.
+                return targetMinOffset.getMappedFile().selectMappedBuffer(mid);
+            }
+        } finally {
+            sbr.release();
+        }
+        return null;
+    }
+
+    private MappedFile searchOffsetFromFiles(long msgOffset) {
+        MappedFile targetBcq = null;
+        // find the mapped file one by one reversely
+        int mappedFileNum = this.mappedFileQueue.getMappedFiles().size();
+        for (int i = mappedFileNum - 1; i >= 0; i--) {
+            MappedFile mappedFile = mappedFileQueue.getMappedFiles().get(i);
+            BatchOffsetIndex tmpMinMsgOffset = getMinMsgOffset(mappedFile, false, false);
+            if (null != tmpMinMsgOffset && tmpMinMsgOffset.getMsgOffset() <= msgOffset) {
+                targetBcq = mappedFile;
+                break;
+            }
+        }
+
+        return targetBcq;
+    }
+
+    /**
+     * Find the message whose timestamp is the smallest, greater than or equal to the given time.
+     * @param timestamp
+     * @return
+     */
+    @Override
+    public long getOffsetInQueueByTime(final long timestamp) {
+        MappedFile targetBcq;
+        BatchOffsetIndex targetMinOffset;
+
+        // first check the last bcq
+        MappedFile lastBcq = mappedFileQueue.getLastMappedFile();
+        BatchOffsetIndex minForLastBcq = getMinMsgOffset(lastBcq, false, true);
+        if (null != minForLastBcq && minForLastBcq.getStoreTimestamp() <= timestamp) {
+            // found, it's the last bcq.
+            targetBcq = lastBcq;
+            targetMinOffset = minForLastBcq;
+        } else {
+            boolean searchBcqByCacheEnable = this.defaultMessageStore.getMessageStoreConfig().isSearchBcqByCacheEnable();
+            if (searchBcqByCacheEnable) {
+                // it's not the last BCQ file, so search it through cache.
+                targetBcq = this.searchTimeFromCache(timestamp);
+                if (targetBcq == null) {
+                    // not found in cache
+                    MappedFile firstBcq = mappedFileQueue.getFirstMappedFile();
+                    BatchOffsetIndex minForFirstBcq = getMinMsgOffset(firstBcq, false, true);
+                    if (minForFirstBcq != null && minForFirstBcq.getStoreTimestamp() <= timestamp && timestamp < minForLastBcq.getStoreTimestamp()) {
+                        // old search logic
+                        targetBcq = this.searchTimeFromFiles(timestamp);
+                    }
+                    log.warn("cache is not working on BCQ [Topic: {}, QueueId: {}] for timestamp: {}, targetBcq: {}", this.topic, this.queueId, timestamp, targetBcq);
+                }
+            } else {
+                // old search logic
+                targetBcq = this.searchTimeFromFiles(timestamp);
+            }
+
+            if (targetBcq == null) {
+                return -1;
+            }
+            targetMinOffset = getMinMsgOffset(targetBcq, false, true);
+        }
+
+        BatchOffsetIndex targetMaxOffset = getMaxMsgOffset(targetBcq, false, true);
+        if (null == targetMinOffset || null == targetMaxOffset) {
+            return -1;
+        }
+
+        //then use binary search to find the indexed position
+        SelectMappedBufferResult sbr = targetMinOffset.getMappedFile().selectMappedBuffer(0);
+        try {
+            ByteBuffer byteBuffer = sbr.getByteBuffer();
+            int left = targetMinOffset.getIndexPos(), right = targetMaxOffset.getIndexPos();
+            int mid = binarySearchRight(byteBuffer, left, right, CQ_STORE_UNIT_SIZE, MSG_STORE_TIME_OFFSET_INDEX, timestamp);
+            if (mid != -1) {
+                return byteBuffer.getLong(mid + MSG_BASE_OFFSET_INDEX);
+            }
+        } finally {
+            sbr.release();
+        }
+
+        return -1;
+    }
+
+    private MappedFile searchTimeFromFiles(long timestamp) {
+        MappedFile targetBcq = null;
+
+        int mappedFileNum = this.mappedFileQueue.getMappedFiles().size();
+        for (int i = mappedFileNum - 1; i >= 0; i--) {
+            MappedFile mappedFile = mappedFileQueue.getMappedFiles().get(i);
+            BatchOffsetIndex tmpMinMsgOffset = getMinMsgOffset(mappedFile, false, true);
+            if (tmpMinMsgOffset == null) {
+                //Maybe the new file
+                continue;
+            }
+            BatchOffsetIndex tmpMaxMsgOffset = getMaxMsgOffset(mappedFile, false, true);
+            //Here should not be null
+            if (tmpMaxMsgOffset == null) {
+                break;
+            }
+            if (tmpMaxMsgOffset.getStoreTimestamp() >= timestamp) {
+                if (tmpMinMsgOffset.getStoreTimestamp() <= timestamp) {
+                    targetBcq = mappedFile;
+                    break;
+                } else {
+                    if (i - 1 < 0) {
+                        //This is the first file
+                        targetBcq = mappedFile;
+                        break;
+                    } else {
+                        //The min timestamp of this file is larger than the given timestamp, so check the next file
+                        continue;
+                    }
+                }
+            } else {
+                //The max timestamp of this file is smaller than the given timestamp, so double check the previous file
+                if (i + 1 <=  mappedFileNum - 1) {
+                    mappedFile =  mappedFileQueue.getMappedFiles().get(i + 1);
+                    targetBcq = mappedFile;
+                    break;
+                } else {
+                    //There is no timestamp larger than the given timestamp
+                    break;
+                }
+            }
+        }
+
+        return targetBcq;
+    }
+
+    /**
+     * Find the offset of which the value is equal or larger than the given targetValue.
+     * If there are many values equal to the target, then find the earliest one.
+     */
+    public static int binarySearchRight(ByteBuffer byteBuffer, int left, int right, final int unitSize, final int unitShift,
+        long targetValue) {
+        int mid = -1;
+        while (left <= right) {
+            mid = ceil((left + right) / 2);
+            long tmpValue = byteBuffer.getLong(mid + unitShift);
+            if (mid == right) {
+                //Means left and the right are the same
+                if (tmpValue >= targetValue) {
+                    return mid;
+                } else {
+                    return -1;
+                }
+            } else if (mid == left) {
+                //Means the left + unitSize = right
+                if (tmpValue >= targetValue) {
+                    return mid;
+                } else {
+                    left =  mid + unitSize;
+                }
+            } else {
+                //mid is actully in the mid
+                if (tmpValue < targetValue) {
+                    left = mid + unitSize;
+                } else {
+                    right = mid;
+                }
+            }
+        }
+        return -1;
+    }
+
+    /**
+     * Here is vulnerable, the min value of the bytebuffer must be smaller or equal then the given value.
+     * Otherwise it may get -1
+     */
+    private int binarySearch(ByteBuffer byteBuffer, int left, int right, final int unitSize, final int unitShift,
+        long targetValue) {
+        int maxRight = right;
+        int mid = -1;
+        while (left <= right) {
+            mid = ceil((left + right) / 2);
+            long tmpValue = byteBuffer.getLong(mid + unitShift);
+            if (tmpValue == targetValue) {
+                return mid;
+            }
+            if (tmpValue > targetValue) {
+                right = mid - unitSize;
+            } else {
+                if (mid == left) {
+                    //the binary search is converging to the left, so maybe the one on the right of mid is the exactly correct one
+                    if (mid + unitSize <= maxRight
+                        && byteBuffer.getLong(mid + unitSize + unitShift) <= targetValue) {
+                        return mid + unitSize;
+                    } else {
+                        return mid;
+                    }
+                } else {
+                    left = mid;
+                }
+            }
+        }
+        return -1;
+    }
+
+    private class BatchConsumeQueueIterator implements ReferredIterator<CqUnit> {
+        private SelectMappedBufferResult sbr;
+        private int relativePos = 0;
+
+        public BatchConsumeQueueIterator(SelectMappedBufferResult sbr) {
+            this.sbr = sbr;
+            if (sbr != null && sbr.getByteBuffer() != null) {
+                relativePos = sbr.getByteBuffer().position();
+            }
+        }
+
+        @Override
+        public boolean hasNext() {
+            if (sbr == null || sbr.getByteBuffer() == null) {
+                return false;
+            }
+
+            return sbr.getByteBuffer().hasRemaining();
+        }
+
+        @Override
+        public CqUnit next() {
+            if (!hasNext()) {
+                return null;
+            }
+            ByteBuffer tmpBuffer = sbr.getByteBuffer().slice();
+            tmpBuffer.position(MSG_COMPACT_OFFSET_INDEX);
+            ByteBuffer compactOffsetStoreBuffer = tmpBuffer.slice();
+            compactOffsetStoreBuffer.limit(MSG_COMPACT_OFFSET_LENGTH);
+
+            int relativePos = sbr.getByteBuffer().position();
+            long offsetPy = sbr.getByteBuffer().getLong();
+            int sizePy = sbr.getByteBuffer().getInt();
+            long tagsCode = sbr.getByteBuffer().getLong(); //tagscode
+            sbr.getByteBuffer().getLong();//timestamp
+            long msgBaseOffset = sbr.getByteBuffer().getLong();
+            short batchSize = sbr.getByteBuffer().getShort();
+            int compactedOffset = sbr.getByteBuffer().getInt();
+            sbr.getByteBuffer().position(relativePos + CQ_STORE_UNIT_SIZE);
+
+            return new CqUnit(msgBaseOffset, offsetPy, sizePy, tagsCode, batchSize, compactedOffset, compactOffsetStoreBuffer);
+        }
+
+        @Override
+        public void remove() {
+            throw new UnsupportedOperationException("remove");
+        }
+
+        @Override
+        public void release() {
+            if (sbr != null) {
+                sbr.release();
+                sbr = null;
+            }
+        }
+
+        @Override
+        public CqUnit nextAndRelease() {
+            try {
+                return next();
+            } finally {
+                release();
+            }
+        }
+    }
+
+    @Override
+    public String getTopic() {
+        return topic;
+    }
+
+    @Override
+    public int getQueueId() {
+        return queueId;
+    }
+
+    @Override
+    public CQType getCQType() {
+        return CQType.BatchCQ;
+    }
+
+    @Override
+    public void destroy() {
+        this.maxMsgPhyOffsetInCommitLog = -1;
+        this.minOffsetInQueue = -1;
+        this.maxOffsetInQueue = 0;
+        this.mappedFileQueue.destroy();
+        this.destroyCache();
+    }
+
+    @Override
+    public long getMessageTotalInQueue() {
+        return this.getMaxOffsetInQueue() - this.getMinOffsetInQueue();
+    }
+
+    @Override
+    public long rollNextFile(long offset) {
+        return 0;
+    }
+
+    /**
+     * Batch msg offset (deep logic offset)
+     * @return max deep offset
+     */
+    @Override
+    public long getMaxOffsetInQueue() {
+        return maxOffsetInQueue;
+    }
+
+    @Override
+    public long getMinOffsetInQueue() {
+        return minOffsetInQueue;
+    }
+
+    @Override
+    public void checkSelf() {
+        mappedFileQueue.checkSelf();
+    }
+
+    @Override
+    public void swapMap(int reserveNum, long forceSwapIntervalMs, long normalSwapIntervalMs) {
+        mappedFileQueue.swapMap(reserveNum, forceSwapIntervalMs, normalSwapIntervalMs);
+    }
+
+    @Override
+    public void cleanSwappedMap(long forceCleanSwapIntervalMs) {
+        mappedFileQueue.cleanSwappedMap(forceCleanSwapIntervalMs);
+    }
+
+    public MappedFileQueue getMappedFileQueue() {
+        return mappedFileQueue;
+    }
+}
diff --git a/store/src/main/java/org/apache/rocketmq/store/queue/BatchOffsetIndex.java b/store/src/main/java/org/apache/rocketmq/store/queue/BatchOffsetIndex.java
new file mode 100644
index 0000000..8ca85c6
--- /dev/null
+++ b/store/src/main/java/org/apache/rocketmq/store/queue/BatchOffsetIndex.java
@@ -0,0 +1,57 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.rocketmq.store.queue;
+
+import org.apache.rocketmq.store.logfile.MappedFile;
+
+public class BatchOffsetIndex {
+
+    private final MappedFile mappedFile;
+    private final int indexPos;
+    private final long msgOffset;
+    private final short batchSize;
+    private final long storeTimestamp;
+
+    public BatchOffsetIndex(MappedFile file, int pos, long msgOffset, short size, long storeTimestamp) {
+        mappedFile = file;
+        indexPos = pos;
+        this.msgOffset = msgOffset;
+        batchSize = size;
+        this.storeTimestamp = storeTimestamp;
+    }
+
+    public MappedFile getMappedFile() {
+        return mappedFile;
+    }
+
+    public int getIndexPos() {
+        return indexPos;
+    }
+
+    public long getMsgOffset() {
+        return msgOffset;
+    }
+
+    public short getBatchSize() {
+        return batchSize;
+    }
+
+    public long getStoreTimestamp() {
+        return storeTimestamp;
+    }
+}
diff --git a/common/src/main/java/org/apache/rocketmq/common/message/MessageExtBatch.java b/store/src/main/java/org/apache/rocketmq/store/queue/CQType.java
similarity index 55%
copy from common/src/main/java/org/apache/rocketmq/common/message/MessageExtBatch.java
copy to store/src/main/java/org/apache/rocketmq/store/queue/CQType.java
index a2713cb..efd65af 100644
--- a/common/src/main/java/org/apache/rocketmq/common/message/MessageExtBatch.java
+++ b/store/src/main/java/org/apache/rocketmq/store/queue/CQType.java
@@ -1,4 +1,4 @@
-/**
+/*
  * Licensed to the Apache Software Foundation (ASF) under one or more
  * contributor license agreements.  See the NOTICE file distributed with
  * this work for additional information regarding copyright ownership.
@@ -6,7 +6,7 @@
  * (the "License"); you may not use this file except in compliance with
  * the License.  You may obtain a copy of the License at
  *
- * http://www.apache.org/licenses/LICENSE-2.0
+ *     http://www.apache.org/licenses/LICENSE-2.0
  *
  * Unless required by applicable law or agreed to in writing, software
  * distributed under the License is distributed on an "AS IS" BASIS,
@@ -15,26 +15,10 @@
  * limitations under the License.
  */
 
-package org.apache.rocketmq.common.message;
+package org.apache.rocketmq.store.queue;
 
-import java.nio.ByteBuffer;
-
-public class MessageExtBatch extends MessageExt {
-
-    private static final long serialVersionUID = -2353110995348498537L;
-
-    public ByteBuffer wrap() {
-        assert getBody() != null;
-        return ByteBuffer.wrap(getBody(), 0, getBody().length);
-    }
-
-    private ByteBuffer encodedBuff;
-
-    public ByteBuffer getEncodedBuff() {
-        return encodedBuff;
-    }
-
-    public void setEncodedBuff(ByteBuffer encodedBuff) {
-        this.encodedBuff = encodedBuff;
-    }
+public enum CQType {
+    SimpleCQ,
+    BatchCQ,
+    MillionCQ;
 }
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
new file mode 100644
index 0000000..48f717d
--- /dev/null
+++ b/store/src/main/java/org/apache/rocketmq/store/queue/ConsumeQueueInterface.java
@@ -0,0 +1,112 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.rocketmq.store.queue;
+
+public interface ConsumeQueueInterface {
+    /**
+     * Get the topic name
+     * @return the topic this cq belongs to.
+     */
+    String getTopic();
+
+    /**
+     * Get queue id
+     * @return the queue id this cq belongs to.
+     */
+    int getQueueId();
+
+    /**
+     * Get the units from the start offset.
+     *
+     * @param startIndex start index
+     * @return the unit iterateFrom
+     */
+    ReferredIterator<CqUnit> iterateFrom(long startIndex);
+
+    /**
+     * Get cq unit at specified index
+     * @param index index
+     * @return the cq unit at index
+     */
+    CqUnit get(long index);
+
+    /**
+     * Get earliest cq unit
+     * @return earliest cq unit
+     */
+    CqUnit getEarliestUnit();
+
+    /**
+     * Get last cq unit
+     * @return last cq unit
+     */
+    CqUnit getLatestUnit();
+
+    /**
+     * Get last commit log offset
+     * @return last commit log offset
+     */
+    long getLastOffset();
+
+    /**
+     * Get min offset(index) in queue
+     * @return the min offset(index) in queue
+     */
+    long getMinOffsetInQueue();
+
+    /**
+     * Get max offset(index) in queue
+     * @return the max offset(index) in queue
+     */
+    long getMaxOffsetInQueue();
+
+    /**
+     * Get total message count
+     * @return total message count
+     */
+    long getMessageTotalInQueue();
+
+    /**
+     * Get the message whose timestamp is the smallest, greater than or equal to the given time.
+     * @param timestamp timestamp
+     * @return the offset(index)
+     */
+    long getOffsetInQueueByTime(final long timestamp);
+
+    /**
+     * The max physical offset of commitlog has been dispatched to this queue.
+     * It should be exclusive.
+     *
+     * @return the max physical offset point to commitlog
+     */
+    long getMaxPhysicOffset();
+
+    /**
+     * Usually, the cq files are not exactly consistent with the commitlog, there maybe some redundant data in the first
+     * cq file.
+     *
+     * @return the minimal effective pos of the cq file.
+     */
+    long getMinLogicOffset();
+
+    /**
+     * Get cq type
+     * @return cq type
+     */
+    CQType getCQType();
+}
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
new file mode 100644
index 0000000..bf42e74
--- /dev/null
+++ b/store/src/main/java/org/apache/rocketmq/store/queue/ConsumeQueueStore.java
@@ -0,0 +1,176 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF 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.queue;
+
+import org.apache.rocketmq.store.ConsumeQueue;
+import org.apache.rocketmq.store.DefaultMessageStore;
+import org.apache.rocketmq.store.DispatchRequest;
+import org.apache.rocketmq.store.MessageStore;
+import org.apache.rocketmq.store.StoreUtil;
+import org.apache.rocketmq.store.config.MessageStoreConfig;
+import org.apache.rocketmq.store.config.StorePathConfigHelper;
+
+import java.util.concurrent.ConcurrentHashMap;
+import java.util.concurrent.ConcurrentMap;
+
+public class ConsumeQueueStore {
+
+    protected final MessageStore messageStore;
+    protected final MessageStoreConfig messageStoreConfig;
+    protected final ConcurrentMap<String/* topic */, ConcurrentMap<Integer/* queueId */, ConsumeQueueInterface>> consumeQueueTable;
+
+    public ConsumeQueueStore(MessageStore messageStore, MessageStoreConfig messageStoreConfig, ConcurrentMap<String, ConcurrentMap<Integer, ConsumeQueueInterface>> consumeQueueTable) {
+        this.messageStore = messageStore;
+        this.messageStoreConfig = messageStoreConfig;
+        this.consumeQueueTable = consumeQueueTable;
+    }
+
+    private FileQueueLifeCycle getLifeCycle(String topic, int queueId) {
+        return (FileQueueLifeCycle) findOrCreateConsumeQueue(topic, queueId);
+    }
+
+    public long rollNextFile(ConsumeQueueInterface consumeQueue, final long offset) {
+        FileQueueLifeCycle fileQueueLifeCycle = getLifeCycle(consumeQueue.getTopic(), consumeQueue.getQueueId());
+        return fileQueueLifeCycle.rollNextFile(offset);
+    }
+
+    public void correctMinOffset(ConsumeQueueInterface consumeQueue, long minCommitLogOffset) {
+        FileQueueLifeCycle fileQueueLifeCycle = getLifeCycle(consumeQueue.getTopic(), consumeQueue.getQueueId());
+        fileQueueLifeCycle.correctMinOffset(minCommitLogOffset);
+    }
+
+    /**
+     * Apply the dispatched request and build the consume queue.
+     * This function should be idempotent.
+     *
+     * @param request
+     */
+    public void putMessagePositionInfoWrapper(ConsumeQueueInterface consumeQueue, DispatchRequest request) {
+        FileQueueLifeCycle fileQueueLifeCycle = getLifeCycle(consumeQueue.getTopic(), consumeQueue.getQueueId());
+        fileQueueLifeCycle.putMessagePositionInfoWrapper(request);
+    }
+
+    public boolean load(ConsumeQueueInterface consumeQueue) {
+        FileQueueLifeCycle fileQueueLifeCycle = getLifeCycle(consumeQueue.getTopic(), consumeQueue.getQueueId());
+        return fileQueueLifeCycle.load();
+    }
+
+    public void recover(ConsumeQueueInterface consumeQueue) {
+        FileQueueLifeCycle fileQueueLifeCycle = getLifeCycle(consumeQueue.getTopic(), consumeQueue.getQueueId());
+        fileQueueLifeCycle.recover();
+    }
+
+    public void checkSelf(ConsumeQueueInterface consumeQueue) {
+        FileQueueLifeCycle fileQueueLifeCycle = getLifeCycle(consumeQueue.getTopic(), consumeQueue.getQueueId());
+        fileQueueLifeCycle.checkSelf();
+    }
+
+    public boolean flush(ConsumeQueueInterface consumeQueue, int flushLeastPages) {
+        FileQueueLifeCycle fileQueueLifeCycle = getLifeCycle(consumeQueue.getTopic(), consumeQueue.getQueueId());
+        return fileQueueLifeCycle.flush(flushLeastPages);
+    }
+
+    public void destroy(ConsumeQueueInterface consumeQueue) {
+        FileQueueLifeCycle fileQueueLifeCycle = getLifeCycle(consumeQueue.getTopic(), consumeQueue.getQueueId());
+        fileQueueLifeCycle.destroy();
+    }
+
+    public int deleteExpiredFile(ConsumeQueueInterface consumeQueue, long minOffset) {
+        FileQueueLifeCycle fileQueueLifeCycle = getLifeCycle(consumeQueue.getTopic(), consumeQueue.getQueueId());
+        return fileQueueLifeCycle.deleteExpiredFile(minOffset);
+    }
+
+    public void truncateDirtyLogicFiles(ConsumeQueueInterface consumeQueue, long phyOffset) {
+        FileQueueLifeCycle fileQueueLifeCycle = getLifeCycle(consumeQueue.getTopic(), consumeQueue.getQueueId());
+        fileQueueLifeCycle.truncateDirtyLogicFiles(phyOffset);
+    }
+
+    public void swapMap(ConsumeQueueInterface consumeQueue, int reserveNum, long forceSwapIntervalMs, long normalSwapIntervalMs) {
+        FileQueueLifeCycle fileQueueLifeCycle = getLifeCycle(consumeQueue.getTopic(), consumeQueue.getQueueId());
+        fileQueueLifeCycle.swapMap(reserveNum, forceSwapIntervalMs, normalSwapIntervalMs);
+    }
+
+    public void cleanSwappedMap(ConsumeQueueInterface consumeQueue, long forceCleanSwapIntervalMs) {
+        FileQueueLifeCycle fileQueueLifeCycle = getLifeCycle(consumeQueue.getTopic(), consumeQueue.getQueueId());
+        fileQueueLifeCycle.cleanSwappedMap(forceCleanSwapIntervalMs);
+    }
+
+    public boolean isFirstFileAvailable(ConsumeQueueInterface consumeQueue) {
+        FileQueueLifeCycle fileQueueLifeCycle = getLifeCycle(consumeQueue.getTopic(), consumeQueue.getQueueId());
+        return fileQueueLifeCycle.isFirstFileAvailable();
+    }
+
+    public boolean isFirstFileExist(ConsumeQueueInterface consumeQueue) {
+        FileQueueLifeCycle fileQueueLifeCycle = getLifeCycle(consumeQueue.getTopic(), consumeQueue.getQueueId());
+        return fileQueueLifeCycle.isFirstFileExist();
+    }
+
+    public ConsumeQueueInterface findOrCreateConsumeQueue(String topic, int queueId) {
+        return doFindOrCreateConsumeQueue(topic, queueId);
+    }
+
+    private ConsumeQueueInterface doFindOrCreateConsumeQueue(String topic, int queueId) {
+
+        ConcurrentMap<Integer, ConsumeQueueInterface> map = consumeQueueTable.get(topic);
+        if (null == map) {
+            ConcurrentMap<Integer, ConsumeQueueInterface> newMap = new ConcurrentHashMap<Integer, ConsumeQueueInterface>(128);
+            ConcurrentMap<Integer, ConsumeQueueInterface> oldMap = consumeQueueTable.putIfAbsent(topic, newMap);
+            if (oldMap != null) {
+                map = oldMap;
+            } else {
+                map = newMap;
+            }
+        }
+
+        ConsumeQueueInterface logic = map.get(queueId);
+        if (logic != null) {
+            return logic;
+        }
+
+        ConsumeQueueInterface newLogic;
+
+        if (StoreUtil.isStreamMode(this.messageStore)) {
+            newLogic = new BatchConsumeQueue(
+                    topic,
+                    queueId,
+                    StorePathConfigHelper.getStorePathBatchConsumeQueue(this.messageStoreConfig.getStorePathRootDir()),
+                    this.messageStoreConfig.getMapperFileSizeBatchConsumeQueue(),
+                    this.messageStore);
+            ConsumeQueueInterface oldLogic = map.putIfAbsent(queueId, newLogic);
+            if (oldLogic != null) {
+                logic = oldLogic;
+            } else {
+                logic = newLogic;
+            }
+        } else {
+            newLogic = new ConsumeQueue(
+                    topic,
+                    queueId,
+                    StorePathConfigHelper.getStorePathConsumeQueue(this.messageStoreConfig.getStorePathRootDir()),
+                    this.messageStoreConfig.getMappedFileSizeConsumeQueue(),
+                    (DefaultMessageStore) this.messageStore);
+            ConsumeQueueInterface oldLogic = map.putIfAbsent(queueId, newLogic);
+            if (oldLogic != null) {
+                logic = oldLogic;
+            } else {
+                logic = newLogic;
+            }
+        }
+
+        return logic;
+    }
+}
diff --git a/store/src/main/java/org/apache/rocketmq/store/queue/CqUnit.java b/store/src/main/java/org/apache/rocketmq/store/queue/CqUnit.java
new file mode 100644
index 0000000..b8865fd
--- /dev/null
+++ b/store/src/main/java/org/apache/rocketmq/store/queue/CqUnit.java
@@ -0,0 +1,115 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
... 2685 lines suppressed ...