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