Skip to content

[ISSUE #9254] Refactor CQ-related in DefaultMessageStorage #9256

New issue

Have a question about this project? Sign up for a free GitHub account to open an issue and contact its maintainers and the community.

By clicking “Sign up for GitHub”, you agree to our terms of service and privacy statement. We’ll occasionally send you account related emails.

Already on GitHub? Sign in to your account

Open
wants to merge 9 commits into
base: develop
Choose a base branch
from
Open
Show file tree
Hide file tree
Changes from all commits
Commits
File filter

Filter by extension

Filter by extension

Conversations
Failed to load comments.
Loading
Jump to
Jump to file
Failed to load files.
Loading
Diff view
Diff view
Original file line number Diff line number Diff line change
Expand Up @@ -60,6 +60,15 @@
import org.apache.rocketmq.broker.client.rebalance.RebalanceLockManager;
import org.apache.rocketmq.broker.coldctr.ColdDataCgCtrService;
import org.apache.rocketmq.broker.coldctr.ColdDataPullRequestHoldService;
import org.apache.rocketmq.broker.config.v1.RocksDBConsumerOffsetManager;
import org.apache.rocketmq.broker.config.v1.RocksDBLmqSubscriptionGroupManager;
import org.apache.rocketmq.broker.config.v1.RocksDBLmqTopicConfigManager;
import org.apache.rocketmq.broker.config.v1.RocksDBSubscriptionGroupManager;
import org.apache.rocketmq.broker.config.v1.RocksDBTopicConfigManager;
import org.apache.rocketmq.broker.config.v2.ConfigStorage;
import org.apache.rocketmq.broker.config.v2.ConsumerOffsetManagerV2;
import org.apache.rocketmq.broker.config.v2.SubscriptionGroupManagerV2;
import org.apache.rocketmq.broker.config.v2.TopicConfigManagerV2;
import org.apache.rocketmq.broker.controller.ReplicasManager;
import org.apache.rocketmq.broker.dledger.DLedgerRoleChangeHandler;
import org.apache.rocketmq.broker.failover.EscapeBridge;
Expand All @@ -76,7 +85,6 @@
import org.apache.rocketmq.broker.offset.ConsumerOffsetManager;
import org.apache.rocketmq.broker.offset.ConsumerOrderInfoManager;
import org.apache.rocketmq.broker.offset.LmqConsumerOffsetManager;
import org.apache.rocketmq.broker.config.v1.RocksDBConsumerOffsetManager;
import org.apache.rocketmq.broker.out.BrokerOuterAPI;
import org.apache.rocketmq.broker.plugin.BrokerAttachedPlugin;
import org.apache.rocketmq.broker.pop.PopConsumerService;
Expand All @@ -100,16 +108,8 @@
import org.apache.rocketmq.broker.schedule.ScheduleMessageService;
import org.apache.rocketmq.broker.slave.SlaveSynchronize;
import org.apache.rocketmq.broker.subscription.LmqSubscriptionGroupManager;
import org.apache.rocketmq.broker.config.v1.RocksDBLmqSubscriptionGroupManager;
import org.apache.rocketmq.broker.config.v1.RocksDBSubscriptionGroupManager;
import org.apache.rocketmq.broker.subscription.SubscriptionGroupManager;
import org.apache.rocketmq.broker.config.v2.ConsumerOffsetManagerV2;
import org.apache.rocketmq.broker.config.v2.SubscriptionGroupManagerV2;
import org.apache.rocketmq.broker.config.v2.TopicConfigManagerV2;
import org.apache.rocketmq.broker.config.v2.ConfigStorage;
import org.apache.rocketmq.broker.topic.LmqTopicConfigManager;
import org.apache.rocketmq.broker.config.v1.RocksDBLmqTopicConfigManager;
import org.apache.rocketmq.broker.config.v1.RocksDBTopicConfigManager;
import org.apache.rocketmq.broker.topic.TopicConfigManager;
import org.apache.rocketmq.broker.topic.TopicQueueMappingCleanService;
import org.apache.rocketmq.broker.topic.TopicQueueMappingManager;
Expand Down Expand Up @@ -815,9 +815,6 @@ public boolean initializeMessageStore() {
defaultMessageStore = new RocksDBMessageStore(this.messageStoreConfig, this.brokerStatsManager, this.messageArrivingListener, this.brokerConfig, topicConfigManager.getTopicConfigTable());
} else {
defaultMessageStore = new DefaultMessageStore(this.messageStoreConfig, this.brokerStatsManager, this.messageArrivingListener, this.brokerConfig, topicConfigManager.getTopicConfigTable());
if (messageStoreConfig.isRocksdbCQDoubleWriteEnable()) {
defaultMessageStore.enableRocksdbCQWrite();
}
}

if (messageStoreConfig.isEnableDLegerCommitLog()) {
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -72,14 +72,12 @@
import org.apache.rocketmq.broker.plugin.BrokerAttachedPlugin;
import org.apache.rocketmq.broker.subscription.SubscriptionGroupManager;
import org.apache.rocketmq.broker.transaction.queue.TransactionalMessageUtil;
import org.apache.rocketmq.common.BoundaryType;
import org.apache.rocketmq.common.BrokerConfig;
import org.apache.rocketmq.common.CheckRocksdbCqWriteResult;
import org.apache.rocketmq.common.KeyBuilder;
import org.apache.rocketmq.common.LockCallback;
import org.apache.rocketmq.common.MQVersion;
import org.apache.rocketmq.common.MixAll;
import org.apache.rocketmq.common.Pair;
import org.apache.rocketmq.common.PlainAccessConfig;
import org.apache.rocketmq.common.TopicAttributes;
import org.apache.rocketmq.common.TopicConfig;
Expand Down Expand Up @@ -223,13 +221,14 @@
import org.apache.rocketmq.store.MessageStore;
import org.apache.rocketmq.store.PutMessageResult;
import org.apache.rocketmq.store.PutMessageStatus;
import org.apache.rocketmq.store.RocksDBMessageStore;
import org.apache.rocketmq.store.SelectMappedBufferResult;
import org.apache.rocketmq.store.StoreType;
import org.apache.rocketmq.store.config.BrokerRole;
import org.apache.rocketmq.store.exception.ConsumeQueueException;
import org.apache.rocketmq.store.plugin.AbstractPluginMessageStore;
import org.apache.rocketmq.store.queue.CombineConsumeQueueStore;
import org.apache.rocketmq.store.queue.ConsumeQueueInterface;
import org.apache.rocketmq.store.queue.ConsumeQueueStoreInterface;
import org.apache.rocketmq.store.queue.CqUnit;
import org.apache.rocketmq.store.queue.ReferredIterator;
import org.apache.rocketmq.store.timer.TimerCheckpoint;
Expand Down Expand Up @@ -3479,129 +3478,24 @@ private boolean validateBlackListConfigExist(Properties properties) {
private CheckRocksdbCqWriteResult doCheckRocksdbCqWriteProgress(ChannelHandlerContext ctx,
RemotingCommand request) throws RemotingCommandException {
CheckRocksdbCqWriteProgressRequestHeader requestHeader = request.decodeCommandCustomHeader(CheckRocksdbCqWriteProgressRequestHeader.class);
String requestTopic = requestHeader.getTopic();
MessageStore messageStore = brokerController.getMessageStore();
DefaultMessageStore defaultMessageStore;
if (messageStore instanceof AbstractPluginMessageStore) {
defaultMessageStore = (DefaultMessageStore) ((AbstractPluginMessageStore) messageStore).getNext();
} else {
defaultMessageStore = (DefaultMessageStore) messageStore;
}
RocksDBMessageStore rocksDBMessageStore = defaultMessageStore.getRocksDBMessageStore();
CheckRocksdbCqWriteResult result = new CheckRocksdbCqWriteResult();
ConsumeQueueStoreInterface consumeQueueStore = defaultMessageStore.getQueueStore();

if (defaultMessageStore.getMessageStoreConfig().getStoreType().equals(StoreType.DEFAULT_ROCKSDB.getStoreType())) {
result.setCheckResult("storeType is DEFAULT_ROCKSDB, no need check");
if (!(consumeQueueStore instanceof CombineConsumeQueueStore)) {
CheckRocksdbCqWriteResult result = new CheckRocksdbCqWriteResult();
result.setCheckResult("is not CombineConsumeQueueStore, no need check");
Copy link
Contributor

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

It's not CombineConsumeQueueStore, no need to check.

result.setCheckStatus(CheckRocksdbCqWriteResult.CheckStatus.CHECK_OK.getValue());
return result;
}

if (!defaultMessageStore.getMessageStoreConfig().isRocksdbCQDoubleWriteEnable()) {
result.setCheckResult("rocksdbCQWriteEnable is false, checkRocksdbCqWriteProgressCommand is invalid");
result.setCheckStatus(CheckRocksdbCqWriteResult.CheckStatus.CHECK_NOT_OK.getValue());
return result;
}

ConcurrentMap<String, ConcurrentMap<Integer, ConsumeQueueInterface>> cqTable = defaultMessageStore.getConsumeQueueTable();
StringBuilder diffResult = new StringBuilder();
try {
if (StringUtils.isNotBlank(requestTopic)) {
boolean checkResult = processConsumeQueuesForTopic(cqTable.get(requestTopic), requestTopic, rocksDBMessageStore, diffResult, true, requestHeader.getCheckStoreTime());
result.setCheckResult(diffResult.toString());
result.setCheckStatus(checkResult ? CheckRocksdbCqWriteResult.CheckStatus.CHECK_OK.getValue() : CheckRocksdbCqWriteResult.CheckStatus.CHECK_NOT_OK.getValue());
return result;
}
int successNum = 0;
int checkSize = 0;
for (Map.Entry<String, ConcurrentMap<Integer, ConsumeQueueInterface>> topicEntry : cqTable.entrySet()) {
boolean checkResult = processConsumeQueuesForTopic(topicEntry.getValue(), topicEntry.getKey(), rocksDBMessageStore, diffResult, false, requestHeader.getCheckStoreTime());
successNum += checkResult ? 1 : 0;
checkSize++;
}
// check all topic finish, all topic is ready, checkSize: 100, currentQueueNum: 110 -> ready (The currentQueueNum means when we do checking, new topics are added.)
// check all topic finish, success/all : 89/100, currentQueueNum: 110 -> not ready
boolean checkReady = successNum == checkSize;
String checkResultString = checkReady ? String.format("all topic is ready, checkSize: %s, currentQueueNum: %s", checkSize, cqTable.size()) :
String.format("success/all : %s/%s, currentQueueNum: %s", successNum, checkSize, cqTable.size());
diffResult.append("check all topic finish, ").append(checkResultString);
result.setCheckResult(diffResult.toString());
result.setCheckStatus(checkReady ? CheckRocksdbCqWriteResult.CheckStatus.CHECK_OK.getValue() : CheckRocksdbCqWriteResult.CheckStatus.CHECK_NOT_OK.getValue());
} catch (Exception e) {
LOGGER.error("CheckRocksdbCqWriteProgressCommand error", e);
result.setCheckResult(e.getMessage() + Arrays.toString(e.getStackTrace()));
result.setCheckStatus(CheckRocksdbCqWriteResult.CheckStatus.CHECK_ERROR.getValue());
}
return result;
}

private boolean processConsumeQueuesForTopic(ConcurrentMap<Integer, ConsumeQueueInterface> queueMap, String topic,
RocksDBMessageStore rocksDBMessageStore, StringBuilder diffResult, boolean printDetail,
long checkpointByStoreTime) {
boolean processResult = true;
for (Map.Entry<Integer, ConsumeQueueInterface> queueEntry : queueMap.entrySet()) {
Integer queueId = queueEntry.getKey();
ConsumeQueueInterface jsonCq = queueEntry.getValue();
ConsumeQueueInterface kvCq = rocksDBMessageStore.getConsumeQueue(topic, queueId);
if (printDetail) {
String format = String.format("[topic: %s, queue: %s] \n kvEarliest : %s | kvLatest : %s \n fileEarliest: %s | fileEarliest: %s ",
topic, queueId, kvCq.getEarliestUnit(), kvCq.getLatestUnit(), jsonCq.getEarliestUnit(), jsonCq.getLatestUnit());
diffResult.append(format).append("\n");
}

long minOffsetByTime = 0L;
try {
minOffsetByTime = rocksDBMessageStore.getConsumeQueueStore().getOffsetInQueueByTime(topic, queueId, checkpointByStoreTime, BoundaryType.UPPER);
} catch (Exception e) {
// ignore
}
long minOffsetInQueue = kvCq.getMinOffsetInQueue();
long checkFrom = Math.max(minOffsetInQueue, minOffsetByTime);
long checkTo = jsonCq.getMaxOffsetInQueue() - 1;
/*
checkTo(maxOffsetInQueue - 1)
v
fileCq +------------------------------------------------------+
kvCq +----------------------------------------------+
^ ^
minOffsetInQueue minOffsetByTime
^
checkFrom = max(minOffsetInQueue, minOffsetByTime)
*/
// The latest message is earlier than the check time
Pair<CqUnit, Long> fileLatestCq = jsonCq.getCqUnitAndStoreTime(checkTo);
if (fileLatestCq != null) {
if (fileLatestCq.getObject2() < checkpointByStoreTime) {
continue;
}
}
for (long i = checkFrom; i <= checkTo; i++) {
Pair<CqUnit, Long> fileCqUnit = jsonCq.getCqUnitAndStoreTime(i);
Pair<CqUnit, Long> kvCqUnit = kvCq.getCqUnitAndStoreTime(i);
if (fileCqUnit == null || kvCqUnit == null || !checkCqUnitEqual(kvCqUnit.getObject1(), fileCqUnit.getObject1())) {
LOGGER.error(String.format("[topic: %s, queue: %s, offset: %s] \n file : %s \n kv : %s \n",
topic, queueId, i, kvCqUnit != null ? kvCqUnit.getObject1() : "null", fileCqUnit != null ? fileCqUnit.getObject1() : "null"));
processResult = false;
break;
}
}
}
return processResult;
}

private boolean checkCqUnitEqual(CqUnit cqUnit1, CqUnit cqUnit2) {
if (cqUnit1.getQueueOffset() != cqUnit2.getQueueOffset()) {
return false;
}
if (cqUnit1.getSize() != cqUnit2.getSize()) {
return false;
}
if (cqUnit1.getPos() != cqUnit2.getPos()) {
return false;
}
if (cqUnit1.getBatchNum() != cqUnit2.getBatchNum()) {
return false;
}
return cqUnit1.getTagsCode() == cqUnit2.getTagsCode();
return ((CombineConsumeQueueStore) consumeQueueStore).
doCheckCqWriteProgress(requestHeader.getTopic(), requestHeader.getCheckStoreTime(), StoreType.DEFAULT, StoreType.DEFAULT_ROCKSDB);
}

private RemotingCommand transferPopToFsStore(ChannelHandlerContext ctx, RemotingCommand request) {
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -236,7 +236,7 @@ public boolean correctDelayOffset() {
try {
for (int delayLevel : delayLevelTable.keySet()) {
ConsumeQueueInterface cq =
brokerController.getMessageStore().getQueueStore().findOrCreateConsumeQueue(TopicValidator.RMQ_SYS_SCHEDULE_TOPIC,
brokerController.getMessageStore().findConsumeQueue(TopicValidator.RMQ_SYS_SCHEDULE_TOPIC,
delayLevel2QueueId(delayLevel));
Long currentDelayOffset = offsetTable.get(delayLevel);
if (currentDelayOffset == null || cq == null) {
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -28,15 +28,18 @@
import org.apache.rocketmq.broker.BrokerController;
import org.apache.rocketmq.broker.config.v1.RocksDBConsumerOffsetManager;
import org.apache.rocketmq.common.BrokerConfig;
import org.apache.rocketmq.common.CheckRocksdbCqWriteResult;
import org.apache.rocketmq.common.Pair;
import org.apache.rocketmq.common.TopicConfig;
import org.apache.rocketmq.store.DefaultMessageStore;
import org.apache.rocketmq.store.DispatchRequest;
import org.apache.rocketmq.store.RocksDBMessageStore;
import org.apache.rocketmq.store.StoreType;
import org.apache.rocketmq.store.config.MessageStoreConfig;
import org.apache.rocketmq.store.queue.CombineConsumeQueueStore;
import org.apache.rocketmq.store.queue.ConsumeQueueInterface;
import org.apache.rocketmq.store.queue.ConsumeQueueStore;
import org.apache.rocketmq.store.queue.ConsumeQueueStoreInterface;
import org.apache.rocketmq.store.queue.CqUnit;
import org.apache.rocketmq.store.queue.RocksDBConsumeQueueStore;
import org.apache.rocketmq.store.stats.BrokerStatsManager;
import org.awaitility.Awaitility;
import org.junit.Assert;
Expand Down Expand Up @@ -81,9 +84,8 @@ public void init() throws IOException {
Mockito.lenient().when(brokerController.getBrokerConfig()).thenReturn(brokerConfig);
Mockito.lenient().when(brokerController.getMessageStoreConfig()).thenReturn(messageStoreConfig);

defaultMessageStore = new DefaultMessageStore(messageStoreConfig, new BrokerStatsManager("aaa", true), null,
brokerConfig, new ConcurrentHashMap<String, TopicConfig>());
defaultMessageStore.enableRocksdbCQWrite();
defaultMessageStore = new DefaultMessageStore(messageStoreConfig, new BrokerStatsManager("for-test", true), null,
brokerConfig, new ConcurrentHashMap<>());
defaultMessageStore.loadCheckPoint();

consumerOffsetManager = new ConsumerOffsetManager(brokerController);
Expand Down Expand Up @@ -134,23 +136,35 @@ public void testRocksdbCqWrite() throws RocksDBException {
if (notToBeExecuted()) {
return;
}
RocksDBMessageStore kvStore = defaultMessageStore.getRocksDBMessageStore();
ConsumeQueueStoreInterface store = kvStore.getConsumeQueueStore();
store.start();
ConsumeQueueInterface rocksdbCq = defaultMessageStore.getRocksDBMessageStore().findConsumeQueue(topic, queueId);
ConsumeQueueInterface fileCq = defaultMessageStore.findConsumeQueue(topic, queueId);
long startTimestamp = System.currentTimeMillis();

ConsumeQueueStoreInterface combineConsumeQueueStore = defaultMessageStore.getQueueStore();
Assert.assertTrue(combineConsumeQueueStore instanceof CombineConsumeQueueStore);
combineConsumeQueueStore.load();
combineConsumeQueueStore.recover(false);
combineConsumeQueueStore.start();

RocksDBConsumeQueueStore rocksDBConsumeQueueStore = ((CombineConsumeQueueStore) combineConsumeQueueStore).getRocksDBConsumeQueueStore();
ConsumeQueueStore consumeQueueStore = ((CombineConsumeQueueStore) combineConsumeQueueStore).getConsumeQueueStore();

for (int i = 0; i < 200; i++) {
DispatchRequest request = new DispatchRequest(topic, queueId, i, 200, 0, System.currentTimeMillis(), i, "", "", 0, 0, new HashMap<>());
fileCq.putMessagePositionInfoWrapper(request);
store.putMessagePositionInfoWrapper(request);
combineConsumeQueueStore.putMessagePositionInfoWrapper(request);
}

ConsumeQueueInterface rocksdbCq = rocksDBConsumeQueueStore.findOrCreateConsumeQueue(topic, queueId);
ConsumeQueueInterface fileCq = consumeQueueStore.findOrCreateConsumeQueue(topic, queueId);

Awaitility.await()
.pollInterval(100, TimeUnit.MILLISECONDS)
.atMost(3, TimeUnit.SECONDS)
.until(() -> rocksdbCq.getMaxOffsetInQueue() == 200);
Pair<CqUnit, Long> unit = rocksdbCq.getCqUnitAndStoreTime(100);
Pair<CqUnit, Long> unit1 = fileCq.getCqUnitAndStoreTime(100);
Assert.assertEquals(unit.getObject1().getPos(), unit1.getObject1().getPos());

CheckRocksdbCqWriteResult result = ((CombineConsumeQueueStore) combineConsumeQueueStore).doCheckCqWriteProgress(topic, startTimestamp, StoreType.DEFAULT, StoreType.DEFAULT_ROCKSDB);
Assert.assertEquals(CheckRocksdbCqWriteResult.CheckStatus.CHECK_OK.getValue(), result.getCheckStatus());
}

/**
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -16,6 +16,11 @@
*/
package org.apache.rocketmq.client.impl.consumer;

import java.lang.reflect.Field;
import java.util.ArrayList;
import java.util.Collections;
import java.util.List;
import java.util.TreeMap;
import org.apache.commons.lang3.reflect.FieldUtils;
import org.apache.rocketmq.client.consumer.DefaultMQPushConsumer;
import org.apache.rocketmq.client.exception.MQBrokerException;
Expand All @@ -29,12 +34,6 @@
import org.junit.runner.RunWith;
import org.mockito.junit.MockitoJUnitRunner;

import java.lang.reflect.Field;
import java.util.ArrayList;
import java.util.Collections;
import java.util.List;
import java.util.TreeMap;

import static org.assertj.core.api.Assertions.assertThat;
import static org.junit.Assert.assertEquals;
import static org.junit.Assert.assertFalse;
Expand Down Expand Up @@ -158,7 +157,6 @@ public void testProcessQueue() {
ProcessQueue processQueue2 = createProcessQueue();
assertEquals(processQueue1.getMsgAccCnt(), processQueue2.getMsgAccCnt());
assertEquals(processQueue1.getTryUnlockTimes(), processQueue2.getTryUnlockTimes());
assertEquals(processQueue1.getLastLockTimestamp(), processQueue2.getLastLockTimestamp());
assertEquals(processQueue1.getLastPullTimestamp(), processQueue2.getLastPullTimestamp());
}

Expand Down
Loading
Loading