diff --git a/broker/src/main/java/org/apache/rocketmq/broker/BrokerController.java b/broker/src/main/java/org/apache/rocketmq/broker/BrokerController.java index a715ec3a4e8..cb20e66a0d4 100644 --- a/broker/src/main/java/org/apache/rocketmq/broker/BrokerController.java +++ b/broker/src/main/java/org/apache/rocketmq/broker/BrokerController.java @@ -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; @@ -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; @@ -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; @@ -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()) { diff --git a/broker/src/main/java/org/apache/rocketmq/broker/processor/AdminBrokerProcessor.java b/broker/src/main/java/org/apache/rocketmq/broker/processor/AdminBrokerProcessor.java index 4ff4bed814d..12698be0366 100644 --- a/broker/src/main/java/org/apache/rocketmq/broker/processor/AdminBrokerProcessor.java +++ b/broker/src/main/java/org/apache/rocketmq/broker/processor/AdminBrokerProcessor.java @@ -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; @@ -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; @@ -3479,7 +3478,6 @@ 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) { @@ -3487,121 +3485,17 @@ private CheckRocksdbCqWriteResult doCheckRocksdbCqWriteProgress(ChannelHandlerCo } 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"); 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> 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> 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 queueMap, String topic, - RocksDBMessageStore rocksDBMessageStore, StringBuilder diffResult, boolean printDetail, - long checkpointByStoreTime) { - boolean processResult = true; - for (Map.Entry 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 fileLatestCq = jsonCq.getCqUnitAndStoreTime(checkTo); - if (fileLatestCq != null) { - if (fileLatestCq.getObject2() < checkpointByStoreTime) { - continue; - } - } - for (long i = checkFrom; i <= checkTo; i++) { - Pair fileCqUnit = jsonCq.getCqUnitAndStoreTime(i); - Pair 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) { diff --git a/broker/src/main/java/org/apache/rocketmq/broker/schedule/ScheduleMessageService.java b/broker/src/main/java/org/apache/rocketmq/broker/schedule/ScheduleMessageService.java index a5b02c9a63c..25c24aff987 100644 --- a/broker/src/main/java/org/apache/rocketmq/broker/schedule/ScheduleMessageService.java +++ b/broker/src/main/java/org/apache/rocketmq/broker/schedule/ScheduleMessageService.java @@ -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) { diff --git a/broker/src/test/java/org/apache/rocketmq/broker/offset/RocksdbTransferOffsetAndCqTest.java b/broker/src/test/java/org/apache/rocketmq/broker/offset/RocksdbTransferOffsetAndCqTest.java index 6a805b04340..3745b994a53 100644 --- a/broker/src/test/java/org/apache/rocketmq/broker/offset/RocksdbTransferOffsetAndCqTest.java +++ b/broker/src/test/java/org/apache/rocketmq/broker/offset/RocksdbTransferOffsetAndCqTest.java @@ -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; @@ -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()); - defaultMessageStore.enableRocksdbCQWrite(); + defaultMessageStore = new DefaultMessageStore(messageStoreConfig, new BrokerStatsManager("for-test", true), null, + brokerConfig, new ConcurrentHashMap<>()); defaultMessageStore.loadCheckPoint(); consumerOffsetManager = new ConsumerOffsetManager(brokerController); @@ -134,16 +136,25 @@ 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) @@ -151,6 +162,9 @@ public void testRocksdbCqWrite() throws RocksDBException { Pair unit = rocksdbCq.getCqUnitAndStoreTime(100); Pair 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()); } /** diff --git a/client/src/test/java/org/apache/rocketmq/client/impl/consumer/ProcessQueueTest.java b/client/src/test/java/org/apache/rocketmq/client/impl/consumer/ProcessQueueTest.java index a8afd4a233a..dd7ffa757f8 100644 --- a/client/src/test/java/org/apache/rocketmq/client/impl/consumer/ProcessQueueTest.java +++ b/client/src/test/java/org/apache/rocketmq/client/impl/consumer/ProcessQueueTest.java @@ -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; @@ -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; @@ -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()); } 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 b061aa7a0d4..84a55c8b53a 100644 --- a/store/src/main/java/org/apache/rocketmq/store/CommitLog.java +++ b/store/src/main/java/org/apache/rocketmq/store/CommitLog.java @@ -17,6 +17,8 @@ package org.apache.rocketmq.store; import com.google.common.base.Strings; +import com.sun.jna.NativeLong; +import com.sun.jna.Pointer; import java.net.Inet6Address; import java.net.InetSocketAddress; import java.nio.ByteBuffer; @@ -34,8 +36,6 @@ import java.util.concurrent.TimeoutException; import java.util.function.Supplier; import java.util.stream.Collectors; -import com.sun.jna.NativeLong; -import com.sun.jna.Pointer; import org.apache.rocketmq.common.MixAll; import org.apache.rocketmq.common.ServiceThread; import org.apache.rocketmq.common.SystemClock; @@ -66,7 +66,6 @@ import org.apache.rocketmq.store.logfile.MappedFile; import org.apache.rocketmq.store.util.LibC; import org.rocksdb.RocksDBException; - import sun.nio.ch.DirectBuffer; /** @@ -321,7 +320,7 @@ public boolean getLastMappedFile(final long startOffset) { * * @throws RocksDBException only in rocksdb mode */ - public void recoverNormally(long maxPhyOffsetOfConsumeQueue) throws RocksDBException { + public void recoverNormally(long dispatchFromPhyOffset) throws RocksDBException { boolean checkCRCOnRecover = this.defaultMessageStore.getMessageStoreConfig().isCheckCRCOnRecover(); boolean checkDupInfo = this.defaultMessageStore.getMessageStoreConfig().isDuplicationEnable(); final List mappedFiles = this.mappedFileQueue.getMappedFiles(); @@ -329,7 +328,7 @@ public void recoverNormally(long maxPhyOffsetOfConsumeQueue) throws RocksDBExcep int index = mappedFiles.size() - 1; while (index > 0) { MappedFile mappedFile = mappedFiles.get(index); - if (mappedFile.getFileFromOffset() <= maxPhyOffsetOfConsumeQueue) { + if (isMappedFileMatchedRecover(mappedFile, true)) { // It's safe to recover from this mapped file break; } @@ -345,7 +344,7 @@ public void recoverNormally(long maxPhyOffsetOfConsumeQueue) throws RocksDBExcep while (true) { DispatchRequest dispatchRequest = this.checkMessageAndReturnSize(byteBuffer, checkCRCOnRecover, checkDupInfo); int size = dispatchRequest.getMsgSize(); - boolean doDispatch = dispatchRequest.getCommitLogOffset() > maxPhyOffsetOfConsumeQueue; + boolean doDispatch = dispatchRequest.getCommitLogOffset() > dispatchFromPhyOffset; // Normal data if (dispatchRequest.isSuccess() && size > 0) { lastValidMsgPhyOffset = processOffset + mappedFileOffset; @@ -395,10 +394,7 @@ else if (!dispatchRequest.isSuccess()) { } // Clear ConsumeQueue redundant data - if (maxPhyOffsetOfConsumeQueue >= processOffset) { - log.warn("maxPhyOffsetOfConsumeQueue({}) >= processOffset({}), truncate dirty logic files", maxPhyOffsetOfConsumeQueue, processOffset); - this.defaultMessageStore.truncateDirtyLogicFiles(processOffset); - } + this.defaultMessageStore.truncateDirtyLogicFiles(processOffset); this.mappedFileQueue.setFlushedWhere(processOffset); this.mappedFileQueue.setCommittedWhere(processOffset); @@ -408,8 +404,7 @@ else if (!dispatchRequest.isSuccess()) { log.warn("The commitlog files are deleted, and delete the consume queue files"); this.mappedFileQueue.setFlushedWhere(0); this.mappedFileQueue.setCommittedWhere(0); - this.defaultMessageStore.getQueueStore().destroy(); - this.defaultMessageStore.getQueueStore().loadAfterDestroy(); + this.defaultMessageStore.destroyConsumeQueueStore(true); } } @@ -717,7 +712,7 @@ public void recoverAbnormally(long maxPhyOffsetOfConsumeQueue) throws RocksDBExc MappedFile mappedFile = null; for (; index >= 0; index--) { mappedFile = mappedFiles.get(index); - if (this.isMappedFileMatchedRecover(mappedFile)) { + if (this.isMappedFileMatchedRecover(mappedFile, false)) { log.info("recover from this mapped file " + mappedFile.getFileName()); break; } @@ -804,10 +799,7 @@ else if (size == 0) { } // Clear ConsumeQueue redundant data - if (maxPhyOffsetOfConsumeQueue >= processOffset) { - log.warn("maxPhyOffsetOfConsumeQueue({}) >= processOffset({}), truncate dirty logic files", maxPhyOffsetOfConsumeQueue, processOffset); - this.defaultMessageStore.truncateDirtyLogicFiles(processOffset); - } + this.defaultMessageStore.truncateDirtyLogicFiles(processOffset); this.mappedFileQueue.setFlushedWhere(processOffset); this.mappedFileQueue.setCommittedWhere(processOffset); @@ -818,8 +810,7 @@ else if (size == 0) { log.warn("The commitlog files are deleted, and delete the consume queue files"); this.mappedFileQueue.setFlushedWhere(0); this.mappedFileQueue.setCommittedWhere(0); - this.defaultMessageStore.getQueueStore().destroy(); - this.defaultMessageStore.getQueueStore().loadAfterDestroy(); + this.defaultMessageStore.destroyConsumeQueueStore(true); } } @@ -842,7 +833,8 @@ protected void onCommitLogAppend(MessageExtBrokerInner msg, AppendMessageResult this.getMessageStore().onCommitLogAppend(msg, result, commitLogFile); } - private boolean isMappedFileMatchedRecover(final MappedFile mappedFile) throws RocksDBException { + private boolean isMappedFileMatchedRecover(final MappedFile mappedFile, + boolean recoverNormally) throws RocksDBException { ByteBuffer byteBuffer = mappedFile.sliceByteBuffer(); int magicCode = byteBuffer.getInt(MessageDecoder.MESSAGE_MAGIC_CODE_POSITION); @@ -850,41 +842,23 @@ private boolean isMappedFileMatchedRecover(final MappedFile mappedFile) throws R return false; } - if (this.defaultMessageStore.getMessageStoreConfig().isEnableRocksDBStore()) { - final long maxPhyOffsetInConsumeQueue = this.defaultMessageStore.getQueueStore().getMaxPhyOffsetInConsumeQueue(); - long phyOffset = byteBuffer.getLong(MessageDecoder.MESSAGE_PHYSIC_OFFSET_POSITION); - if (phyOffset <= maxPhyOffsetInConsumeQueue) { - log.info("find check. beginPhyOffset: {}, maxPhyOffsetInConsumeQueue: {}", phyOffset, maxPhyOffsetInConsumeQueue); - return true; - } - } else { - int sysFlag = byteBuffer.getInt(MessageDecoder.SYSFLAG_POSITION); - int bornHostLength = (sysFlag & MessageSysFlag.BORNHOST_V6_FLAG) == 0 ? 8 : 20; - int msgStoreTimePos = 4 + 4 + 4 + 4 + 4 + 8 + 8 + 4 + 8 + bornHostLength; - long storeTimestamp = byteBuffer.getLong(msgStoreTimePos); - if (0 == storeTimestamp) { - return false; - } + int sysFlag = byteBuffer.getInt(MessageDecoder.SYSFLAG_POSITION); + int bornHostLength = (sysFlag & MessageSysFlag.BORNHOST_V6_FLAG) == 0 ? 8 : 20; + int msgStoreTimePos = 4 + 4 + 4 + 4 + 4 + 8 + 8 + 4 + 8 + bornHostLength; + long storeTimestamp = byteBuffer.getLong(msgStoreTimePos); + if (0 == storeTimestamp) { + return false; + } + long phyOffset = byteBuffer.getLong(MessageDecoder.MESSAGE_PHYSIC_OFFSET_POSITION); - if (this.defaultMessageStore.getMessageStoreConfig().isMessageIndexEnable() - && this.defaultMessageStore.getMessageStoreConfig().isMessageIndexSafe()) { - if (storeTimestamp <= this.defaultMessageStore.getStoreCheckpoint().getMinTimestampIndex()) { - log.info("find check timestamp, {} {}", - storeTimestamp, - UtilAll.timeMillisToHumanString(storeTimestamp)); - return true; - } - } else { - if (storeTimestamp <= this.defaultMessageStore.getStoreCheckpoint().getMinTimestamp()) { - log.info("find check timestamp, {} {}", - storeTimestamp, - UtilAll.timeMillisToHumanString(storeTimestamp)); - return true; - } - } + if (this.defaultMessageStore.getMessageStoreConfig().isMessageIndexEnable() && + this.defaultMessageStore.getMessageStoreConfig().isMessageIndexSafe() && + storeTimestamp > this.defaultMessageStore.getStoreCheckpoint().getIndexMsgTimestamp()) { + return false; } - return false; + return this.defaultMessageStore.getQueueStore() + .isMappedFileMatchedRecover(phyOffset, storeTimestamp, recoverNormally); } public boolean resetOffset(long offset) { 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 b6b9cff538d..2903f6afd3f 100644 --- a/store/src/main/java/org/apache/rocketmq/store/ConsumeQueue.java +++ b/store/src/main/java/org/apache/rocketmq/store/ConsumeQueue.java @@ -35,6 +35,7 @@ import org.apache.rocketmq.store.config.StorePathConfigHelper; 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.FileQueueLifeCycle; import org.apache.rocketmq.store.queue.MultiDispatchUtils; @@ -61,6 +62,7 @@ public class ConsumeQueue implements ConsumeQueueInterface, FileQueueLifeCycle { private static final Logger LOG_ERROR = LoggerFactory.getLogger(LoggerName.STORE_ERROR_LOGGER_NAME); private final MessageStore messageStore; + private final ConsumeQueueStore consumeQueueStore; private final MappedFileQueue mappedFileQueue; private final String topic; @@ -83,9 +85,20 @@ public ConsumeQueue( final String storePath, final int mappedFileSize, final MessageStore messageStore) { + this(topic, queueId, storePath, mappedFileSize, messageStore, (ConsumeQueueStore) messageStore.getQueueStore()); + } + + public ConsumeQueue( + final String topic, + final int queueId, + final String storePath, + final int mappedFileSize, + final MessageStore messageStore, + final ConsumeQueueStore consumeQueueStore) { this.storePath = storePath; this.mappedFileSize = mappedFileSize; this.messageStore = messageStore; + this.consumeQueueStore = consumeQueueStore; this.topic = topic; this.queueId = queueId; @@ -899,14 +912,14 @@ public CqUnit get(long offset) { @Override public Pair getCqUnitAndStoreTime(long index) { CqUnit cqUnit = get(index); - Long messageStoreTime = this.messageStore.getQueueStore().getStoreTime(cqUnit); + Long messageStoreTime = this.consumeQueueStore.getStoreTime(cqUnit); return new Pair<>(cqUnit, messageStoreTime); } @Override public Pair getEarliestUnitAndStoreTime() { CqUnit cqUnit = getEarliestUnit(); - Long messageStoreTime = this.messageStore.getQueueStore().getStoreTime(cqUnit); + Long messageStoreTime = this.consumeQueueStore.getStoreTime(cqUnit); return new Pair<>(cqUnit, messageStoreTime); } @@ -1204,4 +1217,18 @@ public long estimateMessageCount(long from, long to, MessageFilter filter) { log.debug("Result={}, raw={}, match={}, sample={}", result, raw, match, sample); return result; } + + public void initializeWithOffset(long offset) { + destroy(); + + // correct min offset + // TODO: when min commitLog offset is 0 and restart store, min offset of cq may be set to 0 incorrectly + setMinLogicOffset(offset * ConsumeQueue.CQ_STORE_UNIT_SIZE); + + // transientStorePool is null, only need set wrote position here + MappedFile mappedFile = mappedFileQueue.getLastMappedFile(offset * ConsumeQueue.CQ_STORE_UNIT_SIZE, true); + fillPreBlank(mappedFile, offset * ConsumeQueue.CQ_STORE_UNIT_SIZE); + + flush(0); + } } 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 d6134683861..e343c141181 100644 --- a/store/src/main/java/org/apache/rocketmq/store/DefaultMessageStore.java +++ b/store/src/main/java/org/apache/rocketmq/store/DefaultMessageStore.java @@ -43,7 +43,6 @@ import java.util.Optional; import java.util.Set; import java.util.concurrent.CompletableFuture; -import java.util.concurrent.ConcurrentHashMap; import java.util.concurrent.ConcurrentLinkedQueue; import java.util.concurrent.ConcurrentMap; import java.util.concurrent.ConcurrentSkipListMap; @@ -70,7 +69,6 @@ import org.apache.rocketmq.common.ThreadFactoryImpl; import org.apache.rocketmq.common.TopicConfig; import org.apache.rocketmq.common.UtilAll; -import org.apache.rocketmq.common.attribute.CQType; import org.apache.rocketmq.common.attribute.CleanupPolicy; import org.apache.rocketmq.common.constant.LoggerName; import org.apache.rocketmq.common.message.MessageConst; @@ -106,6 +104,7 @@ import org.apache.rocketmq.store.kv.CompactionStore; import org.apache.rocketmq.store.logfile.MappedFile; import org.apache.rocketmq.store.metrics.DefaultStoreMetricsManager; +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; @@ -128,14 +127,8 @@ public class DefaultMessageStore implements MessageStore { protected final ConsumeQueueStoreInterface consumeQueueStore; - private final FlushConsumeQueueService flushConsumeQueueService; - protected final CleanCommitLogService cleanCommitLogService; - private final CleanConsumeQueueService cleanConsumeQueueService; - - private final CorrectLogicOffsetService correctLogicOffsetService; - protected final IndexService indexService; private final AllocateMappedFileService allocateMappedFileService; @@ -167,9 +160,7 @@ public class DefaultMessageStore implements MessageStore { protected StoreCheckpoint storeCheckpoint; private TimerMessageStore timerMessageStore; - private final LinkedList dispatcherList; - - private RocksDBMessageStore rocksDBMessageStore; + private final LinkedList dispatcherList = new LinkedList<>(); private final RandomAccessFile lockFile; @@ -226,48 +217,26 @@ public DefaultMessageStore(final MessageStoreConfig messageStoreConfig, final Br this.brokerStatsManager = brokerStatsManager; this.topicConfigTable = topicConfigTable; this.allocateMappedFileService = new AllocateMappedFileService(this); - if (messageStoreConfig.isEnableDLegerCommitLog()) { - this.commitLog = new DLedgerCommitLog(this); - } else { - this.commitLog = new CommitLog(this); - } + this.commitLog = messageStoreConfig.isEnableDLegerCommitLog() ? + new DLedgerCommitLog(this) : new CommitLog(this); this.consumeQueueStore = createConsumeQueueStore(); - - this.flushConsumeQueueService = createFlushConsumeQueueService(); this.cleanCommitLogService = new CleanCommitLogService(); - this.cleanConsumeQueueService = createCleanConsumeQueueService(); - this.correctLogicOffsetService = createCorrectLogicOffsetService(); this.storeStatsService = new StoreStatsService(getBrokerIdentity()); this.indexService = new IndexService(this); + this.dispatcherList.addLast(new CommitLogDispatcherBuildConsumeQueue()); + this.dispatcherList.addLast(new CommitLogDispatcherBuildIndex()); - if (!messageStoreConfig.isEnableDLegerCommitLog() && !this.messageStoreConfig.isDuplicationEnable()) { - if (brokerConfig.isEnableControllerMode()) { - this.haService = new AutoSwitchHAService(); - LOGGER.warn("Load AutoSwitch HA Service: {}", AutoSwitchHAService.class.getSimpleName()); - } else { - this.haService = ServiceProvider.loadClass(HAService.class); - if (null == this.haService) { - this.haService = new DefaultHAService(); - LOGGER.warn("Load default HA Service: {}", DefaultHAService.class.getSimpleName()); - } - } - } + initializeHAService(); - if (!messageStoreConfig.isEnableBuildConsumeQueueConcurrently()) { - this.reputMessageService = new ReputMessageService(); - } else { - this.reputMessageService = new ConcurrentReputMessageService(); - } + this.reputMessageService = messageStoreConfig.isEnableBuildConsumeQueueConcurrently() ? + new ConcurrentReputMessageService() : new ReputMessageService(); this.transientStorePool = new TransientStorePool(messageStoreConfig.getTransientStorePoolSize(), messageStoreConfig.getMappedFileSizeCommitLog()); this.scheduledExecutorService = ThreadUtils.newSingleThreadScheduledExecutor(new ThreadFactoryImpl("StoreScheduledThread", getBrokerIdentity())); - this.dispatcherList = new LinkedList<>(); - this.dispatcherList.addLast(new CommitLogDispatcherBuildConsumeQueue()); - this.dispatcherList.addLast(new CommitLogDispatcherBuildIndex()); if (messageStoreConfig.isEnableCompaction()) { this.compactionStore = new CompactionStore(this); this.compactionService = new CompactionService(commitLog, this, compactionStore); @@ -284,21 +253,12 @@ public DefaultMessageStore(final MessageStoreConfig messageStoreConfig, final Br } public ConsumeQueueStoreInterface createConsumeQueueStore() { + if (messageStoreConfig.isRocksdbCQDoubleWriteEnable()) { + return new CombineConsumeQueueStore(this); + } return new ConsumeQueueStore(this); } - public CleanConsumeQueueService createCleanConsumeQueueService() { - return new CleanConsumeQueueService(); - } - - public FlushConsumeQueueService createFlushConsumeQueueService() { - return new FlushConsumeQueueService(); - } - - public CorrectLogicOffsetService createCorrectLogicOffsetService() { - return new CorrectLogicOffsetService(); - } - public boolean parseDelayLevel() { HashMap timeUnitTable = new HashMap<>(); timeUnitTable.put("s", 1000L); @@ -330,11 +290,6 @@ public boolean parseDelayLevel() { return true; } - @Override - public void truncateDirtyLogicFiles(long phyOffset) throws RocksDBException { - this.consumeQueueStore.truncateDirty(phyOffset); - } - /** * @throws IOException */ @@ -364,7 +319,6 @@ public boolean load() { LOGGER.info("message store recover end, and the max phy offset = {}", this.getMaxPhyOffset()); } - long maxOffset = this.getMaxPhyOffset(); this.setBrokerInitMaxOffset(maxOffset); LOGGER.info("load over, and the max phy offset = {}", maxOffset); @@ -388,6 +342,31 @@ public void loadCheckPoint() throws IOException { setConfirmOffset(this.storeCheckpoint.getConfirmPhyOffset()); } + private void recover(final boolean lastExitOK) throws RocksDBException { + // recover consume queue + long recoverConsumeQueueStart = System.currentTimeMillis(); + this.consumeQueueStore.recover(this.brokerConfig.isRecoverConcurrently()); + long dispatchFromPhyOffset = this.consumeQueueStore.getDispatchFromPhyOffset(); + long recoverConsumeQueueEnd = System.currentTimeMillis(); + + // recover commitlog + if (lastExitOK) { + this.commitLog.recoverNormally(dispatchFromPhyOffset); + } else { + this.commitLog.recoverAbnormally(dispatchFromPhyOffset); + } + + // recover consume offset table + long recoverCommitLogEnd = System.currentTimeMillis(); + this.recoverTopicQueueTable(); + long recoverConsumeOffsetEnd = System.currentTimeMillis(); + + LOGGER.info("message store recover total cost: {} ms, " + + "recoverConsumeQueue: {} ms, recoverCommitLog: {} ms, recoverOffsetTable: {} ms", + recoverConsumeOffsetEnd - recoverConsumeQueueStart, recoverConsumeQueueEnd - recoverConsumeQueueStart, + recoverCommitLogEnd - recoverConsumeQueueEnd, recoverConsumeOffsetEnd - recoverCommitLogEnd); + } + /** * @throws Exception */ @@ -420,7 +399,6 @@ public void start() throws Exception { // which is eliminating the dispatch inconsistency between the commitLog and consumeQueue at the end of recovery. this.doRecheckReputOffsetFromCq(); - this.flushConsumeQueueService.start(); this.commitLog.start(); this.consumeQueueStore.start(); this.storeStatsService.start(); @@ -517,15 +495,9 @@ public void shutdown() { if (this.compactionService != null) { this.compactionService.shutdown(); } - if (this.rocksDBMessageStore != null && this.rocksDBMessageStore.consumeQueueStore != null) { - this.rocksDBMessageStore.consumeQueueStore.shutdown(); - } - this.flushConsumeQueueService.shutdown(); this.allocateMappedFileService.shutdown(); this.storeCheckpoint.shutdown(); - this.perfs.shutdown(); - if (this.runningFlags.isWriteable() && dispatchBehindBytes() == 0) { this.deleteFile(StorePathConfigHelper.getAbortFile(this.messageStoreConfig.getStorePathRootDir())); shutDownNormal = true; @@ -540,14 +512,14 @@ public void shutdown() { try { lock.release(); lockFile.close(); - } catch (IOException e) { + } catch (IOException ignored) { } } } @Override public void destroy() { - this.consumeQueueStore.destroy(); + this.consumeQueueStore.destroy(false); this.commitLog.destroy(); this.indexService.destroy(); this.deleteFile(StorePathConfigHelper.getAbortFile(this.messageStoreConfig.getStorePathRootDir())); @@ -749,6 +721,11 @@ public void truncateDirtyFiles(long offsetToTruncate) throws RocksDBException { this.reputMessageService.start(); } + @Override + public void truncateDirtyLogicFiles(long phyOffset) throws RocksDBException { + this.consumeQueueStore.truncateDirty(phyOffset); + } + @Override public boolean truncateFiles(long offsetToTruncate) throws RocksDBException { if (offsetToTruncate >= this.getMaxPhyOffset()) { @@ -859,7 +836,7 @@ public GetMessageResult getMessage(final String group, final String topic, final if (bufferConsumeQueue == null) { status = GetMessageStatus.OFFSET_FOUND_NULL; - nextBeginOffset = nextOffsetCorrection(nextBeginOffset, this.consumeQueueStore.rollNextFile(consumeQueue, nextBeginOffset)); + nextBeginOffset = nextOffsetCorrection(nextBeginOffset, consumeQueue.rollNextFile(nextBeginOffset)); LOGGER.warn("consumer request topic: " + topic + ", offset: " + offset + ", minOffset: " + minOffset + ", maxOffset: " + maxOffset + ", but access logic queue failed. Correct nextBeginOffset to " + nextBeginOffset); break; @@ -1386,25 +1363,10 @@ public int deleteTopics(final Set deleteTopics) { int deleteCount = 0; for (String topic : deleteTopics) { - ConcurrentMap queueTable = this.consumeQueueStore.findConsumeQueueMap(topic); - - if (queueTable == null || queueTable.isEmpty()) { + if (!consumeQueueStore.deleteTopic(topic)) { continue; } - for (ConsumeQueueInterface cq : queueTable.values()) { - try { - this.consumeQueueStore.destroy(cq); - } catch (RocksDBException e) { - LOGGER.error("DeleteTopic: ConsumeQueue cleans error!, topic={}, queueId={}", cq.getTopic(), cq.getQueueId(), e); - } - LOGGER.info("DeleteTopic: ConsumeQueue has been cleaned, topic={}, queueId={}", cq.getTopic(), cq.getQueueId()); - this.consumeQueueStore.removeTopicQueueTable(cq.getTopic(), cq.getQueueId()); - } - - // remove topic from cq table - this.consumeQueueStore.getConsumeQueueTable().remove(topic); - if (this.brokerConfig.isAutoDeleteUnusedStats()) { this.brokerStatsManager.onTopicDeleted(topic); } @@ -1576,61 +1538,6 @@ public long getFlushedWhere() { return this.commitLog.getFlushedWhere(); } - @Override - public boolean resetWriteOffset(long phyOffset) { - //copy a new map - ConcurrentHashMap 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; - } - } - // Fetch and compute the newest confirmOffset. // Even if it is just inited. @Override @@ -1849,32 +1756,27 @@ public void run0() { DefaultMessageStore.this.storeCheckpoint.flush(); } }, 1, 1, TimeUnit.SECONDS); + } - this.scheduledCleanQueueExecutorService.scheduleAtFixedRate(new Runnable() { - @Override - public void run() { - DefaultMessageStore.this.cleanQueueFilesPeriodically(); + private void initializeHAService() { + if (!this.messageStoreConfig.isEnableDLegerCommitLog() && !this.messageStoreConfig.isDuplicationEnable()) { + if (brokerConfig.isEnableControllerMode()) { + this.haService = new AutoSwitchHAService(); + LOGGER.warn("Load AutoSwitch HA Service: {}", AutoSwitchHAService.class.getSimpleName()); + } else { + this.haService = ServiceProvider.loadClass(HAService.class); + if (null == this.haService) { + this.haService = new DefaultHAService(); + LOGGER.warn("Load default HA Service: {}", DefaultHAService.class.getSimpleName()); + } } - }, 1000 * 60, this.messageStoreConfig.getCleanResourceInterval(), TimeUnit.MILLISECONDS); - - - // this.scheduledExecutorService.scheduleAtFixedRate(new Runnable() { - // @Override - // public void run() { - // DefaultMessageStore.this.cleanExpiredConsumerQueue(); - // } - // }, 1, 1, TimeUnit.HOURS); + } } private void cleanFilesPeriodically() { this.cleanCommitLogService.run(); } - private void cleanQueueFilesPeriodically() { - this.correctLogicOffsetService.run(); - this.cleanConsumeQueueService.run(); - } - private void checkSelf() { this.commitLog.checkSelf(); this.consumeQueueStore.checkSelf(); @@ -1885,39 +1787,6 @@ private boolean isTempFileExist() { File file = new File(fileName); return file.exists(); } - - private boolean isRecoverConcurrently() { - return this.brokerConfig.isRecoverConcurrently() && !this.messageStoreConfig.isEnableRocksDBStore(); - } - - private void recover(final boolean lastExitOK) throws RocksDBException { - boolean recoverConcurrently = this.isRecoverConcurrently(); - LOGGER.info("message store recover mode: {}", recoverConcurrently ? "concurrent" : "normal"); - - // recover consume queue - long recoverConsumeQueueStart = System.currentTimeMillis(); - this.recoverConsumeQueue(); - long maxPhyOffsetOfConsumeQueue = this.consumeQueueStore.getMaxPhyOffsetInConsumeQueue(); - long recoverConsumeQueueEnd = System.currentTimeMillis(); - - // recover commitlog - if (lastExitOK) { - this.commitLog.recoverNormally(maxPhyOffsetOfConsumeQueue); - } else { - this.commitLog.recoverAbnormally(maxPhyOffsetOfConsumeQueue); - } - - // recover consume offset table - long recoverCommitLogEnd = System.currentTimeMillis(); - this.recoverTopicQueueTable(); - long recoverConsumeOffsetEnd = System.currentTimeMillis(); - - LOGGER.info("message store recover total cost: {} ms, " + - "recoverConsumeQueue: {} ms, recoverCommitLog: {} ms, recoverOffsetTable: {} ms", - recoverConsumeOffsetEnd - recoverConsumeQueueStart, recoverConsumeQueueEnd - recoverConsumeQueueStart, - recoverCommitLogEnd - recoverConsumeQueueEnd, recoverConsumeOffsetEnd - recoverCommitLogEnd); - } - @Override public long getTimingMessageCount(String topic) { if (null == timerMessageStore) { @@ -1937,14 +1806,6 @@ public TransientStorePool getTransientStorePool() { return transientStorePool; } - private void recoverConsumeQueue() { - if (!this.isRecoverConcurrently()) { - this.consumeQueueStore.recover(); - } else { - this.consumeQueueStore.recoverConcurrently(); - } - } - @Override public void recoverTopicQueueTable() { long minPhyOffset = this.commitLog.getMinOffset(); @@ -2070,11 +1931,7 @@ public boolean getData(long offset, int size, ByteBuffer byteBuffer) { @Override public ConsumeQueueInterface getConsumeQueue(String topic, int queueId) { - ConcurrentMap map = this.getConsumeQueueTable().get(topic); - if (map == null) { - return null; - } - return map.get(queueId); + return this.consumeQueueStore.getConsumeQueue(topic, queueId); } @Override @@ -2189,6 +2046,15 @@ public void dispatch(DispatchRequest request) { } } + public boolean isTimeToDelete() { + String when = messageStoreConfig.getDeleteWhen(); + if (UtilAll.isItTimeToDo(when)) { + LOGGER.info("it's time to reclaim disk space, " + when); + return true; + } + return false; + } + class CleanCommitLogService { private final static int MAX_MANUAL_DELETE_FILE_TIMES = 20; @@ -2262,7 +2128,7 @@ private void deleteExpiredFiles() { int destroyMappedFileIntervalForcibly = DefaultMessageStore.this.getMessageStoreConfig().getDestroyMapedFileIntervalForcibly(); int deleteFileBatchMax = DefaultMessageStore.this.getMessageStoreConfig().getDeleteFileBatchMax(); - boolean isTimeUp = this.isTimeToDelete(); + boolean isTimeUp = DefaultMessageStore.this.isTimeToDelete(); boolean isUsageExceedsThreshold = this.isSpaceToDelete(); boolean isManualDelete = this.manualDeleteFileSeveralTimes.get() > 0; @@ -2316,16 +2182,6 @@ public String getServiceName() { return DefaultMessageStore.this.brokerConfig.getIdentifier() + CleanCommitLogService.class.getSimpleName(); } - protected boolean isTimeToDelete() { - String when = DefaultMessageStore.this.getMessageStoreConfig().getDeleteWhen(); - if (UtilAll.isItTimeToDo(when)) { - DefaultMessageStore.LOGGER.info("it's time to reclaim disk space, " + when); - return true; - } - - return false; - } - private boolean isSpaceToDelete() { cleanImmediately = false; @@ -2473,244 +2329,6 @@ public boolean isSpaceFull() { } } - class CleanConsumeQueueService { - protected long lastPhysicalMinOffset = 0; - - public void run() { - try { - this.deleteExpiredFiles(); - } catch (Throwable e) { - DefaultMessageStore.LOGGER.warn(this.getServiceName() + " service has exception. ", e); - } - } - - protected void deleteExpiredFiles() { - int deleteLogicsFilesInterval = DefaultMessageStore.this.getMessageStoreConfig().getDeleteConsumeQueueFilesInterval(); - - long minOffset = DefaultMessageStore.this.commitLog.getMinOffset(); - if (minOffset > this.lastPhysicalMinOffset) { - this.lastPhysicalMinOffset = minOffset; - - ConcurrentMap> tables = DefaultMessageStore.this.getConsumeQueueTable(); - - for (ConcurrentMap maps : tables.values()) { - for (ConsumeQueueInterface logic : maps.values()) { - int deleteCount = DefaultMessageStore.this.consumeQueueStore.deleteExpiredFile(logic, minOffset); - if (deleteCount > 0 && deleteLogicsFilesInterval > 0) { - try { - Thread.sleep(deleteLogicsFilesInterval); - } catch (InterruptedException ignored) { - } - } - } - } - - DefaultMessageStore.this.indexService.deleteExpiredFile(minOffset); - } - } - - public String getServiceName() { - return DefaultMessageStore.this.brokerConfig.getIdentifier() + CleanConsumeQueueService.class.getSimpleName(); - } - } - - class CorrectLogicOffsetService { - private long lastForceCorrectTime = -1L; - - public void run() { - try { - this.correctLogicMinOffset(); - } catch (Throwable e) { - LOGGER.warn(this.getServiceName() + " service has exception. ", e); - } - } - - private boolean needCorrect(ConsumeQueueInterface logic, long minPhyOffset, long lastForeCorrectTimeCurRun) { - if (logic == null) { - return false; - } - // If first exist and not available, it means first file may destroy failed, delete it. - if (DefaultMessageStore.this.consumeQueueStore.isFirstFileExist(logic) && !DefaultMessageStore.this.consumeQueueStore.isFirstFileAvailable(logic)) { - 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; - } - - // logic.getMaxPhysicOffset() or minPhyOffset = -1 - // means there is no message in current queue, so no need to correct. - if (logic.getMaxPhysicOffset() == -1 || minPhyOffset == -1) { - return false; - } - - if (logic.getMaxPhysicOffset() < minPhyOffset) { - if (logic.getMinOffsetInQueue() < logic.getMaxOffsetInQueue()) { - 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 { - 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; - } - } - //the logic.getMaxPhysicOffset() >= minPhyOffset - int forceCorrectInterval = DefaultMessageStore.this.getMessageStoreConfig().getCorrectLogicMinOffsetForceInterval(); - if ((System.currentTimeMillis() - lastForeCorrectTimeCurRun) > forceCorrectInterval) { - lastForceCorrectTime = System.currentTimeMillis(); - CqUnit cqUnit = logic.getEarliestUnit(); - if (cqUnit == null) { - if (logic.getMinOffsetInQueue() == logic.getMaxOffsetInQueue()) { - return false; - } else { - 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) { - 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; - } - - if (cqUnit.getPos() >= minPhyOffset) { - - // Normal case, do not need to correct. - return false; - } - } - - return false; - } - - private void correctLogicMinOffset() { - - long lastForeCorrectTimeCurRun = lastForceCorrectTime; - long minPhyOffset = getMinPhyOffset(); - ConcurrentMap> tables = DefaultMessageStore.this.getConsumeQueueTable(); - for (ConcurrentMap maps : tables.values()) { - for (ConsumeQueueInterface logic : maps.values()) { - if (Objects.equals(CQType.SimpleCQ, logic.getCQType())) { - // cq is not supported for now. - continue; - } - if (needCorrect(logic, minPhyOffset, lastForeCorrectTimeCurRun)) { - doCorrect(logic, minPhyOffset); - } - } - } - } - - private void doCorrect(ConsumeQueueInterface logic, long minPhyOffset) { - DefaultMessageStore.this.consumeQueueStore.deleteExpiredFile(logic, minPhyOffset); - int sleepIntervalWhenCorrectMinOffset = DefaultMessageStore.this.getMessageStoreConfig().getCorrectLogicMinOffsetSleepInterval(); - if (sleepIntervalWhenCorrectMinOffset > 0) { - try { - Thread.sleep(sleepIntervalWhenCorrectMinOffset); - } catch (InterruptedException ignored) { - } - } - } - - public String getServiceName() { - if (brokerConfig.isInBrokerContainer()) { - return brokerConfig.getIdentifier() + CorrectLogicOffsetService.class.getSimpleName(); - } - return CorrectLogicOffsetService.class.getSimpleName(); - } - } - - class FlushConsumeQueueService extends ServiceThread { - private static final int RETRY_TIMES_OVER = 3; - private long lastFlushTimestamp = 0; - - private void doFlush(int retryTimes) { - int flushConsumeQueueLeastPages = DefaultMessageStore.this.getMessageStoreConfig().getFlushConsumeQueueLeastPages(); - - if (retryTimes == RETRY_TIMES_OVER) { - flushConsumeQueueLeastPages = 0; - } - - long logicsMsgTimestamp = 0; - - int flushConsumeQueueThoroughInterval = DefaultMessageStore.this.getMessageStoreConfig().getFlushConsumeQueueThoroughInterval(); - long currentTimeMillis = System.currentTimeMillis(); - if (currentTimeMillis >= (this.lastFlushTimestamp + flushConsumeQueueThoroughInterval)) { - this.lastFlushTimestamp = currentTimeMillis; - flushConsumeQueueLeastPages = 0; - logicsMsgTimestamp = DefaultMessageStore.this.getStoreCheckpoint().getLogicsMsgTimestamp(); - } - - ConcurrentMap> tables = DefaultMessageStore.this.getConsumeQueueTable(); - - for (ConcurrentMap maps : tables.values()) { - for (ConsumeQueueInterface cq : maps.values()) { - boolean result = false; - for (int i = 0; i < retryTimes && !result; i++) { - result = DefaultMessageStore.this.consumeQueueStore.flush(cq, flushConsumeQueueLeastPages); - } - } - } - - if (messageStoreConfig.isEnableCompaction()) { - compactionStore.flush(flushConsumeQueueLeastPages); - } - - if (0 == flushConsumeQueueLeastPages) { - if (logicsMsgTimestamp > 0) { - DefaultMessageStore.this.getStoreCheckpoint().setLogicsMsgTimestamp(logicsMsgTimestamp); - } - DefaultMessageStore.this.getStoreCheckpoint().flush(); - } - } - - @Override - public void run() { - DefaultMessageStore.LOGGER.info(this.getServiceName() + " service started"); - - while (!this.isStopped()) { - try { - int interval = DefaultMessageStore.this.getMessageStoreConfig().getFlushIntervalConsumeQueue(); - this.waitForRunning(interval); - this.doFlush(1); - } catch (Exception e) { - DefaultMessageStore.LOGGER.warn(this.getServiceName() + " service has exception. ", e); - } - } - - this.doFlush(RETRY_TIMES_OVER); - - DefaultMessageStore.LOGGER.info(this.getServiceName() + " service end"); - } - - @Override - public String getServiceName() { - if (DefaultMessageStore.this.brokerConfig.isInBrokerContainer()) { - return DefaultMessageStore.this.getBrokerIdentity().getIdentifier() + FlushConsumeQueueService.class.getSimpleName(); - } - return FlushConsumeQueueService.class.getSimpleName(); - } - - @Override - public long getJoinTime() { - return 1000 * 60; - } - } - static class BatchDispatchRequest { private final ByteBuffer byteBuffer; @@ -3272,17 +2890,6 @@ public HARuntimeInfo getHARuntimeInfo() { } } - public void enableRocksdbCQWrite() { - try { - RocksDBMessageStore store = new RocksDBMessageStore(this.messageStoreConfig, this.brokerStatsManager, this.messageArrivingListener, this.brokerConfig, this.topicConfigTable); - this.rocksDBMessageStore = store; - store.loadAndStartConsumerServiceOnly(); - addDispatcher(store.getDispatcherBuildRocksdbConsumeQueue()); - } catch (Exception e) { - LOGGER.error("enableRocksdbCqWrite error", e); - } - } - public int getMaxDelayLevel() { return maxDelayLevel; } @@ -3371,11 +2978,19 @@ public long getReputFromOffset() { return this.reputMessageService.getReputFromOffset(); } - public RocksDBMessageStore getRocksDBMessageStore() { - return this.rocksDBMessageStore; + public CompactionStore getCompactionStore() { + return compactionStore; } - public ConsumeQueueStoreInterface getConsumeQueueStore() { - return consumeQueueStore; + public IndexService getIndexService() { + return indexService; + } + + public ScheduledExecutorService getScheduledCleanQueueExecutorService() { + return scheduledCleanQueueExecutorService; + } + + public void destroyConsumeQueueStore(boolean loadAfterDestroy) { + consumeQueueStore.destroy(loadAfterDestroy); } } 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 4bbee142a17..9c9a556f6d3 100644 --- a/store/src/main/java/org/apache/rocketmq/store/MessageStore.java +++ b/store/src/main/java/org/apache/rocketmq/store/MessageStore.java @@ -16,6 +16,10 @@ */ package org.apache.rocketmq.store; +import io.opentelemetry.api.common.AttributesBuilder; +import io.opentelemetry.api.metrics.Meter; +import io.opentelemetry.sdk.metrics.InstrumentSelector; +import io.opentelemetry.sdk.metrics.ViewBuilder; import java.nio.ByteBuffer; import java.util.HashMap; import java.util.LinkedList; @@ -43,10 +47,6 @@ import org.apache.rocketmq.store.timer.TimerMessageStore; import org.apache.rocketmq.store.util.PerfCounter; import org.rocksdb.RocksDBException; -import io.opentelemetry.api.common.AttributesBuilder; -import io.opentelemetry.api.metrics.Meter; -import io.opentelemetry.sdk.metrics.InstrumentSelector; -import io.opentelemetry.sdk.metrics.ViewBuilder; /** * This class defines contracting interfaces to implement, allowing third-party vendor to use customized message store. @@ -532,14 +532,6 @@ CompletableFuture queryMessageAsync(final String topic, fina */ long getFlushedWhere(); - /** - * Reset written offset. - * - * @param phyOffset new offset. - * @return true if success; false otherwise. - */ - boolean resetWriteOffset(long phyOffset); - /** * Get confirm offset. * diff --git a/store/src/main/java/org/apache/rocketmq/store/RocksDBMessageStore.java b/store/src/main/java/org/apache/rocketmq/store/RocksDBMessageStore.java index 321689ac8f5..8f0a075ff4c 100644 --- a/store/src/main/java/org/apache/rocketmq/store/RocksDBMessageStore.java +++ b/store/src/main/java/org/apache/rocketmq/store/RocksDBMessageStore.java @@ -16,31 +16,17 @@ */ package org.apache.rocketmq.store; -import io.opentelemetry.api.common.AttributesBuilder; -import io.opentelemetry.api.metrics.Meter; import java.io.IOException; -import java.util.concurrent.ConcurrentHashMap; import java.util.concurrent.ConcurrentMap; -import java.util.function.Supplier; import org.apache.rocketmq.common.BrokerConfig; import org.apache.rocketmq.common.TopicConfig; -import org.apache.rocketmq.common.UtilAll; -import org.apache.rocketmq.common.sysflag.MessageSysFlag; import org.apache.rocketmq.store.config.MessageStoreConfig; -import org.apache.rocketmq.store.config.StorePathConfigHelper; -import org.apache.rocketmq.store.metrics.DefaultStoreMetricsManager; -import org.apache.rocketmq.store.metrics.RocksDBStoreMetricsManager; -import org.apache.rocketmq.store.queue.ConsumeQueueInterface; import org.apache.rocketmq.store.queue.ConsumeQueueStoreInterface; -import org.apache.rocketmq.store.queue.RocksDBConsumeQueue; import org.apache.rocketmq.store.queue.RocksDBConsumeQueueStore; import org.apache.rocketmq.store.stats.BrokerStatsManager; -import org.rocksdb.RocksDBException; public class RocksDBMessageStore extends DefaultMessageStore { - private CommitLogDispatcherBuildRocksdbConsumeQueue dispatcherBuildRocksdbConsumeQueue; - public RocksDBMessageStore(final MessageStoreConfig messageStoreConfig, final BrokerStatsManager brokerStatsManager, final MessageArrivingListener messageArrivingListener, final BrokerConfig brokerConfig, final ConcurrentMap topicConfigTable) throws IOException { @@ -52,157 +38,4 @@ public RocksDBMessageStore(final MessageStoreConfig messageStoreConfig, final Br public ConsumeQueueStoreInterface createConsumeQueueStore() { return new RocksDBConsumeQueueStore(this); } - - @Override - public CleanConsumeQueueService createCleanConsumeQueueService() { - return new RocksDBCleanConsumeQueueService(); - } - - @Override - public FlushConsumeQueueService createFlushConsumeQueueService() { - return new RocksDBFlushConsumeQueueService(); - } - - @Override - public CorrectLogicOffsetService createCorrectLogicOffsetService() { - return new RocksDBCorrectLogicOffsetService(); - } - - /** - * Try to set topicQueueTable = new HashMap<>(), otherwise it will cause bug when broker role changes. - * And unlike method in DefaultMessageStore, we don't need to really recover topic queue table advance, - * because we can recover topic queue table from rocksdb when we need to use it. - * @see RocksDBConsumeQueue#assignQueueOffset - */ - @Override - public void recoverTopicQueueTable() { - this.consumeQueueStore.setTopicQueueTable(new ConcurrentHashMap<>()); - } - - @Override - public ConsumeQueueInterface getConsumeQueue(String topic, int queueId) { - return findConsumeQueue(topic, queueId); - } - - class RocksDBCleanConsumeQueueService extends CleanConsumeQueueService { - private final double diskSpaceWarningLevelRatio = - Double.parseDouble(System.getProperty("rocketmq.broker.diskSpaceWarningLevelRatio", "0.90")); - - private final double diskSpaceCleanForciblyRatio = - Double.parseDouble(System.getProperty("rocketmq.broker.diskSpaceCleanForciblyRatio", "0.85")); - - @Override - protected void deleteExpiredFiles() { - - long minOffset = RocksDBMessageStore.this.commitLog.getMinOffset(); - if (minOffset > this.lastPhysicalMinOffset) { - this.lastPhysicalMinOffset = minOffset; - - boolean spaceFull = isSpaceToDelete(); - boolean timeUp = cleanCommitLogService.isTimeToDelete(); - if (spaceFull || timeUp) { - RocksDBMessageStore.this.consumeQueueStore.cleanExpired(minOffset); - } - - RocksDBMessageStore.this.indexService.deleteExpiredFile(minOffset); - } - } - - private boolean isSpaceToDelete() { - double ratio = RocksDBMessageStore.this.getMessageStoreConfig().getDiskMaxUsedSpaceRatio() / 100.0; - - String storePathLogics = StorePathConfigHelper - .getStorePathConsumeQueue(RocksDBMessageStore.this.getMessageStoreConfig().getStorePathRootDir()); - double logicsRatio = UtilAll.getDiskPartitionSpaceUsedPercent(storePathLogics); - if (logicsRatio > diskSpaceWarningLevelRatio) { - boolean diskOk = RocksDBMessageStore.this.runningFlags.getAndMakeLogicDiskFull(); - if (diskOk) { - RocksDBMessageStore.LOGGER.error("logics disk maybe full soon " + logicsRatio + ", so mark disk full"); - } - } else if (logicsRatio > diskSpaceCleanForciblyRatio) { - } else { - boolean diskOk = RocksDBMessageStore.this.runningFlags.getAndMakeLogicDiskOK(); - if (!diskOk) { - RocksDBMessageStore.LOGGER.info("logics disk space OK " + logicsRatio + ", so mark disk ok"); - } - } - - if (logicsRatio < 0 || logicsRatio > ratio) { - RocksDBMessageStore.LOGGER.info("logics disk maybe full soon, so reclaim space, " + logicsRatio); - return true; - } - - return false; - } - } - - class RocksDBFlushConsumeQueueService extends FlushConsumeQueueService { - /** - * There is no need to flush consume queue, - * we put all consume queues in RocksDBConsumeQueueStore, - * it depends on rocksdb to flush consume queue to disk(sorted string table), - * we even don't flush WAL of consume store, since we think it can recover consume queue from commitlog. - */ - @Override - public void run() { - - } - } - - class RocksDBCorrectLogicOffsetService extends CorrectLogicOffsetService { - /** - * There is no need to correct min offset of consume queue, we already fix this problem. - * @see org.apache.rocketmq.store.queue.RocksDBConsumeQueueOffsetTable#getMinCqOffset - */ - public void run() { - - } - } - - @Override - public void initMetrics(Meter meter, Supplier attributesBuilderSupplier) { - DefaultStoreMetricsManager.init(meter, attributesBuilderSupplier, this); - // Also add some metrics for rocksdb's monitoring. - RocksDBStoreMetricsManager.init(meter, attributesBuilderSupplier, this); - } - - public CommitLogDispatcherBuildRocksdbConsumeQueue getDispatcherBuildRocksdbConsumeQueue() { - return dispatcherBuildRocksdbConsumeQueue; - } - - class CommitLogDispatcherBuildRocksdbConsumeQueue implements CommitLogDispatcher { - @Override - public void dispatch(DispatchRequest request) throws RocksDBException { - boolean enable = getMessageStoreConfig().isRocksdbCQDoubleWriteEnable(); - if (!enable) { - return; - } - final int tranType = MessageSysFlag.getTransactionValue(request.getSysFlag()); - switch (tranType) { - case MessageSysFlag.TRANSACTION_NOT_TYPE: - case MessageSysFlag.TRANSACTION_COMMIT_TYPE: - putMessagePositionInfo(request); - break; - case MessageSysFlag.TRANSACTION_PREPARED_TYPE: - case MessageSysFlag.TRANSACTION_ROLLBACK_TYPE: - break; - } - } - } - - public void loadAndStartConsumerServiceOnly() { - try { - this.dispatcherBuildRocksdbConsumeQueue = new CommitLogDispatcherBuildRocksdbConsumeQueue(); - boolean loadResult = this.consumeQueueStore.load(); - if (!loadResult) { - throw new RuntimeException("load consume queue failed"); - } - super.loadCheckPoint(); - this.consumeQueueStore.start(); - } catch (Exception e) { - ERROR_LOG.error("loadAndStartConsumerServiceOnly error", e); - throw new RuntimeException(e); - } - } - } diff --git a/store/src/main/java/org/apache/rocketmq/store/StoreType.java b/store/src/main/java/org/apache/rocketmq/store/StoreType.java index 4f9c4d0e448..5e89a3ffccf 100644 --- a/store/src/main/java/org/apache/rocketmq/store/StoreType.java +++ b/store/src/main/java/org/apache/rocketmq/store/StoreType.java @@ -16,6 +16,12 @@ */ package org.apache.rocketmq.store; +import java.util.Arrays; +import java.util.Collections; +import java.util.Objects; +import java.util.Set; +import java.util.stream.Collectors; + public enum StoreType { DEFAULT("default"), DEFAULT_ROCKSDB("defaultRocksDB"); @@ -29,4 +35,26 @@ public enum StoreType { public String getStoreType() { return storeType; } + + /** + * convert string to set of StoreType + * + * @param str example "default;defaultRocksDB" + * @return set of StoreType + */ + public static Set fromString(String str) { + if (str == null || str.trim().isEmpty()) { + return Collections.emptySet(); + } + + return Arrays.stream(str.split(";")) + .map(String::trim) + .filter(s -> !s.isEmpty()) + .map(s -> Arrays.stream(StoreType.values()) + .filter(type -> type.getStoreType().equalsIgnoreCase(s)) + .findFirst() + .orElse(null)) + .filter(Objects::nonNull) + .collect(Collectors.toSet()); + } } 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 0ea58415487..787b4d26245 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 @@ -17,7 +17,6 @@ package org.apache.rocketmq.store.config; import java.io.File; - import org.apache.rocketmq.common.annotation.ImportantField; import org.apache.rocketmq.store.ConsumeQueue; import org.apache.rocketmq.store.StoreType; @@ -427,6 +426,18 @@ public class MessageStoreConfig { private boolean rocksdbCQDoubleWriteEnable = false; + /** + * CombineConsumeQueueStore + * combineCQLoadingCQTypes is used to configure the loading types of CQ. load / recover / start order: [default -> defaultRocksDB] + * combineCQPreferCQType is used to configure the preferred CQ type when reading. Make sure the CQ type is included in combineCQLoadingCQTypes + * combineAssignOffsetCQType is used to configure the CQ type when assign offset. Make sure the CQ type is included in combineCQLoadingCQTypes + */ + private String combineCQLoadingCQTypes = StoreType.DEFAULT.getStoreType() + ";" + StoreType.DEFAULT_ROCKSDB.getStoreType(); + private String combineCQPreferCQType = StoreType.DEFAULT.getStoreType(); + private String combineAssignOffsetCQType = StoreType.DEFAULT.getStoreType(); + private boolean combineCQEnableCheckSelf = false; + private int combineCQMaxExtraLookBackCommitLogFiles = 3; + /** * If ConsumeQueueStore is RocksDB based, this option is to configure bottom-most tier compression type. * The following values are valid: @@ -1950,4 +1961,44 @@ public void setUseABSLock(boolean useABSLock) { public boolean getUseABSLock() { return useABSLock; } + + public String getCombineCQPreferCQType() { + return combineCQPreferCQType; + } + + public void setCombineCQPreferCQType(String combineCQPreferCQType) { + this.combineCQPreferCQType = combineCQPreferCQType; + } + + public String getCombineCQLoadingCQTypes() { + return combineCQLoadingCQTypes; + } + + public void setCombineCQLoadingCQTypes(String combineCQLoadingCQTypes) { + this.combineCQLoadingCQTypes = combineCQLoadingCQTypes; + } + + public String getCombineAssignOffsetCQType() { + return combineAssignOffsetCQType; + } + + public void setCombineAssignOffsetCQType(String combineAssignOffsetCQType) { + this.combineAssignOffsetCQType = combineAssignOffsetCQType; + } + + public boolean isCombineCQEnableCheckSelf() { + return combineCQEnableCheckSelf; + } + + public void setCombineCQEnableCheckSelf(boolean combineCQEnableCheckSelf) { + this.combineCQEnableCheckSelf = combineCQEnableCheckSelf; + } + + public int getCombineCQMaxExtraLookBackCommitLogFiles() { + return combineCQMaxExtraLookBackCommitLogFiles; + } + + public void setCombineCQMaxExtraLookBackCommitLogFiles(int combineCQMaxExtraLookBackCommitLogFiles) { + this.combineCQMaxExtraLookBackCommitLogFiles = combineCQMaxExtraLookBackCommitLogFiles; + } } 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 29be9e7c614..db0b5dda7b0 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 @@ -330,7 +330,7 @@ private void dledgerRecoverAbnormally(long maxPhyOffsetOfConsumeQueue) throws Ro MmapFile mmapFile = null; for (; index >= 0; index--) { mmapFile = mmapFiles.get(index); - if (isMmapFileMatchedRecover(mmapFile)) { + if (isMmapFileMatchedRecover(mmapFile, false)) { log.info("dledger recover from this mappFile " + mmapFile.getFileName()); break; } @@ -426,7 +426,7 @@ private void setRecoverPosition() { log.info("Will set the initial commitlog offset={} for dledger", dividedCommitlogOffset); } - private boolean isMmapFileMatchedRecover(final MmapFile mmapFile) throws RocksDBException { + private boolean isMmapFileMatchedRecover(final MmapFile mmapFile, boolean recoverNormally) throws RocksDBException { ByteBuffer byteBuffer = mmapFile.sliceByteBuffer(); int magicCode = byteBuffer.getInt(DLedgerEntry.BODY_OFFSET + MessageDecoder.MESSAGE_MAGIC_CODE_POSITION); @@ -434,51 +434,34 @@ private boolean isMmapFileMatchedRecover(final MmapFile mmapFile) throws RocksDB return false; } - if (this.defaultMessageStore.getMessageStoreConfig().isEnableRocksDBStore()) { - final long maxPhyOffsetInConsumeQueue = this.defaultMessageStore.getQueueStore().getMaxPhyOffsetInConsumeQueue(); - long phyOffset = byteBuffer.getLong(DLedgerEntry.BODY_OFFSET + MessageDecoder.MESSAGE_PHYSIC_OFFSET_POSITION); - if (phyOffset <= maxPhyOffsetInConsumeQueue) { - log.info("find check. beginPhyOffset: {}, maxPhyOffsetInConsumeQueue: {}", phyOffset, maxPhyOffsetInConsumeQueue); - return true; - } + int storeTimestampPosition; + int sysFlag = byteBuffer.getInt(DLedgerEntry.BODY_OFFSET + MessageDecoder.SYSFLAG_POSITION); + if ((sysFlag & MessageSysFlag.BORNHOST_V6_FLAG) == 0) { + storeTimestampPosition = MessageDecoder.MESSAGE_STORE_TIMESTAMP_POSITION; } else { - int storeTimestampPosition; - int sysFlag = byteBuffer.getInt(DLedgerEntry.BODY_OFFSET + MessageDecoder.SYSFLAG_POSITION); - if ((sysFlag & MessageSysFlag.BORNHOST_V6_FLAG) == 0) { - storeTimestampPosition = MessageDecoder.MESSAGE_STORE_TIMESTAMP_POSITION; - } else { - // v6 address is 12 byte larger than v4 - storeTimestampPosition = MessageDecoder.MESSAGE_STORE_TIMESTAMP_POSITION + 12; - } + // v6 address is 12 byte larger than v4 + storeTimestampPosition = MessageDecoder.MESSAGE_STORE_TIMESTAMP_POSITION + 12; + } - long storeTimestamp = byteBuffer.getLong(DLedgerEntry.BODY_OFFSET + storeTimestampPosition); - if (storeTimestamp == 0) { - return false; - } + long storeTimestamp = byteBuffer.getLong(DLedgerEntry.BODY_OFFSET + storeTimestampPosition); + if (storeTimestamp == 0) { + return false; + } - if (this.defaultMessageStore.getMessageStoreConfig().isMessageIndexEnable() - && this.defaultMessageStore.getMessageStoreConfig().isMessageIndexSafe()) { - if (storeTimestamp <= this.defaultMessageStore.getStoreCheckpoint().getMinTimestampIndex()) { - log.info("dledger find check timestamp, {} {}", - storeTimestamp, - UtilAll.timeMillisToHumanString(storeTimestamp)); - return true; - } - } else { - if (storeTimestamp <= this.defaultMessageStore.getStoreCheckpoint().getMinTimestamp()) { - log.info("dledger find check timestamp, {} {}", - storeTimestamp, - UtilAll.timeMillisToHumanString(storeTimestamp)); - return true; - } + if (this.defaultMessageStore.getMessageStoreConfig().isMessageIndexEnable() + && this.defaultMessageStore.getMessageStoreConfig().isMessageIndexSafe()) { + if (storeTimestamp > this.defaultMessageStore.getStoreCheckpoint().getIndexMsgTimestamp()) { + return false; } } - return false; + + long phyOffset = byteBuffer.getLong(DLedgerEntry.BODY_OFFSET + MessageDecoder.MESSAGE_PHYSIC_OFFSET_POSITION); + return this.defaultMessageStore.getQueueStore().isMappedFileMatchedRecover(phyOffset, storeTimestamp, recoverNormally); } @Override - public void recoverNormally(long maxPhyOffsetOfConsumeQueue) throws RocksDBException { - dledgerRecoverNormally(maxPhyOffsetOfConsumeQueue); + public void recoverNormally(long dispatchFromPhyOffset) throws RocksDBException { + dledgerRecoverNormally(dispatchFromPhyOffset); } @Override diff --git a/store/src/main/java/org/apache/rocketmq/store/metrics/DefaultStoreMetricsManager.java b/store/src/main/java/org/apache/rocketmq/store/metrics/DefaultStoreMetricsManager.java index db4c7bb7662..ef72de8baa8 100644 --- a/store/src/main/java/org/apache/rocketmq/store/metrics/DefaultStoreMetricsManager.java +++ b/store/src/main/java/org/apache/rocketmq/store/metrics/DefaultStoreMetricsManager.java @@ -28,6 +28,10 @@ import io.opentelemetry.sdk.metrics.InstrumentType; import io.opentelemetry.sdk.metrics.View; import io.opentelemetry.sdk.metrics.ViewBuilder; +import java.io.File; +import java.util.Arrays; +import java.util.List; +import java.util.function.Supplier; import org.apache.rocketmq.common.Pair; import org.apache.rocketmq.common.metrics.NopLongCounter; import org.apache.rocketmq.common.metrics.NopLongHistogram; @@ -39,11 +43,6 @@ import org.apache.rocketmq.store.timer.TimerMetrics; import org.apache.rocketmq.store.timer.TimerWheel; -import java.io.File; -import java.util.Arrays; -import java.util.List; -import java.util.function.Supplier; - import static org.apache.rocketmq.store.metrics.DefaultStoreMetricsConstant.COUNTER_TIMER_DEQUEUE_TOTAL; import static org.apache.rocketmq.store.metrics.DefaultStoreMetricsConstant.COUNTER_TIMER_ENQUEUE_TOTAL; import static org.apache.rocketmq.store.metrics.DefaultStoreMetricsConstant.DEFAULT_STORAGE_MEDIUM; @@ -105,6 +104,10 @@ public static List> getMetricsView() { public static void init(Meter meter, Supplier attributesBuilderSupplier, DefaultMessageStore messageStore) { + + // Also add some metrics for rocksdb's monitoring. + RocksDBStoreMetricsManager.init(meter, attributesBuilderSupplier, messageStore.getQueueStore()); + DefaultStoreMetricsManager.attributesBuilderSupplier = attributesBuilderSupplier; DefaultStoreMetricsManager.messageStoreConfig = messageStore.getMessageStoreConfig(); diff --git a/store/src/main/java/org/apache/rocketmq/store/metrics/RocksDBStoreMetricsManager.java b/store/src/main/java/org/apache/rocketmq/store/metrics/RocksDBStoreMetricsManager.java index 6029488056c..5faea8beaf4 100644 --- a/store/src/main/java/org/apache/rocketmq/store/metrics/RocksDBStoreMetricsManager.java +++ b/store/src/main/java/org/apache/rocketmq/store/metrics/RocksDBStoreMetricsManager.java @@ -24,17 +24,17 @@ import io.opentelemetry.api.metrics.ObservableLongGauge; import io.opentelemetry.sdk.metrics.InstrumentSelector; import io.opentelemetry.sdk.metrics.ViewBuilder; +import java.util.List; +import java.util.function.Supplier; import org.apache.rocketmq.common.Pair; import org.apache.rocketmq.common.metrics.NopObservableDoubleGauge; import org.apache.rocketmq.common.metrics.NopObservableLongGauge; -import org.apache.rocketmq.store.RocksDBMessageStore; import org.apache.rocketmq.store.config.MessageStoreConfig; +import org.apache.rocketmq.store.queue.CombineConsumeQueueStore; +import org.apache.rocketmq.store.queue.ConsumeQueueStoreInterface; import org.apache.rocketmq.store.queue.RocksDBConsumeQueueStore; import org.rocksdb.TickerType; -import java.util.List; -import java.util.function.Supplier; - import static org.apache.rocketmq.store.metrics.DefaultStoreMetricsConstant.DEFAULT_STORAGE_MEDIUM; import static org.apache.rocketmq.store.metrics.DefaultStoreMetricsConstant.DEFAULT_STORAGE_TYPE; import static org.apache.rocketmq.store.metrics.DefaultStoreMetricsConstant.GAUGE_BYTES_ROCKSDB_READ; @@ -78,50 +78,62 @@ public static List> getMetricsView() { } public static void init(Meter meter, Supplier attributesBuilderSupplier, - RocksDBMessageStore messageStore) { + ConsumeQueueStoreInterface consumeQueueStore) { + + RocksDBConsumeQueueStore rocksDBMessageStore; + if (consumeQueueStore instanceof RocksDBConsumeQueueStore) { + rocksDBMessageStore = (RocksDBConsumeQueueStore) consumeQueueStore; + } else if (consumeQueueStore instanceof CombineConsumeQueueStore) { + rocksDBMessageStore = ((CombineConsumeQueueStore) consumeQueueStore).getRocksDBConsumeQueueStore(); + } else { + rocksDBMessageStore = null; + } + + if (rocksDBMessageStore == null) { + return; + } + RocksDBStoreMetricsManager.attributesBuilderSupplier = attributesBuilderSupplier; bytesRocksdbWritten = meter.gaugeBuilder(GAUGE_BYTES_ROCKSDB_WRITTEN) .setDescription("The cumulative number of bytes written to the database.") .ofLongs() .buildWithCallback(measurement -> { - measurement.record(((RocksDBConsumeQueueStore)messageStore.getQueueStore()) + measurement.record(rocksDBMessageStore .getStatistics().getTickerCount(TickerType.BYTES_WRITTEN), newAttributesBuilder().put("type", "consume_queue").build()); }); bytesRocksdbRead = meter.gaugeBuilder(GAUGE_BYTES_ROCKSDB_READ) .setDescription("The cumulative number of bytes read from the database.") .ofLongs() .buildWithCallback(measurement -> { - measurement.record(((RocksDBConsumeQueueStore)messageStore.getQueueStore()) + measurement.record(rocksDBMessageStore .getStatistics().getTickerCount(TickerType.BYTES_READ), newAttributesBuilder().put("type", "consume_queue").build()); }); timesRocksdbWrittenSelf = meter.gaugeBuilder(DefaultStoreMetricsConstant.GAUGE_TIMES_ROCKSDB_WRITTEN_SELF) .setDescription("The cumulative number of write operations performed by self.") .ofLongs() .buildWithCallback(measurement -> { - measurement.record(((RocksDBConsumeQueueStore)messageStore.getQueueStore()) + measurement.record(rocksDBMessageStore .getStatistics().getTickerCount(TickerType.WRITE_DONE_BY_SELF), newAttributesBuilder().put("type", "consume_queue").build()); }); timesRocksdbWrittenOther = meter.gaugeBuilder(DefaultStoreMetricsConstant.GAUGE_TIMES_ROCKSDB_WRITTEN_OTHER) .setDescription("The cumulative number of write operations performed by other.") .ofLongs() .buildWithCallback(measurement -> { - measurement.record(((RocksDBConsumeQueueStore)messageStore.getQueueStore()) + measurement.record(rocksDBMessageStore .getStatistics().getTickerCount(TickerType.WRITE_DONE_BY_OTHER), newAttributesBuilder().put("type", "consume_queue").build()); }); timesRocksdbRead = meter.gaugeBuilder(DefaultStoreMetricsConstant.GAUGE_TIMES_ROCKSDB_READ) .setDescription("The cumulative number of write operations performed by other.") .ofLongs() .buildWithCallback(measurement -> { - measurement.record(((RocksDBConsumeQueueStore)messageStore.getQueueStore()) + measurement.record(rocksDBMessageStore .getStatistics().getTickerCount(TickerType.NUMBER_KEYS_READ), newAttributesBuilder().put("type", "consume_queue").build()); }); rocksdbCacheHitRate = meter.gaugeBuilder(DefaultStoreMetricsConstant.GAUGE_RATE_ROCKSDB_CACHE_HIT) .setDescription("The rate at which cache lookups were served from the cache rather than needing to be fetched from disk.") .buildWithCallback(measurement -> { - long newHitTimes = ((RocksDBConsumeQueueStore)messageStore.getQueueStore()) - .getStatistics().getTickerCount(TickerType.BLOCK_CACHE_HIT); - long newMissTimes = ((RocksDBConsumeQueueStore)messageStore.getQueueStore()) - .getStatistics().getTickerCount(TickerType.BLOCK_CACHE_MISS); + long newHitTimes = rocksDBMessageStore.getStatistics().getTickerCount(TickerType.BLOCK_CACHE_HIT); + long newMissTimes = rocksDBMessageStore.getStatistics().getTickerCount(TickerType.BLOCK_CACHE_MISS); long totalPeriod = newHitTimes - blockCacheHitTimes + newMissTimes - blockCacheMissTimes; double hitRate = totalPeriod == 0 ? 0 : (double)(newHitTimes - blockCacheHitTimes) / totalPeriod; blockCacheHitTimes = newHitTimes; @@ -132,13 +144,13 @@ public static void init(Meter meter, Supplier attributesBuild .setDescription("The cumulative number of compressions that have occurred.") .ofLongs() .buildWithCallback(measurement -> { - measurement.record(((RocksDBConsumeQueueStore)messageStore.getQueueStore()) + measurement.record(rocksDBMessageStore .getStatistics().getTickerCount(TickerType.NUMBER_BLOCK_COMPRESSED), newAttributesBuilder().put("type", "consume_queue").build()); }); bytesRocksdbAmplificationRead = meter.gaugeBuilder(DefaultStoreMetricsConstant.GAUGE_BYTES_READ_AMPLIFICATION) .setDescription("The rate at which cache lookups were served from the cache rather than needing to be fetched from disk.") .buildWithCallback(measurement -> { - measurement.record(((RocksDBConsumeQueueStore)messageStore.getQueueStore()) + measurement.record(rocksDBMessageStore .getStatistics().getTickerCount(TickerType.READ_AMP_TOTAL_READ_BYTES), newAttributesBuilder().put("type", "consume_queue").build()); }); } diff --git a/store/src/main/java/org/apache/rocketmq/store/plugin/AbstractPluginMessageStore.java b/store/src/main/java/org/apache/rocketmq/store/plugin/AbstractPluginMessageStore.java index d5d6236458e..74c67d01621 100644 --- a/store/src/main/java/org/apache/rocketmq/store/plugin/AbstractPluginMessageStore.java +++ b/store/src/main/java/org/apache/rocketmq/store/plugin/AbstractPluginMessageStore.java @@ -17,6 +17,10 @@ package org.apache.rocketmq.store.plugin; +import io.opentelemetry.api.common.AttributesBuilder; +import io.opentelemetry.api.metrics.Meter; +import io.opentelemetry.sdk.metrics.InstrumentSelector; +import io.opentelemetry.sdk.metrics.ViewBuilder; import java.nio.ByteBuffer; import java.util.HashMap; import java.util.LinkedList; @@ -58,11 +62,6 @@ import org.apache.rocketmq.store.util.PerfCounter; import org.rocksdb.RocksDBException; -import io.opentelemetry.api.common.AttributesBuilder; -import io.opentelemetry.api.metrics.Meter; -import io.opentelemetry.sdk.metrics.InstrumentSelector; -import io.opentelemetry.sdk.metrics.ViewBuilder; - public abstract class AbstractPluginMessageStore implements MessageStore { protected MessageStore next; protected MessageStorePluginContext context; @@ -303,11 +302,6 @@ public long flush() { return next.flush(); } - @Override - public boolean resetWriteOffset(long phyOffset) { - return next.resetWriteOffset(phyOffset); - } - @Override public long getConfirmOffset() { return next.getConfirmOffset(); diff --git a/store/src/main/java/org/apache/rocketmq/store/queue/AbstractConsumeQueueStore.java b/store/src/main/java/org/apache/rocketmq/store/queue/AbstractConsumeQueueStore.java index ef693dc1e65..1f10a2f7c2d 100644 --- a/store/src/main/java/org/apache/rocketmq/store/queue/AbstractConsumeQueueStore.java +++ b/store/src/main/java/org/apache/rocketmq/store/queue/AbstractConsumeQueueStore.java @@ -46,7 +46,6 @@ public AbstractConsumeQueueStore(DefaultMessageStore messageStore) { } } - @Override public void putMessagePositionInfoWrapper(ConsumeQueueInterface consumeQueue, DispatchRequest request) { consumeQueue.putMessagePositionInfoWrapper(request); } @@ -56,17 +55,11 @@ public Long getMaxOffset(String topic, int queueId) throws ConsumeQueueException return this.queueOffsetOperator.currentQueueOffset(topic + "-" + queueId); } - @Override public void setTopicQueueTable(ConcurrentMap topicQueueTable) { this.queueOffsetOperator.setTopicQueueTable(topicQueueTable); this.queueOffsetOperator.setLmqTopicQueueTable(topicQueueTable); } - @Override - public ConcurrentMap getTopicQueueTable() { - return this.queueOffsetOperator.getTopicQueueTable(); - } - @Override public void assignQueueOffset(MessageExtBrokerInner msg) throws RocksDBException { ConsumeQueueInterface consumeQueue = findOrCreateConsumeQueue(msg.getTopic(), msg.getQueueId()); @@ -89,7 +82,6 @@ public long getLmqQueueOffset(String topic, int queueId) throws ConsumeQueueExce return queueOffsetOperator.getLmqOffset(topic, queueId, (t, q) -> 0L); } - @Override public void removeTopicQueueTable(String topic, Integer queueId) { this.queueOffsetOperator.remove(topic, queueId); } @@ -99,12 +91,6 @@ public ConcurrentMap> getC return this.consumeQueueTable; } - @Override - public ConcurrentMap findConsumeQueueMap(String topic) { - return this.consumeQueueTable.get(topic); - } - - @Override public long getStoreTime(CqUnit cqUnit) { if (cqUnit != null) { try { @@ -117,4 +103,43 @@ public long getStoreTime(CqUnit cqUnit) { } return -1; } + + /** + * get max physic offset in consumeQueue + * + * @return the max physic offset in consumeQueue + * @throws RocksDBException only in rocksdb mode + */ + public abstract long getMaxPhyOffsetInConsumeQueue() throws RocksDBException; + + /** + * destroy the specific consumeQueue + * + * @param consumeQueue consumeQueue to be destroyed + * @throws RocksDBException only in rocksdb mode + */ + protected abstract void destroy(ConsumeQueueInterface consumeQueue) throws RocksDBException; + + @Override + public boolean deleteTopic(String topic) { + ConcurrentMap queueTable = this.consumeQueueTable.get(topic); + + if (queueTable == null || queueTable.isEmpty()) { + return false; + } + + for (ConsumeQueueInterface cq : queueTable.values()) { + try { + destroy(cq); + } catch (RocksDBException e) { + log.error("DeleteTopic: ConsumeQueue cleans error!, topic={}, queueId={}", cq.getTopic(), cq.getQueueId(), e); + } + log.info("DeleteTopic: ConsumeQueue has been cleaned, topic={}, queueId={}", cq.getTopic(), cq.getQueueId()); + removeTopicQueueTable(cq.getTopic(), cq.getQueueId()); + } + + // remove topic from cq table + this.consumeQueueTable.remove(topic); + return true; + } } 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 16171827245..9d882eea3d4 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 @@ -24,8 +24,8 @@ import java.util.concurrent.ConcurrentSkipListMap; import java.util.function.Function; import org.apache.commons.lang3.StringUtils; -import org.apache.rocketmq.common.Pair; import org.apache.rocketmq.common.BoundaryType; +import org.apache.rocketmq.common.Pair; import org.apache.rocketmq.common.attribute.CQType; import org.apache.rocketmq.common.constant.LoggerName; import org.apache.rocketmq.common.message.MessageAccessor; @@ -70,6 +70,7 @@ public class BatchConsumeQueue implements ConsumeQueueInterface { public static final int INVALID_POS = -1; protected final MappedFileQueue mappedFileQueue; protected MessageStore messageStore; + protected ConsumeQueueStore consumeQueueStore; protected final String topic; protected final int queueId; protected final ByteBuffer byteBufferItem; @@ -93,10 +94,12 @@ public BatchConsumeQueue( final String storePath, final int mappedFileSize, final MessageStore messageStore, + final ConsumeQueueStore consumeQueueStore, final String subfolder) { this.storePath = storePath; this.mappedFileSize = mappedFileSize; this.messageStore = messageStore; + this.consumeQueueStore = consumeQueueStore; this.commitLogSize = messageStore.getCommitLog().getCommitLogSize(); this.topic = topic; @@ -113,6 +116,16 @@ public BatchConsumeQueue( this.byteBufferItem = ByteBuffer.allocate(CQ_STORE_UNIT_SIZE); } + public BatchConsumeQueue( + final String topic, + final int queueId, + final String storePath, + final int mappedFileSize, + final MessageStore messageStore, + final String subfolder) { + this(topic, queueId, storePath, mappedFileSize, messageStore, (ConsumeQueueStore) messageStore.getQueueStore(), subfolder); + } + public BatchConsumeQueue( final String topic, final int queueId, @@ -329,14 +342,14 @@ public CqUnit get(long offset) { @Override public Pair getCqUnitAndStoreTime(long index) { CqUnit cqUnit = get(index); - Long messageStoreTime = this.messageStore.getQueueStore().getStoreTime(cqUnit); + Long messageStoreTime = this.consumeQueueStore.getStoreTime(cqUnit); return new Pair<>(cqUnit, messageStoreTime); } @Override public Pair getEarliestUnitAndStoreTime() { CqUnit cqUnit = getEarliestUnit(); - Long messageStoreTime = this.messageStore.getQueueStore().getStoreTime(cqUnit); + Long messageStoreTime = this.consumeQueueStore.getStoreTime(cqUnit); return new Pair<>(cqUnit, messageStoreTime); } @@ -1175,4 +1188,8 @@ public long estimateMessageCount(long from, long to, MessageFilter filter) { log.debug("Result={}, raw={}, match={}, sample={}", result, raw, match, sample); return result; } + + public void initializeWithOffset(long offset) { + // not support now + } } diff --git a/store/src/main/java/org/apache/rocketmq/store/queue/CombineConsumeQueueStore.java b/store/src/main/java/org/apache/rocketmq/store/queue/CombineConsumeQueueStore.java new file mode 100644 index 00000000000..1bd0a888b20 --- /dev/null +++ b/store/src/main/java/org/apache/rocketmq/store/queue/CombineConsumeQueueStore.java @@ -0,0 +1,527 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.rocketmq.store.queue; + +import com.alibaba.fastjson.JSON; +import com.google.common.annotations.VisibleForTesting; +import java.util.Arrays; +import java.util.LinkedList; +import java.util.Map; +import java.util.Set; +import java.util.concurrent.ConcurrentMap; +import java.util.concurrent.atomic.AtomicInteger; +import org.apache.commons.lang3.StringUtils; +import org.apache.rocketmq.common.BoundaryType; +import org.apache.rocketmq.common.CheckRocksdbCqWriteResult; +import org.apache.rocketmq.common.Pair; +import org.apache.rocketmq.common.constant.LoggerName; +import org.apache.rocketmq.common.message.MessageExtBrokerInner; +import org.apache.rocketmq.store.DefaultMessageStore; +import org.apache.rocketmq.store.DispatchRequest; +import org.apache.rocketmq.store.StoreType; +import org.apache.rocketmq.store.config.MessageStoreConfig; +import org.apache.rocketmq.store.exception.ConsumeQueueException; +import org.apache.rocketmq.store.exception.StoreException; +import org.rocksdb.RocksDBException; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +public class CombineConsumeQueueStore implements ConsumeQueueStoreInterface { + private static final Logger log = LoggerFactory.getLogger(LoggerName.STORE_LOGGER_NAME); + private static final Logger BROKER_LOG = LoggerFactory.getLogger(LoggerName.BROKER_LOGGER_NAME); + + private final MessageStoreConfig messageStoreConfig; + + // Inner consume queue store. + private final LinkedList innerConsumeQueueStoreList = new LinkedList<>(); + private final ConsumeQueueStore consumeQueueStore; + private final RocksDBConsumeQueueStore rocksDBConsumeQueueStore; + + // currentReadStore can be dynamically changed during the broker running + private final AbstractConsumeQueueStore currentReadStore; + + // consume queue store for assign offset and increase offset. + private final AbstractConsumeQueueStore assignOffsetStore; + + // used for search recover form which commitLog mappedFile. + // when assignOffsetStore allow recovering from here, but other do not allow, toleranceFailuresNum will minus 1. + // if toleranceFailuresNum is 0, only pay attention to whether assignOffsetStore is allowed + private final AtomicInteger toleranceLookBackFailuresNum; + + public CombineConsumeQueueStore(DefaultMessageStore messageStore) { + this.messageStoreConfig = messageStore.getMessageStoreConfig(); + toleranceLookBackFailuresNum = new AtomicInteger(messageStoreConfig.getCombineCQMaxExtraLookBackCommitLogFiles()); + + Set loadingConsumeQueueTypeSet = StoreType.fromString(messageStoreConfig.getCombineCQLoadingCQTypes()); + if (loadingConsumeQueueTypeSet.isEmpty()) { + throw new IllegalArgumentException("CombineConsumeQueueStore loadingCQTypes is empty"); + } + + if (loadingConsumeQueueTypeSet.contains(StoreType.DEFAULT)) { + this.consumeQueueStore = new ConsumeQueueStore(messageStore); + this.innerConsumeQueueStoreList.addFirst(consumeQueueStore); + } else { + this.consumeQueueStore = null; + } + + if (loadingConsumeQueueTypeSet.contains(StoreType.DEFAULT_ROCKSDB)) { + this.rocksDBConsumeQueueStore = new RocksDBConsumeQueueStore(messageStore); + this.innerConsumeQueueStoreList.addFirst(rocksDBConsumeQueueStore); + } else { + this.rocksDBConsumeQueueStore = null; + } + + if (innerConsumeQueueStoreList.isEmpty()) { + throw new IllegalArgumentException("CombineConsumeQueueStore loadingCQTypes is empty"); + } + + assignOffsetStore = getInnerStoreByString(messageStoreConfig.getCombineAssignOffsetCQType()); + if (assignOffsetStore == null) { + log.error("CombineConsumeQueue chooseAssignOffsetStore fail, prefer={}", messageStoreConfig.getCombineAssignOffsetCQType()); + throw new IllegalArgumentException("CombineConsumeQueue chooseAssignOffsetStore fail"); + } + + currentReadStore = getInnerStoreByString(messageStoreConfig.getCombineCQPreferCQType()); + if (currentReadStore == null) { + log.error("CombineConsumeQueue choosePreferCQ fail, prefer={}", messageStoreConfig.getCombineCQPreferCQType()); + throw new IllegalArgumentException("CombineConsumeQueue choosePreferCQ fail"); + } + + log.info("CombineConsumeQueueStore init, consumeQueueStoreList={}, currentReadStore={}, assignOffsetStore={}", + innerConsumeQueueStoreList, currentReadStore.getClass().getSimpleName(), assignOffsetStore.getClass().getSimpleName()); + } + + @Override + public boolean load() { + for (AbstractConsumeQueueStore store : innerConsumeQueueStoreList) { + if (!store.load()) { + log.error("CombineConsumeQueueStore load fail, loadType={}", store.getClass().getSimpleName()); + return false; + } + } + log.info("CombineConsumeQueueStore load success"); + return true; + } + + @Override + public void recover(boolean concurrently) throws RocksDBException { + for (AbstractConsumeQueueStore store : innerConsumeQueueStoreList) { + store.recover(concurrently); + } + log.info("CombineConsumeQueueStore recover success, concurrently={}", concurrently); + } + + @Override + public boolean isMappedFileMatchedRecover(long phyOffset, long storeTimestamp, + boolean recoverNormally) throws RocksDBException { + if (!assignOffsetStore.isMappedFileMatchedRecover(phyOffset, storeTimestamp, recoverNormally)) { + return false; + } + + for (AbstractConsumeQueueStore store : innerConsumeQueueStoreList) { + if (store == assignOffsetStore || store.isMappedFileMatchedRecover(phyOffset, storeTimestamp, recoverNormally)) { + continue; + } + + if (toleranceLookBackFailuresNum.getAndIncrement() <= 0) { + // toleranceLookBackFailuresNum <= 0, so only can read from assignOffsetStore + if (assignOffsetStore != currentReadStore) { + log.error("CombineConsumeQueueStore currentReadStore not satisfied readable conditions, assignOffsetStore={}, currentReadStore={}", + assignOffsetStore.getClass().getSimpleName(), currentReadStore.getClass().getSimpleName()); + throw new IllegalArgumentException(store.getClass().getSimpleName() + " not satisfied readable conditions, only can read from " + assignOffsetStore.getClass().getSimpleName()); + } + log.warn("CombineConsumeQueueStore can not recover all inner store, maybe some inner store start haven’t started before, store={}", + store.getClass().getSimpleName()); + return true; + } else { + return false; + } + } + return true; + } + + @Override + public long getDispatchFromPhyOffset() { + long dispatchFromPhyOffset = assignOffsetStore.getDispatchFromPhyOffset(); + for (AbstractConsumeQueueStore store : innerConsumeQueueStoreList) { + if (store == assignOffsetStore) { + continue; + } + if (store.getDispatchFromPhyOffset() < dispatchFromPhyOffset) { + dispatchFromPhyOffset = store.getDispatchFromPhyOffset(); + } + } + return dispatchFromPhyOffset; + } + + @Override + public void start() { + boolean success = false; + try { + success = checkAssignOffset(true); + } catch (RocksDBException e) { + log.error("CombineConsumeQueueStore checkAssignOffsetStore fail", e); + } + + if (!success && assignOffsetStore != currentReadStore) { + throw new RuntimeException("CombineConsumeQueueStore checkAssignOffsetStore fail"); + } + + for (AbstractConsumeQueueStore store : innerConsumeQueueStoreList) { + store.start(); + } + } + + public boolean checkAssignOffset(boolean initializeOffset) throws RocksDBException { + if (innerConsumeQueueStoreList.size() <= 1) { + return true; + } + + boolean result = true; + // for each topic and queueId in assignOffsetStore + for (Map.Entry> entry : assignOffsetStore.getConsumeQueueTable().entrySet()) { + for (Map.Entry entry0 : entry.getValue().entrySet()) { + String topic = entry.getKey(); + Integer queueId = entry0.getKey(); + long maxOffsetInAssign = entry0.getValue().getMaxOffsetInQueue(); + + for (AbstractConsumeQueueStore abstractConsumeQueueStore : innerConsumeQueueStoreList) { + // skip compare self + if (abstractConsumeQueueStore == assignOffsetStore) { + continue; + } + + ConsumeQueueInterface queue = abstractConsumeQueueStore.findOrCreateConsumeQueue(topic, queueId); + long maxOffset0 = queue.getMaxOffsetInQueue(); + + if (maxOffsetInAssign == maxOffset0 || maxOffsetInAssign <= 0 && maxOffset0 <= 0) { + continue; + } + + if (maxOffset0 > 0) { + log.error("CombineConsumeQueueStore checkAssignOffsetStore fail, topic={}, queueId={}, maxOffsetInAssign={}, otherCQ={}, maxOffset0={}", + topic, queueId, maxOffsetInAssign, abstractConsumeQueueStore.getClass().getSimpleName(), maxOffset0); + result = false; + } + + if (initializeOffset) { + queue.initializeWithOffset(maxOffsetInAssign); + log.info("CombineConsumeQueueStore initialize offset in queue, topic={}, queueId={}, maxOffsetInAssign={}, otherCQ={}, maxOffset0={}, maxOffsetNew={}", + topic, queueId, maxOffsetInAssign, abstractConsumeQueueStore.getClass().getSimpleName(), maxOffset0, queue.getMaxOffsetInQueue()); + } + } + } + } + return result; + } + + @Override + public boolean shutdown() { + boolean result = true; + for (AbstractConsumeQueueStore store : innerConsumeQueueStoreList) { + if (!store.shutdown()) { + result = false; + } + } + return result; + } + + @Override + public void destroy(boolean loadAfterDestroy) { + for (AbstractConsumeQueueStore store : innerConsumeQueueStoreList) { + store.destroy(loadAfterDestroy); + } + } + + @Override + public boolean deleteTopic(String topic) { + boolean result = false; + for (AbstractConsumeQueueStore store : innerConsumeQueueStoreList) { + if (store.deleteTopic(topic)) { + result = true; + } + } + return result; + } + + @Override + public void flush() throws StoreException { + for (AbstractConsumeQueueStore store : innerConsumeQueueStoreList) { + store.flush(); + } + } + + @Override + public void cleanExpired(long minCommitLogOffset) { + for (AbstractConsumeQueueStore store : innerConsumeQueueStoreList) { + store.cleanExpired(minCommitLogOffset); + } + } + + @Override + public void checkSelf() { + for (AbstractConsumeQueueStore store : innerConsumeQueueStoreList) { + store.checkSelf(); + } + + if (messageStoreConfig.isCombineCQEnableCheckSelf()) { + try { + checkAssignOffset(false); + } catch (RocksDBException e) { + log.error("CombineConsumeQueueStore checkAssignOffsetStore fail in checkSelf", e); + } + CheckRocksdbCqWriteResult checkResult = doCheckCqWriteProgress(null, System.currentTimeMillis() - 10 * 60 * 1000, StoreType.DEFAULT, StoreType.DEFAULT_ROCKSDB); + BROKER_LOG.info("checkRocksdbCqWriteProgress result: {}", JSON.toJSONString(checkResult)); + } + } + + @Override + public void truncateDirty(long offsetToTruncate) throws RocksDBException { + for (AbstractConsumeQueueStore store : innerConsumeQueueStoreList) { + store.truncateDirty(offsetToTruncate); + } + } + + @Override + public void putMessagePositionInfoWrapper(DispatchRequest request) throws RocksDBException { + for (AbstractConsumeQueueStore store : innerConsumeQueueStoreList) { + store.putMessagePositionInfoWrapper(request); + } + } + + @Override + public ConcurrentMap> getConsumeQueueTable() { + return currentReadStore.getConsumeQueueTable(); + } + + @Override + public void assignQueueOffset(MessageExtBrokerInner msg) throws RocksDBException { + assignOffsetStore.assignQueueOffset(msg); + } + + @Override + public void increaseQueueOffset(MessageExtBrokerInner msg, short messageNum) { + assignOffsetStore.increaseQueueOffset(msg, messageNum); + } + + @Override + public void increaseLmqOffset(String topic, int queueId, short delta) throws ConsumeQueueException { + assignOffsetStore.increaseLmqOffset(topic, queueId, delta); + } + + @Override + public long getLmqQueueOffset(String topic, int queueId) throws ConsumeQueueException { + return assignOffsetStore.getLmqQueueOffset(topic, queueId); + } + + @Override + public void recoverOffsetTable(long minPhyOffset) { + for (AbstractConsumeQueueStore store : innerConsumeQueueStoreList) { + store.recoverOffsetTable(minPhyOffset); + } + } + + @Override + public Long getMaxOffset(String topic, int queueId) throws ConsumeQueueException { + return currentReadStore.getMaxOffset(topic, queueId); + } + + @Override + public long getMinOffsetInQueue(String topic, int queueId) throws RocksDBException { + return currentReadStore.getMinOffsetInQueue(topic, queueId); + } + + @Override + public long getOffsetInQueueByTime(String topic, int queueId, long timestamp, + BoundaryType boundaryType) throws RocksDBException { + return currentReadStore.getOffsetInQueueByTime(topic, queueId, timestamp, boundaryType); + } + + @Override + public ConsumeQueueInterface findOrCreateConsumeQueue(String topic, int queueId) { + return currentReadStore.findOrCreateConsumeQueue(topic, queueId); + } + + @Override + public ConsumeQueueInterface getConsumeQueue(String topic, int queueId) { + return currentReadStore.getConsumeQueue(topic, queueId); + } + + @Override + public long getTotalSize() { + long result = 0; + for (AbstractConsumeQueueStore store : innerConsumeQueueStoreList) { + result += store.getTotalSize(); + } + return result; + } + + public RocksDBConsumeQueueStore getRocksDBConsumeQueueStore() { + return rocksDBConsumeQueueStore; + } + + @VisibleForTesting + public ConsumeQueueStore getConsumeQueueStore() { + return consumeQueueStore; + } + + @VisibleForTesting + public AbstractConsumeQueueStore getCurrentReadStore() { + return currentReadStore; + } + + @VisibleForTesting + public AbstractConsumeQueueStore getAssignOffsetStore() { + return assignOffsetStore; + } + + public CheckRocksdbCqWriteResult doCheckCqWriteProgress(String requestTopic, long checkStoreTime, + StoreType baseStoreType, StoreType compareStoreType) { + CheckRocksdbCqWriteResult result = new CheckRocksdbCqWriteResult(); + AbstractConsumeQueueStore baseStore = getInnerStoreByStoreType(baseStoreType); + AbstractConsumeQueueStore compareStore = getInnerStoreByStoreType(compareStoreType); + + if (baseStore == null || compareStore == null) { + result.setCheckResult("baseStore or compareStore is null, no need check"); + result.setCheckStatus(CheckRocksdbCqWriteResult.CheckStatus.CHECK_OK.getValue()); + return result; + } + + ConcurrentMap> cqTable = baseStore.getConsumeQueueTable(); + StringBuilder diffResult = new StringBuilder(); + try { + if (StringUtils.isNotBlank(requestTopic)) { + boolean checkResult = processConsumeQueuesForTopic(cqTable.get(requestTopic), requestTopic, compareStore, diffResult, true, checkStoreTime); + 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> topicEntry : cqTable.entrySet()) { + boolean checkResult = processConsumeQueuesForTopic(topicEntry.getValue(), topicEntry.getKey(), compareStore, diffResult, false, checkStoreTime); + 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) { + log.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 queueMap, String topic, + AbstractConsumeQueueStore abstractConsumeQueueStore, StringBuilder diffResult, boolean printDetail, + long checkpointByStoreTime) { + boolean processResult = true; + for (Map.Entry queueEntry : queueMap.entrySet()) { + Integer queueId = queueEntry.getKey(); + ConsumeQueueInterface baseCQ = queueEntry.getValue(); + ConsumeQueueInterface compareCQ = abstractConsumeQueueStore.findOrCreateConsumeQueue(topic, queueId); + if (printDetail) { + String format = String.format("[topic: %s, queue: %s] \n kvEarliest : %s | kvLatest : %s \n fileEarliest: %s | fileEarliest: %s ", + topic, queueId, compareCQ.getEarliestUnit(), compareCQ.getLatestUnit(), baseCQ.getEarliestUnit(), baseCQ.getLatestUnit()); + diffResult.append(format).append("\n"); + } + + long minOffsetByTime = 0L; + try { + minOffsetByTime = abstractConsumeQueueStore.getOffsetInQueueByTime(topic, queueId, checkpointByStoreTime, BoundaryType.UPPER); + } catch (Exception e) { + // ignore + } + long minOffsetInQueue = compareCQ.getMinOffsetInQueue(); + long checkFrom = Math.max(minOffsetInQueue, minOffsetByTime); + long checkTo = baseCQ.getMaxOffsetInQueue() - 1; + /* + checkTo(maxOffsetInQueue - 1) + v + baseCQ +------------------------------------------------------+ + compareCQ +----------------------------------------------+ + ^ ^ + minOffsetInQueue minOffsetByTime + ^ + checkFrom = max(minOffsetInQueue, minOffsetByTime) + */ + // The latest message is earlier than the check time + Pair fileLatestCq = baseCQ.getCqUnitAndStoreTime(checkTo); + if (fileLatestCq != null) { + if (fileLatestCq.getObject2() < checkpointByStoreTime) { + continue; + } + } + for (long i = checkFrom; i <= checkTo; i++) { + Pair baseCqUnit = baseCQ.getCqUnitAndStoreTime(i); + Pair compareCqUnit = compareCQ.getCqUnitAndStoreTime(i); + if (baseCqUnit == null || compareCqUnit == null || !checkCqUnitEqual(compareCqUnit.getObject1(), baseCqUnit.getObject1())) { + log.error(String.format("[topic: %s, queue: %s, offset: %s] \n file : %s \n kv : %s \n", + topic, queueId, i, compareCqUnit != null ? compareCqUnit.getObject1() : "null", baseCqUnit != null ? baseCqUnit.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(); + } + + private AbstractConsumeQueueStore getInnerStoreByString(String storeTypeString) { + if (StoreType.DEFAULT.getStoreType().equalsIgnoreCase(storeTypeString)) { + return consumeQueueStore; + } else if (StoreType.DEFAULT_ROCKSDB.getStoreType().equalsIgnoreCase(storeTypeString)) { + return rocksDBConsumeQueueStore; + } else { + return null; + } + } + + private AbstractConsumeQueueStore getInnerStoreByStoreType(StoreType storeType) { + switch (storeType) { + case DEFAULT: + return consumeQueueStore; + case DEFAULT_ROCKSDB: + return rocksDBConsumeQueueStore; + default: + return null; + } + } +} 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 768c782b1dd..92857b631a4 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 @@ -198,4 +198,6 @@ public interface ConsumeQueueInterface extends FileQueueLifeCycle { * @return Number of matching records. */ long estimateMessageCount(long from, long to, MessageFilter filter); + + void initializeWithOffset(long offset); } 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 5f9c2f90be3..c04fcf31a97 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 @@ -34,6 +34,7 @@ import java.util.concurrent.LinkedBlockingQueue; import java.util.concurrent.TimeUnit; import org.apache.rocketmq.common.BoundaryType; +import org.apache.rocketmq.common.ServiceThread; import org.apache.rocketmq.common.ThreadFactoryImpl; import org.apache.rocketmq.common.TopicConfig; import org.apache.rocketmq.common.attribute.CQType; @@ -54,16 +55,33 @@ import static org.apache.rocketmq.store.config.StorePathConfigHelper.getStorePathConsumeQueue; public class ConsumeQueueStore extends AbstractConsumeQueueStore { + private final FlushConsumeQueueService flushConsumeQueueService; + private final CorrectLogicOffsetService correctLogicOffsetService; + private final CleanConsumeQueueService cleanConsumeQueueService; + + private long dispatchFromPhyOffset; + private long dispatchFromStoreTimestamp; public ConsumeQueueStore(DefaultMessageStore messageStore) { super(messageStore); + this.flushConsumeQueueService = new FlushConsumeQueueService(); + this.correctLogicOffsetService = new CorrectLogicOffsetService(); + this.cleanConsumeQueueService = new CleanConsumeQueueService(); } @Override public void start() { + this.flushConsumeQueueService.start(); + messageStore.getScheduledCleanQueueExecutorService().scheduleWithFixedDelay(this::cleanQueueFilesPeriodically, + 1000 * 60, this.messageStoreConfig.getCleanResourceInterval(), TimeUnit.MILLISECONDS); log.info("Default ConsumeQueueStore start!"); } + private void cleanQueueFilesPeriodically() { + this.correctLogicOffsetService.run(); + this.cleanConsumeQueueService.run(); + } + @Override public boolean load() { boolean cqLoadResult = loadConsumeQueues(getStorePathConsumeQueue(this.messageStoreConfig.getStorePathRootDir()), CQType.SimpleCQ); @@ -72,20 +90,27 @@ public boolean load() { } @Override - public boolean loadAfterDestroy() { - return true; - } - - @Override - public void recover() { - for (ConcurrentMap maps : this.consumeQueueTable.values()) { - for (ConsumeQueueInterface logic : maps.values()) { - this.recover(logic); + public void recover(boolean concurrently) { + log.info("Start to recover consume queue concurrently={}", concurrently); + if (concurrently) { + recoverConcurrently(); + } else { + for (ConcurrentMap maps : this.consumeQueueTable.values()) { + for (ConsumeQueueInterface logic : maps.values()) { + this.recover(logic); + } } } + + dispatchFromPhyOffset = this.getMaxPhyOffsetInConsumeQueue(); + dispatchFromStoreTimestamp = this.messageStore.getStoreCheckpoint().getMinTimestamp(); } @Override + public long getDispatchFromPhyOffset() { + return getMaxPhyOffsetInConsumeQueue(); + } + public boolean recoverConcurrently() { int count = 0; for (ConcurrentMap maps : this.consumeQueueTable.values()) { @@ -137,6 +162,7 @@ public boolean recoverConcurrently() { public boolean shutdown() { try { flush(); + this.flushConsumeQueueService.shutdown(); } catch (StoreException e) { log.error("Failed to flush all consume queues", e); return false; @@ -144,41 +170,15 @@ public boolean shutdown() { return true; } - @Override - public long rollNextFile(ConsumeQueueInterface consumeQueue, final long offset) { - FileQueueLifeCycle fileQueueLifeCycle = getLifeCycle(consumeQueue.getTopic(), consumeQueue.getQueueId()); - return fileQueueLifeCycle.rollNextFile(offset); - } - public void correctMinOffset(ConsumeQueueInterface consumeQueue, long minCommitLogOffset) { consumeQueue.correctMinOffset(minCommitLogOffset); } - @Override public void putMessagePositionInfoWrapper(DispatchRequest dispatchRequest) { ConsumeQueueInterface cq = this.findOrCreateConsumeQueue(dispatchRequest.getTopic(), dispatchRequest.getQueueId()); this.putMessagePositionInfoWrapper(cq, dispatchRequest); } - @Override - public List rangeQuery(String topic, int queueId, long startIndex, int num) { - return null; - } - - @Override - public ByteBuffer get(String topic, int queueId, long startIndex) { - return null; - } - - @Override - public long getMaxOffsetInQueue(String topic, int queueId) { - ConsumeQueueInterface logic = findOrCreateConsumeQueue(topic, queueId); - if (logic != null) { - return logic.getMaxOffsetInQueue(); - } - return 0; - } - @Override public long getOffsetInQueueByTime(String topic, int queueId, long timestamp, BoundaryType boundaryType) { ConsumeQueueInterface logic = findOrCreateConsumeQueue(topic, queueId); @@ -243,7 +243,8 @@ private ConsumeQueueInterface createConsumeQueueByType(CQType cqType, String top queueId, storePath, this.messageStoreConfig.getMappedFileSizeConsumeQueue(), - this.messageStore); + this.messageStore, + this); } else if (Objects.equals(CQType.BatchCQ, cqType)) { return new BatchConsumeQueue( topic, @@ -281,15 +282,6 @@ public void recover(ConsumeQueueInterface consumeQueue) { fileQueueLifeCycle.recover(); } - @Override - public Long getMaxPhyOffsetInConsumeQueue(String topic, int queueId) { - ConsumeQueueInterface logic = findOrCreateConsumeQueue(topic, queueId); - if (logic != null) { - return logic.getMaxPhysicOffset(); - } - return null; - } - @Override public long getMaxPhyOffsetInConsumeQueue() { long maxPhysicOffset = -1L; @@ -327,13 +319,11 @@ public void checkSelf() { } } - @Override public boolean flush(ConsumeQueueInterface consumeQueue, int flushLeastPages) { FileQueueLifeCycle fileQueueLifeCycle = getLifeCycle(consumeQueue.getTopic(), consumeQueue.getQueueId()); return fileQueueLifeCycle.flush(flushLeastPages); } - @Override public void flush() throws StoreException { for (Map.Entry> topicEntry : this.consumeQueueTable.entrySet()) { for (Map.Entry cqEntry : topicEntry.getValue().entrySet()) { @@ -348,7 +338,6 @@ public void destroy(ConsumeQueueInterface consumeQueue) { fileQueueLifeCycle.destroy(); } - @Override public int deleteExpiredFile(ConsumeQueueInterface consumeQueue, long minCommitLogPos) { FileQueueLifeCycle fileQueueLifeCycle = getLifeCycle(consumeQueue.getTopic(), consumeQueue.getQueueId()); return fileQueueLifeCycle.deleteExpiredFile(minCommitLogPos); @@ -370,13 +359,11 @@ public void cleanSwappedMap(ConsumeQueueInterface consumeQueue, long forceCleanS fileQueueLifeCycle.cleanSwappedMap(forceCleanSwapIntervalMs); } - @Override public boolean isFirstFileAvailable(ConsumeQueueInterface consumeQueue) { FileQueueLifeCycle fileQueueLifeCycle = getLifeCycle(consumeQueue.getTopic(), consumeQueue.getQueueId()); return fileQueueLifeCycle.isFirstFileAvailable(); } - @Override public boolean isFirstFileExist(ConsumeQueueInterface consumeQueue) { FileQueueLifeCycle fileQueueLifeCycle = getLifeCycle(consumeQueue.getTopic(), consumeQueue.getQueueId()); return fileQueueLifeCycle.isFirstFileExist(); @@ -417,7 +404,7 @@ public ConsumeQueueInterface findOrCreateConsumeQueue(String topic, int queueId) queueId, getStorePathConsumeQueue(this.messageStoreConfig.getStorePathRootDir()), this.messageStoreConfig.getMappedFileSizeConsumeQueue(), - this.messageStore); + this.messageStore, this); } ConsumeQueueInterface oldLogic = map.putIfAbsent(queueId, newLogic); @@ -430,6 +417,15 @@ public ConsumeQueueInterface findOrCreateConsumeQueue(String topic, int queueId) return logic; } + @Override + public ConsumeQueueInterface getConsumeQueue(String topic, int queueId) { + ConcurrentMap map = this.getConsumeQueueTable().get(topic); + if (map == null) { + return null; + } + return map.get(queueId); + } + public void setBatchTopicQueueTable(ConcurrentMap batchTopicQueueTable) { this.queueOffsetOperator.setBatchTopicQueueTable(batchTopicQueueTable); } @@ -520,7 +516,7 @@ private void compensateForHA(ConcurrentMap cqOffsetTable) { } @Override - public void destroy() { + public void destroy(boolean loadAfterDestroy) { for (ConcurrentMap maps : this.consumeQueueTable.values()) { for (ConsumeQueueInterface logic : maps.values()) { this.destroy(logic); @@ -573,9 +569,13 @@ public void cleanExpired(long minCommitLogOffset) { @Override public void truncateDirty(long offsetToTruncate) { - for (ConcurrentMap maps : this.consumeQueueTable.values()) { - for (ConsumeQueueInterface logic : maps.values()) { - this.truncateDirtyLogicFiles(logic, offsetToTruncate); + long maxPhyOffsetOfConsumeQueue = getMaxPhyOffsetInConsumeQueue(); + if (maxPhyOffsetOfConsumeQueue >= offsetToTruncate) { + log.warn("maxPhyOffsetOfConsumeQueue({}) >= processOffset({}), truncate dirty logic files", maxPhyOffsetOfConsumeQueue, offsetToTruncate); + for (ConcurrentMap maps : this.consumeQueueTable.values()) { + for (ConsumeQueueInterface logic : maps.values()) { + this.truncateDirtyLogicFiles(logic, offsetToTruncate); + } } } } @@ -590,4 +590,246 @@ public long getTotalSize() { } return totalSize; } + + @Override + public boolean isMappedFileMatchedRecover(long phyOffset, long storeTimestamp, boolean recoverNormally) { + if (recoverNormally) { + return phyOffset <= this.dispatchFromPhyOffset; + } else { + return storeTimestamp <= this.dispatchFromStoreTimestamp; + } + } + + public class FlushConsumeQueueService extends ServiceThread { + private static final int RETRY_TIMES_OVER = 3; + private long lastFlushTimestamp = 0; + + private void doFlush(int retryTimes) { + int flushConsumeQueueLeastPages = messageStoreConfig.getFlushConsumeQueueLeastPages(); + + if (retryTimes == RETRY_TIMES_OVER) { + flushConsumeQueueLeastPages = 0; + } + + long logicsMsgTimestamp = 0; + + int flushConsumeQueueThoroughInterval = messageStoreConfig.getFlushConsumeQueueThoroughInterval(); + long currentTimeMillis = System.currentTimeMillis(); + if (currentTimeMillis >= (this.lastFlushTimestamp + flushConsumeQueueThoroughInterval)) { + this.lastFlushTimestamp = currentTimeMillis; + flushConsumeQueueLeastPages = 0; + logicsMsgTimestamp = messageStore.getStoreCheckpoint().getLogicsMsgTimestamp(); + } + + for (ConcurrentMap maps : consumeQueueTable.values()) { + for (ConsumeQueueInterface cq : maps.values()) { + boolean result = false; + for (int i = 0; i < retryTimes && !result; i++) { + result = flush(cq, flushConsumeQueueLeastPages); + } + } + } + + if (messageStoreConfig.isEnableCompaction()) { + messageStore.getCompactionStore().flush(flushConsumeQueueLeastPages); + } + + if (0 == flushConsumeQueueLeastPages) { + if (logicsMsgTimestamp > 0) { + messageStore.getStoreCheckpoint().setLogicsMsgTimestamp(logicsMsgTimestamp); + } + messageStore.getStoreCheckpoint().flush(); + } + } + + @Override + public void run() { + log.info(this.getServiceName() + " service started"); + + while (!this.isStopped()) { + try { + int interval = messageStoreConfig.getFlushIntervalConsumeQueue(); + this.waitForRunning(interval); + this.doFlush(1); + } catch (Exception e) { + log.warn(this.getServiceName() + " service has exception. ", e); + } + } + + this.doFlush(RETRY_TIMES_OVER); + + log.info(this.getServiceName() + " service end"); + } + + @Override + public String getServiceName() { + if (messageStore.getBrokerConfig().isInBrokerContainer()) { + return messageStore.getBrokerIdentity().getIdentifier() + FlushConsumeQueueService.class.getSimpleName(); + } + return FlushConsumeQueueService.class.getSimpleName(); + } + + @Override + public long getJoinTime() { + return 1000 * 60; + } + } + + class CorrectLogicOffsetService { + private long lastForceCorrectTime = -1L; + + public void run() { + try { + this.correctLogicMinOffset(); + } catch (Throwable e) { + log.warn(this.getServiceName() + " service has exception. ", e); + } + } + + private boolean needCorrect(ConsumeQueueInterface logic, long minPhyOffset, long lastForeCorrectTimeCurRun) { + if (logic == null) { + return false; + } + // If first exist and not available, it means first file may destroy failed, delete it. + if (isFirstFileExist(logic) && !isFirstFileAvailable(logic)) { + log.error("CorrectLogicOffsetService.needCorrect. first file not available, trigger correct." + + " topic:{}, queue:{}, maxPhyOffset in queue:{}, minPhyOffset " + + "in commit log:{}, minOffset in queue:{}, maxOffset in queue:{}, cqType:{}" + , logic.getTopic(), logic.getQueueId(), logic.getMaxPhysicOffset() + , minPhyOffset, logic.getMinOffsetInQueue(), logic.getMaxOffsetInQueue(), logic.getCQType()); + return true; + } + + // logic.getMaxPhysicOffset() or minPhyOffset = -1 + // means there is no message in current queue, so no need to correct. + if (logic.getMaxPhysicOffset() == -1 || minPhyOffset == -1) { + return false; + } + + if (logic.getMaxPhysicOffset() < minPhyOffset) { + if (logic.getMinOffsetInQueue() < logic.getMaxOffsetInQueue()) { + log.error("CorrectLogicOffsetService.needCorrect. logic max phy offset: {} is less than min phy offset: {}, " + + "but min offset: {} is less than max offset: {}. topic:{}, queue:{}, cqType:{}." + , logic.getMaxPhysicOffset(), minPhyOffset, logic.getMinOffsetInQueue() + , logic.getMaxOffsetInQueue(), logic.getTopic(), logic.getQueueId(), logic.getCQType()); + return true; + } else if (logic.getMinOffsetInQueue() == logic.getMaxOffsetInQueue()) { + return false; + } else { + log.error("CorrectLogicOffsetService.needCorrect. It should not happen, logic max phy offset: {} is less than min phy offset: {}," + + " but min offset: {} is larger than max offset: {}. topic:{}, queue:{}, cqType:{}" + , logic.getMaxPhysicOffset(), minPhyOffset, logic.getMinOffsetInQueue() + , logic.getMaxOffsetInQueue(), logic.getTopic(), logic.getQueueId(), logic.getCQType()); + return false; + } + } + //the logic.getMaxPhysicOffset() >= minPhyOffset + int forceCorrectInterval = messageStoreConfig.getCorrectLogicMinOffsetForceInterval(); + if ((System.currentTimeMillis() - lastForeCorrectTimeCurRun) > forceCorrectInterval) { + lastForceCorrectTime = System.currentTimeMillis(); + CqUnit cqUnit = logic.getEarliestUnit(); + if (cqUnit == null) { + if (logic.getMinOffsetInQueue() == logic.getMaxOffsetInQueue()) { + return false; + } else { + log.error("CorrectLogicOffsetService.needCorrect. cqUnit is null, logic max phy offset: {} is greater than min phy offset: {}, " + + "but min offset: {} is not equal to max offset: {}. topic:{}, queue:{}, cqType:{}." + , logic.getMaxPhysicOffset(), minPhyOffset, logic.getMinOffsetInQueue() + , logic.getMaxOffsetInQueue(), logic.getTopic(), logic.getQueueId(), logic.getCQType()); + return true; + } + } + + if (cqUnit.getPos() < minPhyOffset) { + log.error("CorrectLogicOffsetService.needCorrect. logic max phy offset: {} is greater than min phy offset: {}, " + + "but minPhyPos in cq is: {}. min offset in queue: {}, max offset in queue: {}, topic:{}, queue:{}, cqType:{}." + , logic.getMaxPhysicOffset(), minPhyOffset, cqUnit.getPos(), logic.getMinOffsetInQueue() + , logic.getMaxOffsetInQueue(), logic.getTopic(), logic.getQueueId(), logic.getCQType()); + return true; + } + + if (cqUnit.getPos() >= minPhyOffset) { + + // Normal case, do not need to correct. + return false; + } + } + + return false; + } + + private void correctLogicMinOffset() { + + long lastForeCorrectTimeCurRun = lastForceCorrectTime; + long minPhyOffset = messageStore.getMinPhyOffset(); + for (ConcurrentMap maps : consumeQueueTable.values()) { + for (ConsumeQueueInterface logic : maps.values()) { + if (Objects.equals(CQType.SimpleCQ, logic.getCQType())) { + // cq is not supported for now. + continue; + } + if (needCorrect(logic, minPhyOffset, lastForeCorrectTimeCurRun)) { + doCorrect(logic, minPhyOffset); + } + } + } + } + + private void doCorrect(ConsumeQueueInterface logic, long minPhyOffset) { + deleteExpiredFile(logic, minPhyOffset); + int sleepIntervalWhenCorrectMinOffset = messageStoreConfig.getCorrectLogicMinOffsetSleepInterval(); + if (sleepIntervalWhenCorrectMinOffset > 0) { + try { + Thread.sleep(sleepIntervalWhenCorrectMinOffset); + } catch (InterruptedException ignored) { + } + } + } + + public String getServiceName() { + if (messageStore.getBrokerConfig().isInBrokerContainer()) { + return messageStore.getBrokerConfig().getIdentifier() + CorrectLogicOffsetService.class.getSimpleName(); + } + return CorrectLogicOffsetService.class.getSimpleName(); + } + } + + public class CleanConsumeQueueService { + protected long lastPhysicalMinOffset = 0; + + public void run() { + try { + this.deleteExpiredFiles(); + } catch (Throwable e) { + log.warn(this.getServiceName() + " service has exception. ", e); + } + } + + protected void deleteExpiredFiles() { + int deleteLogicsFilesInterval = messageStoreConfig.getDeleteConsumeQueueFilesInterval(); + + long minOffset = messageStore.getCommitLog().getMinOffset(); + if (minOffset > this.lastPhysicalMinOffset) { + this.lastPhysicalMinOffset = minOffset; + + for (ConcurrentMap maps : consumeQueueTable.values()) { + for (ConsumeQueueInterface logic : maps.values()) { + int deleteCount = deleteExpiredFile(logic, minOffset); + if (deleteCount > 0 && deleteLogicsFilesInterval > 0) { + try { + Thread.sleep(deleteLogicsFilesInterval); + } catch (InterruptedException ignored) { + } + } + } + } + + messageStore.getIndexService().deleteExpiredFile(minOffset); + } + } + + public String getServiceName() { + return messageStore.getBrokerConfig().getIdentifier() + CleanConsumeQueueService.class.getSimpleName(); + } + } } diff --git a/store/src/main/java/org/apache/rocketmq/store/queue/ConsumeQueueStoreInterface.java b/store/src/main/java/org/apache/rocketmq/store/queue/ConsumeQueueStoreInterface.java index 72a481bd57b..e2cba346eec 100644 --- a/store/src/main/java/org/apache/rocketmq/store/queue/ConsumeQueueStoreInterface.java +++ b/store/src/main/java/org/apache/rocketmq/store/queue/ConsumeQueueStoreInterface.java @@ -16,8 +16,6 @@ */ package org.apache.rocketmq.store.queue; -import java.nio.ByteBuffer; -import java.util.List; import java.util.concurrent.ConcurrentMap; import org.apache.rocketmq.common.BoundaryType; import org.apache.rocketmq.common.message.MessageExtBrokerInner; @@ -28,11 +26,6 @@ public interface ConsumeQueueStoreInterface { - /** - * Start the consumeQueueStore - */ - void start(); - /** * Load from file. * @return true if loaded successfully. @@ -40,20 +33,27 @@ public interface ConsumeQueueStoreInterface { boolean load(); /** - * load after destroy + * Recover from file. + * @param concurrently whether to recover concurrently */ - boolean loadAfterDestroy(); + void recover(boolean concurrently) throws RocksDBException; + + long getDispatchFromPhyOffset(); /** - * Recover from file. + * Start the consumeQueueStore */ - void recover(); + void start(); /** - * Recover concurrently from file. - * @return true if recovered successfully. + * Used to determine whether to start doDispatch from this commitLog mappedFile + * + * @param phyOffset the offset of the first message in this commitlog mappedFile + * @param storeTimestamp the timestamp of the first message in this commitlog mappedFile + * @return whether to start recovering from this MappedFile */ - boolean recoverConcurrently(); + boolean isMappedFileMatchedRecover(long phyOffset, long storeTimestamp, + boolean recoverNormally) throws RocksDBException; /** * Shutdown the consumeQueueStore @@ -63,22 +63,14 @@ public interface ConsumeQueueStoreInterface { /** * destroy all consumeQueues + * @param loadAfterDestroy reload store after destroy, only used in RocksDB mode */ - void destroy(); - - /** - * destroy the specific consumeQueue - * @throws RocksDBException only in rocksdb mode - */ - void destroy(ConsumeQueueInterface consumeQueue) throws RocksDBException; + void destroy(boolean loadAfterDestroy); /** - * Flush cache to file. - * @param consumeQueue the consumeQueue will be flushed - * @param flushLeastPages the minimum number of pages to be flushed - * @return true if any data has been flushed. + * delete topic */ - boolean flush(ConsumeQueueInterface consumeQueue, int flushLeastPages); + boolean deleteTopic(String topic); /** * Flush all nested consume queues to disk @@ -98,36 +90,6 @@ public interface ConsumeQueueStoreInterface { */ void checkSelf(); - /** - * Delete expired files ending at min commit log position. - * @param consumeQueue - * @param minCommitLogOffset min commit log position - * @return deleted file numbers. - */ - int deleteExpiredFile(ConsumeQueueInterface consumeQueue, long minCommitLogOffset); - - /** - * Is the first file available? - * @param consumeQueue - * @return true if it's available - */ - boolean isFirstFileAvailable(ConsumeQueueInterface consumeQueue); - - /** - * Does the first file exist? - * @param consumeQueue - * @return true if it exists - */ - boolean isFirstFileExist(ConsumeQueueInterface consumeQueue); - - /** - * Roll to next file. - * @param consumeQueue - * @param offset next beginning offset - * @return the beginning offset of the next file - */ - long rollNextFile(ConsumeQueueInterface consumeQueue, final long offset); - /** * truncate dirty data * @param offsetToTruncate @@ -135,14 +97,6 @@ public interface ConsumeQueueStoreInterface { */ void truncateDirty(long offsetToTruncate) throws RocksDBException; - /** - * Apply the dispatched request and build the consume queue. This function should be idempotent. - * - * @param consumeQueue consume queue - * @param request dispatch request - */ - void putMessagePositionInfoWrapper(ConsumeQueueInterface consumeQueue, DispatchRequest request); - /** * Apply the dispatched request. This function should be idempotent. * @@ -151,27 +105,6 @@ public interface ConsumeQueueStoreInterface { */ void putMessagePositionInfoWrapper(DispatchRequest request) throws RocksDBException; - /** - * range query cqUnit(ByteBuffer) in rocksdb - * @param topic - * @param queueId - * @param startIndex - * @param num - * @return the byteBuffer list of the topic-queueId in rocksdb - * @throws RocksDBException only in rocksdb mode - */ - List rangeQuery(final String topic, final int queueId, final long startIndex, final int num) throws RocksDBException; - - /** - * query cqUnit(ByteBuffer) in rocksdb - * @param topic - * @param queueId - * @param startIndex - * @return the byteBuffer of the topic-queueId in rocksdb - * @throws RocksDBException only in rocksdb mode - */ - ByteBuffer get(final String topic, final int queueId, final long startIndex) throws RocksDBException; - /** * get consumeQueue table * @return the consumeQueue table @@ -214,33 +147,6 @@ public interface ConsumeQueueStoreInterface { */ void recoverOffsetTable(long minPhyOffset); - /** - * set topicQueue table - * @param topicQueueTable - */ - void setTopicQueueTable(ConcurrentMap topicQueueTable); - - /** - * remove topic-queueId from topicQueue table - * @param topic - * @param queueId - */ - void removeTopicQueueTable(String topic, Integer queueId); - - /** - * get topicQueue table - * @return the topicQueue table - */ - ConcurrentMap getTopicQueueTable(); - - /** - * get the max physical offset in consumeQueue - * @param topic - * @param queueId - * @return - */ - Long getMaxPhyOffsetInConsumeQueue(String topic, int queueId); - /** * get maxOffset of specific topic-queueId in topicQueue table * @@ -251,13 +157,6 @@ public interface ConsumeQueueStoreInterface { */ Long getMaxOffset(String topic, int queueId) throws ConsumeQueueException; - /** - * get max physic offset in consumeQueue - * @return the max physic offset in consumeQueue - * @throws RocksDBException only in rocksdb mode - */ - long getMaxPhyOffsetInConsumeQueue() throws RocksDBException; - /** * get min logic offset of specific topic-queueId in consumeQueue * @param topic @@ -267,15 +166,6 @@ public interface ConsumeQueueStoreInterface { */ long getMinOffsetInQueue(final String topic, final int queueId) throws RocksDBException; - /** - * get max logic offset of specific topic-queueId in consumeQueue - * @param topic - * @param queueId - * @return the max logic offset of specific topic-queueId in consumeQueue - * @throws RocksDBException only in rocksdb mode - */ - long getMaxOffsetInQueue(final String topic, final int queueId) throws RocksDBException; - /** * Get the message whose timestamp is the smallest, greater than or equal to the given time and when there are more * than one message satisfy the condition, decide which one to return based on boundaryType. @@ -295,11 +185,13 @@ public interface ConsumeQueueStoreInterface { ConsumeQueueInterface findOrCreateConsumeQueue(String topic, int queueId); /** - * find the consumeQueueMap of topic + * only find consumeQueue + * * @param topic - * @return the consumeQueueMap of topic + * @param queueId + * @return the consumeQueue */ - ConcurrentMap findConsumeQueueMap(String topic); + ConsumeQueueInterface getConsumeQueue(String topic, int queueId); /** * get the total size of all consumeQueue @@ -307,10 +199,4 @@ public interface ConsumeQueueStoreInterface { */ long getTotalSize(); - /** - * Get store time from commitlog by cqUnit - * @param cqUnit - * @return - */ - long getStoreTime(CqUnit cqUnit); } diff --git a/store/src/main/java/org/apache/rocketmq/store/queue/RocksDBConsumeQueue.java b/store/src/main/java/org/apache/rocketmq/store/queue/RocksDBConsumeQueue.java index 7bd3c2e3057..61ae8d97a8b 100644 --- a/store/src/main/java/org/apache/rocketmq/store/queue/RocksDBConsumeQueue.java +++ b/store/src/main/java/org/apache/rocketmq/store/queue/RocksDBConsumeQueue.java @@ -28,27 +28,29 @@ import org.apache.rocketmq.store.ConsumeQueue; import org.apache.rocketmq.store.DispatchRequest; import org.apache.rocketmq.store.MessageFilter; -import org.apache.rocketmq.store.MessageStore; +import org.apache.rocketmq.store.config.MessageStoreConfig; import org.rocksdb.RocksDBException; public class RocksDBConsumeQueue implements ConsumeQueueInterface { private static final Logger log = LoggerFactory.getLogger(LoggerName.STORE_LOGGER_NAME); private static final Logger ERROR_LOG = LoggerFactory.getLogger(LoggerName.STORE_ERROR_LOGGER_NAME); - private final MessageStore messageStore; + private final MessageStoreConfig messageStoreConfig; + private final RocksDBConsumeQueueStore consumeQueueStore; private final String topic; private final int queueId; - public RocksDBConsumeQueue(final MessageStore messageStore, final String topic, final int queueId) { - this.messageStore = messageStore; + public RocksDBConsumeQueue(final MessageStoreConfig messageStoreConfig, + final RocksDBConsumeQueueStore consumeQueueStore, + final String topic, final int queueId) { + this.messageStoreConfig = messageStoreConfig; + this.consumeQueueStore = consumeQueueStore; this.topic = topic; this.queueId = queueId; } public RocksDBConsumeQueue(final String topic, final int queueId) { - this.messageStore = null; - this.topic = topic; - this.queueId = queueId; + this(null, null, topic, queueId); } @Override @@ -114,7 +116,7 @@ public void cleanSwappedMap(long forceCleanSwapIntervalMs) { @Override public long getMaxOffsetInQueue() { try { - return this.messageStore.getQueueStore().getMaxOffsetInQueue(topic, queueId); + return this.consumeQueueStore.getMaxOffsetInQueue(topic, queueId); } catch (RocksDBException e) { ERROR_LOG.error("getMaxOffsetInQueue Failed. topic: {}, queueId: {}", topic, queueId, e); return 0; @@ -124,8 +126,8 @@ public long getMaxOffsetInQueue() { @Override public long getMessageTotalInQueue() { try { - long maxOffsetInQueue = this.messageStore.getQueueStore().getMaxOffsetInQueue(topic, queueId); - long minOffsetInQueue = this.messageStore.getQueueStore().getMinOffsetInQueue(topic, queueId); + long maxOffsetInQueue = this.consumeQueueStore.getMaxOffsetInQueue(topic, queueId); + long minOffsetInQueue = this.consumeQueueStore.getMinOffsetInQueue(topic, queueId); return maxOffsetInQueue - minOffsetInQueue; } catch (RocksDBException e) { ERROR_LOG.error("getMessageTotalInQueue Failed. topic: {}, queueId: {}, {}", topic, queueId, e); @@ -158,7 +160,7 @@ public long getOffsetInQueueByTime(long timestamp, BoundaryType boundaryType) { @Override public long getMaxPhysicOffset() { - Long maxPhyOffset = this.messageStore.getQueueStore().getMaxPhyOffsetInConsumeQueue(topic, queueId); + Long maxPhyOffset = this.consumeQueueStore.getMaxPhyOffsetInConsumeQueue(topic, queueId); return maxPhyOffset == null ? -1 : maxPhyOffset; } @@ -195,7 +197,6 @@ public void correctMinOffset(long minCommitLogOffset) { /** * Ignored, in rocksdb mode, we build cq in RocksDBConsumeQueueStore - * @see org.apache.rocketmq.store.queue.RocksDBConsumeQueueStore#putMessagePosition() */ @Override public void putMessagePositionInfoWrapper(DispatchRequest request) { @@ -208,7 +209,7 @@ public void assignQueueOffset(QueueOffsetOperator queueOffsetOperator, MessageEx Long queueOffset = queueOffsetOperator.getTopicQueueNextOffset(topicQueueKey); if (queueOffset == null) { // we will recover topic queue table from rocksdb when we use it. - queueOffset = this.messageStore.getQueueStore().getMaxOffsetInQueue(topic, queueId); + queueOffset = this.consumeQueueStore.getMaxOffsetInQueue(topic, queueId); queueOffsetOperator.updateQueueOffset(topicQueueKey, queueOffset); } msg.setQueueOffset(queueOffset); @@ -236,10 +237,10 @@ public long estimateMessageCount(long from, long to, MessageFilter filter) { to = getMaxOffsetInQueue(); } - int maxSampleSize = messageStore.getMessageStoreConfig().getMaxConsumeQueueScan(); + int maxSampleSize = messageStoreConfig.getMaxConsumeQueueScan(); int sampleSize = to - from > maxSampleSize ? maxSampleSize : (int) (to - from); - int matchThreshold = messageStore.getMessageStoreConfig().getSampleCountThreshold(); + int matchThreshold = messageStoreConfig.getSampleCountThreshold(); int matchSize = 0; for (int i = 0; i < sampleSize; i++) { @@ -265,7 +266,12 @@ public long estimateMessageCount(long from, long to, MessageFilter filter) { @Override public long getMinOffsetInQueue() { - return this.messageStore.getMinOffsetInQueue(this.topic, this.queueId); + try { + return this.consumeQueueStore.getMinOffsetInQueue(topic, queueId); + } catch (RocksDBException e) { + ERROR_LOG.error("getMinOffsetInQueue Failed. topic: {}, queueId: {}", topic, queueId, e); + return -1; + } } private int pullNum(long cqOffset, long maxCqOffset) { @@ -279,7 +285,7 @@ private int pullNum(long cqOffset, long maxCqOffset) { @Override public ReferredIterator iterateFrom(final long startIndex) { long maxCqOffset = getMaxOffsetInQueue(); - if (startIndex < maxCqOffset) { + if (startIndex < maxCqOffset && startIndex >= 0) { int num = pullNum(startIndex, maxCqOffset); return new LargeRocksDBConsumeQueueIterator(startIndex, num); } @@ -306,7 +312,7 @@ public CqUnit get(long index) { public Pair getCqUnitAndStoreTime(long index) { ByteBuffer byteBuffer; try { - byteBuffer = this.messageStore.getQueueStore().get(topic, queueId, index); + byteBuffer = this.consumeQueueStore.get(topic, queueId, index); } catch (RocksDBException e) { ERROR_LOG.error("getUnitAndStoreTime Failed. topic: {}, queueId: {}", topic, queueId, e); return null; @@ -324,7 +330,7 @@ public Pair getCqUnitAndStoreTime(long index) { @Override public Pair getEarliestUnitAndStoreTime() { try { - long minOffset = this.messageStore.getQueueStore().getMinOffsetInQueue(topic, queueId); + long minOffset = this.consumeQueueStore.getMinOffsetInQueue(topic, queueId); return getCqUnitAndStoreTime(minOffset); } catch (RocksDBException e) { ERROR_LOG.error("getEarliestUnitAndStoreTime Failed. topic: {}, queueId: {}", topic, queueId, e); @@ -341,7 +347,7 @@ public CqUnit getEarliestUnit() { @Override public CqUnit getLatestUnit() { try { - long maxOffset = this.messageStore.getQueueStore().getMaxOffsetInQueue(topic, queueId); + long maxOffset = this.consumeQueueStore.getMaxOffsetInQueue(topic, queueId); return get(maxOffset > 0 ? maxOffset - 1 : maxOffset); } catch (RocksDBException e) { ERROR_LOG.error("getLatestUnit Failed. topic: {}, queueId: {}, {}", topic, queueId, e.getMessage()); @@ -355,9 +361,9 @@ public long getLastOffset() { } private ReferredIterator iterateFrom0(final long startIndex, final int count) throws RocksDBException { - List byteBufferList = this.messageStore.getQueueStore().rangeQuery(topic, queueId, startIndex, count); + List byteBufferList = this.consumeQueueStore.rangeQuery(topic, queueId, startIndex, count); if (byteBufferList == null || byteBufferList.isEmpty()) { - if (this.messageStore.getMessageStoreConfig().isEnableRocksDBLog()) { + if (this.messageStoreConfig.isEnableRocksDBLog()) { log.warn("iterateFrom0 - find nothing, startIndex:{}, count:{}", startIndex, count); } return null; @@ -449,7 +455,7 @@ public CqUnit next() { final ByteBuffer byteBuffer; try { - byteBuffer = messageStore.getQueueStore().get(topic, queueId, startIndex + currentIndex); + byteBuffer = consumeQueueStore.get(topic, queueId, startIndex + currentIndex); } catch (RocksDBException e) { ERROR_LOG.error("get cq from rocksdb failed. topic: {}, queueId: {}", topic, queueId, e); return null; @@ -480,4 +486,21 @@ public CqUnit nextAndRelease() { } } } + + public void initializeWithOffset(long offset) { + log.info("RocksDBConsumeQueue initializeWithOffset topic={}, queueId={}, offset={}, oldMax={}, oldMin={}", + topic, queueId, offset, getMaxOffsetInQueue(), getMinOffsetInQueue()); + try { + // destroy the consume queue will clear the cqUnit and offset + consumeQueueStore.destroy(this); + + // update the max and min offset + if (offset > 0) { + this.consumeQueueStore.updateCqOffset(topic, queueId, 0L, offset - 1, true); + this.consumeQueueStore.updateCqOffset(topic, queueId, 0L, offset, false); + } + } catch (RocksDBException e) { + ERROR_LOG.error("RocksDBConsumeQueue initializeWithOffset Failed. topic={}, queueId={}, offset={}", topic, queueId, offset, e); + } + } } diff --git a/store/src/main/java/org/apache/rocketmq/store/queue/RocksDBConsumeQueueOffsetTable.java b/store/src/main/java/org/apache/rocketmq/store/queue/RocksDBConsumeQueueOffsetTable.java index cb989852fb9..821cb23baa2 100644 --- a/store/src/main/java/org/apache/rocketmq/store/queue/RocksDBConsumeQueueOffsetTable.java +++ b/store/src/main/java/org/apache/rocketmq/store/queue/RocksDBConsumeQueueOffsetTable.java @@ -26,7 +26,6 @@ import java.util.Set; import java.util.concurrent.ConcurrentHashMap; import java.util.concurrent.ConcurrentMap; - import java.util.function.Consumer; import java.util.function.Function; import org.apache.rocketmq.common.MixAll; @@ -563,7 +562,7 @@ private Long removeHeapMaxCqOffset(String topicQueueId) { return this.topicQueueMaxCqOffset.remove(topicQueueId); } - private void updateCqOffset(final String topic, final int queueId, final long phyOffset, + public void updateCqOffset(final String topic, final int queueId, final long phyOffset, final long cqOffset, boolean max) throws RocksDBException { if (!this.rocksDBStorage.hold()) { return; diff --git a/store/src/main/java/org/apache/rocketmq/store/queue/RocksDBConsumeQueueStore.java b/store/src/main/java/org/apache/rocketmq/store/queue/RocksDBConsumeQueueStore.java index 94ed0c926a7..2e0a2088ad3 100644 --- a/store/src/main/java/org/apache/rocketmq/store/queue/RocksDBConsumeQueueStore.java +++ b/store/src/main/java/org/apache/rocketmq/store/queue/RocksDBConsumeQueueStore.java @@ -38,6 +38,7 @@ import org.apache.rocketmq.common.Pair; import org.apache.rocketmq.common.ServiceState; import org.apache.rocketmq.common.ThreadFactoryImpl; +import org.apache.rocketmq.common.UtilAll; import org.apache.rocketmq.common.constant.LoggerName; import org.apache.rocketmq.common.message.MessageConst; import org.apache.rocketmq.common.message.MessageExtBrokerInner; @@ -88,6 +89,10 @@ public class RocksDBConsumeQueueStore extends AbstractConsumeQueueStore { private final AtomicReference serviceState = new AtomicReference<>(ServiceState.CREATE_JUST); + private final RocksDBCleanConsumeQueueService cleanConsumeQueueService; + + private long dispatchFromPhyOffset; + public RocksDBConsumeQueueStore(DefaultMessageStore messageStore) { super(messageStore); @@ -108,6 +113,7 @@ public RocksDBConsumeQueueStore(DefaultMessageStore messageStore) { this.tempTopicQueueMaxOffsetMap = new HashMap<>(); this.scheduledExecutorService = Executors.newSingleThreadScheduledExecutor( new ThreadFactoryImpl("RocksDBConsumeQueueStoreScheduledThread", messageStore.getBrokerIdentity())); + this.cleanConsumeQueueService = new RocksDBCleanConsumeQueueService(); } private Pair getCQByteBufferPair() { @@ -138,6 +144,9 @@ public void start() { this.scheduledExecutorService.scheduleWithFixedDelay(() -> { cleanDirty(messageStore.getTopicConfigs().keySet()); }, 10, this.messageStoreConfig.getCleanRocksDBDirtyCQIntervalMin(), TimeUnit.MINUTES); + + messageStore.getScheduledCleanQueueExecutorService().scheduleAtFixedRate(this.cleanConsumeQueueService::run, + 1000 * 60, this.messageStoreConfig.getCleanResourceInterval(), TimeUnit.MILLISECONDS); } } @@ -165,21 +174,15 @@ public boolean load() { log.info("load rocksdb consume queue {}.", result ? "OK" : "Failed"); return result; } - @Override - public boolean loadAfterDestroy() { - return this.load(); - } - - @Override - public void recover() { + public void recover(boolean concurrently) throws RocksDBException { start(); + this.dispatchFromPhyOffset = getMaxPhyOffsetInConsumeQueue(); } @Override - public boolean recoverConcurrently() { - start(); - return true; + public long getDispatchFromPhyOffset() { + return dispatchFromPhyOffset; } @Override @@ -343,37 +346,40 @@ public Statistics getStatistics() { return rocksDBStorage.getStatistics(); } - @Override public List rangeQuery(final String topic, final int queueId, final long startIndex, final int num) throws RocksDBException { return this.rocksDBConsumeQueueTable.rangeQuery(topic, queueId, startIndex, num); } - @Override public ByteBuffer get(final String topic, final int queueId, final long cqOffset) throws RocksDBException { return this.rocksDBConsumeQueueTable.getCQInKV(topic, queueId, cqOffset); } /** - * Ignored, we do not need to recover topicQueueTable and correct minLogicOffset. Because we will correct them - * when we use them, we call it lazy correction. + * Try to set topicQueueTable = new HashMap<>(), otherwise it will cause bug when broker role changes. + * And unlike method in DefaultMessageStore, we don't need to really recover topic queue table advance, + * because we can recover topic queue table from rocksdb when we need to use it. + * @see RocksDBConsumeQueue#assignQueueOffset * * @see RocksDBConsumeQueue#increaseQueueOffset(QueueOffsetOperator, MessageExtBrokerInner, short) * @see org.apache.rocketmq.store.queue.RocksDBConsumeQueueOffsetTable#getMinCqOffset(String, int) */ @Override public void recoverOffsetTable(long minPhyOffset) { - + this.setTopicQueueTable(new ConcurrentHashMap<>()); } @Override - public void destroy() { + public void destroy(boolean loadAfterDestroy) { try { shutdownInner(); FileUtils.deleteDirectory(new File(this.storePath)); } catch (Exception e) { ERROR_LOG.error("destroy cq Failed. {}", this.storePath, e); } + if (loadAfterDestroy) { + load(); + } } @Override @@ -397,16 +403,6 @@ public void destroy(ConsumeQueueInterface consumeQueue) throws RocksDBException } } - @Override - public boolean flush(ConsumeQueueInterface consumeQueue, int flushLeastPages) { - try { - this.rocksDBStorage.flushWAL(); - } catch (Exception e) { - log.error("Failed to flush WAL", e); - } - return true; - } - @Override public void flush() throws StoreException { try (FlushOptions flushOptions = new FlushOptions()) { @@ -423,12 +419,6 @@ public void checkSelf() { // ignored } - @Override - public int deleteExpiredFile(ConsumeQueueInterface consumeQueue, long minCommitLogPos) { - // ignored - return 0; - } - /** * We do not need to truncate dirty CQ in RocksDBConsumeQueueTable, Because dirty CQ in RocksDBConsumeQueueTable * will be rewritten by new KV when new messages are appended or will be cleaned up when topics are deleted. @@ -475,7 +465,6 @@ public long getOffsetInQueueByTime(String topic, int queueId, long timestamp, * @return Index of the next slot to push into * @throws RocksDBException if RocksDB fails to fulfill the request. */ - @Override public long getMaxOffsetInQueue(String topic, int queueId) throws RocksDBException { Long maxOffset = this.rocksDBConsumeQueueOffsetTable.getMaxCqOffset(topic, queueId); return (maxOffset != null) ? maxOffset + 1 : 0; @@ -486,7 +475,6 @@ public long getMinOffsetInQueue(String topic, int queueId) throws RocksDBExcepti return this.rocksDBConsumeQueueOffsetTable.getMinCqOffset(topic, queueId); } - @Override public Long getMaxPhyOffsetInConsumeQueue(String topic, int queueId) { return this.rocksDBConsumeQueueOffsetTable.getMaxPhyOffset(topic, queueId); } @@ -520,30 +508,26 @@ public ConsumeQueueInterface findOrCreateConsumeQueue(String topic, int queueId) return logic; } - ConsumeQueueInterface newLogic = new RocksDBConsumeQueue(this.messageStore, topic, queueId); + ConsumeQueueInterface newLogic = new RocksDBConsumeQueue(this.messageStore.getMessageStoreConfig(), this, topic, queueId); ConsumeQueueInterface oldLogic = map.putIfAbsent(queueId, newLogic); return oldLogic != null ? oldLogic : newLogic; } @Override - public long rollNextFile(ConsumeQueueInterface consumeQueue, long offset) { - return 0; - } - - @Override - public boolean isFirstFileExist(ConsumeQueueInterface consumeQueue) { - return true; + public ConsumeQueueInterface getConsumeQueue(String topic, int queueId) { + return findOrCreateConsumeQueue(topic, queueId); } @Override - public boolean isFirstFileAvailable(ConsumeQueueInterface consumeQueue) { - return true; + public long getTotalSize() { + return 0; } @Override - public long getTotalSize() { - return 0; + public boolean isMappedFileMatchedRecover(long phyOffset, long storeTimestamp, + boolean recoverNormally) { + return phyOffset <= dispatchFromPhyOffset; } @Override @@ -562,4 +546,74 @@ public Long getMaxOffset(String topic, int queueId) throws ConsumeQueueException public boolean isStopped() { return ServiceState.SHUTDOWN_ALREADY == serviceState.get(); } + + public void updateCqOffset(final String topic, final int queueId, final long phyOffset, + final long cqOffset, boolean max) throws RocksDBException { + this.rocksDBConsumeQueueOffsetTable.updateCqOffset(topic, queueId, phyOffset, cqOffset, max); + } + + class RocksDBCleanConsumeQueueService { + protected long lastPhysicalMinOffset = 0; + + private final double diskSpaceWarningLevelRatio = + Double.parseDouble(System.getProperty("rocketmq.broker.diskSpaceWarningLevelRatio", "0.90")); + + private final double diskSpaceCleanForciblyRatio = + Double.parseDouble(System.getProperty("rocketmq.broker.diskSpaceCleanForciblyRatio", "0.85")); + + public void run() { + try { + this.deleteExpiredFiles(); + } catch (Throwable e) { + log.warn(this.getServiceName() + " service has exception. ", e); + } + } + + public String getServiceName() { + return messageStore.getBrokerConfig().getIdentifier() + ConsumeQueueStore.CleanConsumeQueueService.class.getSimpleName(); + } + + protected void deleteExpiredFiles() { + + long minOffset = messageStore.getCommitLog().getMinOffset(); + if (minOffset > this.lastPhysicalMinOffset) { + this.lastPhysicalMinOffset = minOffset; + + boolean spaceFull = isSpaceToDelete(); + boolean timeUp = messageStore.isTimeToDelete(); + if (spaceFull || timeUp) { + cleanExpired(minOffset); + } + + messageStore.getIndexService().deleteExpiredFile(minOffset); + } + } + + private boolean isSpaceToDelete() { + double ratio = messageStoreConfig.getDiskMaxUsedSpaceRatio() / 100.0; + + String storePathLogics = StorePathConfigHelper + .getStorePathConsumeQueue(messageStoreConfig.getStorePathRootDir()); + double logicsRatio = UtilAll.getDiskPartitionSpaceUsedPercent(storePathLogics); + if (logicsRatio > diskSpaceWarningLevelRatio) { + boolean diskOk = messageStore.getRunningFlags().getAndMakeLogicDiskFull(); + if (diskOk) { + log.error("logics disk maybe full soon " + logicsRatio + ", so mark disk full"); + } + } else if (logicsRatio > diskSpaceCleanForciblyRatio) { + } else { + boolean diskOk = messageStore.getRunningFlags().getAndMakeLogicDiskOK(); + if (!diskOk) { + log.info("logics disk space OK " + logicsRatio + ", so mark disk ok"); + } + } + + if (logicsRatio < 0 || logicsRatio > ratio) { + log.info("logics disk maybe full soon, so reclaim space, " + logicsRatio); + return true; + } + + return false; + } + } } 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 2e08369bde9..ef33c083831 100644 --- a/store/src/test/java/org/apache/rocketmq/store/ConsumeQueueTest.java +++ b/store/src/test/java/org/apache/rocketmq/store/ConsumeQueueTest.java @@ -18,6 +18,7 @@ package org.apache.rocketmq.store; import java.io.File; +import java.io.IOException; import java.lang.reflect.Method; import java.net.InetAddress; import java.net.InetSocketAddress; @@ -27,7 +28,7 @@ import java.util.concurrent.ConcurrentHashMap; import java.util.concurrent.ConcurrentMap; import java.util.concurrent.TimeUnit; - +import org.apache.commons.io.FileUtils; import org.apache.rocketmq.common.BrokerConfig; import org.apache.rocketmq.common.MixAll; import org.apache.rocketmq.common.UtilAll; @@ -41,9 +42,9 @@ import org.apache.rocketmq.store.stats.BrokerStatsManager; import org.awaitility.Awaitility; import org.junit.Assert; +import org.junit.Assume; import org.junit.Test; import org.mockito.Mockito; -import org.junit.Assume; import static java.util.concurrent.TimeUnit.SECONDS; import static org.assertj.core.api.Assertions.assertThat; @@ -503,4 +504,57 @@ public void testCorrectMinOffset() { Assert.assertEquals(max * ConsumeQueue.CQ_STORE_UNIT_SIZE, consumeQueue.getMinLogicOffset()); consumeQueue.destroy(); } + + @Test + public void testFillBankThenCorrectMinOffset() throws IOException { + String topic = "T1"; + int queueId = 0; + MessageStoreConfig storeConfig = new MessageStoreConfig(); + File tmpDir = new File(System.getProperty("java.io.tmpdir"), "testFillBankThenCorrectMinOffset"); + FileUtils.deleteDirectory(tmpDir); + storeConfig.setStorePathRootDir(tmpDir.getAbsolutePath()); + storeConfig.setEnableConsumeQueueExt(false); + DefaultMessageStore messageStore = Mockito.mock(DefaultMessageStore.class); + Mockito.when(messageStore.getMessageStoreConfig()).thenReturn(storeConfig); + + RunningFlags runningFlags = new RunningFlags(); + Mockito.when(messageStore.getRunningFlags()).thenReturn(runningFlags); + + StoreCheckpoint storeCheckpoint = Mockito.mock(StoreCheckpoint.class); + Mockito.when(messageStore.getStoreCheckpoint()).thenReturn(storeCheckpoint); + + { + ConsumeQueue consumeQueue = new ConsumeQueue(topic, queueId, storeConfig.getStorePathRootDir(), + storeConfig.getMappedFileSizeConsumeQueue(), messageStore); + Assert.assertTrue(consumeQueue.load()); + consumeQueue.recover(); + consumeQueue.initializeWithOffset(100); + Assert.assertEquals(100, consumeQueue.getMinOffsetInQueue()); + Assert.assertEquals(100, consumeQueue.getMaxOffsetInQueue()); + } + + { + ConsumeQueue consumeQueue = new ConsumeQueue(topic, queueId, storeConfig.getStorePathRootDir(), + storeConfig.getMappedFileSizeConsumeQueue(), messageStore); + Assert.assertTrue(consumeQueue.load()); + consumeQueue.recover(); + consumeQueue.correctMinOffset(1L); + Assert.assertEquals(100, consumeQueue.getMinOffsetInQueue()); + Assert.assertEquals(100, consumeQueue.getMaxOffsetInQueue()); + } + +// { +// ConsumeQueue consumeQueue = new ConsumeQueue(topic, queueId, storeConfig.getStorePathRootDir(), +// storeConfig.getMappedFileSizeConsumeQueue(), messageStore); +// Assert.assertTrue(consumeQueue.load()); +// consumeQueue.recover(); +// consumeQueue.correctMinOffset(0L); +// Assert.assertEquals(100, consumeQueue.getMinOffsetInQueue()); +// Assert.assertEquals(100, consumeQueue.getMaxOffsetInQueue()); +// } + + ConsumeQueue consumeQueue0 = new ConsumeQueue(topic, queueId, storeConfig.getStorePathRootDir(), + storeConfig.getMappedFileSizeConsumeQueue(), messageStore); + consumeQueue0.destroy(); + } } 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 083aabc48b3..0f6772e937a 100644 --- a/store/src/test/java/org/apache/rocketmq/store/DefaultMessageStoreCleanFilesTest.java +++ b/store/src/test/java/org/apache/rocketmq/store/DefaultMessageStoreCleanFilesTest.java @@ -17,6 +17,16 @@ package org.apache.rocketmq.store; +import java.io.File; +import java.lang.reflect.Field; +import java.net.InetAddress; +import java.net.InetSocketAddress; +import java.net.SocketAddress; +import java.util.ArrayList; +import java.util.Calendar; +import java.util.HashMap; +import java.util.Map; +import java.util.UUID; import java.util.concurrent.ConcurrentHashMap; import org.apache.rocketmq.common.BrokerConfig; import org.apache.rocketmq.common.MixAll; @@ -30,22 +40,12 @@ import org.apache.rocketmq.store.index.IndexService; 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.stats.BrokerStatsManager; import org.junit.After; import org.junit.Before; import org.junit.Test; -import java.io.File; -import java.lang.reflect.Field; -import java.net.InetAddress; -import java.net.InetSocketAddress; -import java.net.SocketAddress; -import java.util.ArrayList; -import java.util.Calendar; -import java.util.HashMap; -import java.util.Map; -import java.util.UUID; - import static org.apache.rocketmq.common.message.MessageDecoder.CHARSET_UTF8; import static org.apache.rocketmq.store.ConsumeQueue.CQ_STORE_UNIT_SIZE; import static org.junit.Assert.assertEquals; @@ -59,7 +59,7 @@ public class DefaultMessageStoreCleanFilesTest { private DefaultMessageStore messageStore; private DefaultMessageStore.CleanCommitLogService cleanCommitLogService; - private DefaultMessageStore.CleanConsumeQueueService cleanConsumeQueueService; + private ConsumeQueueStore.CleanConsumeQueueService cleanConsumeQueueService; private SocketAddress bornHost; private SocketAddress storeHost; @@ -350,12 +350,12 @@ private DefaultMessageStore.CleanCommitLogService getCleanCommitLogService() return cleanCommitLogService; } - private DefaultMessageStore.CleanConsumeQueueService getCleanConsumeQueueService() + private ConsumeQueueStore.CleanConsumeQueueService getCleanConsumeQueueService() throws Exception { - Field serviceField = messageStore.getClass().getDeclaredField("cleanConsumeQueueService"); + Field serviceField = messageStore.getQueueStore().getClass().getDeclaredField("cleanConsumeQueueService"); serviceField.setAccessible(true); - DefaultMessageStore.CleanConsumeQueueService cleanConsumeQueueService = - (DefaultMessageStore.CleanConsumeQueueService) serviceField.get(messageStore); + ConsumeQueueStore.CleanConsumeQueueService cleanConsumeQueueService = + (ConsumeQueueStore.CleanConsumeQueueService) serviceField.get(messageStore.getQueueStore()); serviceField.setAccessible(false); return cleanConsumeQueueService; } diff --git a/store/src/test/java/org/apache/rocketmq/store/RocksDBMessageStoreTest.java b/store/src/test/java/org/apache/rocketmq/store/RocksDBMessageStoreTest.java index f934f803641..20a7770c5eb 100644 --- a/store/src/test/java/org/apache/rocketmq/store/RocksDBMessageStoreTest.java +++ b/store/src/test/java/org/apache/rocketmq/store/RocksDBMessageStoreTest.java @@ -17,6 +17,7 @@ package org.apache.rocketmq.store; +import com.google.common.collect.Sets; import java.io.File; import java.io.RandomAccessFile; import java.lang.reflect.InvocationTargetException; @@ -40,9 +41,6 @@ import java.util.concurrent.ConcurrentMap; import java.util.concurrent.TimeUnit; import java.util.concurrent.atomic.AtomicInteger; - -import com.google.common.collect.Sets; - import org.apache.rocketmq.common.BrokerConfig; import org.apache.rocketmq.common.MixAll; import org.apache.rocketmq.common.TopicConfig; @@ -59,6 +57,8 @@ import org.apache.rocketmq.store.exception.ConsumeQueueException; import org.apache.rocketmq.store.queue.ConsumeQueueInterface; import org.apache.rocketmq.store.queue.CqUnit; +import org.apache.rocketmq.store.queue.RocksDBConsumeQueue; +import org.apache.rocketmq.store.queue.RocksDBConsumeQueueStore; import org.apache.rocketmq.store.stats.BrokerStatsManager; import org.assertj.core.util.Strings; import org.awaitility.Awaitility; @@ -1026,8 +1026,8 @@ public void testDeleteTopics() { ConcurrentMap cqTable = new ConcurrentHashMap<>(); String topicName = "topic-" + i; for (int j = 0; j < 4; j++) { - ConsumeQueue consumeQueue = new ConsumeQueue(topicName, j, messageStoreConfig.getStorePathRootDir(), - messageStoreConfig.getMappedFileSizeConsumeQueue(), messageStore); + RocksDBConsumeQueue consumeQueue = new RocksDBConsumeQueue(messageStoreConfig, + (RocksDBConsumeQueueStore) messageStore.getQueueStore(), topicName, i); cqTable.put(j, consumeQueue); } consumeQueueTable.put(topicName, cqTable); @@ -1051,8 +1051,8 @@ public void testCleanUnusedTopic() { ConcurrentMap cqTable = new ConcurrentHashMap<>(); String topicName = "topic-" + i; for (int j = 0; j < 4; j++) { - ConsumeQueue consumeQueue = new ConsumeQueue(topicName, j, messageStoreConfig.getStorePathRootDir(), - messageStoreConfig.getMappedFileSizeConsumeQueue(), messageStore); + RocksDBConsumeQueue consumeQueue = new RocksDBConsumeQueue(messageStoreConfig, + (RocksDBConsumeQueueStore) messageStore.getQueueStore(), topicName, i); cqTable.put(j, consumeQueue); } consumeQueueTable.put(topicName, cqTable); diff --git a/store/src/test/java/org/apache/rocketmq/store/StoreTestUtil.java b/store/src/test/java/org/apache/rocketmq/store/StoreTestUtil.java index 17a2b5e19d7..ccb16f390ea 100644 --- a/store/src/test/java/org/apache/rocketmq/store/StoreTestUtil.java +++ b/store/src/test/java/org/apache/rocketmq/store/StoreTestUtil.java @@ -19,18 +19,17 @@ import io.openmessaging.storage.dledger.store.file.DefaultMmapFile; import io.openmessaging.storage.dledger.store.file.MmapFile; import java.io.IOException; +import java.lang.reflect.Field; +import java.lang.reflect.InvocationTargetException; +import java.lang.reflect.Method; +import java.util.ArrayList; import java.util.List; import org.apache.commons.lang3.SystemUtils; import org.apache.rocketmq.logging.org.slf4j.Logger; import org.apache.rocketmq.logging.org.slf4j.LoggerFactory; import org.apache.rocketmq.store.index.IndexFile; import org.apache.rocketmq.store.index.IndexService; - -import java.lang.reflect.Field; -import java.lang.reflect.InvocationTargetException; -import java.lang.reflect.Method; -import java.util.ArrayList; - +import org.apache.rocketmq.store.queue.ConsumeQueueStore; public class StoreTestUtil { @@ -58,12 +57,12 @@ public static boolean isCommitLogAvailable(DefaultMessageStore store) { } public static void flushConsumeQueue(DefaultMessageStore store) throws Exception { - Field field = store.getClass().getDeclaredField("flushConsumeQueueService"); + Field field = store.getQueueStore().getClass().getDeclaredField("flushConsumeQueueService"); field.setAccessible(true); - DefaultMessageStore.FlushConsumeQueueService flushService = (DefaultMessageStore.FlushConsumeQueueService) field.get(store); + ConsumeQueueStore.FlushConsumeQueueService flushService = (ConsumeQueueStore.FlushConsumeQueueService) field.get(store.getQueueStore()); final int retryTimesOver = 3; - Method method = DefaultMessageStore.FlushConsumeQueueService.class.getDeclaredMethod("doFlush", int.class); + Method method = ConsumeQueueStore.FlushConsumeQueueService.class.getDeclaredMethod("doFlush", int.class); method.setAccessible(true); method.invoke(flushService, retryTimesOver); } diff --git a/store/src/test/java/org/apache/rocketmq/store/queue/CombineConsumeQueueStoreTest.java b/store/src/test/java/org/apache/rocketmq/store/queue/CombineConsumeQueueStoreTest.java new file mode 100644 index 00000000000..a52b0228287 --- /dev/null +++ b/store/src/test/java/org/apache/rocketmq/store/queue/CombineConsumeQueueStoreTest.java @@ -0,0 +1,358 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.rocketmq.store.queue; + +import java.io.File; +import java.net.InetSocketAddress; +import java.util.UUID; +import java.util.concurrent.ConcurrentHashMap; +import java.util.concurrent.ConcurrentMap; +import java.util.concurrent.TimeUnit; +import org.apache.commons.io.FileUtils; +import org.apache.rocketmq.common.TopicConfig; +import org.apache.rocketmq.common.UtilAll; +import org.apache.rocketmq.common.attribute.CQType; +import org.apache.rocketmq.common.constant.PermName; +import org.apache.rocketmq.common.message.MessageExtBrokerInner; +import org.apache.rocketmq.store.ConsumeQueue; +import org.apache.rocketmq.store.DefaultMessageStore; +import org.apache.rocketmq.store.DispatchRequest; +import org.apache.rocketmq.store.config.MessageStoreConfig; +import org.junit.After; +import org.junit.Assert; +import org.junit.Before; +import org.junit.Test; +import org.junit.runner.RunWith; +import org.mockito.junit.MockitoJUnitRunner; + +import static org.apache.rocketmq.common.TopicFilterType.SINGLE_TAG; +import static org.awaitility.Awaitility.await; +import static org.junit.Assert.assertNotNull; +import static org.junit.Assert.assertNull; +import static org.junit.Assert.assertTrue; + +@RunWith(MockitoJUnitRunner.class) +public class CombineConsumeQueueStoreTest extends QueueTestBase { + private DefaultMessageStore messageStore; + private MessageStoreConfig messageStoreConfig; + private ConcurrentMap topicConfigTableMap; + + String topic = UUID.randomUUID().toString(); + final int queueId = 0; + final int msgNum = 100; + final int msgSize = 1000; + + @Before + public void init() throws Exception { + this.topicConfigTableMap = new ConcurrentHashMap<>(); + messageStoreConfig = new MessageStoreConfig(); + } + + @After + public void destroy() { + messageStore.shutdown(); + messageStore.destroy(); + + File file = new File(messageStore.getMessageStoreConfig().getStorePathRootDir()); + UtilAll.deleteFile(file); + } + + @Test(expected = IllegalArgumentException.class) + public void CombineConsumeQueueStore_EmptyLoadingCQTypes_ThrowsException() throws Exception { + messageStore = (DefaultMessageStore) createMessageStore(null, false, topicConfigTableMap, messageStoreConfig); + + messageStoreConfig.setCombineCQLoadingCQTypes(""); + new CombineConsumeQueueStore(messageStore); + } + + @Test + public void CombineConsumeQueueStore_InitializesConsumeQueueStore() throws Exception { + messageStore = (DefaultMessageStore) createMessageStore(null, false, topicConfigTableMap, messageStoreConfig); + { + messageStoreConfig.setCombineCQLoadingCQTypes("default"); + messageStoreConfig.setCombineCQPreferCQType("default"); + CombineConsumeQueueStore store = new CombineConsumeQueueStore(messageStore); + assertNotNull(store.getConsumeQueueStore()); + assertNull(store.getRocksDBConsumeQueueStore()); + } + + { + messageStoreConfig.setCombineCQLoadingCQTypes("defaultRocksDB"); + messageStoreConfig.setCombineCQPreferCQType("defaultRocksDB"); + messageStoreConfig.setCombineAssignOffsetCQType("defaultRocksDB"); + CombineConsumeQueueStore store = new CombineConsumeQueueStore(messageStore); + assertNull(store.getConsumeQueueStore()); + assertNotNull(store.getRocksDBConsumeQueueStore()); + assertTrue(store.getCurrentReadStore() instanceof RocksDBConsumeQueueStore); + } + + { + messageStoreConfig.setCombineCQLoadingCQTypes(";;default;defaultRocksDB;"); + messageStoreConfig.setCombineCQPreferCQType("defaultRocksDB"); + CombineConsumeQueueStore store = new CombineConsumeQueueStore(messageStore); + assertNotNull(store.getConsumeQueueStore()); + assertNotNull(store.getRocksDBConsumeQueueStore()); + assertTrue(store.getCurrentReadStore() instanceof RocksDBConsumeQueueStore); + } + + { + messageStoreConfig.setCombineCQLoadingCQTypes("default;defaultRocksDB"); + messageStoreConfig.setCombineCQPreferCQType("defaultRocksDB"); + CombineConsumeQueueStore store = new CombineConsumeQueueStore(messageStore); + assertTrue(store.getCurrentReadStore() instanceof RocksDBConsumeQueueStore); + } + + } + + @Test + public void testIterator() throws Exception { + messageStoreConfig.setRocksdbCQDoubleWriteEnable(true); + messageStore = (DefaultMessageStore) createMessageStore(null, false, topicConfigTableMap, messageStoreConfig); + messageStore.load(); + messageStore.start(); + + //The initial min max offset, before and after the creation of consume queue + Assert.assertEquals(0, messageStore.getMaxOffsetInQueue(topic, queueId)); + Assert.assertEquals(0, messageStore.getMinOffsetInQueue(topic, queueId)); + + ConsumeQueueInterface consumeQueue = messageStore.getConsumeQueue(topic, queueId); + Assert.assertEquals(CQType.SimpleCQ, consumeQueue.getCQType()); + Assert.assertEquals(0, consumeQueue.getMaxOffsetInQueue()); + Assert.assertEquals(0, consumeQueue.getMinOffsetInQueue()); + Assert.assertEquals(0, messageStore.getMaxOffsetInQueue(topic, queueId)); + Assert.assertEquals(0, messageStore.getMinOffsetInQueue(topic, queueId)); + + for (int i = 0; i < msgNum; i++) { + DispatchRequest request = new DispatchRequest(topic, queueId, i * msgSize, msgSize, i, + System.currentTimeMillis(), i, null, null, 0, 0, null); + messageStore.getQueueStore().putMessagePositionInfoWrapper(request); + } + + await().atMost(1, TimeUnit.SECONDS).untilAsserted(() -> { + checkCQ(consumeQueue, msgNum, msgSize); + + CombineConsumeQueueStore combineConsumeQueueStore = (CombineConsumeQueueStore) messageStore.getQueueStore(); + ConsumeQueueInterface rocksDBConsumeQueue = combineConsumeQueueStore.getRocksDBConsumeQueueStore().getConsumeQueue(topic, queueId); + Assert.assertEquals(CQType.RocksDBCQ, rocksDBConsumeQueue.getCQType()); + Assert.assertEquals(msgNum, rocksDBConsumeQueue.getMaxOffsetInQueue()); + checkCQ(rocksDBConsumeQueue, msgNum, msgSize); + }); + } + + private void checkCQ(ConsumeQueueInterface consumeQueue, int msgNum, + int msgSize) { + Assert.assertEquals(0, consumeQueue.getMinLogicOffset()); + Assert.assertEquals(0, consumeQueue.getMinOffsetInQueue()); + Assert.assertEquals(msgNum, consumeQueue.getMaxOffsetInQueue()); + Assert.assertEquals(msgNum, consumeQueue.getMessageTotalInQueue()); + + assertNull(consumeQueue.iterateFrom(-1)); + assertNull(consumeQueue.iterateFrom(msgNum)); + + { + CqUnit first = consumeQueue.getEarliestUnit(); + assertNotNull(first); + Assert.assertEquals(0, first.getQueueOffset()); + Assert.assertEquals(msgSize, first.getSize()); + assertTrue(first.isTagsCodeValid()); + } + { + CqUnit last = consumeQueue.getLatestUnit(); + assertNotNull(last); + Assert.assertEquals(msgNum - 1, last.getQueueOffset()); + Assert.assertEquals(msgSize, last.getSize()); + assertTrue(last.isTagsCodeValid()); + } + + for (int i = 0; i < msgNum; i++) { + ReferredIterator iterator = consumeQueue.iterateFrom(i); + assertNotNull(iterator); + long queueOffset = i; + while (iterator.hasNext()) { + CqUnit cqUnit = iterator.next(); + Assert.assertEquals(queueOffset, cqUnit.getQueueOffset()); + Assert.assertEquals(queueOffset * msgSize, cqUnit.getPos()); + Assert.assertEquals(msgSize, cqUnit.getSize()); + assertTrue(cqUnit.isTagsCodeValid()); + Assert.assertEquals(queueOffset, cqUnit.getTagsCode()); + Assert.assertEquals(queueOffset, cqUnit.getValidTagsCodeAsLong().longValue()); + Assert.assertEquals(1, cqUnit.getBatchNum()); + assertNull(cqUnit.getCqExtUnit()); + queueOffset++; + } + Assert.assertEquals(msgNum, queueOffset); + } + } + + @Test + public void testInitializeWithOffset() throws Exception { + final String path = createBaseDir(); + FileUtils.deleteDirectory(new File(path)); + topicConfigTableMap.put(topic, new TopicConfig(topic, 1, 1, PermName.PERM_WRITE | PermName.PERM_READ)); + + { + messageStoreConfig.setRocksdbCQDoubleWriteEnable(true); + messageStore = (DefaultMessageStore) createMessageStore(path, false, topicConfigTableMap, messageStoreConfig); + messageStore.load(); + + ConsumeQueueStoreInterface consumeQueueStoreInterface = messageStore.getQueueStore(); + assertTrue(consumeQueueStoreInterface instanceof CombineConsumeQueueStore); + CombineConsumeQueueStore combineConsumeQueueStore = (CombineConsumeQueueStore) consumeQueueStoreInterface; + ConsumeQueueStore consumeQueueStore = combineConsumeQueueStore.getConsumeQueueStore(); + RocksDBConsumeQueueStore rocksDBConsumeQueueStore = combineConsumeQueueStore.getRocksDBConsumeQueueStore(); + assertNotNull(consumeQueueStore); + assertNotNull(rocksDBConsumeQueueStore); + + ConsumeQueueInterface rocksDBConsumeQueue = rocksDBConsumeQueueStore.findOrCreateConsumeQueue(topic, queueId); + consumeQueueStore.findOrCreateConsumeQueue(topic, queueId); + rocksDBConsumeQueue.initializeWithOffset(100); + + Assert.assertEquals(100, rocksDBConsumeQueue.getMaxOffsetInQueue()); + Assert.assertEquals(100, rocksDBConsumeQueue.getMinOffsetInQueue()); + + rocksDBConsumeQueue.initializeWithOffset(200); + + Assert.assertEquals(200, rocksDBConsumeQueue.getMaxOffsetInQueue()); + Assert.assertEquals(200, rocksDBConsumeQueue.getMinOffsetInQueue()); + + messageStore.start(); + + Assert.assertEquals(0, rocksDBConsumeQueue.getMaxOffsetInQueue()); + Assert.assertEquals(0, rocksDBConsumeQueue.getMinOffsetInQueue()); + + ConsumeQueue consumeQueue = (ConsumeQueue) consumeQueueStore.findOrCreateConsumeQueue(topic, queueId); + + for (int i = 0; i < msgNum; i++) { + MessageExtBrokerInner msg = new MessageExtBrokerInner(); + + msg.setQueueId(queueId); + msg.setBody(new byte[msgSize]); + msg.setTopic(topic); + msg.setTags("TAG1"); + + msg.setTagsCode(MessageExtBrokerInner.tagsString2tagsCode(SINGLE_TAG, msg.getTags())); + msg.setBornTimestamp(System.currentTimeMillis()); + msg.setReconsumeTimes(0); + + msg.setBornHost(new InetSocketAddress(9999)); + msg.setStoreHost(new InetSocketAddress(8888)); + + messageStore.putMessage(msg); + } + + await().atMost(5, TimeUnit.SECONDS).untilAsserted(() -> { + Assert.assertEquals(msgNum, consumeQueue.getMaxOffsetInQueue()); + Assert.assertEquals(0, consumeQueue.getMinOffsetInQueue()); + }); + + messageStore.shutdown(); + } + + { + messageStoreConfig.setRocksdbCQDoubleWriteEnable(true); + messageStore = (DefaultMessageStore) createMessageStore(path, false, topicConfigTableMap, messageStoreConfig); + messageStore.load(); + + ConsumeQueueStoreInterface consumeQueueStoreInterface = messageStore.getQueueStore(); + assertTrue(consumeQueueStoreInterface instanceof CombineConsumeQueueStore); + CombineConsumeQueueStore combineConsumeQueueStore = (CombineConsumeQueueStore) consumeQueueStoreInterface; + ConsumeQueueStore consumeQueueStore = combineConsumeQueueStore.getConsumeQueueStore(); + RocksDBConsumeQueueStore rocksDBConsumeQueueStore = combineConsumeQueueStore.getRocksDBConsumeQueueStore(); + assertNotNull(consumeQueueStore); + assertNotNull(rocksDBConsumeQueueStore); + + consumeQueueStore.findOrCreateConsumeQueue(topic, queueId).initializeWithOffset(200); + + ConsumeQueueInterface cq = rocksDBConsumeQueueStore.findOrCreateConsumeQueue(topic, queueId); + Assert.assertEquals(msgNum, cq.getMaxOffsetInQueue()); + Assert.assertEquals(0, cq.getMinOffsetInQueue()); + + combineConsumeQueueStore.checkAssignOffset(true); + + Assert.assertEquals(200, cq.getMaxOffsetInQueue()); + Assert.assertEquals(200, cq.getMinOffsetInQueue()); + + messageStore.shutdown(); + } + + } + + @Test + public void testCheckAssignOffset() throws Exception { + final String path = createBaseDir(); + topicConfigTableMap.put(topic, new TopicConfig(topic, 1, 1, PermName.PERM_WRITE | PermName.PERM_READ)); + + { + messageStoreConfig.setRocksdbCQDoubleWriteEnable(false); + messageStore = (DefaultMessageStore) createMessageStore(path, false, topicConfigTableMap, messageStoreConfig); + messageStore.load(); + messageStore.start(); + + assertTrue(messageStore.getQueueStore() instanceof ConsumeQueueStore); + + for (int i = 0; i < msgNum; i++) { + MessageExtBrokerInner msg = new MessageExtBrokerInner(); + + msg.setQueueId(queueId); + msg.setBody(new byte[msgSize]); + msg.setTopic(topic); + msg.setTags("TAG1"); + + msg.setTagsCode(MessageExtBrokerInner.tagsString2tagsCode(SINGLE_TAG, msg.getTags())); + msg.setBornTimestamp(System.currentTimeMillis()); + msg.setReconsumeTimes(0); + + msg.setBornHost(new InetSocketAddress(9999)); + msg.setStoreHost(new InetSocketAddress(8888)); + + messageStore.putMessage(msg); + } + + await().atMost(5, TimeUnit.SECONDS).untilAsserted(() -> { + File cq = new File(path + File.separator + "consumequeue" + File.separator + topic + File.separator + queueId + File.separator + "00000000000000000000"); + assertTrue(cq.exists()); + Assert.assertEquals(msgNum, (long) messageStore.getQueueStore().getMaxOffset(topic, queueId)); + Assert.assertEquals(0, messageStore.getQueueStore().getMinOffsetInQueue(topic, queueId)); + }); + + messageStore.shutdown(); + } + + { + messageStoreConfig.setRocksdbCQDoubleWriteEnable(true); + messageStore = (DefaultMessageStore) createMessageStore(path, false, topicConfigTableMap, messageStoreConfig); + messageStore.load(); + await().atMost(1, TimeUnit.SECONDS).untilAsserted(() -> { + assertTrue(((CombineConsumeQueueStore) messageStore.getQueueStore()).checkAssignOffset(false)); + }); + messageStore.start(); + messageStore.shutdown(); + } + + { + messageStoreConfig.setRocksdbCQDoubleWriteEnable(true); + messageStore = (DefaultMessageStore) createMessageStore(path, false, topicConfigTableMap, messageStoreConfig); + messageStore.load(); + await().atMost(1, TimeUnit.SECONDS).untilAsserted(() -> { + assertTrue(((CombineConsumeQueueStore) messageStore.getQueueStore()).checkAssignOffset(false)); + }); + messageStore.start(); + messageStore.shutdown(); + } + } +} diff --git a/store/src/test/java/org/apache/rocketmq/store/queue/ConsumeQueueTest.java b/store/src/test/java/org/apache/rocketmq/store/queue/ConsumeQueueTest.java index bf3b1eeca83..cae009ab66e 100644 --- a/store/src/test/java/org/apache/rocketmq/store/queue/ConsumeQueueTest.java +++ b/store/src/test/java/org/apache/rocketmq/store/queue/ConsumeQueueTest.java @@ -151,7 +151,7 @@ public void testIterator() throws Exception { DispatchRequest request = new DispatchRequest(consumeQueue.getTopic(), consumeQueue.getQueueId(), i * msgSize, msgSize, i, System.currentTimeMillis(), i, null, null, 0, 0, null); request.setBitMap(new byte[10]); - messageStore.getQueueStore().putMessagePositionInfoWrapper(consumeQueue, request); + ((AbstractConsumeQueueStore) messageStore.getQueueStore()).putMessagePositionInfoWrapper(consumeQueue, request); } Assert.assertEquals(0, consumeQueue.getMinLogicOffset()); Assert.assertEquals(0, consumeQueue.getMinOffsetInQueue()); @@ -200,7 +200,7 @@ public void testIterator() throws Exception { } Assert.assertEquals(msgNum, queueOffset); } - messageStore.getQueueStore().destroy(consumeQueue); + ((AbstractConsumeQueueStore) messageStore.getQueueStore()).destroy(consumeQueue); } @Test diff --git a/store/src/test/java/org/apache/rocketmq/store/queue/QueueTestBase.java b/store/src/test/java/org/apache/rocketmq/store/queue/QueueTestBase.java index 81dc158db53..df3da24ccb0 100644 --- a/store/src/test/java/org/apache/rocketmq/store/queue/QueueTestBase.java +++ b/store/src/test/java/org/apache/rocketmq/store/queue/QueueTestBase.java @@ -16,6 +16,12 @@ */ package org.apache.rocketmq.store.queue; +import java.io.File; +import java.util.HashMap; +import java.util.Map; +import java.util.concurrent.Callable; +import java.util.concurrent.ConcurrentHashMap; +import java.util.concurrent.ConcurrentMap; import org.apache.rocketmq.common.BrokerConfig; import org.apache.rocketmq.common.TopicAttributes; import org.apache.rocketmq.common.TopicConfig; @@ -23,22 +29,15 @@ 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.common.sysflag.MessageSysFlag; import org.apache.rocketmq.store.ConsumeQueue; import org.apache.rocketmq.store.DefaultMessageStore; -import org.apache.rocketmq.common.message.MessageExtBrokerInner; import org.apache.rocketmq.store.MessageStore; import org.apache.rocketmq.store.StoreTestBase; import org.apache.rocketmq.store.config.MessageStoreConfig; import org.apache.rocketmq.store.stats.BrokerStatsManager; -import java.io.File; -import java.util.HashMap; -import java.util.Map; -import java.util.concurrent.Callable; -import java.util.concurrent.ConcurrentHashMap; -import java.util.concurrent.ConcurrentMap; - public class QueueTestBase extends StoreTestBase { protected ConcurrentMap createTopicConfigTable(String topic, CQType cqType) { @@ -58,12 +57,17 @@ protected Callable fullyDispatched(MessageStore messageStore) { return () -> messageStore.dispatchBehindBytes() == 0; } - protected MessageStore createMessageStore(String baseDir, boolean extent, ConcurrentMap topicConfigTable) throws Exception { + protected MessageStore createMessageStore(String baseDir, boolean extent, + ConcurrentMap topicConfigTable) throws Exception { + return createMessageStore(baseDir, extent, topicConfigTable, new MessageStoreConfig()); + } + + protected MessageStore createMessageStore(String baseDir, boolean extent, + ConcurrentMap topicConfigTable, MessageStoreConfig messageStoreConfig) throws Exception { if (baseDir == null) { baseDir = createBaseDir(); } baseDirs.add(baseDir); - MessageStoreConfig messageStoreConfig = new MessageStoreConfig(); messageStoreConfig.setMappedFileSizeCommitLog(1024 * 8); messageStoreConfig.setMappedFileSizeConsumeQueue(100 * ConsumeQueue.CQ_STORE_UNIT_SIZE); messageStoreConfig.setMapperFileSizeBatchConsumeQueue(20 * BatchConsumeQueue.CQ_STORE_UNIT_SIZE); diff --git a/store/src/test/java/org/apache/rocketmq/store/queue/RocksDBConsumeQueueTest.java b/store/src/test/java/org/apache/rocketmq/store/queue/RocksDBConsumeQueueTest.java index b907ce59519..acf4a6a8023 100644 --- a/store/src/test/java/org/apache/rocketmq/store/queue/RocksDBConsumeQueueTest.java +++ b/store/src/test/java/org/apache/rocketmq/store/queue/RocksDBConsumeQueueTest.java @@ -16,14 +16,13 @@ */ package org.apache.rocketmq.store.queue; +import java.nio.ByteBuffer; import org.apache.rocketmq.common.MixAll; import org.apache.rocketmq.store.DefaultMessageStore; import org.junit.Test; import org.mockito.invocation.InvocationOnMock; import org.mockito.stubbing.Answer; -import java.nio.ByteBuffer; - import static org.apache.rocketmq.store.queue.RocksDBConsumeQueueTable.CQ_UNIT_SIZE; import static org.junit.Assert.assertEquals; import static org.junit.Assert.assertFalse; @@ -60,7 +59,7 @@ public ByteBuffer answer(InvocationOnMock mock) throws Throwable { } }); - RocksDBConsumeQueue consumeQueue = new RocksDBConsumeQueue(messageStore, "topic", 0); + RocksDBConsumeQueue consumeQueue = new RocksDBConsumeQueue(messageStore.getMessageStoreConfig(), rocksDBConsumeQueueStore, "topic", 0); ReferredIterator it = consumeQueue.iterateFrom(9000); for (int i = 0; i < 1000; i++) { assertTrue(it.hasNext());