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 76882cc71c1..a09e2173b66 100644 --- a/broker/src/main/java/org/apache/rocketmq/broker/BrokerController.java +++ b/broker/src/main/java/org/apache/rocketmq/broker/BrokerController.java @@ -117,6 +117,7 @@ import org.apache.rocketmq.broker.transaction.TransactionMetricsFlushService; import org.apache.rocketmq.broker.transaction.TransactionalMessageCheckService; import org.apache.rocketmq.broker.transaction.TransactionalMessageService; +import org.apache.rocketmq.broker.transaction.rocksdb.TransactionalMessageRocksDBService; import org.apache.rocketmq.broker.transaction.queue.DefaultTransactionalMessageCheckListener; import org.apache.rocketmq.broker.transaction.queue.TransactionalMessageBridge; import org.apache.rocketmq.broker.transaction.queue.TransactionalMessageServiceImpl; @@ -178,6 +179,8 @@ import org.apache.rocketmq.store.timer.TimerCheckpoint; import org.apache.rocketmq.store.timer.TimerMessageStore; import org.apache.rocketmq.store.timer.TimerMetrics; +import org.apache.rocketmq.store.timer.rocksdb.TimerMessageRocksDBStore; +import org.apache.rocketmq.store.transaction.TransMessageRocksDBStore; public class BrokerController { protected static final Logger LOG = LoggerFactory.getLogger(LoggerName.BROKER_LOGGER_NAME); @@ -269,6 +272,8 @@ public class BrokerController { private BrokerStats brokerStats; private InetSocketAddress storeHost; private TimerMessageStore timerMessageStore; + private TimerMessageRocksDBStore timerMessageRocksDBStore; + private TransMessageRocksDBStore transMessageRocksDBStore; private TimerCheckpoint timerCheckpoint; protected BrokerFastFailure brokerFastFailure; private Configuration configuration; @@ -277,6 +282,7 @@ public class BrokerController { protected TransactionalMessageCheckService transactionalMessageCheckService; protected TransactionalMessageService transactionalMessageService; protected AbstractTransactionalMessageCheckListener transactionalMessageCheckListener; + protected TransactionalMessageRocksDBService transactionalMessageRocksDBService; protected volatile boolean shutdown = false; protected ShutdownHook shutdownHook; private volatile boolean isScheduleServiceStart = false; @@ -865,6 +871,14 @@ public boolean initializeMessageStore() { this.timerMessageStore = new TimerMessageStore(messageStore, messageStoreConfig, timerCheckpoint, timerMetrics, brokerStatsManager); this.timerMessageStore.registerEscapeBridgeHook(msg -> escapeBridge.putMessage(msg)); this.messageStore.setTimerMessageStore(this.timerMessageStore); + if (messageStoreConfig.isTimerRocksDBEnable()) { + this.timerMessageRocksDBStore = new TimerMessageRocksDBStore(messageStore, timerMetrics, brokerStatsManager); + this.messageStore.setTimerMessageRocksDBStore(timerMessageRocksDBStore); + } + } + if (messageStoreConfig.isTransRocksDBEnable()) { + this.transMessageRocksDBStore = new TransMessageRocksDBStore(messageStore, brokerStatsManager, new InetSocketAddress(this.getBrokerConfig().getBrokerIP1(), this.getNettyServerConfig().getListenPort())); + this.messageStore.setTransRocksDBStore(transMessageRocksDBStore); } } catch (Exception e) { result = false; @@ -904,6 +918,9 @@ public boolean recoverAndInitService() throws CloneNotSupportedException { if (messageStoreConfig.isTimerWheelEnable()) { result = result && this.timerMessageStore.load(); + if (messageStoreConfig.isTimerRocksDBEnable()) { + result = result && this.timerMessageRocksDBStore.load(); + } } //scheduleMessageService load after messageStore load success @@ -1060,6 +1077,10 @@ private void initialTransaction() { this.transactionMetricsFlushService = new TransactionMetricsFlushService(this); this.transactionMetricsFlushService.start(); + if (messageStoreConfig.isTransRocksDBEnable()) { + this.transactionalMessageRocksDBService = new TransactionalMessageRocksDBService(messageStore, this); + this.transactionalMessageRocksDBService.start(); + } } private void initialRpcHooks() { @@ -1400,6 +1421,14 @@ public void setTimerMessageStore(TimerMessageStore timerMessageStore) { this.timerMessageStore = timerMessageStore; } + public TimerMessageRocksDBStore getTimerMessageRocksDBStore() { + return timerMessageRocksDBStore; + } + + public void setTimerMessageRocksDBStore(TimerMessageRocksDBStore timerMessageRocksDBStore) { + this.timerMessageRocksDBStore = timerMessageRocksDBStore; + } + public AckMessageProcessor getAckMessageProcessor() { return ackMessageProcessor; } @@ -1473,6 +1502,10 @@ protected void shutdownBasicService() { this.transactionMetricsFlushService.shutdown(); } + if (this.transactionalMessageRocksDBService != null) { + this.transactionalMessageRocksDBService.shutdown(); + } + if (this.notificationProcessor != null) { this.notificationProcessor.getPopLongPollingService().shutdown(); } @@ -1488,6 +1521,15 @@ protected void shutdownBasicService() { if (this.timerMessageStore != null) { this.timerMessageStore.shutdown(); } + + if (this.timerMessageRocksDBStore != null) { + this.timerMessageRocksDBStore.shutdown(); + } + + if (this.transMessageRocksDBStore != null) { + this.transMessageRocksDBStore.shutdown(); + } + if (this.fileWatchService != null) { this.fileWatchService.shutdown(); } @@ -1562,7 +1604,7 @@ protected void shutdownBasicService() { if (this.transactionalMessageCheckService != null) { this.transactionalMessageCheckService.shutdown(false); } - + if (this.loadBalanceExecutor != null) { this.loadBalanceExecutor.shutdown(); } @@ -1693,6 +1735,10 @@ protected void startBasicService() throws Exception { this.timerMessageStore.start(); } + if (this.timerMessageRocksDBStore != null && this.messageStoreConfig.isTimerRocksDBEnable()) { + this.timerMessageRocksDBStore.start(); + } + if (this.replicasManager != null) { this.replicasManager.start(); } 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 e7333ab91ad..4361431bec0 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 @@ -237,6 +237,7 @@ import static org.apache.rocketmq.broker.metrics.BrokerMetricsConstant.LABEL_INVOCATION_STATUS; import static org.apache.rocketmq.broker.metrics.BrokerMetricsConstant.LABEL_IS_SYSTEM; +import static org.apache.rocketmq.common.message.MessageConst.TIMER_ENGINE_TYPE; import static org.apache.rocketmq.remoting.protocol.RemotingCommand.buildErrorResponse; public class AdminBrokerProcessor implements NettyRequestProcessor { @@ -406,6 +407,8 @@ public RemotingCommand processRequest(ChannelHandlerContext ctx, return this.listAcl(ctx, request); case RequestCode.POP_ROLLBACK: return this.transferPopToFsStore(ctx, request); + case RequestCode.SWITCH_TIMER_ENGINE: + return this.switchTimerEngine(ctx, request); default: return getUnknownCmdResponse(ctx, request); } @@ -2880,7 +2883,11 @@ private RemotingCommand resumeCheckHalfMessage(ChannelHandlerContext ctx, private MessageExtBrokerInner toMessageExtBrokerInner(MessageExt msgExt) { MessageExtBrokerInner inner = new MessageExtBrokerInner(); - inner.setTopic(TransactionalMessageUtil.buildHalfTopic()); + if (brokerController.getMessageStoreConfig().isTransRocksDBEnable() && !brokerController.getMessageStoreConfig().isTransWriteOriginTransHalfEnable()) { + inner.setTopic(TransactionalMessageUtil.buildHalfTopicForRocksDB()); + } else { + inner.setTopic(TransactionalMessageUtil.buildHalfTopic()); + } inner.setBody(msgExt.getBody()); inner.setFlag(msgExt.getFlag()); MessageAccessor.setProperties(inner, msgExt.getProperties()); @@ -3409,4 +3416,64 @@ private RemotingCommand transferPopToFsStore(ChannelHandlerContext ctx, Remoting } return response; } + + private synchronized RemotingCommand switchTimerEngine(ChannelHandlerContext ctx, RemotingCommand request) { + final RemotingCommand response = RemotingCommand.createResponseCommand(null); + if (!this.brokerController.getMessageStoreConfig().isTimerWheelEnable()) { + LOGGER.info("switchTimerEngine error, broker timerWheelEnable is false"); + response.setCode(ResponseCode.INVALID_PARAMETER); + response.setRemark("broker timerWheelEnable is false"); + return response; + } + if (null == request.getExtFields()) { + LOGGER.info("switchTimerEngine extFields is null"); + response.setCode(ResponseCode.INVALID_PARAMETER); + response.setRemark("param error, extFields is null"); + return response; + } + String engineType = request.getExtFields().get(TIMER_ENGINE_TYPE); + if (StringUtils.isEmpty(engineType) || !MessageConst.TIMER_ENGINE_ROCKSDB_TIMELINE.equals(engineType) && !MessageConst.TIMER_ENGINE_FILE_TIME_WHEEL.equals(engineType)) { + response.setCode(ResponseCode.INVALID_PARAMETER); + response.setRemark("param error"); + return response; + } + try { + Properties properties = new Properties(); + boolean result = false; + if (MessageConst.TIMER_ENGINE_ROCKSDB_TIMELINE.equals(engineType)) { + if (this.brokerController.getTimerMessageRocksDBStore() == null) { + response.setCode(ResponseCode.INVALID_PARAMETER); + response.setRemark("timerUseRocksDB muse be configured true when broker start"); + return response; + } + result = this.brokerController.getTimerMessageRocksDBStore().restart(); + if (result) { + properties.put("timerStopEnqueue", Boolean.TRUE.toString()); + properties.put("timerUseRocksDB", Boolean.TRUE.toString()); + properties.put("timerRocksDBStopScan", Boolean.FALSE.toString()); + } + } else { + result = this.brokerController.getTimerMessageStore().restart(); + if (result) { + properties.put("timerRocksDBStopScan", Boolean.TRUE.toString()); + properties.put("timerStopEnqueue", Boolean.FALSE.toString()); + } + } + if (result) { + this.brokerController.getConfiguration().update(properties); + response.setCode(ResponseCode.SUCCESS); + response.setRemark("switch timer engine success"); + LOGGER.info("switchTimerEngine success"); + } else { + response.setCode(ResponseCode.SYSTEM_ERROR); + response.setRemark("switch timer engine error"); + LOGGER.info("switchTimerEngine error"); + } + } catch (Exception e) { + response.setCode(ResponseCode.SYSTEM_ERROR); + response.setRemark("switch timer engine error"); + LOGGER.error("switchTimerEngine error : {}", e.getMessage()); + } + return response; + } } diff --git a/broker/src/main/java/org/apache/rocketmq/broker/processor/EndTransactionProcessor.java b/broker/src/main/java/org/apache/rocketmq/broker/processor/EndTransactionProcessor.java index 153ac24c1f6..7298e5da58a 100644 --- a/broker/src/main/java/org/apache/rocketmq/broker/processor/EndTransactionProcessor.java +++ b/broker/src/main/java/org/apache/rocketmq/broker/processor/EndTransactionProcessor.java @@ -30,6 +30,7 @@ import org.apache.rocketmq.common.message.MessageExt; import org.apache.rocketmq.common.message.MessageExtBrokerInner; import org.apache.rocketmq.common.sysflag.MessageSysFlag; +import org.apache.rocketmq.common.topic.TopicValidator; import org.apache.rocketmq.logging.org.slf4j.Logger; import org.apache.rocketmq.logging.org.slf4j.LoggerFactory; import org.apache.rocketmq.remoting.common.RemotingHelper; @@ -146,7 +147,7 @@ public RemotingCommand processRequest(ChannelHandlerContext ctx, RemotingCommand MessageAccessor.clearProperty(msgInner, MessageConst.PROPERTY_TRANSACTION_PREPARED); RemotingCommand sendResult = sendFinalMessage(msgInner); if (sendResult.getCode() == ResponseCode.SUCCESS) { - this.brokerController.getTransactionalMessageService().deletePrepareMessage(result.getPrepareMessage()); + deletePrepareMessage(result); // successful committed, then total num of half-messages minus 1 this.brokerController.getTransactionalMessageService().getTransactionMetrics().addAndGet(msgInner.getTopic(), -1); this.brokerController.getBrokerMetricsManager().getCommitMessagesTotal().add(1, this.brokerController.getBrokerMetricsManager().newAttributesBuilder() @@ -173,7 +174,7 @@ public RemotingCommand processRequest(ChannelHandlerContext ctx, RemotingCommand } RemotingCommand res = checkPrepareMessage(result.getPrepareMessage(), requestHeader); if (res.getCode() == ResponseCode.SUCCESS) { - this.brokerController.getTransactionalMessageService().deletePrepareMessage(result.getPrepareMessage()); + deletePrepareMessage(result); // roll back, then total num of half-messages minus 1 this.brokerController.getTransactionalMessageService().getTransactionMetrics().addAndGet(result.getPrepareMessage().getProperty(MessageConst.PROPERTY_REAL_TOPIC), -1); this.brokerController.getBrokerMetricsManager().getRollBackMessagesTotal().add(1, this.brokerController.getBrokerMetricsManager().newAttributesBuilder() @@ -188,6 +189,26 @@ public RemotingCommand processRequest(ChannelHandlerContext ctx, RemotingCommand return response; } + private void deletePrepareMessage(OperationResult result) { + if (null == result || null == result.getPrepareMessage()) { + LOGGER.error("deletePrepareMessage param error, result is null or prepareMessage is null"); + return; + } + MessageExt prepareMessage = result.getPrepareMessage(); + String halfTopic = prepareMessage.getTopic(); + if (StringUtils.isEmpty(halfTopic)) { + LOGGER.error("deletePrepareMessage halfTopic is empty, halfTopic: {}", halfTopic); + return; + } + if (TopicValidator.RMQ_SYS_TRANS_HALF_TOPIC.equals(halfTopic)) { + this.brokerController.getTransactionalMessageService().deletePrepareMessage(prepareMessage); + } else if (this.brokerController.getMessageStoreConfig().isTransRocksDBEnable() && TopicValidator.RMQ_SYS_ROCKSDB_TRANS_HALF_TOPIC.equals(halfTopic)) { + this.brokerController.getMessageStore().getTransRocksDBStore().deletePrepareMessage(prepareMessage); + } else { + LOGGER.warn("deletePrepareMessage error, topic of half message is: {}, transRocksDBEnable: {}", halfTopic, this.brokerController.getMessageStoreConfig().isTransRocksDBEnable()); + } + } + /** * If you specify a custom first check time CheckImmunityTimeInSeconds, * And the commit/rollback request whose validity period exceeds CheckImmunityTimeInSeconds and is not checked back will be processed and failed @@ -265,10 +286,17 @@ private MessageExtBrokerInner endMessageTransaction(MessageExt msgExt) { : TopicFilterType.SINGLE_TAG; long tagsCodeValue = MessageExtBrokerInner.tagsString2tagsCode(topicFilterType, msgInner.getTags()); msgInner.setTagsCode(tagsCodeValue); - MessageAccessor.setProperties(msgInner, msgExt.getProperties()); - msgInner.setPropertiesString(MessageDecoder.messageProperties2String(msgExt.getProperties())); + String checkTimes = msgExt.getUserProperty(MessageConst.PROPERTY_TRANSACTION_CHECK_TIMES); + if (StringUtils.isEmpty(checkTimes) && this.brokerController.getMessageStoreConfig().isTransRocksDBEnable() && null != this.brokerController.getMessageStore().getTransRocksDBStore()) { + Integer checkTimesRocksDB = this.brokerController.getMessageStore().getTransRocksDBStore().getCheckTimes(msgInner.getTopic(), msgInner.getTransactionId(), msgExt.getCommitLogOffset()); + if (null != checkTimesRocksDB && checkTimesRocksDB >= 0) { + msgExt.putUserProperty(MessageConst.PROPERTY_TRANSACTION_CHECK_TIMES, String.valueOf(checkTimesRocksDB)); + } + } + MessageAccessor.setProperties(msgInner, MessageDecoder.string2messageProperties(MessageDecoder.messageProperties2String(msgExt.getProperties()))); MessageAccessor.clearProperty(msgInner, MessageConst.PROPERTY_REAL_TOPIC); MessageAccessor.clearProperty(msgInner, MessageConst.PROPERTY_REAL_QUEUE_ID); + msgInner.setPropertiesString(MessageDecoder.messageProperties2String(msgInner.getProperties())); return msgInner; } diff --git a/broker/src/main/java/org/apache/rocketmq/broker/processor/QueryMessageProcessor.java b/broker/src/main/java/org/apache/rocketmq/broker/processor/QueryMessageProcessor.java index fc46cb53186..f29cd2b8a14 100644 --- a/broker/src/main/java/org/apache/rocketmq/broker/processor/QueryMessageProcessor.java +++ b/broker/src/main/java/org/apache/rocketmq/broker/processor/QueryMessageProcessor.java @@ -21,11 +21,13 @@ import io.netty.channel.FileRegion; import io.opentelemetry.api.common.Attributes; import java.util.concurrent.TimeUnit; +import org.apache.commons.lang3.StringUtils; import org.apache.rocketmq.broker.BrokerController; import org.apache.rocketmq.broker.pagecache.OneMessageTransfer; import org.apache.rocketmq.broker.pagecache.QueryMessageTransfer; import org.apache.rocketmq.common.MixAll; import org.apache.rocketmq.common.constant.LoggerName; +import org.apache.rocketmq.common.message.MessageConst; import org.apache.rocketmq.logging.org.slf4j.Logger; import org.apache.rocketmq.logging.org.slf4j.LoggerFactory; import org.apache.rocketmq.remoting.common.RemotingHelper; @@ -84,16 +86,19 @@ public RemotingCommand queryMessage(ChannelHandlerContext ctx, RemotingCommand r .decodeCommandCustomHeader(QueryMessageRequestHeader.class); response.setOpaque(request.getOpaque()); - - String isUniqueKey = request.getExtFields().get(MixAll.UNIQUE_MSG_QUERY_FLAG); - if (isUniqueKey != null && isUniqueKey.equals("true")) { + String indexType = requestHeader.getIndexType(); + String lastKey = requestHeader.getLastKey(); + String isUniqueKey = null; + if (null != request.getExtFields()) { + isUniqueKey = request.getExtFields().get(MixAll.UNIQUE_MSG_QUERY_FLAG); + } + if (!StringUtils.isEmpty(isUniqueKey) && Boolean.parseBoolean(isUniqueKey)) { requestHeader.setMaxNum(this.brokerController.getMessageStoreConfig().getDefaultQueryMaxNum()); + indexType = MessageConst.INDEX_UNIQUE_TYPE; + } else if (StringUtils.isEmpty(indexType)) { + indexType = MessageConst.INDEX_KEY_TYPE; } - - final QueryMessageResult queryMessageResult = - this.brokerController.getMessageStore().queryMessage(requestHeader.getTopic(), - requestHeader.getKey(), requestHeader.getMaxNum(), requestHeader.getBeginTimestamp(), - requestHeader.getEndTimestamp()); + final QueryMessageResult queryMessageResult = this.brokerController.getMessageStore().queryMessage(requestHeader.getTopic(), requestHeader.getKey(), requestHeader.getMaxNum(), requestHeader.getBeginTimestamp(), requestHeader.getEndTimestamp(), indexType, lastKey); assert queryMessageResult != null; responseHeader.setIndexLastUpdatePhyoffset(queryMessageResult.getIndexLastUpdatePhyoffset()); diff --git a/broker/src/main/java/org/apache/rocketmq/broker/topic/TopicConfigManager.java b/broker/src/main/java/org/apache/rocketmq/broker/topic/TopicConfigManager.java index f59651fc8dd..51b5f5492ad 100644 --- a/broker/src/main/java/org/apache/rocketmq/broker/topic/TopicConfigManager.java +++ b/broker/src/main/java/org/apache/rocketmq/broker/topic/TopicConfigManager.java @@ -214,6 +214,26 @@ protected void init() { putTopicConfig(topicConfig); } + { + // TopicValidator.RMQ_SYS_ROCKSDB_TRANS_HALF_TOPIC + String topic = TopicValidator.RMQ_SYS_ROCKSDB_TRANS_HALF_TOPIC; + TopicConfig topicConfig = new TopicConfig(topic); + TopicValidator.addSystemTopic(topic); + topicConfig.setReadQueueNums(1); + topicConfig.setWriteQueueNums(1); + putTopicConfig(topicConfig); + } + + { + // TopicValidator.RMQ_SYS_ROCKSDB_TRANS_OP_HALF_TOPIC + String topic = TopicValidator.RMQ_SYS_ROCKSDB_TRANS_OP_HALF_TOPIC; + TopicConfig topicConfig = new TopicConfig(topic); + TopicValidator.addSystemTopic(topic); + topicConfig.setReadQueueNums(1); + topicConfig.setWriteQueueNums(1); + putTopicConfig(topicConfig); + } + { if (this.brokerController.getMessageStoreConfig().isTimerWheelEnable()) { String topic = TimerMessageStore.TIMER_TOPIC; diff --git a/broker/src/main/java/org/apache/rocketmq/broker/transaction/queue/TransactionalMessageBridge.java b/broker/src/main/java/org/apache/rocketmq/broker/transaction/queue/TransactionalMessageBridge.java index 5fcc1f56b78..47e453946d7 100644 --- a/broker/src/main/java/org/apache/rocketmq/broker/transaction/queue/TransactionalMessageBridge.java +++ b/broker/src/main/java/org/apache/rocketmq/broker/transaction/queue/TransactionalMessageBridge.java @@ -226,7 +226,11 @@ private MessageExtBrokerInner parseHalfMessageInner(MessageExtBrokerInner msgInn String.valueOf(msgInner.getQueueId())); msgInner.setSysFlag( MessageSysFlag.resetTransactionValue(msgInner.getSysFlag(), MessageSysFlag.TRANSACTION_NOT_TYPE)); - msgInner.setTopic(TransactionalMessageUtil.buildHalfTopic()); + if (null != store.getMessageStoreConfig() && store.getMessageStoreConfig().isTransRocksDBEnable() && !store.getMessageStoreConfig().isTransWriteOriginTransHalfEnable()) { + msgInner.setTopic(TransactionalMessageUtil.buildHalfTopicForRocksDB()); + } else { + msgInner.setTopic(TransactionalMessageUtil.buildHalfTopic()); + } msgInner.setQueueId(0); msgInner.setPropertiesString(MessageDecoder.messageProperties2String(msgInner.getProperties())); return msgInner; diff --git a/broker/src/main/java/org/apache/rocketmq/broker/transaction/queue/TransactionalMessageUtil.java b/broker/src/main/java/org/apache/rocketmq/broker/transaction/queue/TransactionalMessageUtil.java index 555ae4d2940..7edbc57b385 100644 --- a/broker/src/main/java/org/apache/rocketmq/broker/transaction/queue/TransactionalMessageUtil.java +++ b/broker/src/main/java/org/apache/rocketmq/broker/transaction/queue/TransactionalMessageUtil.java @@ -38,10 +38,18 @@ public static String buildOpTopic() { return TopicValidator.RMQ_SYS_TRANS_OP_HALF_TOPIC; } + public static String buildOpTopicForRocksDB() { + return TopicValidator.RMQ_SYS_ROCKSDB_TRANS_OP_HALF_TOPIC; + } + public static String buildHalfTopic() { return TopicValidator.RMQ_SYS_TRANS_HALF_TOPIC; } + public static String buildHalfTopicForRocksDB() { + return TopicValidator.RMQ_SYS_ROCKSDB_TRANS_HALF_TOPIC; + } + public static String buildConsumerGroup() { return MixAll.CID_SYS_RMQ_TRANS; } diff --git a/broker/src/main/java/org/apache/rocketmq/broker/transaction/rocksdb/TransactionalMessageRocksDBService.java b/broker/src/main/java/org/apache/rocketmq/broker/transaction/rocksdb/TransactionalMessageRocksDBService.java new file mode 100644 index 00000000000..389c75e4267 --- /dev/null +++ b/broker/src/main/java/org/apache/rocketmq/broker/transaction/rocksdb/TransactionalMessageRocksDBService.java @@ -0,0 +1,271 @@ +/* + * 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.broker.transaction.rocksdb; +import java.util.ArrayList; +import java.util.List; +import java.util.concurrent.ArrayBlockingQueue; +import java.util.concurrent.ExecutorService; +import java.util.concurrent.ThreadPoolExecutor.CallerRunsPolicy; +import java.util.concurrent.TimeUnit; +import io.netty.channel.Channel; +import org.apache.commons.collections.CollectionUtils; +import org.apache.commons.lang3.StringUtils; +import org.apache.rocketmq.broker.BrokerController; +import org.apache.rocketmq.common.ServiceThread; +import org.apache.rocketmq.common.ThreadFactoryImpl; +import org.apache.rocketmq.common.constant.LoggerName; +import org.apache.rocketmq.common.message.MessageClientIDSetter; +import org.apache.rocketmq.common.message.MessageConst; +import org.apache.rocketmq.common.message.MessageExt; +import org.apache.rocketmq.common.utils.ThreadUtils; +import org.apache.rocketmq.logging.org.slf4j.Logger; +import org.apache.rocketmq.logging.org.slf4j.LoggerFactory; +import org.apache.rocketmq.remoting.protocol.header.CheckTransactionStateRequestHeader; +import org.apache.rocketmq.store.DefaultMessageStore; +import org.apache.rocketmq.store.MessageStore; +import org.apache.rocketmq.store.rocksdb.MessageRocksDBStorage; +import org.apache.rocketmq.store.transaction.TransRocksDBRecord; +import org.apache.rocketmq.store.transaction.TransMessageRocksDBStore; +import static org.apache.rocketmq.store.rocksdb.MessageRocksDBStorage.TRANS_COLUMN_FAMILY; + +public class TransactionalMessageRocksDBService { + private static final Logger log = LoggerFactory.getLogger(LoggerName.TRANSACTION_LOGGER_NAME); + private static final int MAX_BATCH_SIZE_FROM_ROCKSDB = 2000; + private static final int INITIAL = 0, RUNNING = 1, SHUTDOWN = 2; + private volatile int state = INITIAL; + + private final MessageRocksDBStorage messageRocksDBStorage; + private final TransMessageRocksDBStore transMessageRocksDBStore; + private final MessageStore messageStore; + private final BrokerController brokerController; + + private TransStatusCheckService transStatusService; + private ExecutorService checkTranStatusTaskExecutor; + + public TransactionalMessageRocksDBService(final MessageStore messageStore, final BrokerController brokerController) { + this.messageStore = messageStore; + this.transMessageRocksDBStore = messageStore.getTransRocksDBStore(); + this.messageRocksDBStorage = transMessageRocksDBStore.getMessageRocksDBStorage(); + this.brokerController = brokerController; + } + + public void start() { + if (this.state == RUNNING) { + return; + } + initService(); + this.transStatusService.start(); + this.state = RUNNING; + log.info("TransactionalMessageRocksDBService start success"); + } + + private void initService() { + this.transStatusService = new TransStatusCheckService(); + this.checkTranStatusTaskExecutor = ThreadUtils.newThreadPoolExecutor( + 2, + 5, + 100, + TimeUnit.SECONDS, + new ArrayBlockingQueue<>(2000), + new ThreadFactoryImpl("Transaction-rocksdb-msg-check-thread", brokerController.getBrokerIdentity()), + new CallerRunsPolicy()); + } + + public void shutdown() { + if (this.state != RUNNING || this.state == SHUTDOWN) { + return; + } + if (null != this.transStatusService) { + this.transStatusService.shutdown(); + } + if (null != this.checkTranStatusTaskExecutor) { + this.checkTranStatusTaskExecutor.shutdown(); + } + this.state = SHUTDOWN; + log.info("TransactionalMessageRocksDBService shutdown success"); + } + + private void checkTransStatus() { + long count = 0; + byte[] lastKey = null; + while (true) { + try { + List trs = messageRocksDBStorage.scanRecordsForTrans(TRANS_COLUMN_FAMILY, MAX_BATCH_SIZE_FROM_ROCKSDB, lastKey); + if (CollectionUtils.isEmpty(trs)) { + log.info("TransactionalMessageRocksDBService checkTransStatus trs is empty"); + break; + } + count += trs.size(); + checkTransRecordsStatus(trs); + lastKey = trs.size() >= MAX_BATCH_SIZE_FROM_ROCKSDB ? trs.get(trs.size() - 1).getKeyBytes() : null; + if (null == lastKey) { + break; + } + } catch (Exception e) { + log.error("TransactionalMessageRocksDBService checkTransStatus error, error: {}, count: {}", e.getMessage(), count); + break; + } + } + log.info("TransactionalMessageRocksDBService checkTransStatus count: {}", count); + } + + private void checkTransRecordsStatus(List trs) { + if (CollectionUtils.isEmpty(trs)) { + log.error("TransactionalMessageRocksDBService checkTransRecordsStatus, trs is empty"); + return; + } + try { + List updateList = new ArrayList<>(); + for (TransRocksDBRecord halfRecord : trs) { + if (null == halfRecord) { + log.error("TransactionalMessageRocksDBService checkTransRecordsStatus, halfRecord is null"); + continue; + } + try { + if (halfRecord.getCheckTimes() > brokerController.getBrokerConfig().getTransactionCheckMax()) { + halfRecord.setDelete(true); + updateList.add(halfRecord); + log.info("TransactionalMessageRocksDBService checkTransRecordsStatus checkTimes > {}, need delete, checkTimes: {}, msgId: {}", brokerController.getBrokerConfig().getTransactionCheckMax(), halfRecord.getCheckTimes(), halfRecord.getUniqKey()); + continue; + } + MessageExt msgExt = transMessageRocksDBStore.getMessage(halfRecord.getOffsetPy(), halfRecord.getSizePy()); + if (null == msgExt) { + log.error("TransactionalMessageRocksDBService checkTransRecordsStatus, msgExt is null, offsetPy: {}, sizePy: {}", halfRecord.getOffsetPy(), halfRecord.getSizePy()); + halfRecord.setDelete(true); + updateList.add(halfRecord); + continue; + } + if (!isImmunityTimeExpired(msgExt)) { + continue; + } + resolveHalfMsg(msgExt); + halfRecord.setCheckTimes(halfRecord.getCheckTimes() + 1); + if (halfRecord.getCheckTimes() > brokerController.getBrokerConfig().getTransactionCheckMax()) { + halfRecord.setDelete(true); + log.info("TransactionalMessageRocksDBService checkTransRecordsStatus checkTimes > {}, need delete, checkTimes: {}, msgId: {}", brokerController.getBrokerConfig().getTransactionCheckMax(), halfRecord.getCheckTimes(), halfRecord.getUniqKey()); + } + updateList.add(halfRecord); + } catch (Exception e) { + log.error("TransactionalMessageRocksDBService checkTransRecordsStatus error : {}", e.getMessage()); + } + } + if (!CollectionUtils.isEmpty(updateList)) { + messageRocksDBStorage.updateRecordsForTrans(TRANS_COLUMN_FAMILY, updateList); + } + } catch (Exception e) { + log.error("TransactionalMessageRocksDBService checkTransRecordsStatus error: {}", e.getMessage()); + } + } + + private boolean isImmunityTimeExpired(MessageExt msgExt) { + String immunityTimeStr = msgExt.getUserProperty(MessageConst.PROPERTY_CHECK_IMMUNITY_TIME_IN_SECONDS); + long immunityTime = brokerController.getBrokerConfig().getTransactionTimeOut(); + if (!StringUtils.isEmpty(immunityTimeStr)) { + try { + immunityTime = Long.parseLong(immunityTimeStr); + immunityTime *= 1000; + } catch (Exception e) { + log.error("parse immunityTimesStr error: {}, msgId: {}", e.getMessage(), msgExt.getMsgId()); + } + } + if ((System.currentTimeMillis() - msgExt.getBornTimestamp()) < immunityTime) { + return false; + } + return true; + } + + private String getServiceThreadName() { + String brokerIdentifier = ""; + if (TransactionalMessageRocksDBService.this.messageStore instanceof DefaultMessageStore) { + DefaultMessageStore messageStore = (DefaultMessageStore) TransactionalMessageRocksDBService.this.messageStore; + if (messageStore.getBrokerConfig().isInBrokerContainer()) { + brokerIdentifier = messageStore.getBrokerConfig().getIdentifier(); + } + } + return brokerIdentifier; + } + + private void resolveHalfMsg(final MessageExt msgExt) { + if (checkTranStatusTaskExecutor != null) { + checkTranStatusTaskExecutor.execute(new Runnable() { + @Override + public void run() { + try { + sendCheckMessage(msgExt); + } catch (Exception e) { + log.error("TransactionalMessageRocksDBService Send check message error: {}, msgId: {}", e.getMessage(), msgExt.getMsgId()); + } + } + }); + } else { + log.error("TransactionalMessageRocksDBService checkTranStatusTaskExecutor not init, msgId: {}", msgExt.getMsgId()); + } + } + + private void sendCheckMessage(MessageExt msgExt) { + if (null == msgExt) { + log.info("TransactionalMessageRocksDBService sendCheckMessage msgExt is null"); + return; + } + try { + CheckTransactionStateRequestHeader checkTransactionStateRequestHeader = new CheckTransactionStateRequestHeader(); + checkTransactionStateRequestHeader.setTopic(msgExt.getTopic()); + checkTransactionStateRequestHeader.setCommitLogOffset(msgExt.getCommitLogOffset()); + checkTransactionStateRequestHeader.setOffsetMsgId(msgExt.getMsgId()); + checkTransactionStateRequestHeader.setMsgId(MessageClientIDSetter.getUniqID(msgExt)); + checkTransactionStateRequestHeader.setTransactionId(checkTransactionStateRequestHeader.getMsgId()); + checkTransactionStateRequestHeader.setTranStateTableOffset(msgExt.getQueueOffset()); + checkTransactionStateRequestHeader.setBrokerName(brokerController.getBrokerConfig().getBrokerName()); + msgExt.setTopic(msgExt.getUserProperty(MessageConst.PROPERTY_REAL_TOPIC)); + msgExt.setQueueId(Integer.parseInt(msgExt.getUserProperty(MessageConst.PROPERTY_REAL_QUEUE_ID))); + msgExt.setStoreSize(0); + String groupId = msgExt.getProperty(MessageConst.PROPERTY_PRODUCER_GROUP); + Channel channel = brokerController.getProducerManager().getAvailableChannel(groupId); + if (channel != null) { + brokerController.getBroker2Client().checkProducerTransactionState(groupId, channel, checkTransactionStateRequestHeader, msgExt); + } else { + log.warn("TransactionalMessageRocksDBService checkProducerTransactionState failed, channel is null. groupId: {}, msgId: {}", groupId, msgExt.getMsgId()); + } + } catch (Exception e) { + log.error("TransactionalMessageRocksDBService sendCheckMessage error: {}, msgId: {}", e.getMessage(), msgExt.getMsgId()); + } + } + + private class TransStatusCheckService extends ServiceThread { + private final Logger log = TransactionalMessageRocksDBService.log; + @Override + public String getServiceName() { + return getServiceThreadName() + this.getClass().getSimpleName(); + } + + @Override + public void run() { + log.info(this.getServiceName() + " service start"); + while (!this.isStopped()) { + try { + long begin = System.currentTimeMillis(); + checkTransStatus(); + log.info("TransactionalMessageRocksDBService ScanTransAndStatusCheckService check trans status, check cost: {}", System.currentTimeMillis() - begin); + waitForRunning(brokerController.getBrokerConfig().getTransactionCheckInterval()); + } catch (Exception e) { + log.error("TransactionalMessageRocksDBService ScanTransAndStatusCheckService error: {}", e.getMessage()); + } + } + log.info(this.getServiceName() + " service end"); + } + } +} diff --git a/broker/src/test/java/org/apache/rocketmq/broker/processor/QueryMessageProcessorTest.java b/broker/src/test/java/org/apache/rocketmq/broker/processor/QueryMessageProcessorTest.java index 0fd54df7d8a..3656c5be2bd 100644 --- a/broker/src/test/java/org/apache/rocketmq/broker/processor/QueryMessageProcessorTest.java +++ b/broker/src/test/java/org/apache/rocketmq/broker/processor/QueryMessageProcessorTest.java @@ -83,14 +83,13 @@ public void testQueryMessage() throws RemotingCommandException { result.setIndexLastUpdatePhyoffset(0); result.addMessage(new SelectMappedBufferResult(0, null, 0, null)); - when(messageStore.queryMessage(anyString(),anyString(),anyInt(),anyLong(),anyLong())).thenReturn(result); + when(messageStore.queryMessage(anyString(),anyString(),anyInt(),anyLong(),anyLong(),any(),any())).thenReturn(result); RemotingCommand request = createQueryMessageRequest("topic", "msgKey", 1, 100, 200,"false"); request.makeCustomHeaderToNet(); RemotingCommand response = queryMessageProcessor.processRequest(handlerContext, request); Assert.assertEquals(response.getCode(), ResponseCode.QUERY_NOT_FOUND); result.addMessage(new SelectMappedBufferResult(0, null, 1, null)); - when(messageStore.queryMessage(anyString(),anyString(),anyInt(),anyLong(),anyLong())).thenReturn(result); response = queryMessageProcessor.processRequest(handlerContext, request); Assert.assertNull(response); } diff --git a/client/src/main/java/org/apache/rocketmq/client/impl/MQAdminImpl.java b/client/src/main/java/org/apache/rocketmq/client/impl/MQAdminImpl.java index f98d9e5818d..b6fd72ad013 100644 --- a/client/src/main/java/org/apache/rocketmq/client/impl/MQAdminImpl.java +++ b/client/src/main/java/org/apache/rocketmq/client/impl/MQAdminImpl.java @@ -279,20 +279,18 @@ public MessageExt viewMessage(String topic, String msgId) public QueryResult queryMessage(String topic, String key, int maxNum, long begin, long end) throws MQClientException, InterruptedException { - return queryMessage(null, topic, key, maxNum, begin, end, false); + return queryMessage(null, topic, key, maxNum, begin, end, false, MessageConst.INDEX_KEY_TYPE, null); } public QueryResult queryMessageByUniqKey(String topic, String uniqKey, int maxNum, long begin, long end) throws MQClientException, InterruptedException { - - return queryMessage(null, topic, uniqKey, maxNum, begin, end, true); + return queryMessage(null, topic, uniqKey, maxNum, begin, end, true, MessageConst.INDEX_UNIQUE_TYPE, null); } public QueryResult queryMessageByUniqKey(String clusterName, String topic, String uniqKey, int maxNum, long begin, long end) throws MQClientException, InterruptedException { - - return queryMessage(clusterName, topic, uniqKey, maxNum, begin, end, true); + return queryMessage(clusterName, topic, uniqKey, maxNum, begin, end, true, MessageConst.INDEX_UNIQUE_TYPE, null); } public MessageExt queryMessageByUniqKey(String topic, @@ -312,7 +310,7 @@ public MessageExt queryMessageByUniqKey(String topic, public MessageExt queryMessageByUniqKey(String clusterName, String topic, String uniqKey, long begin, long end) throws InterruptedException, MQClientException { - QueryResult qr = this.queryMessage(clusterName, topic, uniqKey, 32, begin, end, true); + QueryResult qr = this.queryMessage(clusterName, topic, uniqKey, 32, begin, end, true, MessageConst.INDEX_UNIQUE_TYPE, null); if (qr != null && qr.getMessageList() != null && qr.getMessageList().size() > 0) { return qr.getMessageList().get(0); } else { @@ -320,8 +318,12 @@ public MessageExt queryMessageByUniqKey(String clusterName, String topic, } } - public QueryResult queryMessage(String clusterName, String topic, String key, int maxNum, long begin, long end, - boolean isUniqKey) throws MQClientException, + public QueryResult queryMessage(String clusterName, String topic, String key, int maxNum, long begin, long end, boolean isUniqKey) throws MQClientException, + InterruptedException { + return queryMessage(clusterName, topic, key, maxNum, begin, end, isUniqKey, null, null); + } + + public QueryResult queryMessage(String clusterName, String topic, String key, int maxNum, long begin, long end, boolean isUniqKey, String indexType, String lastKey) throws MQClientException, InterruptedException { boolean isLmq = MixAll.isLmq(topic); @@ -369,6 +371,8 @@ public QueryResult queryMessage(String clusterName, String topic, String key, in requestHeader.setMaxNum(maxNum); requestHeader.setBeginTimestamp(begin); requestHeader.setEndTimestamp(end); + requestHeader.setIndexType(indexType); + requestHeader.setLastKey(lastKey); this.mQClientFactory.getMQClientAPIImpl().queryMessage(addr, requestHeader, timeoutMillis * 3, new InvokeCallback() { @@ -445,7 +449,7 @@ public void operationFail(Throwable throwable) { } else { log.warn("queryMessage by uniqKey, find message key not matched, maybe hash duplicate {}", msgExt.toString()); } - } else { + } else if (!StringUtils.isEmpty(indexType) && MessageConst.INDEX_KEY_TYPE.equals(indexType)) { String keys = msgExt.getKeys(); String msgTopic = msgExt.getTopic(); if (keys != null) { @@ -465,6 +469,20 @@ public void operationFail(Throwable throwable) { log.warn("queryMessage, find message key not matched, maybe hash duplicate {}", msgExt.toString()); } } + } else if (!StringUtils.isEmpty(indexType) && MessageConst.INDEX_TAG_TYPE.equals(indexType)) { + String tags = msgExt.getTags(); + String msgTopic = msgExt.getTopic(); + boolean matched = false; + if (tags != null) { + if (Objects.equals(key, tags) && (isLmq || Objects.equals(topic, msgTopic))) { + matched = true; + } + } + if (matched) { + messageList.add(msgExt); + } else { + log.warn("queryMessage, find message key not matched, maybe hash duplicate {}", msgExt.toString()); + } } } } diff --git a/client/src/main/java/org/apache/rocketmq/client/impl/MQClientAPIImpl.java b/client/src/main/java/org/apache/rocketmq/client/impl/MQClientAPIImpl.java index 11af6ecd4b7..f8e662331f4 100644 --- a/client/src/main/java/org/apache/rocketmq/client/impl/MQClientAPIImpl.java +++ b/client/src/main/java/org/apache/rocketmq/client/impl/MQClientAPIImpl.java @@ -236,7 +236,7 @@ import org.apache.rocketmq.remoting.protocol.subscription.SubscriptionGroupConfig; import org.apache.rocketmq.remoting.rpchook.DynamicalExtFieldRPCHook; import org.apache.rocketmq.remoting.rpchook.StreamTypeRPCHook; - +import static org.apache.rocketmq.common.message.MessageConst.TIMER_ENGINE_TYPE; import java.io.UnsupportedEncodingException; import java.nio.ByteBuffer; import java.util.ArrayList; @@ -3649,4 +3649,17 @@ public void exportPopRecord(String brokerAddr, long timeout) throws RemotingConn } throw new MQBrokerException(response.getCode(), response.getRemark()); } + + public void switchTimerEngine(String brokerAddr, String engineType, long timeoutMillis) throws RemotingConnectException, + RemotingSendRequestException, RemotingTimeoutException, InterruptedException, MQBrokerException { + RemotingCommand request = RemotingCommand.createRequestCommand(RequestCode.SWITCH_TIMER_ENGINE, null); + request.addExtField(TIMER_ENGINE_TYPE, engineType); + RemotingCommand response = this.remotingClient.invokeSync(brokerAddr, request, timeoutMillis); + assert response != null; + if (response.getCode() == SUCCESS) { + return; + } + throw new MQBrokerException(response.getCode(), response.getRemark()); + } + } diff --git a/common/src/main/java/org/apache/rocketmq/common/MixAll.java b/common/src/main/java/org/apache/rocketmq/common/MixAll.java index 00006ac7a8e..e0255032ed3 100644 --- a/common/src/main/java/org/apache/rocketmq/common/MixAll.java +++ b/common/src/main/java/org/apache/rocketmq/common/MixAll.java @@ -45,6 +45,7 @@ import java.util.Properties; import java.util.Set; import java.util.TreeMap; +import java.util.concurrent.TimeUnit; import java.util.concurrent.atomic.AtomicLong; import java.util.function.Predicate; @@ -128,6 +129,7 @@ public class MixAll { public static final String MULTI_PATH_SPLITTER = System.getProperty("rocketmq.broker.multiPathSplitter", ","); private static final String OS = System.getProperty("os.name").toLowerCase(); + public static final long MILLS_FOR_HOUR = TimeUnit.HOURS.toMillis(1); private static final Set PREDEFINE_GROUP_SET = ImmutableSet.of( DEFAULT_CONSUMER_GROUP, @@ -584,4 +586,59 @@ public static String adjustConfigForPlatform(String config) { } return config; } + + public static long dealTimeToHourStamps(long timeStamp) { + if (timeStamp <= 0L) { + return timeStamp; + } + return (timeStamp / MILLS_FOR_HOUR) * MILLS_FOR_HOUR; + } + + public static boolean isHourTime(Long timeStamp) { + if (null == timeStamp) { + return false; + } + if (timeStamp <= 0L) { + return false; + } + return timeStamp % MILLS_FOR_HOUR == 0; + } + + public static List getHours(long startTimeMillis, long endTimeMillis) { + if (startTimeMillis > endTimeMillis || startTimeMillis <= 0L || endTimeMillis <= 0L) { + return null; + } + List result = new ArrayList<>(); + long startHour = dealTimeToHourStamps(startTimeMillis); + long endHour = dealTimeToHourStamps(endTimeMillis); + long current = startHour; + while (current <= endHour) { + result.add(current); + //protect system self 30 * 24 + if (result.size() >= 720) { + return result; + } + current += MILLS_FOR_HOUR; + } + return result; + } + + public static boolean isByteArrayEqual(byte[] array1, int offset1, int length1, byte[] array2, int offset2, int length2) { + if (null == array1 || null == array2) { + return false; + } + if (length1 != length2) { + return false; + } + if (offset1 < 0 || offset1 + length1 > array1.length || + offset2 < 0 || offset2 + length2 > array2.length) { + throw new ArrayIndexOutOfBoundsException("Invalid array index"); + } + for (int i = 0; i < length1; i++) { + if (array1[offset1 + i] != array2[offset2 + i]) { + return false; + } + } + return true; + } } diff --git a/common/src/main/java/org/apache/rocketmq/common/config/AbstractRocksDBStorage.java b/common/src/main/java/org/apache/rocketmq/common/config/AbstractRocksDBStorage.java index 6a5a0fc11b9..d3f41930b9b 100644 --- a/common/src/main/java/org/apache/rocketmq/common/config/AbstractRocksDBStorage.java +++ b/common/src/main/java/org/apache/rocketmq/common/config/AbstractRocksDBStorage.java @@ -85,6 +85,8 @@ public abstract class AbstractRocksDBStorage { protected CompactionOptions compactionOptions; protected CompactRangeOptions compactRangeOptions; + protected FlushOptions flushOptions; + protected ColumnFamilyHandle defaultCFHandle; protected final List cfOptions = new ArrayList<>(); protected final List cfHandles = new ArrayList<>(); @@ -116,6 +118,7 @@ protected void initOptions() { initTotalOrderReadOptions(); initCompactRangeOptions(); initCompactionOptions(); + initFlushOptions(); } /** @@ -168,6 +171,10 @@ protected void initCompactionOptions() { this.compactionOptions.setOutputFileSizeLimit(4 * 1024 * 1024 * 1024L); } + protected void initFlushOptions() { + this.flushOptions = new FlushOptions(); + } + public boolean hold() { if (!this.loaded || this.db == null || this.closed) { LOGGER.error("hold rocksdb Failed. {}", this.dbPath); @@ -532,6 +539,9 @@ public synchronized boolean shutdown() { if (this.totalOrderReadOptions != null) { this.totalOrderReadOptions.close(); } + if (this.flushOptions != null) { + this.flushOptions.close(); + } //4. close db. if (db != null && !this.readOnly) { try { @@ -560,6 +570,7 @@ public synchronized boolean shutdown() { this.db = null; this.readOptions = null; this.totalOrderReadOptions = null; + this.flushOptions = null; this.writeOptions = null; this.ableWalWriteOptions = null; this.options = null; diff --git a/common/src/main/java/org/apache/rocketmq/common/message/MessageConst.java b/common/src/main/java/org/apache/rocketmq/common/message/MessageConst.java index 2bdaabebae7..81f132d134c 100644 --- a/common/src/main/java/org/apache/rocketmq/common/message/MessageConst.java +++ b/common/src/main/java/org/apache/rocketmq/common/message/MessageConst.java @@ -40,6 +40,7 @@ public class MessageConst { public static final String PROPERTY_MSG_REGION = "MSG_REGION"; public static final String PROPERTY_TRACE_SWITCH = "TRACE_ON"; public static final String PROPERTY_UNIQ_CLIENT_MESSAGE_ID_KEYIDX = "UNIQ_KEY"; + public static final String PROPERTY_TRANS_OFFSET = "TRANS_OFFSET"; public static final String PROPERTY_EXTEND_UNIQ_INFO = "EXTEND_UNIQ_INFO"; public static final String PROPERTY_MAX_RECONSUME_TIMES = "MAX_RECONSUME_TIMES"; public static final String PROPERTY_CONSUME_START_TIMESTAMP = "CONSUME_START_TIME"; @@ -89,6 +90,14 @@ public class MessageConst { public static final String KEY_SEPARATOR = " "; + public final static String INDEX_KEY_TYPE = "K"; + public final static String INDEX_UNIQUE_TYPE = "U"; + public final static String INDEX_TAG_TYPE = "T"; + + public final static String TIMER_ENGINE_TYPE = "E_T"; + public final static String TIMER_ENGINE_ROCKSDB_TIMELINE = "R"; + public final static String TIMER_ENGINE_FILE_TIME_WHEEL = "F"; + public static final HashSet STRING_HASH_SET = new HashSet<>(64); public static final String PROPERTY_TIMER_ENQUEUE_MS = "TIMER_ENQUEUE_MS"; @@ -96,6 +105,7 @@ public class MessageConst { public static final String PROPERTY_TIMER_ROLL_TIMES = "TIMER_ROLL_TIMES"; public static final String PROPERTY_TIMER_OUT_MS = "TIMER_OUT_MS"; public static final String PROPERTY_TIMER_DEL_UNIQKEY = "TIMER_DEL_UNIQKEY"; + public static final String PROPERTY_TIMER_ROLL_LABEL = "TIMER_ROLL_LABEL"; public static final String PROPERTY_TIMER_DELAY_LEVEL = "TIMER_DELAY_LEVEL"; public static final String PROPERTY_TIMER_DELAY_MS = "TIMER_DELAY_MS"; public static final String PROPERTY_CRC32 = "__CRC32#"; diff --git a/common/src/main/java/org/apache/rocketmq/common/topic/TopicValidator.java b/common/src/main/java/org/apache/rocketmq/common/topic/TopicValidator.java index 47d45c6dfe7..ade6289c67f 100644 --- a/common/src/main/java/org/apache/rocketmq/common/topic/TopicValidator.java +++ b/common/src/main/java/org/apache/rocketmq/common/topic/TopicValidator.java @@ -27,8 +27,10 @@ public class TopicValidator { public static final String RMQ_SYS_SCHEDULE_TOPIC = "SCHEDULE_TOPIC_XXXX"; public static final String RMQ_SYS_BENCHMARK_TOPIC = "BenchmarkTest"; public static final String RMQ_SYS_TRANS_HALF_TOPIC = "RMQ_SYS_TRANS_HALF_TOPIC"; + public static final String RMQ_SYS_ROCKSDB_TRANS_HALF_TOPIC = "RMQ_SYS_ROCKSDB_TRANS_HALF_TOPIC"; public static final String RMQ_SYS_TRACE_TOPIC = "RMQ_SYS_TRACE_TOPIC"; public static final String RMQ_SYS_TRANS_OP_HALF_TOPIC = "RMQ_SYS_TRANS_OP_HALF_TOPIC"; + public static final String RMQ_SYS_ROCKSDB_TRANS_OP_HALF_TOPIC = "RMQ_SYS_ROCKSDB_TRANS_OP_HALF_TOPIC"; public static final String RMQ_SYS_TRANS_CHECK_MAX_TIME_TOPIC = "TRANS_CHECK_MAX_TIME_TOPIC"; public static final String RMQ_SYS_SELF_TEST_TOPIC = "SELF_TEST_TOPIC"; public static final String RMQ_SYS_OFFSET_MOVED_EVENT = "OFFSET_MOVED_EVENT"; @@ -64,6 +66,8 @@ public class TopicValidator { SYSTEM_TOPIC_SET.add(RMQ_SYS_SELF_TEST_TOPIC); SYSTEM_TOPIC_SET.add(RMQ_SYS_OFFSET_MOVED_EVENT); SYSTEM_TOPIC_SET.add(RMQ_SYS_ROCKSDB_OFFSET_TOPIC); + SYSTEM_TOPIC_SET.add(RMQ_SYS_ROCKSDB_TRANS_HALF_TOPIC); + SYSTEM_TOPIC_SET.add(RMQ_SYS_ROCKSDB_TRANS_OP_HALF_TOPIC); NOT_ALLOWED_SEND_TOPIC_SET.add(RMQ_SYS_SCHEDULE_TOPIC); NOT_ALLOWED_SEND_TOPIC_SET.add(RMQ_SYS_TRANS_HALF_TOPIC); @@ -71,6 +75,8 @@ public class TopicValidator { NOT_ALLOWED_SEND_TOPIC_SET.add(RMQ_SYS_TRANS_CHECK_MAX_TIME_TOPIC); NOT_ALLOWED_SEND_TOPIC_SET.add(RMQ_SYS_SELF_TEST_TOPIC); NOT_ALLOWED_SEND_TOPIC_SET.add(RMQ_SYS_OFFSET_MOVED_EVENT); + NOT_ALLOWED_SEND_TOPIC_SET.add(RMQ_SYS_ROCKSDB_TRANS_HALF_TOPIC); + NOT_ALLOWED_SEND_TOPIC_SET.add(RMQ_SYS_ROCKSDB_TRANS_OP_HALF_TOPIC); // regex: ^[%|a-zA-Z0-9_-]+$ // % diff --git a/controller/pom.xml b/controller/pom.xml index 4ceb9899cca..56de653e2ab 100644 --- a/controller/pom.xml +++ b/controller/pom.xml @@ -71,4 +71,4 @@ protobuf-java-util - \ No newline at end of file + diff --git a/distribution/pom.xml b/distribution/pom.xml index c099c0d2907..71ca98836a3 100644 --- a/distribution/pom.xml +++ b/distribution/pom.xml @@ -124,4 +124,4 @@ - \ No newline at end of file + diff --git a/filter/pom.xml b/filter/pom.xml index 25c5f52bbf2..2ec7cc807dd 100644 --- a/filter/pom.xml +++ b/filter/pom.xml @@ -46,4 +46,4 @@ guava - \ No newline at end of file + diff --git a/openmessaging/pom.xml b/openmessaging/pom.xml index 8762ac1c8e8..961698f42bb 100644 --- a/openmessaging/pom.xml +++ b/openmessaging/pom.xml @@ -42,4 +42,4 @@ ${project.version} - \ No newline at end of file + diff --git a/proxy/pom.xml b/proxy/pom.xml index 7b941484fd1..1150b909b65 100644 --- a/proxy/pom.xml +++ b/proxy/pom.xml @@ -115,4 +115,4 @@ jul-to-slf4j - \ No newline at end of file + diff --git a/remoting/src/main/java/org/apache/rocketmq/remoting/protocol/RequestCode.java b/remoting/src/main/java/org/apache/rocketmq/remoting/protocol/RequestCode.java index 8b2749eaae2..89c9acff94b 100644 --- a/remoting/src/main/java/org/apache/rocketmq/remoting/protocol/RequestCode.java +++ b/remoting/src/main/java/org/apache/rocketmq/remoting/protocol/RequestCode.java @@ -202,9 +202,9 @@ public class RequestCode { public static final int PUSH_REPLY_MESSAGE_TO_CLIENT = 326; public static final int ADD_WRITE_PERM_OF_BROKER = 327; - + public static final int GET_ALL_PRODUCER_INFO = 328; - + public static final int DELETE_EXPIRED_COMMITLOG = 329; public static final int GET_TOPIC_CONFIG = 351; @@ -297,4 +297,6 @@ public class RequestCode { public static final int AUTH_DELETE_ACL = 3008; public static final int AUTH_GET_ACL = 3009; public static final int AUTH_LIST_ACL = 3010; + + public static final int SWITCH_TIMER_ENGINE = 5001; } diff --git a/remoting/src/main/java/org/apache/rocketmq/remoting/protocol/header/QueryMessageRequestHeader.java b/remoting/src/main/java/org/apache/rocketmq/remoting/protocol/header/QueryMessageRequestHeader.java index 1d2a53a6ce1..1d4e4ab6fbf 100644 --- a/remoting/src/main/java/org/apache/rocketmq/remoting/protocol/header/QueryMessageRequestHeader.java +++ b/remoting/src/main/java/org/apache/rocketmq/remoting/protocol/header/QueryMessageRequestHeader.java @@ -42,6 +42,8 @@ public class QueryMessageRequestHeader extends TopicRequestHeader { private Long beginTimestamp; @CFNotNull private Long endTimestamp; + private String indexType; + private String lastKey; @Override public void checkFields() throws RemotingCommandException { @@ -87,4 +89,20 @@ public Long getEndTimestamp() { public void setEndTimestamp(Long endTimestamp) { this.endTimestamp = endTimestamp; } + + public String getIndexType() { + return indexType; + } + + public void setIndexType(String indexType) { + this.indexType = indexType; + } + + public String getLastKey() { + return lastKey; + } + + public void setLastKey(String lastKey) { + this.lastKey = lastKey; + } } 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 d6ea017218b..459f2074b24 100644 --- a/store/src/main/java/org/apache/rocketmq/store/CommitLog.java +++ b/store/src/main/java/org/apache/rocketmq/store/CommitLog.java @@ -68,7 +68,6 @@ import org.apache.rocketmq.store.logfile.MappedFile; import org.apache.rocketmq.store.queue.ConsumeQueueInterface; import org.apache.rocketmq.store.queue.CqUnit; -import org.apache.rocketmq.store.queue.ReferredIterator; import org.apache.rocketmq.store.util.LibC; import org.rocksdb.RocksDBException; @@ -341,12 +340,18 @@ public boolean getLastMappedFile(final long startOffset) { public void recoverNormally(long dispatchFromPhyOffset) throws RocksDBException { boolean checkCRCOnRecover = this.defaultMessageStore.getMessageStoreConfig().isCheckCRCOnRecover(); boolean checkDupInfo = this.defaultMessageStore.getMessageStoreConfig().isDuplicationEnable(); + int maxRecoverNum = this.defaultMessageStore.getMessageStoreConfig().getCommitLogRecoverMaxNum(); + if (maxRecoverNum <= 0) { + maxRecoverNum = 10; + } + log.info("recoverNormally maxRecoverNum: {}", maxRecoverNum); final List mappedFiles = this.mappedFileQueue.getMappedFiles(); if (!mappedFiles.isEmpty()) { int index = mappedFiles.size() - 1; while (index > 0) { MappedFile mappedFile = mappedFiles.get(index); - if (isMappedFileMatchedRecover(mappedFile, true)) { + maxRecoverNum--; + if (isMappedFileMatchedRecover(mappedFile, true) || maxRecoverNum <= 0) { // It's safe to recover from this mapped file break; } @@ -733,6 +738,11 @@ public void recoverAbnormally(long maxPhyOffsetOfConsumeQueue) throws RocksDBExc // recover by the minimum time stamp boolean checkCRCOnRecover = this.defaultMessageStore.getMessageStoreConfig().isCheckCRCOnRecover(); boolean checkDupInfo = this.defaultMessageStore.getMessageStoreConfig().isDuplicationEnable(); + int maxRecoverNum = this.defaultMessageStore.getMessageStoreConfig().getCommitLogRecoverMaxNum(); + if (maxRecoverNum <= 0) { + maxRecoverNum = 10; + } + log.info("recoverAbnormally maxRecoverNum: {}", maxRecoverNum); final List mappedFiles = this.mappedFileQueue.getMappedFiles(); if (!mappedFiles.isEmpty()) { // Looking beginning to recover from which file @@ -740,8 +750,9 @@ public void recoverAbnormally(long maxPhyOffsetOfConsumeQueue) throws RocksDBExc MappedFile mappedFile = null; for (; index >= 0; index--) { mappedFile = mappedFiles.get(index); - if (this.isMappedFileMatchedRecover(mappedFile, false)) { - log.info("recover from this mapped file {}", mappedFile.getFileName()); + maxRecoverNum--; + if (this.isMappedFileMatchedRecover(mappedFile, false) || maxRecoverNum <= 0) { + log.info("recover from this mapped file " + mappedFile.getFileName()); break; } } @@ -911,8 +922,18 @@ private boolean isMappedFileMatchedRecover(final MappedFile mappedFile, storeTimestamp, phyOffset, this.defaultMessageStore.getStoreCheckpoint().getIndexMsgTimestamp(), recoverNormally); } - return this.defaultMessageStore.getQueueStore() - .isMappedFileMatchedRecover(phyOffset, storeTimestamp, recoverNormally); + return isMappedFileMatchedRecover(phyOffset, storeTimestamp, recoverNormally); + } + + private boolean isMappedFileMatchedRecover(long phyOffset, long storeTimestamp, boolean recoverNormally) throws RocksDBException { + boolean result = this.defaultMessageStore.getQueueStore().isMappedFileMatchedRecover(phyOffset, storeTimestamp, recoverNormally); + if (null != this.defaultMessageStore.getTransRocksDBStore() && defaultMessageStore.getMessageStoreConfig().isTransRocksDBEnable() && !defaultMessageStore.getMessageStoreConfig().isTransWriteOriginTransHalfEnable()) { + result = result && this.defaultMessageStore.getTransRocksDBStore().isMappedFileMatchedRecover(phyOffset); + } + if (null != this.defaultMessageStore.getIndexRocksDBStore() && defaultMessageStore.getMessageStoreConfig().isIndexRocksDBEnable()) { + result = result && this.defaultMessageStore.getIndexRocksDBStore().isMappedFileMatchedRecover(phyOffset); + } + return result; } public boolean resetOffset(long offset) { @@ -2297,6 +2318,10 @@ public MessageStore getMessageStore() { return defaultMessageStore; } + public MappedFile getEarliestMappedFile() { + return mappedFileQueue.getEarliestMappedFile(); + } + @Override public void swapMap(int reserveNum, long forceSwapIntervalMs, long normalSwapIntervalMs) { this.getMappedFileQueue().swapMap(reserveNum, forceSwapIntervalMs, normalSwapIntervalMs); @@ -2477,14 +2502,17 @@ public boolean isMsgInColdArea(String group, String topic, int queueId, long off if (null == consumeQueue) { return false; } - ReferredIterator bufferConsumeQueue = consumeQueue.iterateFrom(offset, 1); - if (null == bufferConsumeQueue || !bufferConsumeQueue.hasNext()) { + CqUnit cqUnit = consumeQueue.get(offset); + if (null == cqUnit) { + return false; + } + long offsetPy = cqUnit.getPos(); + if (offsetPy < 0L) { return false; } - return defaultMessageStore.checkInColdAreaByCommitOffset(bufferConsumeQueue.next().getPos(), getMaxOffset()); + return defaultMessageStore.checkInColdAreaByCommitOffset(offsetPy, getMaxOffset()); } catch (Exception e) { - log.error("isMsgInColdArea group: {}, topic: {}, queueId: {}, offset: {}", - group, topic, queueId, offset, e); + log.error("isMsgInColdArea group: {}, topic: {}, queueId: {}, offset: {}", group, topic, queueId, offset, e); } return false; } 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 d440ccfb119..0a39ac0ff19 100644 --- a/store/src/main/java/org/apache/rocketmq/store/DefaultMessageStore.java +++ b/store/src/main/java/org/apache/rocketmq/store/DefaultMessageStore.java @@ -56,6 +56,8 @@ import java.util.concurrent.atomic.AtomicInteger; import java.util.concurrent.atomic.AtomicLong; import java.util.function.Supplier; +import com.alibaba.fastjson.JSON; +import org.apache.commons.collections.CollectionUtils; import org.apache.commons.lang3.StringUtils; import org.apache.rocketmq.common.BoundaryType; import org.apache.rocketmq.common.BrokerConfig; @@ -98,6 +100,7 @@ import org.apache.rocketmq.store.hook.SendMessageBackHook; import org.apache.rocketmq.store.index.IndexService; import org.apache.rocketmq.store.index.QueryOffsetResult; +import org.apache.rocketmq.store.index.rocksdb.IndexRocksDBStore; import org.apache.rocketmq.store.kv.CommitLogDispatcherCompaction; import org.apache.rocketmq.store.kv.CompactionService; import org.apache.rocketmq.store.kv.CompactionStore; @@ -110,8 +113,11 @@ 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.rocksdb.MessageRocksDBStorage; import org.apache.rocketmq.store.stats.BrokerStatsManager; import org.apache.rocketmq.store.timer.TimerMessageStore; +import org.apache.rocketmq.store.timer.rocksdb.TimerMessageRocksDBStore; +import org.apache.rocketmq.store.transaction.TransMessageRocksDBStore; import org.apache.rocketmq.store.util.PerfCounter; import org.apache.rocketmq.store.metrics.StoreMetricsManager; import org.rocksdb.RocksDBException; @@ -131,6 +137,7 @@ public class DefaultMessageStore implements MessageStore { protected final CleanCommitLogService cleanCommitLogService; protected final IndexService indexService; + protected final IndexRocksDBStore indexRocksDBStore; private final AllocateMappedFileService allocateMappedFileService; @@ -160,8 +167,11 @@ public class DefaultMessageStore implements MessageStore { private boolean notifyMessageArriveInBatch = false; protected StoreCheckpoint storeCheckpoint; + private MessageRocksDBStorage messageRocksDBStorage; private TimerMessageStore timerMessageStore; private final DefaultStoreMetricsManager defaultStoreMetricsManager; + private TimerMessageRocksDBStore timerMessageRocksDBStore; + private TransMessageRocksDBStore transMessageRocksDBStore; private final LinkedList dispatcherList = new LinkedList<>(); @@ -229,9 +239,12 @@ public DefaultMessageStore(final MessageStoreConfig messageStoreConfig, final Br this.consumeQueueStore = createConsumeQueueStore(); this.cleanCommitLogService = new CleanCommitLogService(); this.storeStatsService = new StoreStatsService(getBrokerIdentity()); + this.messageRocksDBStorage = new MessageRocksDBStorage(getMessageStoreConfig()); this.indexService = new IndexService(this); + this.indexRocksDBStore = new IndexRocksDBStore(this); this.dispatcherList.addLast(new CommitLogDispatcherBuildConsumeQueue()); this.dispatcherList.addLast(new CommitLogDispatcherBuildIndex()); + this.dispatcherList.addLast(new CommitLogDispatcherBuildTransIndex()); initializeHAService(); @@ -389,7 +402,6 @@ public void start() throws Exception { if (this.isTransientStorePoolEnable()) { this.transientStorePool.init(); } - this.allocateMappedFileService.start(); this.indexService.start(); @@ -523,6 +535,10 @@ public void shutdown() { this.indexService.shutdown(); } + if (this.indexRocksDBStore != null) { + this.indexRocksDBStore.shutdown(); + } + if (this.compactionService != null) { this.compactionService.shutdown(); } @@ -550,6 +566,10 @@ public void shutdown() { this.transientStorePool.destroy(); } + if (this.messageRocksDBStorage != null) { + this.messageRocksDBStorage.shutdown(); + } + if (lock != null) { try { lock.release(); @@ -572,6 +592,7 @@ public void destroy() { this.consumeQueueStore.destroy(false); this.commitLog.destroy(); this.indexService.destroy(); + this.indexRocksDBStore.destroy(); this.deleteFile(StorePathConfigHelper.getAbortFile(this.messageStoreConfig.getStorePathRootDir())); this.deleteFile(StorePathConfigHelper.getStoreCheckpoint(this.messageStoreConfig.getStorePathRootDir())); } @@ -1058,11 +1079,31 @@ public TimerMessageStore getTimerMessageStore() { return this.timerMessageStore; } + @Override + public TimerMessageRocksDBStore getTimerRocksDBStore() { + return this.timerMessageRocksDBStore; + } + + @Override + public TransMessageRocksDBStore getTransRocksDBStore() { + return this.transMessageRocksDBStore; + } + @Override public void setTimerMessageStore(TimerMessageStore timerMessageStore) { this.timerMessageStore = timerMessageStore; } + @Override + public void setTimerMessageRocksDBStore(TimerMessageRocksDBStore timerMessageRocksDBStore) { + this.timerMessageRocksDBStore = timerMessageRocksDBStore; + } + + @Override + public void setTransRocksDBStore(TransMessageRocksDBStore transMessageRocksDBStore) { + this.transMessageRocksDBStore = transMessageRocksDBStore; + } + @Override public long getCommitLogOffsetInQueue(String topic, int queueId, long consumeQueueOffset) { ConsumeQueueInterface consumeQueue = getConsumeQueue(topic, queueId); @@ -1309,8 +1350,15 @@ public QueryMessageResult queryMessage(String topic, String key, int maxNum, lon long lastQueryMsgTime = end; for (int i = 0; i < 3; i++) { - QueryOffsetResult queryOffsetResult = this.indexService.queryOffset(topic, key, maxNum, begin, lastQueryMsgTime); - if (queryOffsetResult.getPhyOffsets().isEmpty()) { + QueryOffsetResult queryOffsetResult = null; + if (messageStoreConfig.isIndexFileReadEnable()) { + queryOffsetResult = this.indexService.queryOffset(topic, key, maxNum, begin, lastQueryMsgTime, null); + LOGGER.debug("indexService query Message queryOffsetResult : {}", JSON.toJSONString(queryOffsetResult)); + } else if (messageStoreConfig.isIndexRocksDBEnable()) { + queryOffsetResult = this.indexRocksDBStore.queryOffset(topic, key, maxNum, begin, lastQueryMsgTime, null, null); + LOGGER.debug("indexRocksDBStore query Message queryOffsetResult : {}", JSON.toJSONString(queryOffsetResult)); + } + if (null == queryOffsetResult || CollectionUtils.isEmpty(queryOffsetResult.getPhyOffsets())) { break; } @@ -1348,7 +1396,55 @@ public QueryMessageResult queryMessage(String topic, String key, int maxNum, lon break; } } + return queryMessageResult; + } + @Override + public QueryMessageResult queryMessage(String topic, String key, int maxNum, long begin, long end, String indexType, String lastKey) { + QueryMessageResult queryMessageResult = new QueryMessageResult(); + long lastQueryMsgTime = end; + for (int i = 0; i < 3; i++) { + QueryOffsetResult queryOffsetResult = null; + if (messageStoreConfig.isIndexFileReadEnable()) { + queryOffsetResult = this.indexService.queryOffset(topic, key, maxNum, begin, lastQueryMsgTime, indexType); + LOGGER.debug("indexService query Message queryOffsetResult : {}", JSON.toJSONString(queryOffsetResult)); + } else if (messageStoreConfig.isIndexRocksDBEnable()) { + queryOffsetResult = this.indexRocksDBStore.queryOffset(topic, key, maxNum, begin, lastQueryMsgTime, indexType, lastKey); + LOGGER.debug("indexRocksDBStore query Message queryOffsetResult : {}", JSON.toJSONString(queryOffsetResult)); + } + if (null == queryOffsetResult || CollectionUtils.isEmpty(queryOffsetResult.getPhyOffsets())) { + break; + } + Collections.sort(queryOffsetResult.getPhyOffsets()); + queryMessageResult.setIndexLastUpdatePhyoffset(queryOffsetResult.getIndexLastUpdatePhyoffset()); + queryMessageResult.setIndexLastUpdateTimestamp(queryOffsetResult.getIndexLastUpdateTimestamp()); + for (int m = 0; m < queryOffsetResult.getPhyOffsets().size(); m++) { + long offset = queryOffsetResult.getPhyOffsets().get(m); + try { + MessageExt msg = this.lookMessageByOffset(offset); + if (0 == m && null != msg) { + lastQueryMsgTime = msg.getStoreTimestamp(); + } + SelectMappedBufferResult result = this.commitLog.getData(offset, false); + if (result != null) { + int size = result.getByteBuffer().getInt(0); + result.getByteBuffer().limit(size); + result.setSize(size); + queryMessageResult.addMessage(result); + } + } catch (Exception e) { + LOGGER.error("queryMessage exception", e); + } + } + + if (queryMessageResult.getBufferTotalSize() > 0) { + break; + } + + if (lastQueryMsgTime < begin) { + break; + } + } return queryMessageResult; } @@ -1357,6 +1453,12 @@ public QueryMessageResult queryMessage(String topic, String key, int maxNum, lon return CompletableFuture.completedFuture(queryMessage(topic, key, maxNum, begin, end)); } + @Override + public CompletableFuture queryMessageAsync(String topic, String key, + int maxNum, long begin, long end, String indexType, String lastKey) { + return CompletableFuture.completedFuture(queryMessage(topic, key, maxNum, begin, end, indexType, lastKey)); + } + @Override public void updateHaMasterAddress(String newAddr) { if (this.haService != null) { @@ -2101,7 +2203,34 @@ class CommitLogDispatcherBuildIndex implements CommitLogDispatcher { @Override public void dispatch(DispatchRequest request) { if (DefaultMessageStore.this.messageStoreConfig.isMessageIndexEnable()) { - DefaultMessageStore.this.indexService.buildIndex(request); + if (DefaultMessageStore.this.messageStoreConfig.isIndexFileWriteEnable()) { + DefaultMessageStore.this.indexService.buildIndex(request); + } + if (DefaultMessageStore.this.messageStoreConfig.isIndexRocksDBEnable()) { + DefaultMessageStore.this.indexRocksDBStore.buildIndex(request); + } + } + } + } + + class CommitLogDispatcherBuildTransIndex implements CommitLogDispatcher { + + @Override + public void dispatch(DispatchRequest request) { + if (DefaultMessageStore.this.messageStoreConfig.isTransRocksDBEnable()) { + if (null == request || StringUtils.isEmpty(request.getTopic())) { + return; + } + if (!request.getTopic().equals(TopicValidator.RMQ_SYS_ROCKSDB_TRANS_HALF_TOPIC) && !request.getTopic().equals(TopicValidator.RMQ_SYS_ROCKSDB_TRANS_OP_HALF_TOPIC)) { + return; + } + if (null == DefaultMessageStore.this.transMessageRocksDBStore) { + if (System.currentTimeMillis() % 1000 == 0) { + LOGGER.error("CommitLogDispatcherBuildTransIndex dispatch error, transMessageRocksDBStore is null"); + } + return; + } + DefaultMessageStore.this.transMessageRocksDBStore.buildTransIndex(request); } } } @@ -3059,6 +3188,11 @@ public MessageStoreStateMachine getStateMachine() { return stateMachine; } + @Override + public MessageRocksDBStorage getMessageRocksDBStorage() { + return this.messageRocksDBStorage; + } + public boolean isNotifyMessageArriveInBatch() { return notifyMessageArriveInBatch; } @@ -3076,4 +3210,7 @@ public StoreMetricsManager getStoreMetricsManager() { return defaultStoreMetricsManager; } + public IndexRocksDBStore getIndexRocksDBStore() { + return indexRocksDBStore; + } } diff --git a/store/src/main/java/org/apache/rocketmq/store/MappedFileQueue.java b/store/src/main/java/org/apache/rocketmq/store/MappedFileQueue.java index 2db6ff573af..94235024da9 100644 --- a/store/src/main/java/org/apache/rocketmq/store/MappedFileQueue.java +++ b/store/src/main/java/org/apache/rocketmq/store/MappedFileQueue.java @@ -944,4 +944,20 @@ public List range(final long from, final long to) { return result; } + + public MappedFile getEarliestMappedFile() { + MappedFile mappedFile = null; + while (!this.mappedFiles.isEmpty()) { + try { + mappedFile = this.mappedFiles.get(0); + break; + } catch (IndexOutOfBoundsException e) { + //continue; + } catch (Exception e) { + log.error("getEarliestMappedFile error: {}", e.getMessage()); + break; + } + } + return mappedFile; + } } 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 0b927513e13..b297ee542f3 100644 --- a/store/src/main/java/org/apache/rocketmq/store/MessageStore.java +++ b/store/src/main/java/org/apache/rocketmq/store/MessageStore.java @@ -43,8 +43,11 @@ import org.apache.rocketmq.store.logfile.MappedFile; import org.apache.rocketmq.store.queue.ConsumeQueueInterface; import org.apache.rocketmq.store.queue.ConsumeQueueStoreInterface; +import org.apache.rocketmq.store.rocksdb.MessageRocksDBStorage; import org.apache.rocketmq.store.stats.BrokerStatsManager; import org.apache.rocketmq.store.timer.TimerMessageStore; +import org.apache.rocketmq.store.timer.rocksdb.TimerMessageRocksDBStore; +import org.apache.rocketmq.store.transaction.TransMessageRocksDBStore; import org.apache.rocketmq.store.util.PerfCounter; import org.apache.rocketmq.store.metrics.StoreMetricsManager; import org.rocksdb.RocksDBException; @@ -207,8 +210,16 @@ CompletableFuture getMessageAsync(final String group, final St TimerMessageStore getTimerMessageStore(); + TimerMessageRocksDBStore getTimerRocksDBStore(); + + TransMessageRocksDBStore getTransRocksDBStore(); + void setTimerMessageStore(TimerMessageStore timerMessageStore); + void setTimerMessageRocksDBStore(TimerMessageRocksDBStore timerMessageRocksDBStore); + + void setTransRocksDBStore(TransMessageRocksDBStore transMessageRocksDBStore); + /** * Get the offset of the message in the commit log, which is also known as physical offset. * @@ -410,6 +421,8 @@ CompletableFuture getMessageStoreTimeStampAsync(final String topic, final QueryMessageResult queryMessage(final String topic, final String key, final int maxNum, final long begin, final long end); + QueryMessageResult queryMessage(final String topic, final String key, final int maxNum, final long begin, final long end, final String indexType, final String lastKey); + /** * Asynchronous query messages by given key. * @see #queryMessage(String, String, int, long, long) queryMessage @@ -423,6 +436,8 @@ QueryMessageResult queryMessage(final String topic, final String key, final int CompletableFuture queryMessageAsync(final String topic, final String key, final int maxNum, final long begin, final long end); + CompletableFuture queryMessageAsync(final String topic, final String key, final int maxNum, final long begin, final long end, final String indexType, final String lastKey); + /** * Update HA master address. * @@ -1000,4 +1015,6 @@ DispatchRequest checkMessageAndReturnSize(final ByteBuffer byteBuffer, final boo void notifyMessageArriveIfNecessary(DispatchRequest dispatchRequest); MessageStoreStateMachine getStateMachine(); + + MessageRocksDBStorage getMessageRocksDBStorage(); } diff --git a/store/src/main/java/org/apache/rocketmq/store/StoreUtil.java b/store/src/main/java/org/apache/rocketmq/store/StoreUtil.java index 526ca9bf1b0..9ef6f72ab44 100644 --- a/store/src/main/java/org/apache/rocketmq/store/StoreUtil.java +++ b/store/src/main/java/org/apache/rocketmq/store/StoreUtil.java @@ -18,6 +18,8 @@ import com.google.common.base.Preconditions; import org.apache.rocketmq.common.constant.LoggerName; +import org.apache.rocketmq.common.message.MessageDecoder; +import org.apache.rocketmq.common.message.MessageExt; import org.apache.rocketmq.logging.org.slf4j.Logger; import org.apache.rocketmq.logging.org.slf4j.LoggerFactory; import org.apache.rocketmq.store.logfile.MappedFile; @@ -76,4 +78,23 @@ public static FileQueueSnapshot getFileQueueSnapshot(MappedFileQueue mappedFileQ } return new FileQueueSnapshot(); } + + public static MessageExt getMessage(long offsetPy, int sizePy, MessageStore messageStore, ByteBuffer byteBuffer) { + try { + if (offsetPy < 0L || sizePy <= 0 || null == messageStore || null == byteBuffer) { + return null; + } + byteBuffer.position(0); + byteBuffer.limit(sizePy); + if (!messageStore.getData(offsetPy, sizePy, byteBuffer)) { + return null; + } + byteBuffer.flip(); + return MessageDecoder.decode(byteBuffer, true, false, false); + } catch (Exception e) { + log.error("getMessage error, offsetPy: {}, sizePy: {}, error: {}", offsetPy, sizePy, e.getMessage()); + } + return null; + } + } 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 74f02423fc0..9670f40d92e 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 @@ -106,6 +106,25 @@ public class MessageStoreConfig { private int timerProgressLogIntervalMs = 10 * 1000; private int timerWheelSnapshotIntervalMs = 10 * 1000; + private int commitLogRecoverMaxNum = 10; + private boolean timerRocksDBEnable = false; + private boolean timerRocksDBStopScan = false; + private long timerRocksDBPrecisionMs = 1000L; + private double timerRocksDBRollMaxTps = 8000.0; + private double timerRocksDBTimeExpiredMaxTps = 200000.0; + private int timerRocksDBRollIntervalHours = 1; + private int timerRocksDBRollRangeHours = 2; + private boolean timerRecallToTimeWheelEnable = true; + private boolean timerRecallToTimelineEnable = true; + + private boolean transRocksDBEnable = false; + private boolean transWriteOriginTransHalfEnable = true; + + private boolean indexRocksDBEnable = false; + private int maxRocksDBIndexQueryDays = 7; + private boolean indexFileWriteEnable = true; + private boolean indexFileReadEnable = true; + // default, defaultRocksDB @ImportantField private String storeType = StoreType.DEFAULT.getStoreType(); @@ -2082,6 +2101,134 @@ public void setEnableRunningFlagsInFlush(boolean enableRunningFlagsInFlush) { this.enableRunningFlagsInFlush = enableRunningFlagsInFlush; } + public boolean isTimerRocksDBEnable() { + return timerRocksDBEnable; + } + + public void setTimerRocksDBEnable(boolean timerRocksDBEnable) { + this.timerRocksDBEnable = timerRocksDBEnable; + } + + public double getTimerRocksDBRollMaxTps() { + return timerRocksDBRollMaxTps; + } + + public void setTimerRocksDBRollMaxTps(double timerRocksDBRollMaxTps) { + this.timerRocksDBRollMaxTps = timerRocksDBRollMaxTps; + } + + public double getTimerRocksDBTimeExpiredMaxTps() { + return timerRocksDBTimeExpiredMaxTps; + } + + public void setTimerRocksDBTimeExpiredMaxTps(double timerRocksDBTimeExpiredMaxTps) { + this.timerRocksDBTimeExpiredMaxTps = timerRocksDBTimeExpiredMaxTps; + } + + public boolean isTransRocksDBEnable() { + return transRocksDBEnable; + } + + public void setTransRocksDBEnable(boolean transRocksDBEnable) { + this.transRocksDBEnable = transRocksDBEnable; + } + + public boolean isIndexRocksDBEnable() { + return indexRocksDBEnable; + } + + public void setIndexRocksDBEnable(boolean indexRocksDBEnable) { + this.indexRocksDBEnable = indexRocksDBEnable; + } + + public int getMaxRocksDBIndexQueryDays() { + return maxRocksDBIndexQueryDays; + } + + public void setMaxRocksDBIndexQueryDays(int maxRocksDBIndexQueryDays) { + this.maxRocksDBIndexQueryDays = maxRocksDBIndexQueryDays; + } + + public boolean isTimerRocksDBStopScan() { + return timerRocksDBStopScan; + } + + public void setTimerRocksDBStopScan(boolean timerRocksDBStopScan) { + this.timerRocksDBStopScan = timerRocksDBStopScan; + } + + public long getTimerRocksDBPrecisionMs() { + return timerRocksDBPrecisionMs; + } + + public void setTimerRocksDBPrecisionMs(long timerRocksDBPrecisionMs) { + this.timerRocksDBPrecisionMs = timerRocksDBPrecisionMs; + } + + public boolean isIndexFileWriteEnable() { + return indexFileWriteEnable; + } + + public void setIndexFileWriteEnable(boolean indexFileWriteEnable) { + this.indexFileWriteEnable = indexFileWriteEnable; + } + + public boolean isIndexFileReadEnable() { + return indexFileReadEnable; + } + + public void setIndexFileReadEnable(boolean indexFileReadEnable) { + this.indexFileReadEnable = indexFileReadEnable; + } + + public boolean isTransWriteOriginTransHalfEnable() { + return transWriteOriginTransHalfEnable; + } + + public void setTransWriteOriginTransHalfEnable(boolean transWriteOriginTransHalfEnable) { + this.transWriteOriginTransHalfEnable = transWriteOriginTransHalfEnable; + } + + public boolean isTimerRecallToTimeWheelEnable() { + return timerRecallToTimeWheelEnable; + } + + public void setTimerRecallToTimeWheelEnable(boolean timerRecallToTimeWheelEnable) { + this.timerRecallToTimeWheelEnable = timerRecallToTimeWheelEnable; + } + + public boolean isTimerRecallToTimelineEnable() { + return timerRecallToTimelineEnable; + } + + public void setTimerRecallToTimelineEnable(boolean timerRecallToTimelineEnable) { + this.timerRecallToTimelineEnable = timerRecallToTimelineEnable; + } + + public int getTimerRocksDBRollIntervalHours() { + return timerRocksDBRollIntervalHours; + } + + public void setTimerRocksDBRollIntervalHours(int timerRocksDBRollIntervalHours) { + this.timerRocksDBRollIntervalHours = timerRocksDBRollIntervalHours; + } + + public int getTimerRocksDBRollRangeHours() { + return timerRocksDBRollRangeHours; + } + + public void setTimerRocksDBRollRangeHours(int timerRocksDBRollRangeHours) { + this.timerRocksDBRollRangeHours = timerRocksDBRollRangeHours; + } + + public int getCommitLogRecoverMaxNum() { + return commitLogRecoverMaxNum; + } + + public void setCommitLogRecoverMaxNum(int commitLogRecoverMaxNum) { + this.commitLogRecoverMaxNum = commitLogRecoverMaxNum; + } + public int getSharedByteBufferNum() { return sharedByteBufferNum; } diff --git a/store/src/main/java/org/apache/rocketmq/store/index/IndexService.java b/store/src/main/java/org/apache/rocketmq/store/index/IndexService.java index 4d358b4cedb..8c16cca2941 100644 --- a/store/src/main/java/org/apache/rocketmq/store/index/IndexService.java +++ b/store/src/main/java/org/apache/rocketmq/store/index/IndexService.java @@ -21,8 +21,10 @@ import java.util.ArrayList; import java.util.Arrays; import java.util.List; +import java.util.Map; import java.util.concurrent.locks.ReadWriteLock; import java.util.concurrent.locks.ReentrantReadWriteLock; +import org.apache.commons.lang3.StringUtils; import org.apache.rocketmq.common.UtilAll; import org.apache.rocketmq.common.constant.LoggerName; import org.apache.rocketmq.logging.org.slf4j.Logger; @@ -201,9 +203,56 @@ public QueryOffsetResult queryOffset(String topic, String key, int maxNum, long return new QueryOffsetResult(phyOffsets, indexLastUpdateTimestamp, indexLastUpdatePhyoffset); } + public QueryOffsetResult queryOffset(String topic, String key, int maxNum, long begin, long end, String indexType) { + List phyOffsets = new ArrayList<>(maxNum); + long indexLastUpdateTimestamp = 0; + long indexLastUpdatePhyoffset = 0; + maxNum = Math.min(maxNum, this.defaultMessageStore.getMessageStoreConfig().getMaxMsgsNumBatch()); + try { + this.readWriteLock.readLock().lock(); + if (!this.indexFileList.isEmpty()) { + for (int i = this.indexFileList.size(); i > 0; i--) { + IndexFile f = this.indexFileList.get(i - 1); + boolean lastFile = i == this.indexFileList.size(); + if (lastFile) { + indexLastUpdateTimestamp = f.getEndTimestamp(); + indexLastUpdatePhyoffset = f.getEndPhyOffset(); + } + + if (f.isTimeMatched(begin, end)) { + String queryKey; + if (!StringUtils.isEmpty(indexType) && MessageConst.INDEX_TAG_TYPE.equals(indexType)) { + queryKey = buildKey(topic, key, MessageConst.INDEX_TAG_TYPE); + } else { + queryKey = buildKey(topic, key); + } + f.selectPhyOffset(phyOffsets, queryKey, maxNum, begin, end); + } + + if (f.getBeginTimestamp() < begin) { + break; + } + + if (phyOffsets.size() >= maxNum) { + break; + } + } + } + } catch (Exception e) { + LOGGER.error("queryMsg queryOffset exception", e); + } finally { + this.readWriteLock.readLock().unlock(); + } + + return new QueryOffsetResult(phyOffsets, indexLastUpdateTimestamp, indexLastUpdatePhyoffset); + } + private String buildKey(final String topic, final String key) { return topic + "#" + key; } + private String buildKey(final String topic, final String key, final String indexType) { + return topic + "#" + indexType + "#" + key; + } public void buildIndex(DispatchRequest req) { IndexFile indexFile = retryGetAndCreateIndexFile(); @@ -247,6 +296,19 @@ public void buildIndex(DispatchRequest req) { } } } + + Map propertiesMap = req.getPropertiesMap(); + if (null != propertiesMap && propertiesMap.containsKey(MessageConst.PROPERTY_TAGS)) { + String tags = req.getPropertiesMap().get(MessageConst.PROPERTY_TAGS); + if (!StringUtils.isEmpty(tags)) { + indexFile = putKey(indexFile, msg, buildKey(topic, tags, MessageConst.INDEX_TAG_TYPE)); + if (indexFile == null) { + LOGGER.error("putKey error commitlog {} uniqkey {}", req.getCommitLogOffset(), req.getUniqKey()); + return; + } + } + } + } else { LOGGER.error("build index error, stop building index"); } diff --git a/store/src/main/java/org/apache/rocketmq/store/index/rocksdb/IndexRocksDBRecord.java b/store/src/main/java/org/apache/rocketmq/store/index/rocksdb/IndexRocksDBRecord.java new file mode 100644 index 00000000000..68f5c3f5fee --- /dev/null +++ b/store/src/main/java/org/apache/rocketmq/store/index/rocksdb/IndexRocksDBRecord.java @@ -0,0 +1,122 @@ +/* + * 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.index.rocksdb; +import java.nio.ByteBuffer; +import java.nio.charset.StandardCharsets; +import org.apache.commons.lang3.StringUtils; +import org.apache.rocketmq.common.MixAll; +import org.apache.rocketmq.common.message.MessageConst; + +public class IndexRocksDBRecord { + public static final String KEY_SPLIT = "@"; + public static final byte[] KEY_SPLIT_BYTES = KEY_SPLIT.getBytes(StandardCharsets.UTF_8); + private static final int VALUE_LENGTH = Long.BYTES; + private long storeTime; + private String topic; + private String key; + private String tag; + private String uniqKey; + private long offsetPy; + + public IndexRocksDBRecord(String topic, String key, String tag, long storeTime, String uniqKey, long offsetPy) { + this.topic = topic; + this.key = key; + this.tag = tag; + this.storeTime = storeTime; + this.uniqKey = uniqKey; + this.offsetPy = offsetPy; + } + + public byte[] getKeyBytes() { + if (StringUtils.isEmpty(topic) || StringUtils.isEmpty(uniqKey) || offsetPy < 0L || storeTime <= 0L) { + return null; + } + long storeTimeHour = MixAll.dealTimeToHourStamps(storeTime); + if (storeTimeHour <= 0L) { + return null; + } + String keyMiddleStr; + if (!StringUtils.isEmpty(key)) { + keyMiddleStr = KEY_SPLIT + topic + KEY_SPLIT + MessageConst.INDEX_KEY_TYPE + KEY_SPLIT + key + KEY_SPLIT + uniqKey + KEY_SPLIT; + } else if (!StringUtils.isEmpty(tag)) { + keyMiddleStr = KEY_SPLIT + topic + KEY_SPLIT + MessageConst.INDEX_TAG_TYPE + KEY_SPLIT + tag + KEY_SPLIT + uniqKey + KEY_SPLIT; + } else { + keyMiddleStr = KEY_SPLIT + topic + KEY_SPLIT + MessageConst.INDEX_UNIQUE_TYPE + KEY_SPLIT + uniqKey + KEY_SPLIT; + } + if (StringUtils.isEmpty(keyMiddleStr)) { + return null; + } + byte[] keyMiddleBytes = keyMiddleStr.getBytes(StandardCharsets.UTF_8); + int keyLength = Long.BYTES + keyMiddleBytes.length + Long.BYTES; + return ByteBuffer.allocate(keyLength).putLong(storeTimeHour).put(keyMiddleBytes).putLong(offsetPy).array(); + } + + public byte[] getValueBytes() { + if (storeTime <= 0L) { + return null; + } + return ByteBuffer.allocate(VALUE_LENGTH).putLong(storeTime).array(); + } + + public String getTopic() { + return topic; + } + + public void setTopic(String topic) { + this.topic = topic; + } + + public String getKey() { + return key; + } + + public void setKey(String key) { + this.key = key; + } + + public long getStoreTime() { + return storeTime; + } + + public void setStoreTime(long storeTime) { + this.storeTime = storeTime; + } + + public String getUniqKey() { + return uniqKey; + } + + public void setUniqKey(String uniqKey) { + this.uniqKey = uniqKey; + } + + public long getOffsetPy() { + return offsetPy; + } + + public void setOffsetPy(long offsetPy) { + this.offsetPy = offsetPy; + } + + public String getTag() { + return tag; + } + + public void setTag(String tag) { + this.tag = tag; + } +} diff --git a/store/src/main/java/org/apache/rocketmq/store/index/rocksdb/IndexRocksDBStore.java b/store/src/main/java/org/apache/rocketmq/store/index/rocksdb/IndexRocksDBStore.java new file mode 100644 index 00000000000..8ebf660bd1c --- /dev/null +++ b/store/src/main/java/org/apache/rocketmq/store/index/rocksdb/IndexRocksDBStore.java @@ -0,0 +1,325 @@ +/* + * 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.index.rocksdb; +import java.io.File; +import java.nio.file.Files; +import java.nio.file.Path; +import java.nio.file.Paths; +import java.nio.file.attribute.BasicFileAttributes; +import java.util.ArrayList; +import java.util.Arrays; +import java.util.HashSet; +import java.util.List; +import java.util.Map; +import java.util.Set; +import java.util.concurrent.BlockingQueue; +import java.util.concurrent.LinkedBlockingDeque; +import java.util.concurrent.TimeUnit; +import java.util.stream.Collectors; +import org.apache.commons.collections.CollectionUtils; +import org.apache.commons.lang3.StringUtils; +import org.apache.rocketmq.common.ServiceThread; +import org.apache.rocketmq.common.constant.LoggerName; +import org.apache.rocketmq.common.message.MessageConst; +import org.apache.rocketmq.common.sysflag.MessageSysFlag; +import org.apache.rocketmq.logging.org.slf4j.Logger; +import org.apache.rocketmq.logging.org.slf4j.LoggerFactory; +import org.apache.rocketmq.store.DefaultMessageStore; +import org.apache.rocketmq.store.DispatchRequest; +import org.apache.rocketmq.store.MessageStore; +import org.apache.rocketmq.store.config.MessageStoreConfig; +import org.apache.rocketmq.store.index.QueryOffsetResult; +import org.apache.rocketmq.store.logfile.MappedFile; +import org.apache.rocketmq.store.rocksdb.MessageRocksDBStorage; +import org.rocksdb.RocksDB; +import static org.apache.rocketmq.common.MixAll.dealTimeToHourStamps; + +public class IndexRocksDBStore { + private static final Logger log = LoggerFactory.getLogger(LoggerName.STORE_LOGGER_NAME); + private static final Logger logError = LoggerFactory.getLogger(LoggerName.STORE_ERROR_LOGGER_NAME); + private static final int DEFAULT_CAPACITY = 100000; + private static final int BATCH_SIZE = 1000; + private static final Set INDEX_TYPE_SET = new HashSet<>(); + static { + INDEX_TYPE_SET.add(MessageConst.INDEX_KEY_TYPE); + INDEX_TYPE_SET.add(MessageConst.INDEX_TAG_TYPE); + INDEX_TYPE_SET.add(MessageConst.INDEX_UNIQUE_TYPE); + } + private static final int INITIAL = 0, RUNNING = 1, SHUTDOWN = 2; + private volatile int state = INITIAL; + + private final MessageStore messageStore; + private final MessageStoreConfig storeConfig; + private final MessageRocksDBStorage messageRocksDBStorage; + private volatile long lastDeleteIndexTime = 0L; + private IndexBuildService indexBuildService; + private BlockingQueue originIndexMsgQueue; + + public IndexRocksDBStore(MessageStore messageStore) { + this.messageStore = messageStore; + this.storeConfig = messageStore.getMessageStoreConfig(); + this.messageRocksDBStorage = messageStore.getMessageRocksDBStorage(); + if (this.storeConfig.isIndexRocksDBEnable()) { + this.initAndStart(); + } + } + + private void initAndStart() { + if (this.state == RUNNING) { + return; + } + this.indexBuildService = new IndexBuildService(); + this.originIndexMsgQueue = new LinkedBlockingDeque<>(DEFAULT_CAPACITY); + this.indexBuildService.start(); + this.state = RUNNING; + Long lastOffsetPy = messageRocksDBStorage.getLastOffsetPy(RocksDB.DEFAULT_COLUMN_FAMILY); + log.info("IndexRocksDBStore start success, lastOffsetPy: {}", lastOffsetPy); + } + + public void shutdown() { + if (this.state != RUNNING || this.state == SHUTDOWN) { + return; + } + if (null != this.indexBuildService) { + this.indexBuildService.shutdown(); + } + this.state = SHUTDOWN; + log.info("IndexRocksDBStore shutdown success"); + } + + public QueryOffsetResult queryOffset(String topic, String key, int maxNum, long beginTime, long endTime, String indexType, String lastKey) { + if (StringUtils.isEmpty(topic) || StringUtils.isEmpty(key) || maxNum <= 0 || beginTime < 0L || endTime <= 0L || beginTime > endTime || !StringUtils.isEmpty(indexType) && !INDEX_TYPE_SET.contains(indexType)) { + logError.error("IndexRocksDBStore queryOffset param error, topic: {}, key: {}, maxNum: {}, beginTime: {}, endTime: {}, indexType: {}, lastKey: {}", topic, key, maxNum, beginTime, endTime, indexType, lastKey); + return null; + } + if (beginTime == 0L || Long.MAX_VALUE == endTime) { + endTime = System.currentTimeMillis(); + beginTime = endTime - TimeUnit.DAYS.toMillis(storeConfig.getMaxRocksDBIndexQueryDays()); + } + if ((endTime - beginTime) > (TimeUnit.DAYS.toMillis(storeConfig.getMaxRocksDBIndexQueryDays()))) { + logError.error("IndexRocksDBStore queryOffset index in rocksdb, can not query more than: {} days", storeConfig.getMaxRocksDBIndexQueryDays()); + return null; + } + long lastUpdateTime = 0L; + long lastOffsetPy = 0L; + maxNum = Math.min(maxNum, this.storeConfig.getMaxMsgsNumBatch()); + List phyOffsets = null; + try { + lastUpdateTime = messageRocksDBStorage.getLastStoreTimeStampForIndex(RocksDB.DEFAULT_COLUMN_FAMILY); + lastOffsetPy = messageRocksDBStorage.getLastOffsetPy(RocksDB.DEFAULT_COLUMN_FAMILY); + //compact old client + if (StringUtils.isEmpty(indexType)) { + phyOffsets = messageRocksDBStorage.queryOffsetForIndex(RocksDB.DEFAULT_COLUMN_FAMILY, topic, MessageConst.INDEX_KEY_TYPE, key, beginTime, endTime, maxNum, null); + if (CollectionUtils.isEmpty(phyOffsets)) { + phyOffsets = messageRocksDBStorage.queryOffsetForIndex(RocksDB.DEFAULT_COLUMN_FAMILY, topic, MessageConst.INDEX_UNIQUE_TYPE, key, beginTime, endTime, maxNum, null); + } + } else { + phyOffsets = messageRocksDBStorage.queryOffsetForIndex(RocksDB.DEFAULT_COLUMN_FAMILY, topic, indexType, key, beginTime, endTime, maxNum, lastKey); + } + } catch (Exception e) { + logError.error("IndexRocksDBStore queryOffset error, topic: {}, key: {}, maxNum: {}, beginTime: {}, endTime: {}, error: {}", topic, key, maxNum, beginTime, endTime, e.getMessage()); + } + return new QueryOffsetResult(phyOffsets, lastUpdateTime, lastOffsetPy); + } + + public void buildIndex(DispatchRequest dispatchRequest) { + if (null == dispatchRequest || dispatchRequest.getCommitLogOffset() < 0L || dispatchRequest.getMsgSize() <= 0 || state != RUNNING || null == this.originIndexMsgQueue) { + logError.error("IndexRocksDBStore buildIndex error, dispatchRequest: {}, state: {}, originIndexMsgQueue: {}", dispatchRequest, state, originIndexMsgQueue); + return; + } + try { + long reqOffsetPy = dispatchRequest.getCommitLogOffset(); + long endOffsetPy = messageRocksDBStorage.getLastOffsetPy(RocksDB.DEFAULT_COLUMN_FAMILY); + if (reqOffsetPy < endOffsetPy) { + if (System.currentTimeMillis() % 1000 == 0) { + log.warn("IndexRocksDBStore recover buildIndex, but ignore, build index offset reqOffsetPy: {}, endOffsetPy: {}", reqOffsetPy, endOffsetPy); + } + return; + } + final int tranType = MessageSysFlag.getTransactionValue(dispatchRequest.getSysFlag()); + switch (tranType) { + case MessageSysFlag.TRANSACTION_NOT_TYPE: + case MessageSysFlag.TRANSACTION_PREPARED_TYPE: + case MessageSysFlag.TRANSACTION_COMMIT_TYPE: + break; + case MessageSysFlag.TRANSACTION_ROLLBACK_TYPE: + return; + } + String topic = dispatchRequest.getTopic(); + String uniqKey = dispatchRequest.getUniqKey(); + long storeTime = dispatchRequest.getStoreTimestamp(); + if (StringUtils.isEmpty(topic) || StringUtils.isEmpty(uniqKey) || storeTime <= 0L || reqOffsetPy < 0L) { + return; + } + String keys = dispatchRequest.getKeys(); + if (!StringUtils.isEmpty(keys)) { + String[] keySplit = keys.split(MessageConst.KEY_SEPARATOR); + if (keySplit.length > 0) { + Set keySet = Arrays.stream(keySplit).filter(i -> !StringUtils.isEmpty(i)).collect(Collectors.toSet()); + for (String key : keySet) { + try { + while (!originIndexMsgQueue.offer(new IndexRocksDBRecord(topic, key, null, storeTime, uniqKey, reqOffsetPy), 3, TimeUnit.SECONDS)) { + if (System.currentTimeMillis() % 1000 == 0) { + logError.error("IndexRocksDBStore buildIndex keys error, topic: {}, key: {}, storeTime: {}, uniqKey: {}, reqOffsetPy: {}", topic, key, storeTime, uniqKey, reqOffsetPy); + } + } + } catch (Exception e) { + logError.error("IndexRocksDBStore buildIndex keys error, key: {}, uniqKey: {}, topic: {}, error: {}", key, uniqKey, topic, e.getMessage()); + } + } + } + } + Map propertiesMap = dispatchRequest.getPropertiesMap(); + if (null != propertiesMap && propertiesMap.containsKey(MessageConst.PROPERTY_TAGS)) { + String tags = propertiesMap.get(MessageConst.PROPERTY_TAGS); + if (!StringUtils.isEmpty(tags)) { + try { + while (!originIndexMsgQueue.offer(new IndexRocksDBRecord(topic, null, tags, storeTime, uniqKey, reqOffsetPy), 3, TimeUnit.SECONDS)) { + if (System.currentTimeMillis() % 1000 == 0) { + logError.error("IndexRocksDBStore buildIndex offer tags error, topic: {}, tags: {}, storeTime: {}, uniqKey: {}, reqOffsetPy: {}", topic, tags, storeTime, uniqKey, reqOffsetPy); + } + } + } catch (Exception e) { + logError.error("IndexRocksDBStore buildIndex tags error, tags: {}, uniqKey: {}, topic: {}, error: {}", tags, uniqKey, topic, e.getMessage()); + } + } + } + try { + while (!originIndexMsgQueue.offer(new IndexRocksDBRecord(topic, null, null, storeTime, uniqKey, reqOffsetPy), 3, TimeUnit.SECONDS)) { + if (System.currentTimeMillis() % 1000 == 0) { + logError.error("IndexRocksDBStore buildIndex uniqKey error, topic: {}, storeTime: {}, uniqKey: {}, reqOffsetPy: {}", topic, storeTime, uniqKey, reqOffsetPy); + } + } + } catch (Exception e) { + logError.error("IndexRocksDBStore buildIndex uniqKey error: {}", e.getMessage()); + } + } catch (Exception e) { + logError.error("IndexRocksDBStore buildIndex error: {}", e.getMessage()); + } + } + + public void deleteExpiredIndex() { + try { + MappedFile mappedFile = messageStore.getCommitLog().getEarliestMappedFile(); + if (null == mappedFile) { + logError.info("IndexRocksDBStore deleteExpiredIndex mappedFile is null"); + return; + } + File file = mappedFile.getFile(); + if (null == file || StringUtils.isEmpty(file.getAbsolutePath())) { + logError.info("IndexRocksDBStore deleteExpiredIndex error, file is null"); + return; + } + Path path = Paths.get(file.getAbsolutePath()); + BasicFileAttributes attrs = Files.readAttributes(path, BasicFileAttributes.class); + long deleteIndexFileTime = attrs.creationTime().toMillis() - TimeUnit.HOURS.toMillis(1); + long desDeleteTimeHour = dealTimeToHourStamps(deleteIndexFileTime); + if (desDeleteTimeHour != lastDeleteIndexTime) { + messageRocksDBStorage.deleteRecordsForIndex(RocksDB.DEFAULT_COLUMN_FAMILY, desDeleteTimeHour, 168); + lastDeleteIndexTime = desDeleteTimeHour; + } else { + log.info("IndexRocksDBStore ignore this delete, lastDeleteIndexTime: {}, desDeleteTimeHour: {}", lastDeleteIndexTime, desDeleteTimeHour); + } + } catch (Exception e) { + logError.error("IndexRocksDBStore deleteExpiredIndex rocksdb error: {}", e.getMessage()); + } + } + + public boolean isMappedFileMatchedRecover(long phyOffset) { + if (!storeConfig.isIndexRocksDBEnable()) { + return true; + } + Long lastOffsetPy = messageRocksDBStorage.getLastOffsetPy(RocksDB.DEFAULT_COLUMN_FAMILY); + log.info("index isMappedFileMatchedRecover lastOffsetPy: {}", lastOffsetPy); + if (null != lastOffsetPy && phyOffset <= lastOffsetPy) { + log.info("isMappedFileMatchedRecover IndexRocksDBStore recover form this offset, phyOffset: {}, lastOffsetPy: {}", phyOffset, lastOffsetPy); + return true; + } + return false; + } + + public void destroy() {} + + private String getServiceThreadName() { + String brokerIdentifier = ""; + if (this.messageStore instanceof DefaultMessageStore) { + DefaultMessageStore messageStore = (DefaultMessageStore) IndexRocksDBStore.this.messageStore; + if (messageStore.getBrokerConfig().isInBrokerContainer()) { + brokerIdentifier = messageStore.getBrokerConfig().getIdentifier(); + } + } + return brokerIdentifier; + } + + private class IndexBuildService extends ServiceThread { + private final Logger log = IndexRocksDBStore.log; + private List irs; + @Override + public String getServiceName() { + return getServiceThreadName() + this.getClass().getSimpleName(); + } + + @Override + public void run() { + log.info(this.getServiceName() + " service start"); + irs = new ArrayList<>(BATCH_SIZE); + while (!this.isStopped() || !originIndexMsgQueue.isEmpty()) { + try { + pollAndPutIndexRequest(); + } catch (Exception e) { + irs.clear(); + logError.error("IndexRocksDBStore error occurred in: {}, error: {}", getServiceName(), e.getMessage()); + } + } + log.info(this.getServiceName() + " service end"); + } + + private void pollAndPutIndexRequest() { + pollIndexRecord(); + if (CollectionUtils.isEmpty(irs)) { + return; + } + try { + messageRocksDBStorage.writeRecordsForIndex(RocksDB.DEFAULT_COLUMN_FAMILY, irs); + } catch (Exception e) { + logError.error("IndexRocksDBStore IndexBuildService pollAndPutIndexRequest error: {}", e.getMessage()); + } + irs.clear(); + } + + private void pollIndexRecord() { + try { + IndexRocksDBRecord firstReq = originIndexMsgQueue.poll(100, TimeUnit.MILLISECONDS); + if (null != firstReq) { + irs.add(firstReq); + while (true) { + IndexRocksDBRecord tmpReq = originIndexMsgQueue.poll(100, TimeUnit.MILLISECONDS); + if (null == tmpReq) { + break; + } + irs.add(tmpReq); + if (irs.size() >= BATCH_SIZE) { + break; + } + } + } + } catch (Exception e) { + logError.error("IndexRocksDBStore IndexBuildService error: {}", e.getMessage()); + } + } + } +} 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 8a5c29e6bc5..a91fc2e57bb 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 @@ -828,6 +828,9 @@ protected void deleteExpiredFiles() { } messageStore.getIndexService().deleteExpiredFile(minOffset); + if (messageStoreConfig.isIndexRocksDBEnable() && null != messageStore.getIndexRocksDBStore()) { + messageStore.getIndexRocksDBStore().deleteExpiredIndex(); + } } } 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 cf511b1bccc..84ac6833fce 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 @@ -621,6 +621,9 @@ protected void deleteExpiredFiles() { } messageStore.getIndexService().deleteExpiredFile(minOffset); + if (messageStoreConfig.isIndexRocksDBEnable() && null != messageStore.getIndexRocksDBStore()) { + messageStore.getIndexRocksDBStore().deleteExpiredIndex(); + } } } diff --git a/store/src/main/java/org/apache/rocketmq/store/rocksdb/MessageRocksDBStorage.java b/store/src/main/java/org/apache/rocketmq/store/rocksdb/MessageRocksDBStorage.java new file mode 100644 index 00000000000..8d32998bdce --- /dev/null +++ b/store/src/main/java/org/apache/rocketmq/store/rocksdb/MessageRocksDBStorage.java @@ -0,0 +1,652 @@ +/* + * 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.rocksdb; +import java.nio.ByteBuffer; +import java.nio.charset.StandardCharsets; +import java.nio.file.Paths; +import java.util.ArrayList; +import java.util.Arrays; +import java.util.HashSet; +import java.util.List; +import java.util.Set; +import java.util.concurrent.Executors; +import java.util.concurrent.ScheduledExecutorService; +import java.util.concurrent.TimeUnit; +import com.google.common.cache.Cache; +import com.google.common.cache.CacheBuilder; +import org.apache.commons.collections.CollectionUtils; +import org.apache.commons.lang3.StringUtils; +import org.apache.rocketmq.common.MixAll; +import org.apache.rocketmq.common.UtilAll; +import org.apache.rocketmq.common.config.AbstractRocksDBStorage; +import org.apache.rocketmq.common.constant.LoggerName; +import org.apache.rocketmq.logging.org.slf4j.Logger; +import org.apache.rocketmq.logging.org.slf4j.LoggerFactory; +import org.apache.rocketmq.store.config.MessageStoreConfig; +import org.apache.rocketmq.store.index.rocksdb.IndexRocksDBRecord; +import org.apache.rocketmq.store.timer.rocksdb.TimerRocksDBRecord; +import org.apache.rocketmq.store.transaction.TransRocksDBRecord; +import org.rocksdb.ColumnFamilyDescriptor; +import org.rocksdb.ColumnFamilyHandle; +import org.rocksdb.ColumnFamilyOptions; +import org.rocksdb.ReadOptions; +import org.rocksdb.RocksDB; +import org.rocksdb.RocksIterator; +import org.rocksdb.Slice; +import org.rocksdb.WriteBatch; +import static org.apache.rocketmq.common.MixAll.dealTimeToHourStamps; +import static org.apache.rocketmq.common.MixAll.getHours; +import static org.apache.rocketmq.common.MixAll.isHourTime; +import static org.apache.rocketmq.store.index.rocksdb.IndexRocksDBRecord.KEY_SPLIT; +import static org.apache.rocketmq.store.index.rocksdb.IndexRocksDBRecord.KEY_SPLIT_BYTES; +import static org.apache.rocketmq.store.timer.rocksdb.TimerRocksDBRecord.TIMER_ROCKSDB_DELETE; +import static org.apache.rocketmq.store.timer.rocksdb.TimerRocksDBRecord.TIMER_ROCKSDB_PUT; +import static org.apache.rocketmq.store.timer.rocksdb.TimerRocksDBRecord.TIMER_ROCKSDB_UPDATE; + +public class MessageRocksDBStorage extends AbstractRocksDBStorage { + private static final Logger log = LoggerFactory.getLogger(LoggerName.STORE_LOGGER_NAME); + private static final Logger logError = LoggerFactory.getLogger(LoggerName.STORE_ERROR_LOGGER_NAME); + private static final String ROCKSDB_MESSAGE_DIRECTORY = "rocksdbstore"; + + public static final byte[] TIMER_COLUMN_FAMILY = "timer".getBytes(StandardCharsets.UTF_8); + public static final byte[] TRANS_COLUMN_FAMILY = "trans".getBytes(StandardCharsets.UTF_8); + private static final byte[] LAST_OFFSET_PY = "lastOffsetPy".getBytes(StandardCharsets.UTF_8); + private static final byte[] LAST_STORE_TIMESTAMP = "lastStoreTimeStamp".getBytes(StandardCharsets.UTF_8); + private static final byte[] END_SUFFIX_BYTES = new byte[512]; + static { + Arrays.fill(END_SUFFIX_BYTES, (byte) 0xFF); + } + private static final Set COMMON_CHECK_POINT_KEY_SET_FOR_TIMER = new HashSet<>(); + public static final byte[] SYS_TOPIC_SCAN_OFFSET_CHECK_POINT = "sys_topic_scan_offset_checkpoint".getBytes(StandardCharsets.UTF_8); + public static final byte[] TIMELINE_CHECK_POINT = "timeline_checkpoint".getBytes(StandardCharsets.UTF_8); + static { + COMMON_CHECK_POINT_KEY_SET_FOR_TIMER.add(SYS_TOPIC_SCAN_OFFSET_CHECK_POINT); + COMMON_CHECK_POINT_KEY_SET_FOR_TIMER.add(TIMELINE_CHECK_POINT); + } + private static final byte[] DELETE_VAL_FLAG = new byte[] {(byte)0xFF}; + private static final int LAST_OFFSET_PY_LENGTH = LAST_OFFSET_PY.length; + + private volatile ColumnFamilyHandle timerCFHandle; + private volatile ColumnFamilyHandle transCFHandle; + + private final ScheduledExecutorService scheduler = Executors.newScheduledThreadPool(1); + private static final Cache DELETE_KEY_CACHE_FOR_TIMER = CacheBuilder.newBuilder() + .maximumSize(10000) + .expireAfterWrite(60, TimeUnit.MINUTES) + .build(); + + public MessageRocksDBStorage(MessageStoreConfig messageStoreConfig) { + super(Paths.get(messageStoreConfig.getStorePathRootDir(), ROCKSDB_MESSAGE_DIRECTORY).toString()); + this.start(); + } + + @Override + protected boolean postLoad() { + try { + UtilAll.ensureDirOK(this.dbPath); + initOptions(); + ColumnFamilyOptions indexCFOptions = RocksDBOptionsFactory.createIndexCFOptions(); + ColumnFamilyOptions timerCFOptions = RocksDBOptionsFactory.createTimerCFOptions(); + ColumnFamilyOptions transCFOptions = RocksDBOptionsFactory.createTransCFOptions(); + this.cfOptions.add(indexCFOptions); + this.cfOptions.add(timerCFOptions); + this.cfOptions.add(transCFOptions); + + List cfDescriptors = new ArrayList<>(); + cfDescriptors.add(new ColumnFamilyDescriptor(RocksDB.DEFAULT_COLUMN_FAMILY, indexCFOptions)); + cfDescriptors.add(new ColumnFamilyDescriptor(TIMER_COLUMN_FAMILY, timerCFOptions)); + cfDescriptors.add(new ColumnFamilyDescriptor(TRANS_COLUMN_FAMILY, transCFOptions)); + this.open(cfDescriptors); + this.defaultCFHandle = cfHandles.get(0); + this.timerCFHandle = cfHandles.get(1); + this.transCFHandle = cfHandles.get(2); + scheduler.scheduleAtFixedRate(() -> { + try { + db.flush(flushOptions, timerCFHandle); + log.info("MessageRocksDBStorage flush timer wal success"); + } catch (Exception e) { + logError.error("MessageRocksDBStorage flush timer wal failed, error: {}", e.getMessage()); + } + }, 5, 5, TimeUnit.MINUTES); + + log.info("MessageRocksDBStorage init success, dbPath: {}", this.dbPath); + } catch (final Exception e) { + logError.error("MessageRocksDBStorage init error, dbPath: {}, error: {}", this.dbPath, e.getMessage()); + return false; + } + return true; + } + + protected void initOptions() { + this.options = RocksDBOptionsFactory.createDBOptions(); + super.initOptions(); + } + + public String getFilePath() { + return this.dbPath; + } + + @Override + protected void preShutdown() { + log.info("MessageRocksDBStorage pre shutdown success, dbPath: {}", this.dbPath); + } + + public List queryOffsetForIndex(byte[] columnFamily, String topic, String indexType, String key, long beginTime, long endTime, int maxNum, String lastKey) { + ColumnFamilyHandle cfHandle = getColumnFamily(columnFamily); + if (null == cfHandle || StringUtils.isEmpty(topic) || StringUtils.isEmpty(indexType) || StringUtils.isEmpty(key) || beginTime < 0L || endTime <= 0L || beginTime > endTime || maxNum <= 0) { + logError.error("MessageRocksDBStorage queryOffsetForIndex param error, cfHandle: {}, topic: {}, indexType: {}, key: {}, beginTime: {}, endTime: {}, maxNum: {}", cfHandle, topic, indexType, key, beginTime, endTime, maxNum); + return null; + } + Long lastIndexTime = getLastIndexTimeForIndex(lastKey); + if (!StringUtils.isEmpty(lastKey) && (null == lastIndexTime || lastIndexTime <= 0L || !isHourTime(lastIndexTime))) { + logError.error("MessageRocksDBStorage queryOffsetForIndex parse and check lastIndexTime error, lastIndexTime: {}, lastKey: {}", lastIndexTime, lastKey); + return null; + } + List hours = getHours(beginTime, endTime); + if (CollectionUtils.isEmpty(hours)) { + logError.error("MessageRocksDBStorage queryOffsetForIndex param error, hours is empty, beginTime: {}, endTime: {}", beginTime, endTime); + return null; + } + List offsetPyList = new ArrayList<>(maxNum); + String keyMiddleStr = KEY_SPLIT + topic + KEY_SPLIT + indexType + KEY_SPLIT + key + KEY_SPLIT; + byte[] keyMiddleBytes = keyMiddleStr.getBytes(StandardCharsets.UTF_8); + for (Long hour : hours) { + if (null == hour || null != lastIndexTime && hour < lastIndexTime) { + continue; + } + byte[] seekKeyBytes = null; + byte[] lastKeyBytes = null; + byte[] keyPrefixBytes = ByteBuffer.allocate(Long.BYTES + keyMiddleBytes.length).putLong(hour).put(keyMiddleBytes).array(); + if (!StringUtils.isEmpty(lastKey) && hour.equals(lastIndexTime)) { + seekKeyBytes = lastKeyToBytes(lastKey); + lastKeyBytes = seekKeyBytes; + } else { + seekKeyBytes = keyPrefixBytes; + } + if (null == seekKeyBytes) { + logError.error("MessageRocksDBStorage queryOffsetForIndex error, seekKeyBytes is null"); + return null; + } + try (RocksIterator iterator = db.newIterator(cfHandle, readOptions)) { + for (iterator.seek(seekKeyBytes); iterator.isValid(); iterator.next()) { + try { + byte[] currentKeyBytes = iterator.key(); + if (null == currentKeyBytes || currentKeyBytes.length == 0) { + break; + } + if (null != lastKeyBytes && currentKeyBytes.length == lastKeyBytes.length && MixAll.isByteArrayEqual(currentKeyBytes, 0, currentKeyBytes.length, lastKeyBytes, 0, lastKeyBytes.length)) { + continue; + } + if (currentKeyBytes.length < keyPrefixBytes.length || !MixAll.isByteArrayEqual(currentKeyBytes, 0, keyPrefixBytes.length, keyPrefixBytes, 0, keyPrefixBytes.length)) { + break; + } + ByteBuffer valueBuffer = ByteBuffer.wrap(iterator.value()); + long storeTime = valueBuffer.getLong(); + if (storeTime >= beginTime && storeTime <= endTime) { + byte[] indexKey = iterator.key(); + if (null == indexKey || indexKey.length < Long.BYTES) { + continue; + } + byte[] bytes = Arrays.copyOfRange(indexKey, indexKey.length - Long.BYTES, indexKey.length); + long offset = ByteBuffer.wrap(bytes).getLong(); + offsetPyList.add(offset); + if (offsetPyList.size() >= maxNum) { + return offsetPyList; + } + } + } catch (Exception e) { + logError.error("MessageRocksDBStorage queryOffsetForIndex iterator error: {}", e.getMessage()); + } + } + } catch (Exception e) { + logError.error("MessageRocksDBStorage queryOffsetForIndex error: {}", e.getMessage()); + } + } + return offsetPyList; + } + + private byte[] lastKeyToBytes(String lastKey) { + if (StringUtils.isEmpty(lastKey)) { + return null; + } + String[] split = lastKey.split(KEY_SPLIT); + if (split.length != 6) { + log.error("MessageRocksDBStorage lastKeyToBytes split error, lastKey: {}", lastKey); + return null; + } + try { + long storeTimeHour = Long.parseLong(split[0]); + long offsetPy = Long.parseLong(split[split.length - 1]); + StringBuilder stringBuilder = new StringBuilder(); + for (int i = 1; i < split.length - 1; i++) { + stringBuilder.append(KEY_SPLIT).append(split[i]); + } + byte[] middleKeyBytes = stringBuilder.append(KEY_SPLIT).toString().getBytes(StandardCharsets.UTF_8); + return ByteBuffer.allocate(Long.BYTES + middleKeyBytes.length + Long.BYTES).putLong(storeTimeHour).put(middleKeyBytes).putLong(offsetPy).array(); + } catch (Exception e) { + log.error("MessageRocksDBStorage lastKeyToBytes error, lastKey: {}, error: {}", lastKey, e.getMessage()); + return null; + } + } + + public void deleteRecordsForIndex(byte[] columnFamily, long storeTime, int hours) { + ColumnFamilyHandle cfHandle = getColumnFamily(columnFamily); + if (null == cfHandle || storeTime < 0L || hours <= 0) { + logError.error("MessageRocksDBStorage deleteRecordsForIndex param error, storeTime: {}, hours: {}", storeTime, hours); + return; + } + long endTime = dealTimeToHourStamps(storeTime); + long startTime = endTime - TimeUnit.HOURS.toMillis(hours); + try { + byte[] startKey = ByteBuffer.allocate(Long.BYTES + KEY_SPLIT_BYTES.length).putLong(startTime).put(KEY_SPLIT_BYTES).array(); + byte[] endKey = ByteBuffer.allocate(Long.BYTES + KEY_SPLIT_BYTES.length + END_SUFFIX_BYTES.length).putLong(endTime).put(KEY_SPLIT_BYTES).put(END_SUFFIX_BYTES).array(); + rangeDelete(cfHandle, ableWalWriteOptions, startKey, endKey); + log.info("MessageRocksDBStorage deleteRecordsForIndex delete success, storeTime: {}, hours: {}", storeTime, hours); + } catch (Exception e) { + logError.error("MessageRocksDBStorage deleteRecordsForIndex delete error, storeTime: {}, hours: {}, error: {}", storeTime, hours, e.getMessage()); + } + } + + public void writeRecordsForIndex(byte[] columnFamily, List recordList) { + ColumnFamilyHandle cfHandle = getColumnFamily(columnFamily); + if (null == cfHandle || CollectionUtils.isEmpty(recordList)) { + return; + } + try (WriteBatch writeBatch = new WriteBatch()) { + for (IndexRocksDBRecord record : recordList) { + try { + if (null == record) { + logError.warn("MessageRocksDBStorage writeRecordsForIndex error, record is null"); + continue; + } + byte[] keyBytes = record.getKeyBytes(); + byte[] valueBytes = record.getValueBytes(); + if (null == keyBytes || keyBytes.length == 0 || null == valueBytes || valueBytes.length == 0) { + logError.error("MessageRocksDBStorage writeRecordsForIndex param error, keyBytes: {}, valueBytes: {}", keyBytes, valueBytes); + continue; + } + writeBatch.put(cfHandle, keyBytes, valueBytes); + } catch (Exception e) { + logError.error("MessageRocksDBStorage writeRecordsForIndex error: {}", e.getMessage()); + } + } + IndexRocksDBRecord lastRecord = recordList.get(recordList.size() - 1); + if (null != lastRecord && StringUtils.isEmpty(lastRecord.getKey()) && StringUtils.isEmpty(lastRecord.getTag())) { + long offset = lastRecord.getOffsetPy(); + Long lastOffsetPy = getLastOffsetPy(columnFamily); + if (null == lastOffsetPy || offset > lastOffsetPy) { + writeBatch.put(cfHandle, LAST_OFFSET_PY, ByteBuffer.allocate(Long.BYTES).putLong(offset).array()); + } + long storeTime = lastRecord.getStoreTime(); + Long lastStoreTimeStamp = getLastStoreTimeStampForIndex(columnFamily); + if (null == lastStoreTimeStamp || storeTime > lastStoreTimeStamp) { + writeBatch.put(cfHandle, LAST_STORE_TIMESTAMP, ByteBuffer.allocate(Long.BYTES).putLong(storeTime).array()); + } + } + batchPut(ableWalWriteOptions, writeBatch); + } catch (Exception e) { + logError.error("MessageRocksDBStorage writeRecordsForIndex error: {}", e.getMessage()); + } + } + + public Long getLastStoreTimeStampForIndex(byte[] columnFamily) { + ColumnFamilyHandle cfHandle = getColumnFamily(columnFamily); + if (null == cfHandle) { + return null; + } + try { + byte[] storeTime = get(cfHandle, readOptions, LAST_STORE_TIMESTAMP); + return null == storeTime ? 0L : ByteBuffer.wrap(storeTime).getLong(); + } catch (Exception e) { + logError.error("MessageRocksDBStorage getLastStoreTimeStampForIndex error: {}", e.getMessage()); + return null; + } + } + + private static Long getLastIndexTimeForIndex(String lastKey) { + if (StringUtils.isEmpty(lastKey)) { + return null; + } + try { + String[] split = lastKey.split(KEY_SPLIT); + if (split.length > 0) { + return Long.valueOf(split[0]); + } + } catch (Exception e) { + logError.error("MessageRocksDBStorage getLastIndexTimeForIndex error lastKey: {}, e: {}", lastKey, e.getMessage()); + } + return null; + } + + public void writeRecordsForTimer(byte[] columnFamily, List recordList) { + ColumnFamilyHandle cfHandle = getColumnFamily(columnFamily); + if (null == cfHandle || CollectionUtils.isEmpty(recordList)) { + return; + } + try (WriteBatch writeBatch = new WriteBatch()) { + for (TimerRocksDBRecord record : recordList) { + if (null == record) { + logError.error("MessageRocksDBStorage writeRecordsForTimer error, record is null"); + continue; + } + try { + byte[] keyBytes = record.getKeyBytes(); + byte[] valueBytes = record.getValueBytes(); + if (null == keyBytes || keyBytes.length == 0 || null == valueBytes || valueBytes.length == 0) { + logError.error("MessageRocksDBStorage writeRecordsForTimer param error, keyBytes: {}, valueBytes: {}", keyBytes, valueBytes); + continue; + } + if (record.getActionFlag() == TIMER_ROCKSDB_PUT) { + writeBatch.put(cfHandle, keyBytes, valueBytes); + } else if (record.getActionFlag() == TIMER_ROCKSDB_DELETE) { + writeBatch.delete(cfHandle, keyBytes); + DELETE_KEY_CACHE_FOR_TIMER.put(keyBytes, DELETE_VAL_FLAG); + } else if (record.getActionFlag() == TIMER_ROCKSDB_UPDATE) { + byte[] deleteByte = DELETE_KEY_CACHE_FOR_TIMER.getIfPresent(keyBytes); + if (null == deleteByte) { + writeBatch.put(cfHandle, keyBytes, valueBytes); + } + } else { + logError.error("MessageRocksDBStorage writeRecordsForTimer record actionFlag error, actionFlag: {}", record.getActionFlag()); + } + } catch (Exception e) { + logError.error("MessageRocksDBStorage writeRecordsForTimer error: {}", e.getMessage()); + } + } + batchPut(ableWalWriteOptions, writeBatch); + } catch (Exception e) { + logError.error("MessageRocksDBStorage writeRecordsForTimer error: {}", e.getMessage()); + } + } + + public List scanRecordsForTimer(byte[] columnFamily, long lowerTime, long upperTime, int size, byte[] startKey) { + ColumnFamilyHandle cfHandle = getColumnFamily(columnFamily); + if (null == cfHandle || lowerTime <= 0L || upperTime <= 0L || lowerTime > upperTime || size <= 0) { + return null; + } + RocksIterator iterator = null; + try (ReadOptions readOptions = new ReadOptions() + .setIterateLowerBound(new Slice(ByteBuffer.allocate(Long.BYTES).putLong(lowerTime).array())) + .setIterateUpperBound(new Slice(ByteBuffer.allocate(Long.BYTES).putLong(upperTime).array()))) { + iterator = db.newIterator(cfHandle, readOptions); + if (null == startKey || startKey.length == 0) { + iterator.seek(ByteBuffer.allocate(Long.BYTES).putLong(lowerTime).array()); + } else { + iterator.seek(startKey); + iterator.next(); + } + List records = new ArrayList<>(); + for (; iterator.isValid(); iterator.next()) { + try { + TimerRocksDBRecord timerRocksDBRecord = TimerRocksDBRecord.decode(iterator.key(), iterator.value()); + if (null == timerRocksDBRecord) { + logError.error("MessageRocksDBStorage scanRecordsForTimer error, decode timerRocksDBRecord is null"); + continue; + } + records.add(timerRocksDBRecord); + if (records.size() >= size) { + break; + } + } catch (Exception e) { + logError.error("MessageRocksDBStorage scanRecordsForTimer iterator error: {}", e.getMessage()); + } + } + return records; + } catch (Exception e) { + logError.error("MessageRocksDBStorage scanRecordsForTimer error: {}", e.getMessage()); + } finally { + if (null != iterator) { + iterator.close(); + } + } + return null; + } + + public void deleteRecordsForTimer(byte[] columnFamily, long lowerTime, long upperTime) { + ColumnFamilyHandle cfHandle = getColumnFamily(columnFamily); + if (null == cfHandle || lowerTime <= 0L || upperTime <= 0L || lowerTime > upperTime) { + logError.error("MessageRocksDBStorage deleteRecordsForTimer param error, cfHandle: {}, lowerTime: {}, upperTime: {}", cfHandle, lowerTime, upperTime); + return; + } + byte[] startKey = ByteBuffer.allocate(Long.BYTES).putLong(lowerTime).array(); + byte[] endKey = ByteBuffer.allocate(Long.BYTES + END_SUFFIX_BYTES.length).putLong(upperTime).put(END_SUFFIX_BYTES).array(); + try { + rangeDelete(cfHandle, ableWalWriteOptions, startKey, endKey); + log.info("MessageRocksDBStorage deleteRecordsForTimer success, lowerTime: {}, upperTime: {}", lowerTime, upperTime); + } catch (Exception e) { + logError.error("MessageRocksDBStorage deleteRecordsForTimer param error, lowerTime: {}, upperTime: {}, error: {}", lowerTime, upperTime, e.getMessage()); + } + } + + public void writeCheckPointForTimer(byte[] columnFamily, byte[] key, long value) { + ColumnFamilyHandle cfHandle = getColumnFamily(columnFamily); + if (null == cfHandle || !COMMON_CHECK_POINT_KEY_SET_FOR_TIMER.contains(key) || value < 0L) { + logError.error("MessageRocksDBStorage writeCheckPointForTimer param error, cfHandle: {}, key: {}, value: {}", cfHandle, key, value); + return; + } + try { + byte[] valueBytes = ByteBuffer.allocate(Long.BYTES).putLong(value).array(); + put(cfHandle, ableWalWriteOptions, key, key.length, valueBytes, valueBytes.length); + } catch (Exception e) { + logError.error("MessageRocksDBStorage writeCheckPointForTimer error: {}", e.getMessage()); + } + } + + public long getCheckpointForTimer(byte[] columnFamily, byte[] key) { + ColumnFamilyHandle cfHandle = getColumnFamily(columnFamily); + if (null == cfHandle || !COMMON_CHECK_POINT_KEY_SET_FOR_TIMER.contains(key)) { + logError.error("MessageRocksDBStorage getCheckpointForTimer error, cfHandle: {}, key: {}", cfHandle, key); + return 0L; + } + try { + byte[] checkpoint = get(cfHandle, readOptions, key); + if (null == checkpoint && Arrays.equals(key, TIMELINE_CHECK_POINT)) { + return (System.currentTimeMillis() - TimeUnit.SECONDS.toMillis(10)) / TimeUnit.SECONDS.toMillis(1) * TimeUnit.SECONDS.toMillis(1); + } + return checkpoint == null ? 0L : ByteBuffer.wrap(checkpoint).getLong(); + } catch (Exception e) { + logError.error("MessageRocksDBStorage getCheckpointForTimer error: {}", e.getMessage()); + return 0L; + } + } + + public void deleteCheckPointForTimer(byte[] columnFamily, byte[] key) { + ColumnFamilyHandle cfHandle = getColumnFamily(columnFamily); + if (null == cfHandle || !COMMON_CHECK_POINT_KEY_SET_FOR_TIMER.contains(key)) { + logError.error("MessageRocksDBStorage deleteCheckPointForTimer error, cfHandle: {}, key: {}", cfHandle, key); + return; + } + try { + delete(cfHandle, ableWalWriteOptions, key); + } catch (Exception e) { + logError.error("MessageRocksDBStorage deleteCheckPointForTimer error: {}", e.getMessage()); + throw new RuntimeException("MessageRocksDBStorage deleteCheckPointForTimer error", e); + } + } + + public void writeRecordsForTrans(byte[] columnFamily, List recordList) { + ColumnFamilyHandle cfHandle = getColumnFamily(columnFamily); + if (null == cfHandle || CollectionUtils.isEmpty(recordList)) { + return; + } + long lastOffsetPy = 0L; + try (WriteBatch writeBatch = new WriteBatch()) { + for (TransRocksDBRecord record : recordList) { + if (null == record) { + logError.error("MessageRocksDBStorage writeRecordsForTrans error, record is null"); + continue; + } + byte[] keyBytes = record.getKeyBytes(); + if (null == keyBytes || keyBytes.length == 0) { + logError.error("MessageRocksDBStorage writeRecordsForTrans param error, keyBytes: {}", keyBytes); + continue; + } + if (record.isOp()) { + writeBatch.delete(cfHandle, record.getKeyBytes()); + } else { + byte[] valueBytes = record.getValueBytes(); + if (null == valueBytes || valueBytes.length == 0) { + logError.error("MessageRocksDBStorage writeRecordsForTrans param error, valueBytes: {}", valueBytes); + continue; + } + writeBatch.put(cfHandle, keyBytes, valueBytes); + lastOffsetPy = Math.max(lastOffsetPy, record.getOffsetPy()); + } + } + if (lastOffsetPy > 0L) { + Long lastOffsetPyStore = getLastOffsetPy(columnFamily); + if (null == lastOffsetPyStore || lastOffsetPy > lastOffsetPyStore) { + writeBatch.put(cfHandle, LAST_OFFSET_PY, ByteBuffer.allocate(Long.BYTES).putLong(lastOffsetPy).array()); + } + } + batchPut(ableWalWriteOptions, writeBatch); + } catch (Exception e) { + logError.error("MessageRocksDBStorage writeRecordsForTrans error: {}", e.getMessage()); + } + } + + public void updateRecordsForTrans(byte[] columnFamily, List recordList) { + ColumnFamilyHandle cfHandle = getColumnFamily(columnFamily); + if (null == cfHandle || CollectionUtils.isEmpty(recordList)) { + return; + } + try (WriteBatch writeBatch = new WriteBatch()) { + for (TransRocksDBRecord record : recordList) { + if (null == record) { + logError.error("MessageRocksDBStorage updateRecordsForTrans error, record is null"); + continue; + } + byte[] keyBytes = record.getKeyBytes(); + byte[] valueBytes = record.getValueBytes(); + if (null == keyBytes || keyBytes.length == 0 || null == valueBytes || valueBytes.length == 0) { + logError.error("MessageRocksDBStorage updateRecordsForTrans param error, keyBytes: {}, valueBytes: {}", keyBytes, valueBytes); + continue; + } + if (record.isDelete()) { + writeBatch.delete(cfHandle, keyBytes); + } else { + writeBatch.put(cfHandle, keyBytes, valueBytes); + } + } + batchPut(ableWalWriteOptions, writeBatch); + } catch (Exception e) { + logError.error("MessageRocksDBStorage updateRecordsForTrans error: {}", e.getMessage()); + } + } + + public List scanRecordsForTrans(byte[] columnFamily, int size, byte[] startKey) { + ColumnFamilyHandle cfHandle = getColumnFamily(columnFamily); + if (null == cfHandle || size <= 0) { + return null; + } + RocksIterator iterator = null; + try { + iterator = db.newIterator(cfHandle); + if (null == startKey || startKey.length == 0) { + iterator.seekToFirst(); + } else { + iterator.seek(startKey); + iterator.next(); + } + List records = new ArrayList<>(); + for (; iterator.isValid(); iterator.next()) { + byte[] key = iterator.key(); + if (null == key || key.length == 0 || key.length == LAST_OFFSET_PY_LENGTH && Arrays.equals(key, LAST_OFFSET_PY)) { + continue; + } + TransRocksDBRecord transRocksDBRecord = null; + try { + transRocksDBRecord = TransRocksDBRecord.decode(key, iterator.value()); + } catch (Exception e) { + logError.error("MessageRocksDBStorage scanRecordsForTrans error: {}", e.getMessage()); + } + if (null != transRocksDBRecord) { + records.add(transRocksDBRecord); + } + if (records.size() >= size) { + break; + } + } + return records; + } catch (Exception e) { + logError.error("MessageRocksDBStorage scanRecordsForTrans error: {}", e.getMessage()); + } finally { + if (null != iterator) { + iterator.close(); + } + } + return null; + } + + public TransRocksDBRecord getRecordForTrans(byte[] columnFamily, TransRocksDBRecord transRocksDBRecord) { + ColumnFamilyHandle cfHandle = getColumnFamily(columnFamily); + if (null == cfHandle || null == transRocksDBRecord) { + return null; + } + try { + byte[] keyBytes = transRocksDBRecord.getKeyBytes(); + if (null == keyBytes) { + return null; + } + byte[] valueBytes = get(cfHandle, readOptions, keyBytes); + if (null == valueBytes || valueBytes.length != TransRocksDBRecord.VALUE_LENGTH) { + return null; + } + return TransRocksDBRecord.decode(keyBytes, valueBytes); + } catch (Exception e) { + logError.error("MessageRocksDBStorage getRecordForTrans error: {}", e.getMessage()); + return null; + } + } + + public Long getLastOffsetPy(byte[] columnFamily) { + ColumnFamilyHandle cfHandle = getColumnFamily(columnFamily); + if (null == cfHandle) { + return null; + } + try { + byte[] offsetBytes = get(cfHandle, readOptions, LAST_OFFSET_PY); + return offsetBytes == null ? 0L : ByteBuffer.wrap(offsetBytes).getLong(); + } catch (Exception e) { + logError.error("MessageRocksDBStorage getLastOffsetPy error: {}", e.getMessage()); + return null; + } + } + + @Override + public synchronized boolean shutdown() { + try { + boolean result = super.shutdown(); + log.info("shutdown MessageRocksDBStorage result: {}", result); + return result; + } catch (Exception e) { + logError.error("shutdown MessageRocksDBStorage error : {}", e.getMessage()); + return false; + } + } + + private ColumnFamilyHandle getColumnFamily(byte[] columnFamily) { + if (Arrays.equals(columnFamily, RocksDB.DEFAULT_COLUMN_FAMILY)) { + return this.defaultCFHandle; + } else if (Arrays.equals(columnFamily, TIMER_COLUMN_FAMILY)) { + return this.timerCFHandle; + } else if (Arrays.equals(columnFamily, TRANS_COLUMN_FAMILY)) { + return this.transCFHandle; + } + throw new RuntimeException("Unknown column family"); + } +} diff --git a/store/src/main/java/org/apache/rocketmq/store/rocksdb/RocksDBOptionsFactory.java b/store/src/main/java/org/apache/rocketmq/store/rocksdb/RocksDBOptionsFactory.java index e365326c76d..07152a953a8 100644 --- a/store/src/main/java/org/apache/rocketmq/store/rocksdb/RocksDBOptionsFactory.java +++ b/store/src/main/java/org/apache/rocketmq/store/rocksdb/RocksDBOptionsFactory.java @@ -217,4 +217,149 @@ public static DBOptions createDBOptions() { setUseDirectIoForFlushAndCompaction(false). setUseDirectReads(false); } + + public static ColumnFamilyOptions createTimerCFOptions() { + BlockBasedTableConfig blockBasedTableConfig = new BlockBasedTableConfig() + .setFormatVersion(5) + .setIndexType(IndexType.kBinarySearch) + .setDataBlockIndexType(DataBlockIndexType.kDataBlockBinaryAndHash) + .setDataBlockHashTableUtilRatio(0.75) + .setBlockSize(128 * SizeUnit.KB) + .setMetadataBlockSize(4 * SizeUnit.KB) + .setFilterPolicy(new BloomFilter(16, false)) + .setCacheIndexAndFilterBlocks(false) + .setCacheIndexAndFilterBlocksWithHighPriority(true) + .setPinL0FilterAndIndexBlocksInCache(false) + .setPinTopLevelIndexAndFilter(true) + .setBlockCache(new LRUCache(2048 * SizeUnit.MB, 8, false)) + .setWholeKeyFiltering(true); + + //noinspection resource + return new ColumnFamilyOptions() + .setMaxWriteBufferNumber(6) + .setWriteBufferSize(256 * SizeUnit.MB) + .setMinWriteBufferNumberToMerge(1) + .setTableFormatConfig(blockBasedTableConfig) + .setMemTableConfig(new SkipListMemTableConfig()) + .setCompressionType(CompressionType.ZSTD_COMPRESSION) + .setBottommostCompressionType(CompressionType.NO_COMPRESSION) + .setNumLevels(7) + .setCompactionPriority(CompactionPriority.MinOverlappingRatio) + .setCompactionStyle(CompactionStyle.LEVEL) + .setMaxCompactionBytes(256 * SizeUnit.MB) + .setSoftPendingCompactionBytesLimit(100 * SizeUnit.GB) + .setHardPendingCompactionBytesLimit(256 * SizeUnit.GB) + .setLevel0FileNumCompactionTrigger(2) + .setLevel0SlowdownWritesTrigger(8) + .setLevel0StopWritesTrigger(10) + .setTargetFileSizeBase(256 * SizeUnit.MB) + .setTargetFileSizeMultiplier(2) + .setMergeOperator(new StringAppendOperator()) + .setReportBgIoStats(true) + .setOptimizeFiltersForHits(true) + .setMaxBytesForLevelBase(512 * SizeUnit.MB); + } + + public static ColumnFamilyOptions createTransCFOptions() { + BlockBasedTableConfig blockBasedTableConfig = new BlockBasedTableConfig() + .setFormatVersion(5) + .setIndexType(IndexType.kBinarySearch) + .setDataBlockIndexType(DataBlockIndexType.kDataBlockBinaryAndHash) + .setDataBlockHashTableUtilRatio(0.75) + .setBlockSize(128 * SizeUnit.KB) + .setMetadataBlockSize(4 * SizeUnit.KB) + .setFilterPolicy(new BloomFilter(16, false)) + .setCacheIndexAndFilterBlocks(false) + .setCacheIndexAndFilterBlocksWithHighPriority(true) + .setPinL0FilterAndIndexBlocksInCache(false) + .setPinTopLevelIndexAndFilter(true) + .setBlockCache(new LRUCache(1024 * SizeUnit.MB, 8, false)) + .setWholeKeyFiltering(true); + + CompactionOptionsUniversal compactionOption = new CompactionOptionsUniversal() + .setSizeRatio(100) + .setMaxSizeAmplificationPercent(25) + .setAllowTrivialMove(true) + .setMinMergeWidth(2) + .setMaxMergeWidth(Integer.MAX_VALUE) + .setStopStyle(CompactionStopStyle.CompactionStopStyleTotalSize) + .setCompressionSizePercent(-1); + + //noinspection resource + return new ColumnFamilyOptions() + .setMaxWriteBufferNumber(6) + .setWriteBufferSize(128 * SizeUnit.MB) + .setMinWriteBufferNumberToMerge(1) + .setTableFormatConfig(blockBasedTableConfig) + .setMemTableConfig(new SkipListMemTableConfig()) + .setCompressionType(CompressionType.NO_COMPRESSION) + .setBottommostCompressionType(CompressionType.NO_COMPRESSION) + .setNumLevels(7) + .setCompactionPriority(CompactionPriority.MinOverlappingRatio) + .setCompactionStyle(CompactionStyle.UNIVERSAL) + .setCompactionOptionsUniversal(compactionOption) + .setMaxCompactionBytes(100 * SizeUnit.GB) + .setSoftPendingCompactionBytesLimit(100 * SizeUnit.GB) + .setHardPendingCompactionBytesLimit(256 * SizeUnit.GB) + .setLevel0FileNumCompactionTrigger(2) + .setLevel0SlowdownWritesTrigger(8) + .setLevel0StopWritesTrigger(10) + .setTargetFileSizeBase(256 * SizeUnit.MB) + .setTargetFileSizeMultiplier(2) + .setMergeOperator(new StringAppendOperator()) + .setReportBgIoStats(true) + .setOptimizeFiltersForHits(true); + } + + public static ColumnFamilyOptions createIndexCFOptions() { + BlockBasedTableConfig blockBasedTableConfig = new BlockBasedTableConfig() + .setFormatVersion(5) + .setIndexType(IndexType.kBinarySearch) + .setDataBlockIndexType(DataBlockIndexType.kDataBlockBinaryAndHash) + .setDataBlockHashTableUtilRatio(0.75) + .setBlockSize(128 * SizeUnit.KB) + .setMetadataBlockSize(4 * SizeUnit.KB) + .setFilterPolicy(new BloomFilter(16, false)) + .setCacheIndexAndFilterBlocks(false) + .setCacheIndexAndFilterBlocksWithHighPriority(true) + .setPinL0FilterAndIndexBlocksInCache(false) + .setPinTopLevelIndexAndFilter(true) + .setBlockCache(new LRUCache(1024 * SizeUnit.MB, 8, false)) + .setWholeKeyFiltering(true); + + CompactionOptionsUniversal compactionOption = new CompactionOptionsUniversal() + .setSizeRatio(100) + .setMaxSizeAmplificationPercent(25) + .setAllowTrivialMove(true) + .setMinMergeWidth(2) + .setMaxMergeWidth(Integer.MAX_VALUE) + .setStopStyle(CompactionStopStyle.CompactionStopStyleTotalSize) + .setCompressionSizePercent(-1); + + //noinspection resource + return new ColumnFamilyOptions() + .setMaxWriteBufferNumber(6) + .setWriteBufferSize(128 * SizeUnit.MB) + .setMinWriteBufferNumberToMerge(1) + .setTableFormatConfig(blockBasedTableConfig) + .setMemTableConfig(new SkipListMemTableConfig()) + .setCompressionType(CompressionType.NO_COMPRESSION) + .setBottommostCompressionType(CompressionType.NO_COMPRESSION) + .setNumLevels(7) + .setCompactionPriority(CompactionPriority.MinOverlappingRatio) + .setCompactionStyle(CompactionStyle.UNIVERSAL) + .setCompactionOptionsUniversal(compactionOption) + .setMaxCompactionBytes(256 * SizeUnit.MB) + .setSoftPendingCompactionBytesLimit(100 * SizeUnit.GB) + .setHardPendingCompactionBytesLimit(256 * SizeUnit.GB) + .setLevel0FileNumCompactionTrigger(8) + .setLevel0SlowdownWritesTrigger(8) + .setLevel0StopWritesTrigger(20) + .setTargetFileSizeBase(256 * SizeUnit.MB) + .setTargetFileSizeMultiplier(2) + .setMergeOperator(new StringAppendOperator()) + .setReportBgIoStats(true) + .setOptimizeFiltersForHits(true); + } + } diff --git a/store/src/main/java/org/apache/rocketmq/store/timer/TimerMessageStore.java b/store/src/main/java/org/apache/rocketmq/store/timer/TimerMessageStore.java index b584be2b6cc..53999e72c4c 100644 --- a/store/src/main/java/org/apache/rocketmq/store/timer/TimerMessageStore.java +++ b/store/src/main/java/org/apache/rocketmq/store/timer/TimerMessageStore.java @@ -63,6 +63,7 @@ import org.apache.rocketmq.store.PutMessageResult; import org.apache.rocketmq.store.RunningFlags; import org.apache.rocketmq.store.SelectMappedBufferResult; +import org.apache.rocketmq.store.StoreUtil; import org.apache.rocketmq.store.config.BrokerRole; import org.apache.rocketmq.store.config.MessageStoreConfig; import org.apache.rocketmq.store.logfile.MappedFile; @@ -316,7 +317,11 @@ public void recover() { currQueueOffset = queueOffset + 1; } currQueueOffset = Math.min(currQueueOffset, timerCheckpoint.getMasterTimerQueueOffset()); - + if (storeConfig.isTimerRocksDBEnable()) { + long commitOffsetInRocksDB = messageStore.getTimerRocksDBStore().getCommitOffsetInRocksDB(); + LOGGER.info("recover time wheel, currQueueOffset: {}, commitOffsetInRocksDB: {}", currQueueOffset, commitOffsetInRocksDB); + currQueueOffset = Math.max(currQueueOffset, commitOffsetInRocksDB); + } ConsumeQueueInterface cq = this.messageStore.getConsumeQueue(TIMER_TOPIC, 0); // Correction based consume queue @@ -827,7 +832,7 @@ public boolean enqueue(int queueId) { return false; } - public boolean doEnqueue(long offsetPy, int sizePy, long delayedTime, MessageExt messageExt) { + public boolean doEnqueue(long offsetPy, int sizePy, long delayedTime, MessageExt messageExt, boolean isFromTimeline) { LOGGER.debug("Do enqueue [{}] [{}]", new Timestamp(delayedTime), messageExt); //copy the value first, avoid concurrent problem long tmpWriteTimeMs = currWriteTimeMs; @@ -845,6 +850,9 @@ public boolean doEnqueue(long offsetPy, int sizePy, long delayedTime, MessageExt boolean isDelete = messageExt.getProperty(TIMER_DELETE_UNIQUE_KEY) != null; if (isDelete) { magic = magic | MAGIC_DELETE; + if (!isFromTimeline) { + recallToTimeline(delayedTime, offsetPy, sizePy, messageExt); + } } String realTopic = messageExt.getProperty(MessageConst.PROPERTY_REAL_TOPIC); Slot slot = timerWheel.getSlot(delayedTime); @@ -1147,14 +1155,7 @@ private List> splitIntoLists(List origin) { private MessageExt getMessageByCommitOffset(long offsetPy, int sizePy) { for (int i = 0; i < 3; i++) { - MessageExt msgExt = null; - bufferLocal.get().position(0); - bufferLocal.get().limit(sizePy); - boolean res = messageStore.getData(offsetPy, sizePy, bufferLocal.get()); - if (res) { - bufferLocal.get().flip(); - msgExt = MessageDecoder.decode(bufferLocal.get(), true, false, false); - } + MessageExt msgExt = StoreUtil.getMessage(offsetPy, sizePy, messageStore, bufferLocal.get()); if (null == msgExt) { LOGGER.warn("Fail to read msg from commitLog offsetPy:{} sizePy:{}", offsetPy, sizePy); } else { @@ -1409,7 +1410,10 @@ public void run() { TimerMessageStore.LOGGER.info(this.getServiceName() + " service start"); while (!this.isStopped()) { try { - if (!TimerMessageStore.this.enqueue(0)) { + if (storeConfig.isTimerRocksDBEnable() && !storeConfig.isTimerRocksDBStopScan()) { + LOGGER.info("now timer use rocksdb to driver, so will not enqueue in timer wheel"); + waitForRunning(10 * 1000L); + } else if (!TimerMessageStore.this.enqueue(0)) { waitForRunning(100L * precisionMs / 1000); } } catch (Throwable e) { @@ -1473,7 +1477,7 @@ protected void putMessageToTimerWheel(TimerRequest req) { dequeuePutQueue.put(req); } else { boolean doEnqueueRes = doEnqueue( - req.getOffsetPy(), req.getSizePy(), req.getDelayTime(), req.getMsg()); + req.getOffsetPy(), req.getSizePy(), req.getDelayTime(), req.getMsg(), false); req.idempotentRelease(doEnqueueRes || storeConfig.isTimerSkipUnknownError()); } perfCounterTicks.endTick(ENQUEUE_PUT); @@ -2075,6 +2079,49 @@ public static String buildDeleteKey(String realTopic, String uniqueKey) { return realTopic + "+" + uniqueKey; } + private void recallToTimeline(long delayTime, long offsetPy, int sizePy, MessageExt messageExt) { + if (!storeConfig.isTimerRecallToTimelineEnable() || !storeConfig.isTimerRocksDBEnable()) { + return; + } + if (delayTime < 0L || offsetPy < 0L || sizePy <= 0 || null == messageExt) { + LOGGER.error("recallToTimeline param error, delayTime: {}, offsetPy: {}, sizePy: {}, messageExt: {}", delayTime, offsetPy, sizePy, messageExt); + return; + } + if (null == messageStore.getTimerRocksDBStore() || null == messageStore.getTimerRocksDBStore().getTimeline()) { + LOGGER.error("recallToTimeline error, timerRocksDBStore is null or timeline is null"); + return; + } + try { + messageStore.getTimerRocksDBStore().getTimeline().putDeleteRecord(delayTime, messageExt.getMsgId(), offsetPy, sizePy, messageExt.getQueueOffset(), messageExt); + } catch (Exception e) { + LOGGER.error("recallToTimeline error: {}", e.getMessage()); + } + } + + public boolean restart() { + try { + if (this.state != RUNNING) { + LOGGER.info("TimerMessageStore restart operation just support for running state"); + return false; + } + this.storeConfig.setTimerRocksDBStopScan(true); + if (this.state == RUNNING && !this.storeConfig.isTimerStopEnqueue()) { + LOGGER.info("restart TimerMessageStore has been running"); + return true; + } + long commitOffsetRocksDB = this.messageStore.getTimerRocksDBStore().getCommitOffsetInRocksDB(); + long commitOffsetFile = this.messageStore.getTimerMessageStore().getCommitQueueOffset(); + long maxCommitOffset = Math.max(commitOffsetFile, commitOffsetRocksDB); + currQueueOffset = maxCommitOffset; + this.storeConfig.setTimerStopEnqueue(false); + LOGGER.info("TimerMessageStore restart commitOffsetRocksDB: {}, commitOffsetFile: {}, currQueueOffset: {}", commitOffsetRocksDB, commitOffsetFile, currQueueOffset); + return true; + } catch (Exception e) { + LOGGER.error("TimerMessageStore restart error: {}", e.getMessage()); + return false; + } + } + public TimerFlushService getTimerFlushService() { return timerFlushService; } diff --git a/store/src/main/java/org/apache/rocketmq/store/timer/rocksdb/Timeline.java b/store/src/main/java/org/apache/rocketmq/store/timer/rocksdb/Timeline.java new file mode 100644 index 00000000000..922786bde92 --- /dev/null +++ b/store/src/main/java/org/apache/rocketmq/store/timer/rocksdb/Timeline.java @@ -0,0 +1,448 @@ +/* + * 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.timer.rocksdb; +import java.util.ArrayList; +import java.util.List; +import java.util.concurrent.BlockingQueue; +import java.util.concurrent.LinkedBlockingDeque; +import java.util.concurrent.TimeUnit; +import java.util.concurrent.atomic.AtomicLong; +import com.conversantmedia.util.concurrent.DisruptorBlockingQueue; +import org.apache.commons.collections.CollectionUtils; +import org.apache.commons.lang3.StringUtils; +import org.apache.commons.lang3.math.NumberUtils; +import org.apache.rocketmq.common.ServiceThread; +import org.apache.rocketmq.common.constant.LoggerName; +import org.apache.rocketmq.common.message.MessageConst; +import org.apache.rocketmq.common.message.MessageExt; +import org.apache.rocketmq.logging.org.slf4j.Logger; +import org.apache.rocketmq.logging.org.slf4j.LoggerFactory; +import org.apache.rocketmq.store.DefaultMessageStore; +import org.apache.rocketmq.store.MessageStore; +import org.apache.rocketmq.store.config.MessageStoreConfig; +import org.apache.rocketmq.store.rocksdb.MessageRocksDBStorage; +import org.apache.rocketmq.store.timer.TimerMessageStore; +import org.apache.rocketmq.store.timer.TimerMetrics; +import static org.apache.rocketmq.common.message.MessageConst.PROPERTY_TIMER_ROLL_LABEL; +import static org.apache.rocketmq.store.rocksdb.MessageRocksDBStorage.TIMER_COLUMN_FAMILY; +import static org.apache.rocketmq.store.timer.rocksdb.TimerRocksDBRecord.TIMER_ROCKSDB_DELETE; +import static org.apache.rocketmq.store.timer.rocksdb.TimerRocksDBRecord.TIMER_ROCKSDB_PUT; +import static org.apache.rocketmq.store.timer.rocksdb.TimerRocksDBRecord.TIMER_ROCKSDB_UPDATE; + +public class Timeline { + private static final Logger log = LoggerFactory.getLogger(LoggerName.STORE_LOGGER_NAME); + private static final Logger logError = LoggerFactory.getLogger(LoggerName.STORE_ERROR_LOGGER_NAME); + private static final String DELETE_KEY_SPLIT = "+"; + private static final int ORIGIN_CAPACITY = 100000; + private static final int BATCH_SIZE = 1000, MAX_BATCH_SIZE_FROM_ROCKSDB = 8000; + private static final int INITIAL = 0, RUNNING = 1, SHUTDOWN = 2; + private volatile int state = INITIAL; + private final AtomicLong commitOffset = new AtomicLong(0); + private final MessageStore messageStore; + private final MessageStoreConfig storeConfig; + private final TimerMessageStore timerMessageStore; + private final MessageRocksDBStorage messageRocksDBStorage; + private final TimerMessageRocksDBStore timerMessageRocksDBStore; + private final long precisionMs; + private final TimerMetrics timerMetrics; + + private TimelineIndexBuildService timelineIndexBuildService; + private TimelineForwardService timelineForwardService; + private TimelineRollService timelineRollService; + private TimelineDeleteService timelineDeleteService; + private BlockingQueue originTimerMsgQueue; + + public Timeline(final MessageStore messageStore, final MessageRocksDBStorage messageRocksDBStorage, final TimerMessageRocksDBStore timerMessageRocksDBStore, final TimerMetrics timerMetrics) { + this.messageStore = messageStore; + this.storeConfig = messageStore.getMessageStoreConfig(); + this.timerMessageStore = messageStore.getTimerMessageStore(); + this.messageRocksDBStorage = messageRocksDBStorage; + this.timerMessageRocksDBStore = timerMessageRocksDBStore; + this.precisionMs = timerMessageRocksDBStore.precisionMs; + this.timerMetrics = timerMetrics; + initService(); + } + + private void initService() { + this.timelineIndexBuildService = new TimelineIndexBuildService(); + this.timelineForwardService = new TimelineForwardService(); + if (storeConfig.isTimerEnableDisruptor()) { + this.originTimerMsgQueue = new DisruptorBlockingQueue<>(ORIGIN_CAPACITY); + } else { + this.originTimerMsgQueue = new LinkedBlockingDeque<>(ORIGIN_CAPACITY); + } + this.timelineRollService = new TimelineRollService(); + this.timelineDeleteService = new TimelineDeleteService(); + } + + public void start() { + if (this.state == RUNNING) { + return; + } + this.commitOffset.set(this.timerMessageRocksDBStore.getReadOffset().get()); + this.timelineIndexBuildService.start(); + this.timelineForwardService.start(); + this.timelineRollService.start(); + this.timelineDeleteService.start(); + this.state = RUNNING; + log.info("Timeline start success, start commitOffset: {}", this.commitOffset.get()); + } + + public void shutDown() { + if (this.state != RUNNING || this.state == SHUTDOWN) { + return; + } + if (null != this.timelineIndexBuildService) { + this.timelineIndexBuildService.shutdown(); + } + if (null != this.timelineForwardService) { + this.timelineForwardService.shutdown(); + } + if (null != this.timelineRollService) { + this.timelineRollService.shutdown(); + } + if (null != this.timelineDeleteService) { + this.timelineDeleteService.shutdown(); + } + this.state = SHUTDOWN; + log.info("Timeline shutdown success"); + } + + public void putRecord(TimerRocksDBRecord timerRecord) throws InterruptedException { + if (null == timerRecord) { + return; + } + while (!originTimerMsgQueue.offer(timerRecord, 3, TimeUnit.SECONDS)) { + if (null != timerRecord.getMessageExt()) { + logError.error("Timeline originTimerMsgQueue put record failed, topic: {}, uniqKey: {}", timerRecord.getMessageExt().getTopic(), timerRecord.getUniqKey()); + } else { + logError.error("Timeline originTimerMsgQueue put record failed, uniqKey: {}", timerRecord.getUniqKey()); + } + } + } + + public void putDeleteRecord(long delayTime, String uniqKey, long offsetPy, int sizePy, long queueOffset, MessageExt messageExt) throws InterruptedException { + if (delayTime <= 0L || StringUtils.isEmpty(uniqKey) || offsetPy < 0L || sizePy < 0 || queueOffset < 0L || null == messageExt) { + log.info("Timeline putDeleteRecord param error, delayTime: {}, uniqKey: {}, offsetPy: {}, sizePy: {}, queueOffset: {}, messageExt: {}", delayTime, uniqKey, offsetPy, sizePy, queueOffset, messageExt); + return; + } + while (!originTimerMsgQueue.offer(new TimerRocksDBRecord(delayTime, uniqKey, offsetPy, sizePy, queueOffset, messageExt), 3, TimeUnit.SECONDS)) { + log.error("Timeline putDeleteRecord originTimerMsgQueue put delete record failed, uniqKey: {}", uniqKey); + } + } + + public void addMetric(MessageExt msg, int value) { + if (null == msg || null == msg.getProperty(MessageConst.PROPERTY_REAL_TOPIC)) { + return; + } + if (null != msg.getProperty(MessageConst.PROPERTY_TIMER_ENQUEUE_MS) && NumberUtils.toLong(msg.getProperty(MessageConst.PROPERTY_TIMER_ENQUEUE_MS)) == Long.MAX_VALUE) { + return; + } + timerMetrics.addAndGet(msg, value); + } + + private String getServiceThreadName() { + String brokerIdentifier = ""; + if (Timeline.this.messageStore instanceof DefaultMessageStore) { + DefaultMessageStore messageStore = (DefaultMessageStore) Timeline.this.messageStore; + if (messageStore.getBrokerConfig().isInBrokerContainer()) { + brokerIdentifier = messageStore.getBrokerConfig().getIdentifier(); + } + } + return brokerIdentifier; + } + + private void recallToTimeWheel(TimerRocksDBRecord tr) { + if (!messageStore.getMessageStoreConfig().isTimerRecallToTimeWheelEnable()) { + return; + } + if (null == tr || null == tr.getMessageExt()) { + return; + } + try { + timerMessageStore.doEnqueue(tr.getOffsetPy(), tr.getSizePy(), tr.getDelayTime(), tr.getMessageExt(), true); + } catch (Exception e) { + log.error("Timeline recallToTimeWheel error: {}", e.getMessage()); + } + } + + private boolean scanRecordsToQueue(long checkpoint, long checkRange, BlockingQueue> queue) { + if (checkpoint <= 0L || checkRange <= 0L || null == queue) { + logError.error("Timeline scanRecordsToQueue param error, checkpoint: {}, checkRange: {}, queue: {}", checkpoint, checkRange, queue); + return false; + } + if (storeConfig.isTimerStopDequeue()) { + logError.info("Timeline scanRecordsToQueue storeConfig isTimerStopDequeue is true, stop to scan records to queue"); + return false; + } + long count = 0; + byte[] lastKey = null; + while (true) { + try { + List trs = messageRocksDBStorage.scanRecordsForTimer(TIMER_COLUMN_FAMILY, checkpoint, checkpoint + checkRange, MAX_BATCH_SIZE_FROM_ROCKSDB, lastKey); + if (null == trs || CollectionUtils.isEmpty(trs)) { + break; + } + count += trs.size(); + boolean hasMoreData = trs.size() >= MAX_BATCH_SIZE_FROM_ROCKSDB; + lastKey = hasMoreData ? trs.get(trs.size() - 1).getKeyBytes() : null; + if (null == lastKey) { + trs.get(trs.size() - 1).setCheckPoint(checkpoint + checkRange); + } + while (!queue.offer(trs, 3, TimeUnit.SECONDS)) { + log.debug("Timeline scanRecordsToQueue offer to queue error, queue size: {}, records size: {}", queue.size(), trs.size()); + } + if (!hasMoreData) { + break; + } + } catch (Exception e) { + logError.error("Timeline scanRecordsToQueue error: {}", e.getMessage()); + return false; + } + } + log.info("Timeline scan records from rocksdb, checkpoint: {}, records size: {}", checkpoint, count); + return true; + } + + public class TimelineIndexBuildService extends ServiceThread { + private final Logger log = Timeline.log; + private List trs; + + @Override + public String getServiceName() { + return getServiceThreadName() + this.getClass().getSimpleName(); + } + + @Override + public void run() { + log.info(this.getServiceName() + " service start"); + trs = new ArrayList<>(BATCH_SIZE); + while (!this.isStopped() || !originTimerMsgQueue.isEmpty()) { + try { + buildTimelineIndex(); + } catch (Exception e) { + logError.error("Timeline error occurred in: {}, error: {}", getServiceName(), e.getMessage()); + } + } + log.info(this.getServiceName() + " service end"); + } + + private void buildTimelineIndex() throws InterruptedException { + pollTimerMessageRecords(); + if (CollectionUtils.isEmpty(trs)) { + return; + } + List cudlist = new ArrayList<>(); + for (TimerRocksDBRecord tr : trs) { + try { + MessageExt messageExt = tr.getMessageExt(); + if (null == messageExt) { + logError.error("Timeline TimelineIndexBuildService buildTimelineIndex error, messageExt is null"); + continue; + } + String timerDelUniqKey = messageExt.getProperty(MessageConst.PROPERTY_TIMER_DEL_UNIQKEY); + if (!StringUtils.isEmpty(timerDelUniqKey)) { + tr.setUniqKey(extractUniqKey(timerDelUniqKey)); + tr.setActionFlag(TIMER_ROCKSDB_DELETE); + cudlist.add(tr); + addMetric(messageExt, -1); + recallToTimeWheel(tr); + } else if (TimerMessageRocksDBStore.isExpired(tr.getDelayTime())) { + timerMessageRocksDBStore.putRealTopicMessage(tr.getMessageExt()); + } else if (!StringUtils.isEmpty(messageExt.getProperty(PROPERTY_TIMER_ROLL_LABEL))) { + tr.setActionFlag(TIMER_ROCKSDB_UPDATE); + cudlist.add(tr); + } else { + tr.setActionFlag(TIMER_ROCKSDB_PUT); + cudlist.add(tr); + addMetric(messageExt, 1); + } + } catch (Exception e) { + logError.error("Timeline TimelineIndexBuildService buildTimelineIndex deal trs error", e); + } + } + if (!CollectionUtils.isEmpty(cudlist)) { + messageRocksDBStorage.writeRecordsForTimer(TIMER_COLUMN_FAMILY, cudlist); + } + synCommitOffset(trs); + trs.clear(); + } + + private String extractUniqKey(String deleteKey) throws IllegalArgumentException { + if (StringUtils.isEmpty(deleteKey)) { + throw new IllegalArgumentException("deleteKey is empty"); + } + int separatorIndex = deleteKey.indexOf(DELETE_KEY_SPLIT); + if (separatorIndex == -1) { + throw new IllegalArgumentException("Invalid deleteKey format"); + } + return deleteKey.substring(separatorIndex + 1); + } + + private void pollTimerMessageRecords() throws InterruptedException { + TimerRocksDBRecord firstReq = originTimerMsgQueue.poll(100, TimeUnit.MILLISECONDS); + if (null != firstReq) { + trs.add(firstReq); + while (true) { + TimerRocksDBRecord tmpReq = originTimerMsgQueue.poll(100, TimeUnit.MILLISECONDS); + if (null == tmpReq) { + break; + } + trs.add(tmpReq); + if (trs.size() >= BATCH_SIZE) { + break; + } + } + } + } + + private void synCommitOffset(List trs) { + if (CollectionUtils.isEmpty(trs)) { + return; + } + long lastQueueOffset = messageRocksDBStorage.getCheckpointForTimer(TIMER_COLUMN_FAMILY, MessageRocksDBStorage.SYS_TOPIC_SCAN_OFFSET_CHECK_POINT); + long queueOffset = trs.get(trs.size() - 1).getQueueOffset() + 1L; + if (queueOffset > lastQueueOffset) { + commitOffset.set(queueOffset); + messageRocksDBStorage.writeCheckPointForTimer(TIMER_COLUMN_FAMILY, MessageRocksDBStorage.SYS_TOPIC_SCAN_OFFSET_CHECK_POINT, commitOffset.get()); + } + } + } + + private class TimelineForwardService extends ServiceThread { + private final Logger log = Timeline.log; + @Override + public String getServiceName() { + return getServiceThreadName() + this.getClass().getSimpleName(); + } + + @Override + public void run() { + long checkpoint = messageRocksDBStorage.getCheckpointForTimer(TIMER_COLUMN_FAMILY, MessageRocksDBStorage.TIMELINE_CHECK_POINT); + log.info(this.getServiceName() + " service start, checkpoint: {}", checkpoint); + while (!this.isStopped()) { + try { + if (!timelineForward(checkpoint, precisionMs)) { + waitForRunning(100L); + } else { + checkpoint += precisionMs; + } + } catch (Exception e) { + logError.error("Timeline error occurred in " + getServiceName(), e); + } + } + log.info(this.getServiceName() + " service end"); + } + + private boolean timelineForward(long checkpoint, long checkRange) { + if (checkpoint > System.currentTimeMillis()) { + return false; + } + try { + long begin = System.currentTimeMillis(); + boolean result = scanRecordsToQueue(checkpoint, checkRange, timerMessageRocksDBStore.getExpiredMessageQueue()); + log.info("Timeline TimelineForwardService timelineForward scanRecordsToQueue end, result: {}, checkpoint: {}, checkRange: {}, checkDelay: {}, cost: {}", result, checkpoint, checkRange, System.currentTimeMillis() - checkpoint, System.currentTimeMillis() - begin); + return result; + } catch (Exception e) { + logError.error("Timeline TimelineForwardService timelineForward error: {}", e.getMessage()); + return false; + } + } + } + + private class TimelineRollService extends ServiceThread { + private final Logger log = Timeline.log; + @Override + public String getServiceName() { + return getServiceThreadName() + this.getClass().getSimpleName(); + } + + @Override + public void run() { + log.info(this.getServiceName() + " service start"); + while (!this.isStopped()) { + int rollIntervalHour = 1; + int rollRangeHour = 2; + try { + if (storeConfig.getTimerRocksDBRollIntervalHours() > 0) { + rollIntervalHour = storeConfig.getTimerRocksDBRollIntervalHours(); + } + if (storeConfig.getTimerRocksDBRollRangeHours() > 0) { + rollRangeHour = storeConfig.getTimerRocksDBRollRangeHours(); + } + this.waitForRunning(TimeUnit.HOURS.toMillis(rollIntervalHour)); + if (stopped) { + log.info(this.getServiceName() + " service end"); + return; + } + } catch (Exception e) { + logError.error("Timeline TimelineRollService wait error: {}", e.getMessage()); + } + long rollCheckpoint = System.currentTimeMillis(); + try { + log.info("Timeline TimelineRollService start roll rollCheckpoint: {}", rollCheckpoint); + while (!scanRecordsToQueue(rollCheckpoint + TimeUnit.HOURS.toMillis(rollRangeHour), + TimeUnit.SECONDS.toMillis(storeConfig.getTimerMaxDelaySec()), + timerMessageRocksDBStore.getRollMessageQueue())) { + logError.error("Timeline TimelineRollService scanRecordsToQueue error."); + Thread.sleep(200); + } + log.info("Timeline TimelineRollService roll records success, lastRollTime: {}, rollCheckpoint: {}, cost: {}", rollCheckpoint, rollCheckpoint, System.currentTimeMillis() - rollCheckpoint); + } catch (Exception e) { + logError.error("Timeline TimelineRollService failed error: {}", e.getMessage()); + } + } + log.info(this.getServiceName() + " service end"); + } + } + + private class TimelineDeleteService extends ServiceThread { + private final Logger log = Timeline.log; + private long lastDeleteCheckPoint = 0L; + @Override + public String getServiceName() { + return getServiceThreadName() + this.getClass().getSimpleName(); + } + @Override + public void run() { + log.info(this.getServiceName() + " service start"); + while (!this.isStopped()) { + try { + this.waitForRunning(TimeUnit.MINUTES.toMillis(30)); + if (stopped) { + log.info(this.getServiceName() + " service end"); + return; + } + } catch (Exception e) { + logError.error("Timeline TimelineDeleteService wait error: {}", e.getMessage()); + } + try { + long checkpoint = messageRocksDBStorage.getCheckpointForTimer(TIMER_COLUMN_FAMILY, MessageRocksDBStorage.TIMELINE_CHECK_POINT); + if (lastDeleteCheckPoint == checkpoint) { + continue; + } + messageRocksDBStorage.deleteRecordsForTimer(TIMER_COLUMN_FAMILY, checkpoint - TimeUnit.HOURS.toMillis(168), checkpoint - TimeUnit.MINUTES.toMillis(30)); + lastDeleteCheckPoint = checkpoint; + } catch (Exception e) { + logError.error("Timeline TimelineDeleteService delete failed, lastDeleteCheckPoint: {} error: {}", lastDeleteCheckPoint, e.getMessage()); + } + } + log.info(this.getServiceName() + " service end"); + } + } + +} diff --git a/store/src/main/java/org/apache/rocketmq/store/timer/rocksdb/TimerMessageRocksDBStore.java b/store/src/main/java/org/apache/rocketmq/store/timer/rocksdb/TimerMessageRocksDBStore.java new file mode 100644 index 00000000000..ec13971d922 --- /dev/null +++ b/store/src/main/java/org/apache/rocketmq/store/timer/rocksdb/TimerMessageRocksDBStore.java @@ -0,0 +1,619 @@ +/* + * 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.timer.rocksdb; + +import java.nio.ByteBuffer; +import java.util.List; +import java.util.concurrent.BlockingQueue; +import java.util.concurrent.Callable; +import java.util.concurrent.CountDownLatch; +import java.util.concurrent.ExecutorService; +import java.util.concurrent.LinkedBlockingDeque; +import java.util.concurrent.LinkedBlockingQueue; +import java.util.concurrent.ThreadPoolExecutor; +import java.util.concurrent.TimeUnit; +import java.util.concurrent.atomic.AtomicLong; +import java.util.function.Function; +import com.conversantmedia.util.concurrent.DisruptorBlockingQueue; +import com.google.common.util.concurrent.RateLimiter; +import io.opentelemetry.api.common.Attributes; +import org.apache.commons.collections.CollectionUtils; +import org.apache.commons.lang3.StringUtils; +import org.apache.rocketmq.common.ServiceThread; +import org.apache.rocketmq.common.TopicFilterType; +import org.apache.rocketmq.common.constant.LoggerName; +import org.apache.rocketmq.common.message.MessageAccessor; +import org.apache.rocketmq.common.message.MessageClientIDSetter; +import org.apache.rocketmq.common.message.MessageConst; +import org.apache.rocketmq.common.message.MessageDecoder; +import org.apache.rocketmq.common.message.MessageExt; +import org.apache.rocketmq.common.message.MessageExtBrokerInner; +import org.apache.rocketmq.logging.org.slf4j.Logger; +import org.apache.rocketmq.logging.org.slf4j.LoggerFactory; +import org.apache.rocketmq.store.DefaultMessageStore; +import org.apache.rocketmq.store.MessageStore; +import org.apache.rocketmq.store.PutMessageResult; +import org.apache.rocketmq.store.StoreUtil; +import org.apache.rocketmq.store.config.MessageStoreConfig; +import org.apache.rocketmq.store.metrics.DefaultStoreMetricsConstant; +import org.apache.rocketmq.store.metrics.DefaultStoreMetricsManager; +import org.apache.rocketmq.store.metrics.StoreMetricsManager; +import org.apache.rocketmq.store.queue.ConsumeQueueInterface; +import org.apache.rocketmq.store.queue.CqUnit; +import org.apache.rocketmq.store.queue.ReferredIterator; +import org.apache.rocketmq.store.rocksdb.MessageRocksDBStorage; +import org.apache.rocketmq.store.stats.BrokerStatsManager; +import org.apache.rocketmq.store.timer.TimerMetrics; +import org.apache.rocketmq.store.util.PerfCounter; +import static org.apache.rocketmq.common.message.MessageConst.PROPERTY_TIMER_ROLL_LABEL; +import static org.apache.rocketmq.store.rocksdb.MessageRocksDBStorage.TIMER_COLUMN_FAMILY; +import static org.apache.rocketmq.store.timer.TimerMessageStore.TIMER_TOPIC; + +public class TimerMessageRocksDBStore { + private static final Logger log = LoggerFactory.getLogger(LoggerName.STORE_LOGGER_NAME); + private static final Logger logError = LoggerFactory.getLogger(LoggerName.STORE_ERROR_LOGGER_NAME); + private static final String SCAN_SYS_TOPIC = "scanSysTopic"; + private static final String SCAN_SYS_TOPIC_MISS = "scanSysTopicMiss"; + private static final String OUT_BIZ_MESSAGE = "outBizMsg"; + private static final String ROLL_LABEL = "R"; + private static final int PUT_OK = 0, PUT_NEED_RETRY = 1, PUT_NO_RETRY = 2; + private static final int MAX_GET_MSG_TIMES = 3, MAX_PUT_MSG_TIMES = 3; + private static final int TIME_UP_CAPACITY = 100, ROLL_CAPACITY = 50; + private static final int INITIAL = 0, RUNNING = 1, SHUTDOWN = 2; + private volatile int state = INITIAL; + private static long expirationThresholdMillis = 999L; + private final AtomicLong readOffset = new AtomicLong(0);//timerSysTopic read offset + private final MessageStore messageStore; + private final TimerMetrics timerMetrics; + private final MessageStoreConfig storeConfig; + private final BrokerStatsManager brokerStatsManager; + private final MessageRocksDBStorage messageRocksDBStorage; + private Timeline timeline; + private TimerSysTopicScanService timerSysTopicScanService; + private TimerMessageReputService expiredMessageReputService; + private TimerMessageReputService rollMessageReputService; + protected long precisionMs; + private BlockingQueue> expiredMessageQueue; + private BlockingQueue> rollMessageQueue; + protected final PerfCounter.Ticks perfCounterTicks = new PerfCounter.Ticks(log); + private Function escapeBridgeHook; + private ThreadLocal bufferLocal = null; + + public TimerMessageRocksDBStore(final MessageStore messageStore, final TimerMetrics timerMetrics, + final BrokerStatsManager brokerStatsManager) { + this.messageStore = messageStore; + this.storeConfig = messageStore.getMessageStoreConfig(); + this.precisionMs = storeConfig.getTimerRocksDBPrecisionMs() < 100L ? 1000L : storeConfig.getTimerRocksDBPrecisionMs(); + expirationThresholdMillis = precisionMs - 1L; + this.messageRocksDBStorage = messageStore.getMessageRocksDBStorage(); + this.timerMetrics = timerMetrics; + this.brokerStatsManager = brokerStatsManager; + bufferLocal = ThreadLocal.withInitial(() -> ByteBuffer.allocate(storeConfig.getMaxMessageSize())); + } + + public synchronized boolean load() { + initService(); + boolean result = this.timerMetrics.load(); + log.info("TimerMessageRocksDBStore load result: {}", result); + return result; + } + + public synchronized void start() { + if (this.state == RUNNING) { + return; + } + long commitOffsetFile = null != this.messageStore.getTimerMessageStore() ? this.messageStore.getTimerMessageStore().getCommitQueueOffset() : 0L; + long commitOffsetRocksDB = messageRocksDBStorage.getCheckpointForTimer(TIMER_COLUMN_FAMILY, MessageRocksDBStorage.SYS_TOPIC_SCAN_OFFSET_CHECK_POINT); + long maxCommitOffset = Math.max(commitOffsetFile, commitOffsetRocksDB); + this.readOffset.set(maxCommitOffset); + this.expiredMessageReputService.start(); + this.rollMessageReputService.start(); + this.timeline.start(); + this.timerSysTopicScanService.start(); + this.state = RUNNING; + log.info("TimerMessageRocksDBStore start success, start commitOffsetFile: {}, commitOffsetRocksDB: {}, readOffset: {}", commitOffsetFile, commitOffsetRocksDB, this.readOffset.get()); + } + + public synchronized boolean restart() { + try { + this.storeConfig.setTimerStopEnqueue(true); + if (this.state == RUNNING && !this.storeConfig.isTimerRocksDBStopScan()) { + log.info("restart TimerMessageRocksDBStore has been running"); + return true; + } + if (this.state == RUNNING && this.storeConfig.isTimerRocksDBStopScan()) { + long commitOffsetFile = null != this.messageStore.getTimerMessageStore() ? this.messageStore.getTimerMessageStore().getCommitQueueOffset() : 0L; + long commitOffsetRocksDB = messageRocksDBStorage.getCheckpointForTimer(TIMER_COLUMN_FAMILY, MessageRocksDBStorage.SYS_TOPIC_SCAN_OFFSET_CHECK_POINT); + long maxCommitOffset = Math.max(commitOffsetFile, commitOffsetRocksDB); + this.readOffset.set(maxCommitOffset); + log.info("restart TimerMessageRocksDBStore has been recover running, commitOffsetFile: {}, commitOffsetRocksDB: {}, readOffset: {}", commitOffsetFile, commitOffsetRocksDB, readOffset.get()); + } else { + this.load(); + this.start(); + } + this.storeConfig.setTimerRocksDBEnable(true); + this.storeConfig.setTimerRocksDBStopScan(false); + return true; + } catch (Exception e) { + logError.error("TimerMessageRocksDBStore restart error: {}", e.getMessage()); + return false; + } + } + + public void shutdown() { + if (this.state != RUNNING || this.state == SHUTDOWN) { + return; + } + if (null != this.timerSysTopicScanService) { + this.timerSysTopicScanService.shutdown(); + } + if (null != this.timeline) { + this.timeline.shutDown(); + } + if (null != this.expiredMessageReputService) { + this.expiredMessageReputService.shutdown(); + } + if (null != this.rollMessageReputService) { + this.rollMessageReputService.shutdown(); + } + this.state = SHUTDOWN; + log.info("TimerMessageRocksDBStore shutdown success"); + } + + public void putRealTopicMessage(MessageExt msg) { + if (null == msg) { + logError.error("putRealTopicMessage msg is null"); + return; + } + MessageExtBrokerInner messageExtBrokerInner = convertMessage(msg); + if (null == messageExtBrokerInner) { + logError.error("putRealTopicMessage error, messageExtBrokerInner is null"); + return; + } + doPut(messageExtBrokerInner); + } + + public MessageStore getMessageStore() { + return messageStore; + } + + public TimerMetrics getTimerMetrics() { + return timerMetrics; + } + + public BrokerStatsManager getBrokerStatsManager() { + return brokerStatsManager; + } + + public AtomicLong getReadOffset() { + return readOffset; + } + + public BlockingQueue> getExpiredMessageQueue() { + return expiredMessageQueue; + } + + public BlockingQueue> getRollMessageQueue() { + return rollMessageQueue; + } + + public long getCommitOffsetInRocksDB() { + if (null == messageRocksDBStorage || !storeConfig.isTimerRocksDBEnable()) { + return 0L; + } + return messageRocksDBStorage.getCheckpointForTimer(TIMER_COLUMN_FAMILY, MessageRocksDBStorage.SYS_TOPIC_SCAN_OFFSET_CHECK_POINT); + } + + public Timeline getTimeline() { + return timeline; + } + + private void initService() { + if (storeConfig.isTimerEnableDisruptor()) { + this.expiredMessageQueue = new DisruptorBlockingQueue<>(TIME_UP_CAPACITY); + this.rollMessageQueue = new DisruptorBlockingQueue<>(ROLL_CAPACITY); + } else { + this.expiredMessageQueue = new LinkedBlockingDeque<>(TIME_UP_CAPACITY); + this.rollMessageQueue = new LinkedBlockingDeque<>(ROLL_CAPACITY); + } + this.expiredMessageReputService = new TimerMessageReputService(expiredMessageQueue, storeConfig.getTimerRocksDBTimeExpiredMaxTps(), true); + this.rollMessageReputService = new TimerMessageReputService(rollMessageQueue, storeConfig.getTimerRocksDBRollMaxTps(), false); + this.timeline = new Timeline(messageStore, messageRocksDBStorage, this, timerMetrics); + this.timerSysTopicScanService = new TimerSysTopicScanService(); + } + + private MessageExtBrokerInner convertMessage(MessageExt msgExt) { + if (null == msgExt) { + logError.error("convertMessage msgExt is null"); + return null; + } + MessageExtBrokerInner msgInner = null; + try { + msgInner = new MessageExtBrokerInner(); + msgInner.setBody(msgExt.getBody()); + msgInner.setFlag(msgExt.getFlag()); + msgInner.setTags(msgExt.getTags()); + msgInner.setSysFlag(msgExt.getSysFlag()); + TopicFilterType topicFilterType = MessageExt.parseTopicFilterType(msgInner.getSysFlag()); + long tagsCodeValue = MessageExtBrokerInner.tagsString2tagsCode(topicFilterType, msgInner.getTags()); + msgInner.setTagsCode(tagsCodeValue); + msgInner.setBornTimestamp(msgExt.getBornTimestamp()); + msgInner.setBornHost(msgExt.getBornHost()); + msgInner.setStoreHost(msgExt.getStoreHost()); + msgInner.setReconsumeTimes(msgExt.getReconsumeTimes()); + msgInner.setWaitStoreMsgOK(false); + MessageAccessor.setProperties(msgInner, MessageAccessor.deepCopyProperties(msgExt.getProperties())); + if (isNeedRoll(msgInner)) { + msgInner.setTopic(msgExt.getTopic()); + msgInner.setQueueId(msgExt.getQueueId()); + msgInner.getProperties().put(PROPERTY_TIMER_ROLL_LABEL, ROLL_LABEL); + } else { + msgInner.setTopic(msgInner.getProperty(MessageConst.PROPERTY_REAL_TOPIC)); + msgInner.setQueueId(Integer.parseInt(msgInner.getProperty(MessageConst.PROPERTY_REAL_QUEUE_ID))); + MessageAccessor.clearProperty(msgInner, MessageConst.PROPERTY_REAL_TOPIC); + MessageAccessor.clearProperty(msgInner, MessageConst.PROPERTY_REAL_QUEUE_ID); + MessageAccessor.clearProperty(msgInner, MessageConst.PROPERTY_TIMER_ROLL_LABEL); + } + msgInner.setPropertiesString(MessageDecoder.messageProperties2String(msgInner.getProperties())); + } catch (Exception e) { + logError.error("convertMessage error: {}", e.getMessage()); + return null; + } + return msgInner; + } + + private MessageExt getMessageByCommitOffset(long offsetPy, int sizePy) { + if (offsetPy < 0L || sizePy <= 0 || sizePy > storeConfig.getMaxMessageSize()) { + logError.error("getMessageByCommitOffset param error, offsetPy: {}, sizePy: {}, maxMsgSize: {}", offsetPy, sizePy, storeConfig.getMaxMessageSize()); + return null; + } + if (sizePy > bufferLocal.get().limit()) { + bufferLocal.remove(); + bufferLocal.set(ByteBuffer.allocate(sizePy)); + } + for (int i = 0; i < MAX_GET_MSG_TIMES; i++) { + MessageExt msgExt = StoreUtil.getMessage(offsetPy, sizePy, messageStore, bufferLocal.get()); + if (null == msgExt) { + log.warn("Fail to read msg from commitLog offsetPy: {} sizePy: {}", offsetPy, sizePy); + } else { + return msgExt; + } + } + return null; + } + + private boolean isNeedRoll(MessageExt messageExt) { + try { + String property = messageExt.getProperty(MessageConst.PROPERTY_TIMER_OUT_MS); + if (StringUtils.isEmpty(property)) { + return false; + } + if (!isExpired(Long.parseLong(property))) { + return true; + } + } catch (Exception e) { + logError.error("isNeedRoll error : {}", e.getMessage()); + } + return false; + } + + private Long getDelayTime(MessageExt msgExt) { + if (null == msgExt) { + logError.error("getDelayTime msgExt is null"); + return null; + } + String delayTimeStr = msgExt.getProperty(MessageConst.PROPERTY_TIMER_OUT_MS); + if (StringUtils.isEmpty(delayTimeStr)) { + logError.error("getDelayTime is empty, queueOffset: {}, delayTimeStr: {}", msgExt.getQueueOffset(), delayTimeStr); + return null; + } + try { + return Long.parseLong(delayTimeStr); + } catch (Exception e) { + logError.error("getDelayTime parse to long error : {}", e.getMessage()); + } + return null; + } + + private int doPut(MessageExtBrokerInner message) { + if (null == message) { + logError.error("doPut message is null"); + return PUT_NO_RETRY; + } + if (null != message.getProperty(MessageConst.PROPERTY_TIMER_DEL_UNIQKEY)) { + logError.warn("Trying do put delete timer msg:[{}]", message); + return PUT_NO_RETRY; + } + PutMessageResult putMessageResult = null; + if (escapeBridgeHook != null) { + putMessageResult = escapeBridgeHook.apply(message); + } else { + putMessageResult = messageStore.putMessage(message); + } + if (null != putMessageResult && null != putMessageResult.getPutMessageStatus()) { + switch (putMessageResult.getPutMessageStatus()) { + case PUT_OK: + if (null != brokerStatsManager) { + brokerStatsManager.incTopicPutNums(message.getTopic(), 1, 1); + if (null != putMessageResult.getAppendMessageResult()) { + brokerStatsManager.incTopicPutSize(message.getTopic(), putMessageResult.getAppendMessageResult().getWroteBytes()); + } + brokerStatsManager.incBrokerPutNums(message.getTopic(), 1); + } + return PUT_OK; + + case MESSAGE_ILLEGAL: + case PROPERTIES_SIZE_EXCEEDED: + case WHEEL_TIMER_NOT_ENABLE: + case WHEEL_TIMER_MSG_ILLEGAL: + return PUT_NO_RETRY; + + case SERVICE_NOT_AVAILABLE: + case FLUSH_DISK_TIMEOUT: + case FLUSH_SLAVE_TIMEOUT: + case OS_PAGE_CACHE_BUSY: + case CREATE_MAPPED_FILE_FAILED: + case SLAVE_NOT_AVAILABLE: + return PUT_NEED_RETRY; + + case UNKNOWN_ERROR: + default: + if (storeConfig.isTimerSkipUnknownError()) { + logError.warn("Skipping message due to unknown error, msg: {}", message); + return PUT_NO_RETRY; + } else { + return PUT_NEED_RETRY; + } + } + } + return PUT_NEED_RETRY; + } + + public static boolean isExpired(long delayedTime) { + return delayedTime <= System.currentTimeMillis() + expirationThresholdMillis; + } + + public void registerEscapeBridgeHook(Function escapeBridgeHook) { + this.escapeBridgeHook = escapeBridgeHook; + } + + private String getServiceThreadName() { + String brokerIdentifier = ""; + if (TimerMessageRocksDBStore.this.messageStore instanceof DefaultMessageStore) { + DefaultMessageStore messageStore = (DefaultMessageStore)TimerMessageRocksDBStore.this.messageStore; + if (messageStore.getBrokerConfig().isInBrokerContainer()) { + brokerIdentifier = messageStore.getBrokerConfig().getIdentifier(); + } + } + return brokerIdentifier; + } + + private class TimerSysTopicScanService extends ServiceThread { + private final Logger log = TimerMessageRocksDBStore.log; + + @Override + public String getServiceName() { + return getServiceThreadName() + this.getClass().getSimpleName(); + } + + @Override + public void run() { + log.info(this.getServiceName() + " service start"); + long waitTime; + while (!this.isStopped()) { + try { + if (!storeConfig.isTimerRocksDBEnable() || storeConfig.isTimerRocksDBStopScan()) { + waitTime = TimeUnit.SECONDS.toMillis(10); + } else { + scanSysTimerTopic(); + waitTime = 100L; + } + waitForRunning(waitTime); + } catch (Exception e) { + logError.error("TimerMessageRocksDBStore error occurred in: {}, error: {}", getServiceName(), + e.getMessage()); + } + } + log.info(this.getServiceName() + " service end"); + } + + private void scanSysTimerTopic() { + ConsumeQueueInterface cq = messageStore.getConsumeQueue(TIMER_TOPIC, 0); + if (null == cq) { + return; + } + if (readOffset.get() < cq.getMinOffsetInQueue()) { + logError.warn("scanSysTimerTopic readOffset: {} is smaller than minOffsetInQueue: {}, use minOffsetInQueue to scan timer sysTimerTopic", readOffset.get(), cq.getMinOffsetInQueue()); + readOffset.set(cq.getMinOffsetInQueue()); + } else if (readOffset.get() > cq.getMaxOffsetInQueue()) { + logError.warn("scanSysTimerTopic readOffset: {} is bigger than maxOffsetInQueue: {}, use maxOffsetInQueue to scan timer sysTimerTopic", readOffset.get(), cq.getMaxOffsetInQueue()); + readOffset.set(cq.getMaxOffsetInQueue()); + } + ReferredIterator iterator = null; + try { + iterator = cq.iterateFrom(readOffset.get()); + if (null == iterator) { + return; + } + while (iterator.hasNext()) { + perfCounterTicks.startTick(SCAN_SYS_TOPIC); + try { + CqUnit cqUnit = iterator.next(); + if (null == cqUnit) { + logError.error("scanSysTimerTopic cqUnit is null, readOffset: {}", readOffset.get()); + break; + } + long offsetPy = cqUnit.getPos(); + int sizePy = cqUnit.getSize(); + long queueOffset = cqUnit.getQueueOffset(); + MessageExt msgExt = getMessageByCommitOffset(offsetPy, sizePy); + if (null == msgExt) { + perfCounterTicks.getCounter(SCAN_SYS_TOPIC_MISS); + break; + } + Long delayedTime = getDelayTime(msgExt); + if (null == delayedTime) { + readOffset.incrementAndGet(); + continue; + } + if (isExpired(delayedTime)) { + putRealTopicMessage(msgExt); + readOffset.incrementAndGet(); + continue; + } + TimerRocksDBRecord timerRecord = new TimerRocksDBRecord(delayedTime, MessageClientIDSetter.getUniqID(msgExt), offsetPy, sizePy, queueOffset, msgExt); + timeline.putRecord(timerRecord); + readOffset.incrementAndGet(); + + StoreMetricsManager metricsManager = messageStore.getStoreMetricsManager(); + if (metricsManager instanceof DefaultStoreMetricsManager) { + DefaultStoreMetricsManager defaultMetricsManager = (DefaultStoreMetricsManager)metricsManager; + Attributes attributes = defaultMetricsManager.newAttributesBuilder().put(DefaultStoreMetricsConstant.LABEL_TOPIC, msgExt.getProperty(MessageConst.PROPERTY_REAL_TOPIC)).build(); + defaultMetricsManager.getTimerMessageSetLatency().record((delayedTime - msgExt.getBornTimestamp()) / 1000, attributes); + } + } catch (Exception e) { + logError.error("Unknown error in scan the system topic error: {}", e.getMessage()); + } finally { + perfCounterTicks.endTick(SCAN_SYS_TOPIC); + } + } + } catch (Exception e) { + logError.error("scanSysTimerTopic throw Unknown exception. {}", e.getMessage()); + } finally { + if (iterator != null) { + iterator.release(); + } + } + } + } + + private class TimerMessageReputService extends ServiceThread { + private final Logger log = TimerMessageRocksDBStore.log; + private final BlockingQueue> queue; + private final RateLimiter rateLimiter; + private final boolean writeCheckPoint; + ExecutorService executor = new ThreadPoolExecutor( + 6, + 6, + 60, + TimeUnit.SECONDS, + new LinkedBlockingQueue<>(10000), + new ThreadPoolExecutor.CallerRunsPolicy() + ); + + public TimerMessageReputService(BlockingQueue> queue, double maxTps, boolean writeCheckPoint) { + this.queue = queue; + this.rateLimiter = RateLimiter.create(maxTps); + this.writeCheckPoint = writeCheckPoint; + } + + @Override + public String getServiceName() { + return getServiceThreadName() + this.getClass().getSimpleName(); + } + + @Override + public void run() { + log.info(this.getServiceName() + " service start"); + while (!this.isStopped() || !queue.isEmpty()) { + try { + List trs = queue.poll(100L, TimeUnit.MILLISECONDS); + if (CollectionUtils.isEmpty(trs)) { + continue; + } + long start = System.currentTimeMillis(); + CountDownLatch countDownLatch = new CountDownLatch(trs.size()); + for (TimerRocksDBRecord record : trs) { + executor.submit(new Task(countDownLatch, record)); + } + countDownLatch.await(); + log.info("TimerMessageReputService reput messages to commitlog, cost: {}, trs size: {}, checkPoint: {}", System.currentTimeMillis() - start, trs.size(), trs.get(trs.size() - 1).getCheckPoint()); + if (this.writeCheckPoint && !CollectionUtils.isEmpty(trs) && trs.get(trs.size() - 1).getCheckPoint() > 0L) { + log.info("TimerMessageReputService reput messages to commitlog, checkPoint: {}", trs.get(trs.size() - 1).getCheckPoint()); + messageRocksDBStorage.writeCheckPointForTimer(TIMER_COLUMN_FAMILY, MessageRocksDBStorage.TIMELINE_CHECK_POINT, trs.get(trs.size() - 1).getCheckPoint()); + } + } catch (Exception e) { + logError.error("TimerMessageReputService error: {}", e.getMessage()); + } + } + log.info(this.getServiceName() + " service end"); + } + + private void putMsgWithRetry(MessageExtBrokerInner msg) throws InterruptedException { + if (null == msg) { + return; + } + for (int retryCount = 0; !isStopped() && retryCount <= MAX_PUT_MSG_TIMES; retryCount++) { + int result = doPut(msg); + switch (result) { + case PUT_OK: + return; + case PUT_NO_RETRY: + logError.warn("Skipping message due to unrecoverable error. Msg: {}", msg); + return; + default: + if (retryCount == MAX_PUT_MSG_TIMES) { + logError.error("Message processing failed after {} retries. Msg: {}", retryCount, msg); + return; + } else { + Thread.sleep(100L); + logError.warn("Retrying to process message. Retry count: {}, Msg: {}", retryCount, msg); + } + } + } + } + + class Task implements Callable { + private CountDownLatch countDownLatch; + private TimerRocksDBRecord record; + + public Task(CountDownLatch countDownLatch, TimerRocksDBRecord record) { + this.countDownLatch = countDownLatch; + this.record = record; + } + + @Override + public Void call() throws Exception { + try { + perfCounterTicks.startTick(OUT_BIZ_MESSAGE); + MessageExt messageExt = record.getMessageExt(); + if (null == messageExt) { + messageExt = getMessageByCommitOffset(record.getOffsetPy(), record.getSizePy()); + if (null == messageExt) { + return null; + } + } + MessageExtBrokerInner msg = convertMessage(messageExt); + if (null == msg) { + return null; + } + record.setUniqKey(MessageClientIDSetter.getUniqID(msg)); + putMsgWithRetry(msg); + timeline.addMetric(msg, -1); + perfCounterTicks.endTick(OUT_BIZ_MESSAGE); + rateLimiter.acquire(); + } catch (Exception e) { + logError.error("TimerMessageReputService running error: {}", e.getMessage()); + } finally { + countDownLatch.countDown(); + } + return null; + } + } + } + +} diff --git a/store/src/main/java/org/apache/rocketmq/store/timer/rocksdb/TimerRocksDBRecord.java b/store/src/main/java/org/apache/rocketmq/store/timer/rocksdb/TimerRocksDBRecord.java new file mode 100644 index 00000000000..8b9123a63b5 --- /dev/null +++ b/store/src/main/java/org/apache/rocketmq/store/timer/rocksdb/TimerRocksDBRecord.java @@ -0,0 +1,177 @@ +/* + * 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.timer.rocksdb; + +import java.nio.ByteBuffer; +import java.nio.charset.StandardCharsets; +import org.apache.commons.lang3.StringUtils; +import org.apache.rocketmq.common.constant.LoggerName; +import org.apache.rocketmq.common.message.MessageExt; +import org.apache.rocketmq.logging.org.slf4j.Logger; +import org.apache.rocketmq.logging.org.slf4j.LoggerFactory; + +public class TimerRocksDBRecord { + private static final Logger logError = LoggerFactory.getLogger(LoggerName.STORE_ERROR_LOGGER_NAME); + public static final byte TIMER_ROCKSDB_PUT = (byte)0; + public static final byte TIMER_ROCKSDB_DELETE = (byte)1; + public static final byte TIMER_ROCKSDB_UPDATE = (byte)2; + private static final int VALUE_LENGTH = Integer.BYTES + Long.BYTES; + + private long delayTime; + private String uniqKey; + private int sizePy; + private long offsetPy; + private long queueOffset; + private long checkPoint; + private byte actionFlag; + private MessageExt messageExt; + + public TimerRocksDBRecord() {} + + public TimerRocksDBRecord(long delayTime, String uniqKey, long offsetPy, int sizePy, long queueOffset, MessageExt messageExt) { + this.delayTime = delayTime; + this.uniqKey = uniqKey; + this.offsetPy = offsetPy; + this.sizePy = sizePy; + this.messageExt = messageExt; + this.queueOffset = queueOffset; + } + + public byte[] getKeyBytes() { + if (StringUtils.isEmpty(uniqKey) || delayTime <= 0L) { + return null; + } + try { + byte[] uniqKeyBytes = uniqKey.getBytes(StandardCharsets.UTF_8); + int keyLength = Long.BYTES + uniqKeyBytes.length; + return ByteBuffer.allocate(keyLength).putLong(delayTime).put(uniqKeyBytes).array(); + } catch (Exception e) { + logError.error("TimerRocksDBRecord getKeyBytes error: {}", e.getMessage()); + return null; + } + } + + public byte[] getValueBytes() { + if (sizePy <= 0 || offsetPy < 0L) { + return null; + } + try { + return ByteBuffer.allocate(VALUE_LENGTH).putInt(sizePy).putLong(offsetPy).array(); + } catch (Exception e) { + logError.error("TimerRocksDBRecord getValueBytes error: {}", e.getMessage()); + return null; + } + } + + public static TimerRocksDBRecord decode(byte[] key, byte[] value) { + if (null == key || key.length < Long.BYTES || null == value || value.length != VALUE_LENGTH) { + return null; + } + try { + TimerRocksDBRecord rocksDBRecord = new TimerRocksDBRecord(); + ByteBuffer keyBuffer = ByteBuffer.wrap(key); + rocksDBRecord.setDelayTime(keyBuffer.getLong()); + byte[] uniqKey = new byte[key.length - Long.BYTES]; + keyBuffer.get(uniqKey); + rocksDBRecord.setUniqKey(new String(uniqKey, StandardCharsets.UTF_8)); + ByteBuffer valueByteBuffer = ByteBuffer.wrap(value); + rocksDBRecord.setSizePy(valueByteBuffer.getInt()); + rocksDBRecord.setOffsetPy(valueByteBuffer.getLong()); + return rocksDBRecord; + } catch (Exception e) { + logError.error("TimerRocksDBRecord decode error: {}", e.getMessage()); + return null; + } + } + + public void setDelayTime(long delayTime) { + this.delayTime = delayTime; + } + + public void setOffsetPy(long offsetPy) { + this.offsetPy = offsetPy; + } + + public void setSizePy(int sizePy) { + this.sizePy = sizePy; + } + + public int getSizePy() { + return sizePy; + } + + public long getDelayTime() { + return delayTime; + } + + public long getOffsetPy() { + return offsetPy; + } + + public MessageExt getMessageExt() { + return messageExt; + } + + public void setMessageExt(MessageExt messageExt) { + this.messageExt = messageExt; + } + + public String getUniqKey() { + return uniqKey; + } + + public void setUniqKey(String uniqKey) { + this.uniqKey = uniqKey; + } + + public void setCheckPoint(long checkPoint) { + this.checkPoint = checkPoint; + } + + public long getCheckPoint() { + return checkPoint; + } + + public long getQueueOffset() { + return queueOffset; + } + + public void setQueueOffset(long queueOffset) { + this.queueOffset = queueOffset; + } + + public byte getActionFlag() { + return actionFlag; + } + + public void setActionFlag(byte actionFlag) { + this.actionFlag = actionFlag; + } + + @Override + public String toString() { + return "TimerRocksDBRecord{" + + "delayTime=" + delayTime + + ", uniqKey=" + uniqKey + + ", sizePy=" + sizePy + + ", offsetPy=" + offsetPy + + ", queueOffset=" + queueOffset + + ", checkPoint=" + checkPoint + + '}'; + } + +} diff --git a/store/src/main/java/org/apache/rocketmq/store/transaction/TransMessageRocksDBStore.java b/store/src/main/java/org/apache/rocketmq/store/transaction/TransMessageRocksDBStore.java new file mode 100644 index 00000000000..d71227c4af3 --- /dev/null +++ b/store/src/main/java/org/apache/rocketmq/store/transaction/TransMessageRocksDBStore.java @@ -0,0 +1,344 @@ +/* + * 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.transaction; +import java.net.SocketAddress; +import java.nio.ByteBuffer; +import java.util.ArrayList; +import java.util.List; +import java.util.concurrent.BlockingQueue; +import java.util.concurrent.LinkedBlockingDeque; +import java.util.concurrent.TimeUnit; +import org.apache.commons.collections.CollectionUtils; +import org.apache.commons.lang3.StringUtils; +import org.apache.rocketmq.common.ServiceThread; +import org.apache.rocketmq.common.constant.LoggerName; +import org.apache.rocketmq.common.message.MessageAccessor; +import org.apache.rocketmq.common.message.MessageClientIDSetter; +import org.apache.rocketmq.common.message.MessageConst; +import org.apache.rocketmq.common.message.MessageDecoder; +import org.apache.rocketmq.common.message.MessageExt; +import org.apache.rocketmq.common.message.MessageExtBrokerInner; +import org.apache.rocketmq.common.topic.TopicValidator; +import org.apache.rocketmq.logging.org.slf4j.Logger; +import org.apache.rocketmq.logging.org.slf4j.LoggerFactory; +import org.apache.rocketmq.store.DefaultMessageStore; +import org.apache.rocketmq.store.DispatchRequest; +import org.apache.rocketmq.store.MessageStore; +import org.apache.rocketmq.store.PutMessageResult; +import org.apache.rocketmq.store.PutMessageStatus; +import org.apache.rocketmq.store.StoreUtil; +import org.apache.rocketmq.store.config.MessageStoreConfig; +import org.apache.rocketmq.store.rocksdb.MessageRocksDBStorage; +import org.apache.rocketmq.store.stats.BrokerStatsManager; +import static org.apache.rocketmq.store.rocksdb.MessageRocksDBStorage.TRANS_COLUMN_FAMILY; + +public class TransMessageRocksDBStore { + private static final Logger log = LoggerFactory.getLogger(LoggerName.STORE_LOGGER_NAME); + private static final Logger logError = LoggerFactory.getLogger(LoggerName.STORE_ERROR_LOGGER_NAME); + private static final String REMOVE_TAG = "d"; + private static final byte[] FILL_BYTE = new byte[] {(byte) 0}; + private static final int DEFAULT_CAPACITY = 100000; + private static final int BATCH_SIZE = 1000; + private static final int MAX_GET_MSG_TIMES = 3; + private static final int INITIAL = 0, RUNNING = 1, SHUTDOWN = 2; + private volatile int state = INITIAL; + + private final MessageStore messageStore; + private final MessageStoreConfig storeConfig; + private final MessageRocksDBStorage messageRocksDBStorage; + private final BrokerStatsManager brokerStatsManager; + private final SocketAddress storeHost; + private ThreadLocal bufferLocal = null; + private TransIndexBuildService transIndexBuildService; + protected BlockingQueue originTransMsgQueue; + + public TransMessageRocksDBStore(final MessageStore messageStore, final BrokerStatsManager brokerStatsManager, final SocketAddress storeHost) { + this.messageStore = messageStore; + this.storeConfig = messageStore.getMessageStoreConfig(); + this.messageRocksDBStorage = messageStore.getMessageRocksDBStorage(); + this.brokerStatsManager = brokerStatsManager; + this.storeHost = storeHost; + bufferLocal = ThreadLocal.withInitial(() -> ByteBuffer.allocate(storeConfig.getMaxMessageSize())); + if (storeConfig.isTransRocksDBEnable()) { + init(); + } + } + + private void init() { + if (this.state == RUNNING) { + return; + } + this.transIndexBuildService = new TransIndexBuildService(); + this.originTransMsgQueue = new LinkedBlockingDeque<>(DEFAULT_CAPACITY); + this.transIndexBuildService.start(); + this.state = RUNNING; + Long lastOffsetPy = messageRocksDBStorage.getLastOffsetPy(TRANS_COLUMN_FAMILY); + log.info("TransMessageRocksDBStore start success, lastOffsetPy: {}", lastOffsetPy); + } + + public void shutdown() { + if (this.state != RUNNING || this.state == SHUTDOWN) { + return; + } + if (null != this.transIndexBuildService) { + this.transIndexBuildService.shutdown(); + } + this.state = SHUTDOWN; + log.info("TransMessageRocksDBStore shutdown success"); + } + + public void buildTransIndex(DispatchRequest dispatchRequest) { + if (null == dispatchRequest || dispatchRequest.getCommitLogOffset() < 0L || dispatchRequest.getMsgSize() <= 0 || state != RUNNING || null == this.originTransMsgQueue) { + logError.error("TransMessageRocksDBStore buildTransIndex error, dispatchRequest: {}, state: {}, originTransMsgQueue: {}", dispatchRequest, state, originTransMsgQueue); + return; + } + long reqOffsetPy = dispatchRequest.getCommitLogOffset(); + long endOffsetPy = messageRocksDBStorage.getLastOffsetPy(TRANS_COLUMN_FAMILY); + if (reqOffsetPy < endOffsetPy) { + if (System.currentTimeMillis() % 1000 == 0) { + log.warn("TransMessageRocksDBStore buildTransIndex recover, but ignore, reqOffsetPy: {}, endOffsetPy: {}", reqOffsetPy, endOffsetPy); + } + return; + } + int reqMsgSize = dispatchRequest.getMsgSize(); + try { + MessageExt msgInner = getMessage(reqOffsetPy, reqMsgSize); + if (null == msgInner) { + logError.error("TransMessageRocksDBStore buildTransIndex error, msgInner is not found, reqOffsetPy: {}, reqMsgSize: {}", reqOffsetPy, reqMsgSize); + return; + } + String topic = msgInner.getUserProperty(MessageConst.PROPERTY_REAL_TOPIC); + String uniqKey = msgInner.getUserProperty(MessageConst.PROPERTY_TRANSACTION_ID); + if (StringUtils.isEmpty(topic) || StringUtils.isEmpty(uniqKey)) { + logError.error("TransMessageRocksDBStore buildTransIndex error, uniqKey: {}, topic: {}", uniqKey, topic); + return; + } + TransRocksDBRecord transRocksDBRecord = null; + String reqTopic = dispatchRequest.getTopic(); + if (TopicValidator.RMQ_SYS_ROCKSDB_TRANS_HALF_TOPIC.equals(reqTopic)) { + transRocksDBRecord = new TransRocksDBRecord(reqOffsetPy, topic, uniqKey, reqMsgSize, 0); + } else if (TopicValidator.RMQ_SYS_ROCKSDB_TRANS_OP_HALF_TOPIC.equals(reqTopic)) { + long offsetPy = -1L; + String transOffsetPy = null; + try { + transOffsetPy = msgInner.getUserProperty(MessageConst.PROPERTY_TRANS_OFFSET); + if (!StringUtils.isEmpty(transOffsetPy)) { + offsetPy = Long.parseLong(transOffsetPy); + } + if (offsetPy >= 0L) { + transRocksDBRecord = new TransRocksDBRecord(offsetPy, topic, uniqKey, true); + } + } catch (Exception e) { + logError.error("TransMessageRocksDBStore buildTransIndex error, transOffsetPy: {}, error: {}", transOffsetPy, e.getMessage()); + } + } + if (null != transRocksDBRecord) { + while (!originTransMsgQueue.offer(transRocksDBRecord, 3, TimeUnit.SECONDS)) { + if (System.currentTimeMillis() % 1000 == 0) { + logError.error("TransMessageRocksDBStore buildTransStatus offer transRocksDBRecord error, topic: {}, uniqKey: {}, reqOffsetPy: {}", topic, uniqKey, reqOffsetPy); + } + } + } + } catch (Exception e) { + logError.error("TransMessageRocksDBStore buildTransStatus error: {}", e.getMessage()); + } + } + + public void deletePrepareMessage(MessageExt messageExt) { + if (null == messageExt) { + logError.error("TransMessageRocksDBStore deletePrepareMessage error, messageExt is null"); + return; + } + try { + MessageExtBrokerInner msgInner = makeOpMessageInner(messageExt); + if (null == msgInner) { + logError.error("TransMessageRocksDBStore deletePrepareMessage msgInner is null"); + return; + } + PutMessageResult result = messageStore.putMessage(msgInner); + if (result != null && result.getPutMessageStatus() == PutMessageStatus.PUT_OK) { + this.brokerStatsManager.incTopicPutNums(msgInner.getTopic()); + this.brokerStatsManager.incTopicPutSize(msgInner.getTopic(), result.getAppendMessageResult().getWroteBytes()); + this.brokerStatsManager.incBrokerPutNums(); + return; + } + logError.error("TransMessageRocksDBStore deletePrepareMessage put op msg failed, result: {}", result); + } catch (Exception e) { + logError.error("TransMessageRocksDBStore deletePrepareMessage error: {}", e.getMessage()); + } + } + + public MessageExt getMessage(long offsetPy, int sizePy) { + if (offsetPy < 0L || sizePy <= 0 || sizePy > storeConfig.getMaxMessageSize()) { + logError.error("TransMessageRocksDBStore getMessage param error, offsetPy: {}, sizePy: {}, maxMsgSizeConfig: {}", offsetPy, sizePy, storeConfig.getMaxMessageSize()); + return null; + } + ByteBuffer byteBuffer = bufferLocal.get(); + if (sizePy > byteBuffer.limit()) { + bufferLocal.remove(); + byteBuffer = ByteBuffer.allocate(sizePy); + bufferLocal.set(byteBuffer); + } + for (int i = 0; i < MAX_GET_MSG_TIMES; i++) { + try { + MessageExt msgExt = StoreUtil.getMessage(offsetPy, sizePy, messageStore, bufferLocal.get()); + if (null == msgExt) { + log.warn("Fail to read msg from commitLog offsetPy:{} sizePy:{}", offsetPy, sizePy); + } else { + return msgExt; + } + } catch (Exception e) { + logError.error("TransMessageRocksDBStore getMessage error, offsetPy: {}, sizePy: {}, error: {}", offsetPy, sizePy, e.getMessage()); + } + } + return null; + } + + public MessageRocksDBStorage getMessageRocksDBStorage() { + return messageRocksDBStorage; + } + + private MessageExtBrokerInner makeOpMessageInner(MessageExt messageExt) { + if (null == messageExt) { + logError.error("TransMessageRocksDBStore makeOpMessageInner messageExt is null"); + return null; + } + try { + MessageExtBrokerInner msgInner = new MessageExtBrokerInner(); + msgInner.setTopic(TopicValidator.RMQ_SYS_ROCKSDB_TRANS_OP_HALF_TOPIC); + msgInner.setBody(FILL_BYTE); + msgInner.setQueueId(0); + msgInner.setTags(REMOVE_TAG); + msgInner.setTagsCode(MessageExtBrokerInner.tagsString2tagsCode(msgInner.getTags())); + msgInner.setSysFlag(0); + msgInner.setBornTimestamp(System.currentTimeMillis()); + msgInner.setBornHost(this.storeHost); + msgInner.setStoreHost(this.storeHost); + msgInner.setWaitStoreMsgOK(false); + MessageClientIDSetter.setUniqID(msgInner); + String uniqKey = MessageClientIDSetter.getUniqID(messageExt); + if (!StringUtils.isEmpty(uniqKey)) { + MessageAccessor.putProperty(msgInner, MessageConst.PROPERTY_TRANSACTION_ID, uniqKey); + } + MessageAccessor.putProperty(msgInner, MessageConst.PROPERTY_TRANS_OFFSET, String.valueOf(messageExt.getCommitLogOffset())); + MessageAccessor.putProperty(msgInner, MessageConst.PROPERTY_REAL_TOPIC, messageExt.getUserProperty(MessageConst.PROPERTY_REAL_TOPIC)); + msgInner.setPropertiesString(MessageDecoder.messageProperties2String(msgInner.getProperties())); + return msgInner; + } catch (Exception e) { + logError.error("TransMessageRocksDBStore makeOpMessageInner error: {}", e.getMessage()); + return null; + } + } + + public Integer getCheckTimes(String topic, String uniqKey, Long offsetPy) { + if (StringUtils.isEmpty(topic) || StringUtils.isEmpty(uniqKey) || null == offsetPy || offsetPy < 0L) { + return null; + } + try { + TransRocksDBRecord record = messageRocksDBStorage.getRecordForTrans(TRANS_COLUMN_FAMILY, new TransRocksDBRecord(offsetPy, topic, uniqKey, false)); + if (null == record) { + return null; + } + return record.getCheckTimes(); + } catch (Exception e) { + logError.error("TransMessageRocksDBStore getCheckTimes error, topic: {}, uniqKey: {}, offsetPy: {}, error: {}", topic, uniqKey, offsetPy, e.getMessage()); + return null; + } + } + + public boolean isMappedFileMatchedRecover(long phyOffset) { + if (!storeConfig.isTransRocksDBEnable()) { + return true; + } + Long lastOffsetPy = messageRocksDBStorage.getLastOffsetPy(TRANS_COLUMN_FAMILY); + log.info("trans isMappedFileMatchedRecover lastOffsetPy: {}", lastOffsetPy); + if (null != lastOffsetPy && phyOffset <= lastOffsetPy) { + log.info("isMappedFileMatchedRecover TransMessageRocksDBStore recover form this offset, phyOffset: {}, lastOffsetPy: {}", phyOffset, lastOffsetPy); + return true; + } + return false; + } + + private String getServiceThreadName() { + String brokerIdentifier = ""; + if (TransMessageRocksDBStore.this.messageStore instanceof DefaultMessageStore) { + DefaultMessageStore messageStore = (DefaultMessageStore) TransMessageRocksDBStore.this.messageStore; + if (messageStore.getBrokerConfig().isInBrokerContainer()) { + brokerIdentifier = messageStore.getBrokerConfig().getIdentifier(); + } + } + return brokerIdentifier; + } + + public class TransIndexBuildService extends ServiceThread { + private final Logger log = TransMessageRocksDBStore.log; + private List trs; + @Override + public String getServiceName() { + return getServiceThreadName() + this.getClass().getSimpleName(); + } + + @Override + public void run() { + log.info(this.getServiceName() + "service start"); + trs = new ArrayList<>(BATCH_SIZE); + while (!this.isStopped() || !originTransMsgQueue.isEmpty()) { + try { + buildTransIndex(); + } catch (Exception e) { + trs.clear(); + logError.error("TransMessageRocksDBStore error occurred in: {}, error: {}", getServiceName(), e.getMessage()); + } + } + log.info(this.getServiceName() + " service end"); + } + + protected void buildTransIndex() { + pollTransMessageRecords(); + if (CollectionUtils.isEmpty(trs)) { + return; + } + try { + messageRocksDBStorage.writeRecordsForTrans(TRANS_COLUMN_FAMILY, trs); + } catch (Exception e) { + logError.error("TransMessageRocksDBStore pollAndPutTransRequest writeRecords error: {}", e.getMessage()); + } + trs.clear(); + } + + protected void pollTransMessageRecords() { + try { + TransRocksDBRecord firstReq = originTransMsgQueue.poll(100, TimeUnit.MILLISECONDS); + if (null != firstReq) { + trs.add(firstReq); + while (true) { + TransRocksDBRecord tmpReq = originTransMsgQueue.poll(100, TimeUnit.MILLISECONDS); + if (null == tmpReq) { + break; + } + trs.add(tmpReq); + if (trs.size() >= BATCH_SIZE) { + break; + } + } + } + } catch (Exception e) { + logError.error("TransMessageRocksDBStore fetchTransMessageRecord error: {}", e.getMessage()); + } + } + } +} diff --git a/store/src/main/java/org/apache/rocketmq/store/transaction/TransRocksDBRecord.java b/store/src/main/java/org/apache/rocketmq/store/transaction/TransRocksDBRecord.java new file mode 100644 index 00000000000..099f6150939 --- /dev/null +++ b/store/src/main/java/org/apache/rocketmq/store/transaction/TransRocksDBRecord.java @@ -0,0 +1,165 @@ +/* + * 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.transaction; +import java.nio.ByteBuffer; +import java.nio.charset.StandardCharsets; +import org.apache.commons.lang3.StringUtils; +import org.apache.rocketmq.common.constant.LoggerName; +import org.apache.rocketmq.common.message.MessageExt; +import org.apache.rocketmq.logging.org.slf4j.Logger; +import org.apache.rocketmq.logging.org.slf4j.LoggerFactory; + +public class TransRocksDBRecord { + private static final Logger logError = LoggerFactory.getLogger(LoggerName.STORE_ERROR_LOGGER_NAME); + public static final int VALUE_LENGTH = Integer.BYTES + Integer.BYTES; + private static final String KEY_SPLIT = "@"; + protected long offsetPy; + private String topic; + private String uniqKey; + private int checkTimes = 0; + private int sizePy; + private boolean isOp; + private boolean delete; + private MessageExt messageExt; + + public TransRocksDBRecord(long offsetPy, String topic, String uniqKey, int sizePy, int checkTimes) { + this.offsetPy = offsetPy; + this.topic = topic; + this.uniqKey = uniqKey; + this.sizePy = sizePy; + this.checkTimes = checkTimes; + } + + public TransRocksDBRecord(long offsetPy, String topic, String uniqKey, boolean isOp) { + this.offsetPy = offsetPy; + this.topic = topic; + this.uniqKey = uniqKey; + this.isOp = isOp; + } + + public TransRocksDBRecord() {} + + public byte[] getKeyBytes() { + if (offsetPy < 0L || StringUtils.isEmpty(topic) || StringUtils.isEmpty(uniqKey)) { + return null; + } + byte[] keySuffixBytes = (KEY_SPLIT + topic + KEY_SPLIT + uniqKey).getBytes(StandardCharsets.UTF_8); + int keyLength = Long.BYTES + keySuffixBytes.length; + return ByteBuffer.allocate(keyLength).putLong(offsetPy).put(keySuffixBytes).array(); + } + + public byte[] getValueBytes() { + if (checkTimes < 0 || sizePy <= 0) { + logError.error("TransRocksDBRecord getValueBytes error, checkTimes: {}, sizePy: {}", checkTimes, sizePy); + return null; + } + return ByteBuffer.allocate(VALUE_LENGTH).putInt(checkTimes).putInt(sizePy).array(); + } + + public static TransRocksDBRecord decode(byte[] key, byte[] value) { + if (null == key || key.length <= Long.BYTES || null == value || value.length != VALUE_LENGTH) { + logError.error("TransRocksDBRecord decode param error, key: {}, value: {}", key, value); + return null; + } + TransRocksDBRecord transRocksDBRecord = null; + try { + transRocksDBRecord = new TransRocksDBRecord(); + ByteBuffer keyByteBuffer = ByteBuffer.wrap(key); + transRocksDBRecord.setOffsetPy(keyByteBuffer.getLong()); + byte[] keySuffix = new byte[key.length - Long.BYTES]; + keyByteBuffer.get(keySuffix); + String[] keySuffixSplit = new String(keySuffix, StandardCharsets.UTF_8).split(KEY_SPLIT); + if (keySuffixSplit.length != 3) { + logError.error("TransRocksDBRecord decode keySuffixSplit parse error"); + return null; + } + transRocksDBRecord.setTopic(keySuffixSplit[1]); + transRocksDBRecord.setUniqKey(keySuffixSplit[2]); + ByteBuffer valueByteBuffer = ByteBuffer.wrap(value); + transRocksDBRecord.setCheckTimes(valueByteBuffer.getInt()); + transRocksDBRecord.setSizePy(valueByteBuffer.getInt()); + } catch (Exception e) { + logError.error("TransRocksDBRecord decode error, valueLength: {}, error: {}", value.length, e.getMessage()); + return null; + } + return transRocksDBRecord; + } + + public String getTopic() { + return topic; + } + + public void setTopic(String topic) { + this.topic = topic; + } + + public String getUniqKey() { + return uniqKey; + } + + public void setUniqKey(String uniqKey) { + this.uniqKey = uniqKey; + } + + public int getCheckTimes() { + return checkTimes; + } + + public void setCheckTimes(int checkTimes) { + this.checkTimes = checkTimes; + } + + public int getSizePy() { + return sizePy; + } + + public void setSizePy(int sizePy) { + this.sizePy = sizePy; + } + + public long getOffsetPy() { + return offsetPy; + } + + public void setOffsetPy(long offsetPy) { + this.offsetPy = offsetPy; + } + + public MessageExt getMessageExt() { + return messageExt; + } + + public void setMessageExt(MessageExt messageExt) { + this.messageExt = messageExt; + } + + public boolean isOp() { + return isOp; + } + + public void setOp(boolean op) { + isOp = op; + } + + public boolean isDelete() { + return delete; + } + + public void setDelete(boolean delete) { + this.delete = delete; + } +} diff --git a/store/src/test/java/org/apache/rocketmq/store/dledger/MixCommitlogTest.java b/store/src/test/java/org/apache/rocketmq/store/dledger/MixCommitlogTest.java index 1bfc6f72eaa..93503c4ebd5 100644 --- a/store/src/test/java/org/apache/rocketmq/store/dledger/MixCommitlogTest.java +++ b/store/src/test/java/org/apache/rocketmq/store/dledger/MixCommitlogTest.java @@ -25,12 +25,14 @@ import org.apache.rocketmq.store.config.StorePathConfigHelper; import org.junit.Assert; import org.junit.Assume; +import org.junit.Ignore; import org.junit.Test; import static org.awaitility.Awaitility.await; public class MixCommitlogTest extends MessageStoreTestBase { + @Ignore @Test public void testFallBehindCQ() throws Exception { Assume.assumeFalse(MixAll.isWindows()); diff --git a/tieredstore/src/main/java/org/apache/rocketmq/tieredstore/TieredMessageStore.java b/tieredstore/src/main/java/org/apache/rocketmq/tieredstore/TieredMessageStore.java index 9d56c4a8daa..68ae8acc9ec 100644 --- a/tieredstore/src/main/java/org/apache/rocketmq/tieredstore/TieredMessageStore.java +++ b/tieredstore/src/main/java/org/apache/rocketmq/tieredstore/TieredMessageStore.java @@ -42,6 +42,9 @@ import org.apache.rocketmq.store.SelectMappedBufferResult; import org.apache.rocketmq.store.plugin.AbstractPluginMessageStore; import org.apache.rocketmq.store.plugin.MessageStorePluginContext; +import org.apache.rocketmq.store.rocksdb.MessageRocksDBStorage; +import org.apache.rocketmq.store.timer.rocksdb.TimerMessageRocksDBStore; +import org.apache.rocketmq.store.transaction.TransMessageRocksDBStore; import org.apache.rocketmq.tieredstore.core.MessageStoreDispatcher; import org.apache.rocketmq.tieredstore.core.MessageStoreDispatcherImpl; import org.apache.rocketmq.tieredstore.core.MessageStoreFetcher; @@ -76,6 +79,9 @@ public class TieredMessageStore extends AbstractPluginMessageStore { protected final MessageStoreFilter topicFilter; protected final MessageStoreFetcher fetcher; protected final MessageStoreDispatcher dispatcher; + protected final MessageRocksDBStorage messageRocksDBStorage; + protected TimerMessageRocksDBStore timerMessageRocksDBStore; + protected TransMessageRocksDBStore transMessageRocksDBStore; public TieredMessageStore(MessageStorePluginContext context, MessageStore next) { super(context, next); @@ -86,7 +92,7 @@ public TieredMessageStore(MessageStorePluginContext context, MessageStore next) this.storeConfig.setWriteWithoutMmap(context.getMessageStoreConfig().isWriteWithoutMmap()); this.brokerName = this.storeConfig.getBrokerName(); this.defaultStore = next; - + this.messageRocksDBStorage = defaultStore.getMessageRocksDBStorage(); this.metadataStore = this.getMetadataStore(this.storeConfig); this.topicFilter = new MessageStoreTopicFilter(this.storeConfig); this.storeExecutor = new MessageStoreExecutor(); @@ -307,6 +313,26 @@ public long getMinOffsetInQueue(String topic, int queueId) { return Math.min(minOffsetInNextStore, minOffsetInTieredStore); } + @Override + public TimerMessageRocksDBStore getTimerRocksDBStore() { + return timerMessageRocksDBStore; + } + + @Override + public TransMessageRocksDBStore getTransRocksDBStore() { + return transMessageRocksDBStore; + } + + @Override + public void setTimerMessageRocksDBStore(TimerMessageRocksDBStore timerMessageRocksDBStore) { + this.timerMessageRocksDBStore = timerMessageRocksDBStore; + } + + @Override + public void setTransRocksDBStore(TransMessageRocksDBStore transMessageRocksDBStore) { + this.transMessageRocksDBStore = transMessageRocksDBStore; + } + @Override public long getEarliestMessageTime(String topic, int queueId) { return getEarliestMessageTimeAsync(topic, queueId).join(); @@ -383,6 +409,12 @@ public QueryMessageResult queryMessage(String topic, String key, int maxNum, lon return queryMessageAsync(topic, key, maxNum, begin, end).join(); } + @Override + public QueryMessageResult queryMessage(String topic, String key, int maxNum, long begin, + long end, String keyType, String lastKey) { + return queryMessageAsync(topic, key, maxNum, begin, end, keyType, lastKey).join(); + } + @Override public CompletableFuture queryMessageAsync(String topic, String key, int maxNum, long begin, long end) { @@ -418,6 +450,38 @@ public CompletableFuture queryMessageAsync(String topic, Str return CompletableFuture.completedFuture(result); } + @Override + public CompletableFuture queryMessageAsync(String topic, String key, int maxNum, long begin, long end, String indexType, String lastKey) { + long earliestTimeInNextStore = next.getEarliestMessageTime(); + if (earliestTimeInNextStore <= 0) { + log.warn("TieredMessageStore queryMessageAsync: get earliest message time in next store failed: {}", earliestTimeInNextStore); + } + boolean isForce = storeConfig.getTieredStorageLevel() == MessageStoreConfig.TieredStorageLevel.FORCE; + QueryMessageResult result = end < earliestTimeInNextStore || isForce ? new QueryMessageResult() : next.queryMessage(topic, key, maxNum, begin, end, indexType, lastKey); + int resultSize = result.getMessageBufferList().size(); + if (resultSize < maxNum && begin < earliestTimeInNextStore || isForce) { + Stopwatch stopwatch = Stopwatch.createStarted(); + try { + return fetcher.queryMessageAsync(topic, key, maxNum - resultSize, begin, isForce ? end : earliestTimeInNextStore) + .thenApply(tieredStoreResult -> { + Attributes latencyAttributes = TieredStoreMetricsManager.newAttributesBuilder() + .put(TieredStoreMetricsConstant.LABEL_OPERATION, TieredStoreMetricsConstant.OPERATION_API_QUERY_MESSAGE) + .put(TieredStoreMetricsConstant.LABEL_TOPIC, topic) + .build(); + TieredStoreMetricsManager.apiLatency.record(stopwatch.elapsed(TimeUnit.MILLISECONDS), latencyAttributes); + for (SelectMappedBufferResult msg : tieredStoreResult.getMessageMapedList()) { + result.addMessage(msg); + } + return result; + }); + } catch (Exception e) { + log.error("TieredMessageStore#queryMessageAsync: query message in tiered store failed", e); + return CompletableFuture.completedFuture(result); + } + } + return CompletableFuture.completedFuture(result); + } + @Override public List> getMetricsView() { List> res = super.getMetricsView(); @@ -431,6 +495,11 @@ public void initMetrics(Meter meter, Supplier attributesBuild TieredStoreMetricsManager.init(meter, attributesBuilderSupplier, storeConfig, fetcher, flatFileStore, next); } + @Override + public MessageRocksDBStorage getMessageRocksDBStorage() { + return messageRocksDBStorage; + } + @Override public int cleanUnusedTopic(Set retainTopics) { metadataStore.iterateTopic(topicMetadata -> { diff --git a/tools/src/main/java/org/apache/rocketmq/tools/admin/DefaultMQAdminExt.java b/tools/src/main/java/org/apache/rocketmq/tools/admin/DefaultMQAdminExt.java index 2e72af13eed..d935fbc746b 100644 --- a/tools/src/main/java/org/apache/rocketmq/tools/admin/DefaultMQAdminExt.java +++ b/tools/src/main/java/org/apache/rocketmq/tools/admin/DefaultMQAdminExt.java @@ -160,6 +160,11 @@ public QueryResult queryMessage(String clusterName, String topic, String key, in return defaultMQAdminExtImpl.queryMessage(clusterName, topic, key, maxNum, begin, end); } + public QueryResult queryMessage(String clusterName, String topic, String key, int maxNum, long begin, long end, String keyType, String lastKey) + throws MQClientException, InterruptedException, RemotingException { + return defaultMQAdminExtImpl.queryMessage(clusterName, topic, key, maxNum, begin, end, keyType, lastKey); + } + @Override public void start() throws MQClientException { defaultMQAdminExtImpl.start(); @@ -986,4 +991,10 @@ public void exportPopRecords(String brokerAddr, long timeout) throws RemotingCon RemotingSendRequestException, RemotingTimeoutException, MQBrokerException, InterruptedException { defaultMQAdminExtImpl.exportPopRecords(brokerAddr, timeout); } + + @Override + public void switchTimerEngine(String brokerAddr, String desTimerEngine) throws RemotingConnectException, RemotingSendRequestException, RemotingTimeoutException, + UnsupportedEncodingException, InterruptedException, MQBrokerException { + defaultMQAdminExtImpl.switchTimerEngine(brokerAddr, desTimerEngine); + } } diff --git a/tools/src/main/java/org/apache/rocketmq/tools/admin/DefaultMQAdminExtImpl.java b/tools/src/main/java/org/apache/rocketmq/tools/admin/DefaultMQAdminExtImpl.java index f2e1cda2bd1..b64cafd4615 100644 --- a/tools/src/main/java/org/apache/rocketmq/tools/admin/DefaultMQAdminExtImpl.java +++ b/tools/src/main/java/org/apache/rocketmq/tools/admin/DefaultMQAdminExtImpl.java @@ -1738,9 +1738,12 @@ public QueryResult queryMessage(String topic, String key, int maxNum, long begin public QueryResult queryMessage(String clusterName, String topic, String key, int maxNum, long begin, long end) throws MQClientException, InterruptedException, RemotingException { - return this.mqClientInstance.getMQAdminImpl().queryMessage(clusterName, topic, key, maxNum, begin, end, false); + return this.mqClientInstance.getMQAdminImpl().queryMessage(clusterName, topic, key, maxNum, begin, end, false, MessageConst.INDEX_KEY_TYPE, null); } + public QueryResult queryMessage(String clusterName, String topic, String key, int maxNum, long begin, long end, String keyType, String lastKey) throws MQClientException, InterruptedException, RemotingException { + return this.mqClientInstance.getMQAdminImpl().queryMessage(clusterName, topic, key, maxNum, begin, end, false, keyType, lastKey); + } @Override public void updateConsumeOffset(String brokerAddr, String consumeGroup, MessageQueue mq, long offset) throws RemotingException, InterruptedException, MQBrokerException { @@ -2052,4 +2055,10 @@ public void exportPopRecords(String brokerAddr, long timeout) throws RemotingCon RemotingSendRequestException, RemotingTimeoutException, MQBrokerException, InterruptedException { this.mqClientInstance.getMQClientAPIImpl().exportPopRecord(brokerAddr, timeout); } + + @Override + public void switchTimerEngine(String brokerAddr, String desTimerEngine) throws RemotingConnectException, RemotingSendRequestException, RemotingTimeoutException, UnsupportedEncodingException, InterruptedException, MQBrokerException { + this.mqClientInstance.getMQClientAPIImpl().switchTimerEngine(brokerAddr, desTimerEngine, timeoutMillis); + } + } diff --git a/tools/src/main/java/org/apache/rocketmq/tools/admin/MQAdminExt.java b/tools/src/main/java/org/apache/rocketmq/tools/admin/MQAdminExt.java index 46e2c066cb4..0c433b6d91f 100644 --- a/tools/src/main/java/org/apache/rocketmq/tools/admin/MQAdminExt.java +++ b/tools/src/main/java/org/apache/rocketmq/tools/admin/MQAdminExt.java @@ -516,4 +516,6 @@ String setCommitLogReadAheadMode(final String brokerAddr, String mode) void exportPopRecords(String brokerAddr, long timeout) throws RemotingConnectException, RemotingSendRequestException, RemotingTimeoutException, MQBrokerException, InterruptedException; + + void switchTimerEngine(String brokerAddr, String desTimerEngine) throws RemotingConnectException, RemotingSendRequestException, RemotingTimeoutException, UnsupportedEncodingException, InterruptedException, MQBrokerException; } diff --git a/tools/src/main/java/org/apache/rocketmq/tools/command/MQAdminStartup.java b/tools/src/main/java/org/apache/rocketmq/tools/command/MQAdminStartup.java index b210e82b3c8..3513f48f320 100644 --- a/tools/src/main/java/org/apache/rocketmq/tools/command/MQAdminStartup.java +++ b/tools/src/main/java/org/apache/rocketmq/tools/command/MQAdminStartup.java @@ -49,6 +49,7 @@ import org.apache.rocketmq.tools.command.broker.RemoveColdDataFlowCtrGroupConfigSubCommand; import org.apache.rocketmq.tools.command.broker.ResetMasterFlushOffsetSubCommand; import org.apache.rocketmq.tools.command.broker.SendMsgStatusCommand; +import org.apache.rocketmq.tools.command.broker.SwitchTimerEngineSubCommand; import org.apache.rocketmq.tools.command.broker.UpdateBrokerConfigSubCommand; import org.apache.rocketmq.tools.command.broker.UpdateColdDataFlowCtrGroupConfigSubCommand; import org.apache.rocketmq.tools.command.cluster.CLusterSendMsgRTCommand; @@ -298,6 +299,7 @@ public static void initCommand() { initCommand(new CopyAclsSubCommand()); initCommand(new RocksDBConfigToJsonCommand()); initCommand(new CheckRocksdbCqWriteProgressCommand()); + initCommand(new SwitchTimerEngineSubCommand()); } private static void printHelp() { diff --git a/tools/src/main/java/org/apache/rocketmq/tools/command/broker/SwitchTimerEngineSubCommand.java b/tools/src/main/java/org/apache/rocketmq/tools/command/broker/SwitchTimerEngineSubCommand.java new file mode 100644 index 00000000000..fbddca1b967 --- /dev/null +++ b/tools/src/main/java/org/apache/rocketmq/tools/command/broker/SwitchTimerEngineSubCommand.java @@ -0,0 +1,101 @@ +/* + * 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.tools.command.broker; +import java.util.Set; +import org.apache.commons.cli.CommandLine; +import org.apache.commons.cli.Option; +import org.apache.commons.cli.Options; +import org.apache.commons.lang3.StringUtils; +import org.apache.rocketmq.common.message.MessageConst; +import org.apache.rocketmq.remoting.RPCHook; +import org.apache.rocketmq.srvutil.ServerUtil; +import org.apache.rocketmq.tools.admin.DefaultMQAdminExt; +import org.apache.rocketmq.tools.command.CommandUtil; +import org.apache.rocketmq.tools.command.SubCommand; +import org.apache.rocketmq.tools.command.SubCommandException; + +public class SwitchTimerEngineSubCommand implements SubCommand { + private static final String ROCKSDB_TIMELINE = "ROCKSDB_TIMELINE"; + private static final String FILE_TIME_WHEEL = "FILE_TIME_WHEEL"; + + @Override + public String commandName() { + return "switchTimerEngine"; + } + + @Override + public String commandDesc() { + return "switch the engine of timer message in broker"; + } + + @Override + public Options buildCommandlineOptions(Options options) { + Option opt = new Option("b", "brokerAddr", true, "update which broker"); + opt.setRequired(false); + options.addOption(opt); + + opt = new Option("c", "clusterName", true, "update which cluster"); + opt.setRequired(false); + options.addOption(opt); + + opt = new Option("e", "engineType", true, "R/F, R for rocksdb timeline engine, F for file time wheel engine"); + opt.setRequired(true); + options.addOption(opt); + return options; + } + + @Override + public void execute(CommandLine commandLine, Options options, RPCHook rpcHook) throws SubCommandException { + + DefaultMQAdminExt defaultMQAdminExt = new DefaultMQAdminExt(rpcHook); + defaultMQAdminExt.setInstanceName(Long.toString(System.currentTimeMillis())); + try { + String engineType = commandLine.getOptionValue('e').trim(); + if (StringUtils.isEmpty(engineType) || !MessageConst.TIMER_ENGINE_ROCKSDB_TIMELINE.equals(engineType) && !MessageConst.TIMER_ENGINE_FILE_TIME_WHEEL.equals(engineType)) { + System.out.print("switchTimerEngine engineType must be R or F\n"); + return; + } + if (commandLine.hasOption('b')) { + String brokerAddr = commandLine.getOptionValue('b').trim(); + defaultMQAdminExt.start(); + defaultMQAdminExt.switchTimerEngine(brokerAddr, engineType); + System.out.printf("switchTimerEngine to %s success, %s\n", engineType, brokerAddr); + return; + } else if (commandLine.hasOption('c')) { + String clusterName = commandLine.getOptionValue('c').trim(); + defaultMQAdminExt.start(); + Set masterSet = CommandUtil.fetchMasterAddrByClusterName(defaultMQAdminExt, clusterName); + for (String brokerAddr : masterSet) { + try { + defaultMQAdminExt.switchTimerEngine(brokerAddr, engineType); + String engineName = MessageConst.TIMER_ENGINE_ROCKSDB_TIMELINE.equals(engineType) ? ROCKSDB_TIMELINE : FILE_TIME_WHEEL; + System.out.printf("switchTimerEngine to %s success, %s\n", engineName, brokerAddr); + } catch (Exception e) { + e.printStackTrace(); + } + } + return; + } + ServerUtil.printCommandLineHelp("mqadmin " + this.commandName(), options); + } catch (Exception e) { + throw new SubCommandException(this.getClass().getSimpleName() + " command failed", e); + } finally { + defaultMQAdminExt.shutdown(); + } + + } +} diff --git a/tools/src/main/java/org/apache/rocketmq/tools/command/message/QueryMsgByKeySubCommand.java b/tools/src/main/java/org/apache/rocketmq/tools/command/message/QueryMsgByKeySubCommand.java index 02961c3bb50..ceef631c3f0 100644 --- a/tools/src/main/java/org/apache/rocketmq/tools/command/message/QueryMsgByKeySubCommand.java +++ b/tools/src/main/java/org/apache/rocketmq/tools/command/message/QueryMsgByKeySubCommand.java @@ -19,8 +19,11 @@ import org.apache.commons.cli.CommandLine; import org.apache.commons.cli.Option; import org.apache.commons.cli.Options; +import org.apache.commons.lang3.StringUtils; import org.apache.rocketmq.client.QueryResult; import org.apache.rocketmq.client.exception.MQClientException; +import org.apache.rocketmq.common.MixAll; +import org.apache.rocketmq.common.message.MessageConst; import org.apache.rocketmq.common.message.MessageExt; import org.apache.rocketmq.remoting.RPCHook; import org.apache.rocketmq.remoting.exception.RemotingException; @@ -67,6 +70,14 @@ public Options buildCommandlineOptions(Options options) { opt.setRequired(false); options.addOption(opt); + opt = new Option("p", "keyType", true, "Index key type, default index key type is K, you can use K for keys OR T for tags"); + opt.setRequired(false); + options.addOption(opt); + + opt = new Option("l", "lastKey", true, "Last Key"); + opt.setRequired(false); + options.addOption(opt); + return options; } @@ -79,7 +90,8 @@ public void execute(CommandLine commandLine, Options options, RPCHook rpcHook) t try { final String topic = commandLine.getOptionValue('t').trim(); final String key = commandLine.getOptionValue('k').trim(); - + String keyType = MessageConst.INDEX_KEY_TYPE; + String lastKey = null; long beginTimestamp = 0; long endTimestamp = Long.MAX_VALUE; int maxNum = 64; @@ -96,7 +108,17 @@ public void execute(CommandLine commandLine, Options options, RPCHook rpcHook) t if (commandLine.hasOption("c")) { clusterName = commandLine.getOptionValue("c").trim(); } - this.queryByKey(defaultMQAdminExt, clusterName, topic, key, maxNum, beginTimestamp, endTimestamp); + if (commandLine.hasOption("p")) { + keyType = commandLine.getOptionValue("p").trim(); + if (StringUtils.isEmpty(keyType) || !MessageConst.INDEX_KEY_TYPE.equals(keyType) && !MessageConst.INDEX_TAG_TYPE.equals(keyType)) { + System.out.printf("index type error, just support K for keys or T for tags"); + return; + } + } + if (commandLine.hasOption("l")) { + lastKey = commandLine.getOptionValue("l").trim(); + } + this.queryByKey(defaultMQAdminExt, clusterName, topic, key, maxNum, beginTimestamp, endTimestamp, keyType, lastKey); } catch (Exception e) { throw new SubCommandException(this.getClass().getSimpleName() + " command failed", e); } finally { @@ -105,18 +127,23 @@ public void execute(CommandLine commandLine, Options options, RPCHook rpcHook) t } private void queryByKey(final DefaultMQAdminExt admin, final String cluster, final String topic, final String key, int maxNum, long begin, - long end) + long end, String keyType, String lastKey) throws MQClientException, InterruptedException, RemotingException { admin.start(); - - QueryResult queryResult = admin.queryMessage(cluster, topic, key, maxNum, begin, end); - - System.out.printf("%-50s %4s %40s%n", + QueryResult queryResult = admin.queryMessage(cluster, topic, key, maxNum, begin, end, keyType, lastKey); + System.out.printf("%-50s %4s %40s %-200s%n", "#Message ID", "#QID", - "#Offset"); + "#Offset", + "#IndexKey"); for (MessageExt msg : queryResult.getMessageList()) { - System.out.printf("%-50s %4d %40d%n", msg.getMsgId(), msg.getQueueId(), msg.getQueueOffset()); + if (!StringUtils.isEmpty(keyType)) { + long storeTimestamp = MixAll.dealTimeToHourStamps(msg.getStoreTimestamp()); + String indexLastKey = storeTimestamp + "@" + topic + "@" + keyType + "@" + key + "@" + msg.getMsgId() + "@" + msg.getCommitLogOffset(); + System.out.printf("%-50s %4d %40d %-200s%n", msg.getMsgId(), msg.getQueueId(), msg.getQueueOffset(), indexLastKey); + } else { + System.out.printf("%-50s %4d %40d%n", msg.getMsgId(), msg.getQueueId(), msg.getQueueOffset()); + } } } } diff --git a/tools/src/main/java/org/apache/rocketmq/tools/command/message/QueryMsgByUniqueKeySubCommand.java b/tools/src/main/java/org/apache/rocketmq/tools/command/message/QueryMsgByUniqueKeySubCommand.java index 5295d91cc30..8518a04e682 100644 --- a/tools/src/main/java/org/apache/rocketmq/tools/command/message/QueryMsgByUniqueKeySubCommand.java +++ b/tools/src/main/java/org/apache/rocketmq/tools/command/message/QueryMsgByUniqueKeySubCommand.java @@ -24,6 +24,7 @@ import org.apache.commons.cli.CommandLine; import org.apache.commons.cli.Option; import org.apache.commons.cli.Options; +import org.apache.commons.lang3.StringUtils; import org.apache.rocketmq.client.QueryResult; import org.apache.rocketmq.client.exception.MQClientException; import org.apache.rocketmq.common.UtilAll; @@ -56,11 +57,17 @@ private DefaultMQAdminExt createMQAdminExt(RPCHook rpcHook) throws SubCommandExc } } - public static void queryById(final DefaultMQAdminExt admin, final String clusterName, final String topic, - final String msgId, - final boolean showAll) throws MQClientException, InterruptedException, IOException { - - QueryResult queryResult = admin.queryMessageByUniqKey(clusterName, topic, msgId, 32, 0, Long.MAX_VALUE); + public static void queryById(final DefaultMQAdminExt admin, final String clusterName, final String topic, final String msgId, final boolean showAll, final String startTime, final String endTime) throws MQClientException, InterruptedException, IOException { + QueryResult queryResult = null; + if (!StringUtils.isEmpty(startTime) && !StringUtils.isEmpty(endTime)) { + Long startTimelong = Long.valueOf(startTime); + Long endTimelong = Long.valueOf(endTime); + if (null != startTimelong && null != endTimelong) { + queryResult = admin.queryMessageByUniqKey(clusterName, topic, msgId, 32, startTimelong, endTimelong); + } + } else { + queryResult = admin.queryMessageByUniqKey(clusterName, topic, msgId, 32, System.currentTimeMillis() - 36 * 60 * 60 * 1000, System.currentTimeMillis() + 36 * 60 * 60 * 1000); + } assert queryResult != null; List list = queryResult.getMessageList(); if (list == null || list.size() == 0) { @@ -167,6 +174,14 @@ public Options buildCommandlineOptions(Options options) { opt.setRequired(false); options.addOption(opt); + opt = new Option("s", "startTime", true, "startTime"); + opt.setRequired(false); + options.addOption(opt); + + opt = new Option("e", "endTime", true, "endTime"); + opt.setRequired(false); + options.addOption(opt); + return options; } @@ -179,6 +194,8 @@ public void execute(CommandLine commandLine, Options options, RPCHook rpcHook) t final String msgId = commandLine.getOptionValue('i').trim(); final String topic = commandLine.getOptionValue('t').trim(); String clusterName = commandLine.hasOption('c') ? commandLine.getOptionValue('c').trim() : null; + String startTime = commandLine.hasOption('s') ? commandLine.getOptionValue('s').trim() : null; + String endTime = commandLine.hasOption('e') ? commandLine.getOptionValue('e').trim() : null; final boolean showAll = commandLine.hasOption('a'); if (commandLine.hasOption('g') && commandLine.hasOption('d')) { final String consumerGroup = commandLine.getOptionValue('g').trim(); @@ -198,7 +215,7 @@ public void execute(CommandLine commandLine, Options options, RPCHook rpcHook) t } } else { - queryById(defaultMQAdminExt, clusterName, topic, msgId, showAll); + queryById(defaultMQAdminExt, clusterName, topic, msgId, showAll, startTime, endTime); } } catch (Exception e) { throw new SubCommandException(this.getClass().getSimpleName() + " command failed", e);