You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@rocketmq.apache.org by ji...@apache.org on 2022/02/08 02:43:45 UTC
[rocketmq] 02/17: support batch consume-queue.
This is an automated email from the ASF dual-hosted git repository.
jinrongtong pushed a commit to branch 5.0.0-alpha
in repository https://gitbox.apache.org/repos/asf/rocketmq.git
commit 821b91f0a25839550fe2fbdd8183ddb389147e78
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 2891baf..f30953d 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;
@@ -1799,7 +1801,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);
@@ -1830,26 +1832,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,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
... 2680 lines suppressed ...