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 e83bea239753a5314e48876b03edbc97e838d155..5eb916922f61b70a586f7f46316ff10a8cad1d75 100644 --- a/broker/src/main/java/org/apache/rocketmq/broker/BrokerController.java +++ b/broker/src/main/java/org/apache/rocketmq/broker/BrokerController.java @@ -16,13 +16,18 @@ */ package org.apache.rocketmq.broker; +import com.google.common.collect.Maps; import java.io.IOException; import java.net.InetSocketAddress; +import java.util.AbstractMap; import java.util.ArrayList; +import java.util.Collections; import java.util.HashMap; import java.util.Iterator; import java.util.List; import java.util.Map; +import java.util.Objects; +import java.util.Optional; import java.util.concurrent.BlockingQueue; import java.util.concurrent.ConcurrentHashMap; import java.util.concurrent.ConcurrentMap; @@ -33,6 +38,8 @@ import java.util.concurrent.LinkedBlockingQueue; import java.util.concurrent.ScheduledExecutorService; import java.util.concurrent.ThreadPoolExecutor; import java.util.concurrent.TimeUnit; +import java.util.function.Function; +import java.util.stream.Collectors; import org.apache.rocketmq.acl.AccessValidator; import org.apache.rocketmq.broker.client.ClientHousekeepingService; import org.apache.rocketmq.broker.client.ConsumerIdsChangeListener; @@ -42,6 +49,7 @@ import org.apache.rocketmq.broker.client.ProducerManager; import org.apache.rocketmq.broker.client.net.Broker2Client; import org.apache.rocketmq.broker.client.rebalance.RebalanceLockManager; import org.apache.rocketmq.broker.dledger.DLedgerRoleChangeHandler; +import org.apache.rocketmq.broker.domain.LogicalQueuesInfoInBroker; import org.apache.rocketmq.broker.filter.CommitLogDispatcherCalcBitMap; import org.apache.rocketmq.broker.filter.ConsumerFilterManager; import org.apache.rocketmq.broker.filtersrv.FilterServerManager; @@ -79,9 +87,11 @@ import org.apache.rocketmq.broker.transaction.queue.DefaultTransactionalMessageC import org.apache.rocketmq.broker.transaction.queue.TransactionalMessageBridge; import org.apache.rocketmq.broker.transaction.queue.TransactionalMessageServiceImpl; import org.apache.rocketmq.broker.util.ServiceProvider; +import org.apache.rocketmq.client.exception.MQBrokerException; import org.apache.rocketmq.common.BrokerConfig; import org.apache.rocketmq.common.Configuration; import org.apache.rocketmq.common.DataVersion; +import org.apache.rocketmq.common.MixAll; import org.apache.rocketmq.common.ThreadFactoryImpl; import org.apache.rocketmq.common.TopicConfig; import org.apache.rocketmq.common.UtilAll; @@ -89,13 +99,18 @@ import org.apache.rocketmq.common.constant.LoggerName; import org.apache.rocketmq.common.constant.PermName; import org.apache.rocketmq.common.namesrv.RegisterBrokerResult; import org.apache.rocketmq.common.protocol.RequestCode; +import org.apache.rocketmq.common.protocol.body.ClusterInfo; import org.apache.rocketmq.common.protocol.body.TopicConfigSerializeWrapper; +import org.apache.rocketmq.common.protocol.route.LogicalQueuesInfo; import org.apache.rocketmq.common.stats.MomentStatsItem; import org.apache.rocketmq.logging.InternalLogger; import org.apache.rocketmq.logging.InternalLoggerFactory; import org.apache.rocketmq.remoting.RPCHook; import org.apache.rocketmq.remoting.RemotingServer; import org.apache.rocketmq.remoting.common.TlsMode; +import org.apache.rocketmq.remoting.exception.RemotingConnectException; +import org.apache.rocketmq.remoting.exception.RemotingSendRequestException; +import org.apache.rocketmq.remoting.exception.RemotingTimeoutException; import org.apache.rocketmq.remoting.netty.NettyClientConfig; import org.apache.rocketmq.remoting.netty.NettyRemotingServer; import org.apache.rocketmq.remoting.netty.NettyRequestProcessor; @@ -159,6 +174,7 @@ public class BrokerController { private final BrokerStatsManager brokerStatsManager; private final List sendMessageHookList = new ArrayList(); private final List consumeMessageHookList = new ArrayList(); + private final ConcurrentMap brokerName2AddrMap = Maps.newConcurrentMap(); private MessageStore messageStore; private RemotingServer remotingServer; private RemotingServer fastRemotingServer; @@ -277,9 +293,9 @@ public class BrokerController { if (result) { try { - this.messageStore = - new DefaultMessageStore(this.messageStoreConfig, this.brokerStatsManager, this.messageArrivingListener, - this.brokerConfig); + DefaultMessageStore messageStore = new DefaultMessageStore(this.messageStoreConfig, this.brokerStatsManager, this.messageArrivingListener, this.brokerConfig); + messageStore.registerCleanFileHook(topicConfigManager.getLogicalQueueCleanHook()); + this.messageStore = messageStore; if (messageStoreConfig.isEnableDLegerCommitLog()) { DLedgerRoleChangeHandler roleChangeHandler = new DLedgerRoleChangeHandler(this, (DefaultMessageStore) messageStore); ((DLedgerCommitLog)((DefaultMessageStore) messageStore).getCommitLog()).getdLedgerServer().getdLedgerLeaderElector().addRoleChangeHandler(roleChangeHandler); @@ -467,6 +483,14 @@ public class BrokerController { }, 1000 * 10, 1000 * 60 * 2, TimeUnit.MILLISECONDS); } + this.scheduledExecutorService.scheduleAtFixedRate(() -> { + try { + BrokerController.this.refreshBrokerNameMapping(); + } catch (Exception e) { + log.error("ScheduledTask examineBrokerClusterInfo exception", e); + } + }, 10, 10, TimeUnit.SECONDS); + if (!messageStoreConfig.isEnableDLegerCommitLog()) { if (BrokerRole.SLAVE == this.messageStoreConfig.getBrokerRole()) { if (this.messageStoreConfig.getHaMasterAddress() != null && this.messageStoreConfig.getHaMasterAddress().length() >= 6) { @@ -593,6 +617,18 @@ public class BrokerController { } } + private void refreshBrokerNameMapping() throws InterruptedException, MQBrokerException, RemotingTimeoutException, RemotingSendRequestException, RemotingConnectException { + ClusterInfo brokerClusterInfo = this.brokerOuterAPI.getBrokerClusterInfo(); + brokerClusterInfo.getBrokerAddrTable().forEach((brokerName, data) -> { + String masterBrokerAddr = data.getBrokerAddrs().get(MixAll.MASTER_ID); + this.brokerName2AddrMap.put(brokerName, masterBrokerAddr); + }); + } + + public String getBrokerAddrByName(String brokerName) { + return this.brokerName2AddrMap.get(brokerName); + } + public void registerProcessor() { /** * SendMessageProcessor @@ -1009,20 +1045,54 @@ public class BrokerController { } public synchronized void registerIncrementBrokerData(TopicConfig topicConfig, DataVersion dataVersion) { - TopicConfig registerTopicConfig = topicConfig; - if (!PermName.isWriteable(this.getBrokerConfig().getBrokerPermission()) - || !PermName.isReadable(this.getBrokerConfig().getBrokerPermission())) { - registerTopicConfig = - new TopicConfig(topicConfig.getTopicName(), topicConfig.getReadQueueNums(), topicConfig.getWriteQueueNums(), - this.brokerConfig.getBrokerPermission()); + this.registerIncrementBrokerData(Collections.singletonList(topicConfig), dataVersion); + } + + public synchronized void registerIncrementBrokerData(List topicConfigList, DataVersion dataVersion) { + if (topicConfigList == null || topicConfigList.isEmpty()) { + return; } - ConcurrentMap topicConfigTable = new ConcurrentHashMap(); - topicConfigTable.put(topicConfig.getTopicName(), registerTopicConfig); TopicConfigSerializeWrapper topicConfigSerializeWrapper = new TopicConfigSerializeWrapper(); topicConfigSerializeWrapper.setDataVersion(dataVersion); + + ConcurrentMap topicConfigTable = topicConfigList.stream() + .map(topicConfig -> { + TopicConfig registerTopicConfig; + if (!PermName.isWriteable(this.getBrokerConfig().getBrokerPermission()) + || !PermName.isReadable(this.getBrokerConfig().getBrokerPermission())) { + registerTopicConfig = + new TopicConfig(topicConfig.getTopicName(), + topicConfig.getReadQueueNums(), + topicConfig.getWriteQueueNums(), + this.brokerConfig.getBrokerPermission()); + } else { + registerTopicConfig = new TopicConfig(topicConfig); + } + return registerTopicConfig; + }) + .collect(Collectors.toConcurrentMap(TopicConfig::getTopicName, Function.identity())); topicConfigSerializeWrapper.setTopicConfigTable(topicConfigTable); + String brokerName = this.brokerConfig.getBrokerName(); + Map logicalQueuesInfoMap = topicConfigList.stream() + .map(TopicConfig::getTopicName) + .map(topicName -> Optional.ofNullable(this.topicConfigManager.selectLogicalQueuesInfo(topicName)) + .map(info -> { + info.readLock().lock(); + try { + return new AbstractMap.SimpleImmutableEntry<>(topicName, new LogicalQueuesInfoInBroker(info, data -> Objects.equals(data.getBrokerName(), brokerName))); + } finally { + info.readLock().unlock(); + } + }) + .orElse(null)) + .filter(Objects::nonNull) + .collect(Collectors.toMap(Map.Entry::getKey, Map.Entry::getValue)); + if (!logicalQueuesInfoMap.isEmpty()) { + topicConfigSerializeWrapper.setLogicalQueuesInfoMap(logicalQueuesInfoMap); + } + doRegisterBrokerAll(true, false, topicConfigSerializeWrapper); } @@ -1032,13 +1102,21 @@ public class BrokerController { if (!PermName.isWriteable(this.getBrokerConfig().getBrokerPermission()) || !PermName.isReadable(this.getBrokerConfig().getBrokerPermission())) { ConcurrentHashMap topicConfigTable = new ConcurrentHashMap(); + Map logicalQueuesInfoMap = Maps.newHashMapWithExpectedSize(topicConfigWrapper.getTopicConfigTable().size()); for (TopicConfig topicConfig : topicConfigWrapper.getTopicConfigTable().values()) { + String topicName = topicConfig.getTopicName(); TopicConfig tmp = - new TopicConfig(topicConfig.getTopicName(), topicConfig.getReadQueueNums(), topicConfig.getWriteQueueNums(), + new TopicConfig(topicName, topicConfig.getReadQueueNums(), topicConfig.getWriteQueueNums(), this.brokerConfig.getBrokerPermission()); - topicConfigTable.put(topicConfig.getTopicName(), tmp); + topicConfigTable.put(topicName, tmp); + LogicalQueuesInfoInBroker logicalQueuesInfo = this.topicConfigManager.selectLogicalQueuesInfo(topicName); + if (logicalQueuesInfo != null) { + String brokerName = this.brokerConfig.getBrokerName(); + logicalQueuesInfoMap.put(topicName, new LogicalQueuesInfoInBroker(logicalQueuesInfo, data -> Objects.equals(data.getBrokerName(), brokerName))); + } } topicConfigWrapper.setTopicConfigTable(topicConfigTable); + topicConfigWrapper.setLogicalQueuesInfoMap(logicalQueuesInfoMap); } if (forceRegister || needRegister(this.brokerConfig.getBrokerClusterName(), diff --git a/broker/src/main/java/org/apache/rocketmq/broker/domain/LogicalQueuesInfoInBroker.java b/broker/src/main/java/org/apache/rocketmq/broker/domain/LogicalQueuesInfoInBroker.java new file mode 100644 index 0000000000000000000000000000000000000000..a6728c828b0507f0127cc92288e5be5fcd3af25a --- /dev/null +++ b/broker/src/main/java/org/apache/rocketmq/broker/domain/LogicalQueuesInfoInBroker.java @@ -0,0 +1,116 @@ +/* + * 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.domain; + +import com.alibaba.fastjson.parser.ParserConfig; +import com.google.common.collect.Maps; +import java.util.Collections; +import java.util.List; +import java.util.concurrent.ConcurrentMap; +import java.util.concurrent.ConcurrentNavigableMap; +import java.util.concurrent.ConcurrentSkipListMap; +import java.util.function.Predicate; +import java.util.stream.Collectors; +import java.util.stream.Stream; +import org.apache.rocketmq.common.fastjson.GenericMapSuperclassDeserializer; +import org.apache.rocketmq.common.protocol.route.LogicalQueueRouteData; +import org.apache.rocketmq.common.protocol.route.LogicalQueuesInfo; +import org.apache.rocketmq.srvutil.ConcurrentHashMapUtil; + +import static java.util.Optional.ofNullable; + +public class LogicalQueuesInfoInBroker extends LogicalQueuesInfo { + private final ConcurrentMap> queueId2LogicalQueueMap = Maps.newConcurrentMap(); + + public LogicalQueuesInfoInBroker() { + } + + public LogicalQueuesInfoInBroker(LogicalQueuesInfoInBroker other) { + this(other, null); + } + + // deep copy + public LogicalQueuesInfoInBroker(LogicalQueuesInfoInBroker other, Predicate predicate) { + other.readLock().lock(); + try { + for (Entry> entry : other.entrySet()) { + Stream stream = entry.getValue().stream(); + if (predicate != null) { + stream = stream.filter(predicate); + } + this.put(entry.getKey(), stream.map(LogicalQueueRouteData::new).collect(Collectors.toList())); + } + } finally { + other.readLock().unlock(); + } + } + + public void updateQueueRouteDataByQueueId(int queueId, LogicalQueueRouteData queueRouteData) { + if (queueRouteData == null) { + return; + } + ConcurrentHashMapUtil.computeIfAbsent(queueId2LogicalQueueMap, queueId, k -> new ConcurrentSkipListMap<>()).put(queueRouteData.getOffsetDelta(), queueRouteData); + } + + /** + * find logical queue route data for message queues owned by this broker + */ + public LogicalQueueRouteData queryQueueRouteDataByQueueId(int queueId, long offset) { + ConcurrentNavigableMap m = this.queueId2LogicalQueueMap.get(queueId); + if (m == null || m.isEmpty()) { + return null; + } + Entry entry = m.floorEntry(offset); + if (entry == null) { + return null; + } + return entry.getValue(); + } + + public void deleteQueueRouteData(LogicalQueueRouteData logicalQueueRouteData) { + ConcurrentNavigableMap m = this.queueId2LogicalQueueMap.get(logicalQueueRouteData.getQueueId()); + if (m != null) { + m.remove(logicalQueueRouteData.getOffsetDelta(), logicalQueueRouteData); + } + } + + public LogicalQueueRouteData nextAvailableLogicalRouteData(LogicalQueueRouteData queueRouteData, + Predicate predicate) { + this.readLock().lock(); + try { + List queueRouteDataList = ofNullable(this.get(queueRouteData.getLogicalQueueIndex())).orElse(Collections.emptyList()); + int idx = Collections.binarySearch(queueRouteDataList, queueRouteData); + if (idx >= 0) { + for (int i = idx + 1, size = queueRouteDataList.size(); i < size; i++) { + LogicalQueueRouteData tmp = queueRouteDataList.get(i); + if (predicate.test(tmp)) { + return tmp; + } + } + } + } finally { + this.readLock().unlock(); + } + return null; + } + + static { + // workaround https://github.com/alibaba/fastjson/issues/3730 + ParserConfig.getGlobalInstance().putDeserializer(LogicalQueuesInfoInBroker.class, GenericMapSuperclassDeserializer.INSTANCE); + } +} diff --git a/broker/src/main/java/org/apache/rocketmq/broker/out/BrokerOuterAPI.java b/broker/src/main/java/org/apache/rocketmq/broker/out/BrokerOuterAPI.java index 252201a058ed4514cf0361d169deb640ba141958..4d33663bb90d03e91873ad74f372f24c6c586cf8 100644 --- a/broker/src/main/java/org/apache/rocketmq/broker/out/BrokerOuterAPI.java +++ b/broker/src/main/java/org/apache/rocketmq/broker/out/BrokerOuterAPI.java @@ -34,6 +34,7 @@ import org.apache.rocketmq.common.namesrv.RegisterBrokerResult; import org.apache.rocketmq.common.namesrv.TopAddressing; import org.apache.rocketmq.common.protocol.RequestCode; import org.apache.rocketmq.common.protocol.ResponseCode; +import org.apache.rocketmq.common.protocol.body.ClusterInfo; import org.apache.rocketmq.common.protocol.body.ConsumerOffsetSerializeWrapper; import org.apache.rocketmq.common.protocol.body.KVTable; import org.apache.rocketmq.common.protocol.body.RegisterBrokerBody; @@ -48,6 +49,7 @@ import org.apache.rocketmq.common.protocol.header.namesrv.UnRegisterBrokerReques import org.apache.rocketmq.common.protocol.route.TopicRouteData; import org.apache.rocketmq.logging.InternalLogger; import org.apache.rocketmq.logging.InternalLoggerFactory; +import org.apache.rocketmq.remoting.InvokeCallback; import org.apache.rocketmq.remoting.RPCHook; import org.apache.rocketmq.remoting.RemotingClient; import org.apache.rocketmq.remoting.exception.RemotingCommandException; @@ -432,4 +434,23 @@ public class BrokerOuterAPI { throw new MQBrokerException(response.getCode(), response.getRemark()); } + + public ClusterInfo getBrokerClusterInfo() throws InterruptedException, RemotingTimeoutException, RemotingSendRequestException, RemotingConnectException, MQBrokerException { + RemotingCommand request = RemotingCommand.createRequestCommand(RequestCode.GET_BROKER_CLUSTER_INFO, null); + RemotingCommand response = this.remotingClient.invokeSync(null, request, 3_000); + assert response != null; + switch (response.getCode()) { + case ResponseCode.SUCCESS: { + return ClusterInfo.decode(response.getBody(), ClusterInfo.class); + } + default: + break; + } + + throw new MQBrokerException(response.getCode(), response.getRemark()); + } + + public void forwardRequest(String brokerAddr, RemotingCommand request, long timeoutMillis, InvokeCallback invokeCallback) throws InterruptedException, RemotingSendRequestException, RemotingTimeoutException, RemotingTooMuchRequestException, RemotingConnectException { + this.remotingClient.invokeAsync(brokerAddr, request, timeoutMillis, invokeCallback); + } } diff --git a/broker/src/main/java/org/apache/rocketmq/broker/plugin/AbstractPluginMessageStore.java b/broker/src/main/java/org/apache/rocketmq/broker/plugin/AbstractPluginMessageStore.java index 1db019bec810accdf844597dc2783910f05c1e1c..db1b6267b092bc5fc8dbdcfc8db2917944ed852d 100644 --- a/broker/src/main/java/org/apache/rocketmq/broker/plugin/AbstractPluginMessageStore.java +++ b/broker/src/main/java/org/apache/rocketmq/broker/plugin/AbstractPluginMessageStore.java @@ -109,6 +109,11 @@ public abstract class AbstractPluginMessageStore implements MessageStore { return next.getMaxOffsetInQueue(topic, queueId); } + @Override + public long getMaxOffsetInQueue(String topic, int queueId, boolean committed) { + return next.getMaxOffsetInQueue(topic, queueId, committed); + } + @Override public long getMinOffsetInQueue(String topic, int queueId) { return next.getMinOffsetInQueue(topic, queueId); diff --git a/broker/src/main/java/org/apache/rocketmq/broker/processor/AbstractSendMessageProcessor.java b/broker/src/main/java/org/apache/rocketmq/broker/processor/AbstractSendMessageProcessor.java index ced7c2014f4a5d24b5832971142e686ebc404df0..4daa832dd0a48d3577eadd1a9ae50270a3988527 100644 --- a/broker/src/main/java/org/apache/rocketmq/broker/processor/AbstractSendMessageProcessor.java +++ b/broker/src/main/java/org/apache/rocketmq/broker/processor/AbstractSendMessageProcessor.java @@ -16,19 +16,28 @@ */ package org.apache.rocketmq.broker.processor; +import com.google.common.collect.ImmutableList; +import com.google.common.collect.Maps; import io.netty.channel.ChannelHandlerContext; import java.net.InetSocketAddress; import java.net.SocketAddress; import java.util.List; +import java.util.Locale; import java.util.Map; +import java.util.Optional; import java.util.Random; +import java.util.concurrent.CompletableFuture; +import java.util.concurrent.ConcurrentMap; +import java.util.concurrent.atomic.LongAdder; import org.apache.rocketmq.broker.BrokerController; +import org.apache.rocketmq.broker.domain.LogicalQueuesInfoInBroker; import org.apache.rocketmq.broker.mqtrace.SendMessageContext; import org.apache.rocketmq.broker.mqtrace.SendMessageHook; -import org.apache.rocketmq.common.topic.TopicValidator; +import org.apache.rocketmq.broker.topic.TopicConfigManager; import org.apache.rocketmq.common.MixAll; import org.apache.rocketmq.common.TopicConfig; import org.apache.rocketmq.common.TopicFilterType; +import org.apache.rocketmq.common.TopicQueueId; import org.apache.rocketmq.common.constant.DBMsgConstants; import org.apache.rocketmq.common.constant.LoggerName; import org.apache.rocketmq.common.constant.PermName; @@ -42,17 +51,23 @@ import org.apache.rocketmq.common.protocol.ResponseCode; import org.apache.rocketmq.common.protocol.header.SendMessageRequestHeader; import org.apache.rocketmq.common.protocol.header.SendMessageRequestHeaderV2; import org.apache.rocketmq.common.protocol.header.SendMessageResponseHeader; +import org.apache.rocketmq.common.protocol.route.LogicalQueueRouteData; import org.apache.rocketmq.common.sysflag.MessageSysFlag; import org.apache.rocketmq.common.sysflag.TopicSysFlag; +import org.apache.rocketmq.common.topic.TopicValidator; import org.apache.rocketmq.common.utils.ChannelUtil; import org.apache.rocketmq.logging.InternalLogger; import org.apache.rocketmq.logging.InternalLoggerFactory; +import org.apache.rocketmq.remoting.CommandCustomHeader; import org.apache.rocketmq.remoting.common.RemotingHelper; import org.apache.rocketmq.remoting.exception.RemotingCommandException; import org.apache.rocketmq.remoting.netty.AsyncNettyRequestProcessor; import org.apache.rocketmq.remoting.netty.NettyRequestProcessor; import org.apache.rocketmq.remoting.protocol.RemotingCommand; +import org.apache.rocketmq.remoting.protocol.RemotingSerializable; +import org.apache.rocketmq.srvutil.ConcurrentHashMapUtil; import org.apache.rocketmq.store.MessageExtBrokerInner; +import org.apache.rocketmq.store.PutMessageResult; public abstract class AbstractSendMessageProcessor extends AsyncNettyRequestProcessor implements NettyRequestProcessor { protected static final InternalLogger log = InternalLoggerFactory.getLogger(LoggerName.BROKER_LOGGER_NAME); @@ -63,6 +78,8 @@ public abstract class AbstractSendMessageProcessor extends AsyncNettyRequestProc protected final SocketAddress storeHost; private List sendMessageHookList; + private final ConcurrentMap inFlyWritingCounterMap = Maps.newConcurrentMap(); + public AbstractSendMessageProcessor(final BrokerController brokerController) { this.brokerController = brokerController; this.storeHost = @@ -330,4 +347,158 @@ public abstract class AbstractSendMessageProcessor extends AsyncNettyRequestProc public boolean rejectRequest() { return false; } + + + public ConcurrentMap getInFlyWritingCounterMap() { + return inFlyWritingCounterMap; + } + + protected LogicalQueueContext buildLogicalQueueContext(String topic, int queueId, + RemotingCommand response) { + TopicConfigManager topicConfigManager = this.brokerController.getTopicConfigManager(); + LogicalQueuesInfoInBroker logicalQueuesInfo = topicConfigManager.selectLogicalQueuesInfo(topic); + if (logicalQueuesInfo == null) { + return noopLogicalQueueContext; + } + // writable route data will has largest offset + LogicalQueueRouteData curQueueRouteData = logicalQueuesInfo.queryQueueRouteDataByQueueId(queueId, Long.MAX_VALUE); + if (curQueueRouteData == null) { + // topic enabled logical queue, but some message queues are not converted or being converted + String msg = String.format(Locale.ENGLISH, "queueId %d not included in logical queue", queueId); + log.debug("buildLogicalQueueContext unexpected error, topic {} {}", topic, msg); + response.setCode(ResponseCode.SYSTEM_ERROR); + response.setRemark(msg); + return noopLogicalQueueContext; + } + LongAdder inFlyWritingCounter = ConcurrentHashMapUtil.computeIfAbsent(inFlyWritingCounterMap, new TopicQueueId(topic, queueId), ignore -> new LongAdder()); + return new LogicalQueueContext(topic, queueId, logicalQueuesInfo, curQueueRouteData, inFlyWritingCounter); + } + + protected class LogicalQueueContext { + private final String topic; + private final int queueId; + private final LogicalQueuesInfoInBroker logicalQueuesInfo; + private final LogicalQueueRouteData curQueueRouteData; + private final LongAdder inFlyWritingCounter; + + public LogicalQueueContext(String topic, int queueId, + LogicalQueuesInfoInBroker logicalQueuesInfo, + LogicalQueueRouteData curQueueRouteData, LongAdder inFlyWritingCounter) { + this.topic = topic; + this.queueId = queueId; + this.logicalQueuesInfo = logicalQueuesInfo; + this.curQueueRouteData = curQueueRouteData; + this.inFlyWritingCounter = inFlyWritingCounter; + } + + public CompletableFuture hookBeforePut(ChannelHandlerContext ctx, SendMessageRequestHeader requestHeader, + RemotingCommand request, RemotingCommand response) { + if (curQueueRouteData.isWritable()) { + this.inFlyWritingCounter.increment(); + return null; + } + int logicalQueueIdx = curQueueRouteData.getLogicalQueueIndex(); + List queueRouteDataList = logicalQueuesInfo.get(logicalQueueIdx); + LogicalQueueRouteData writableQueueRouteData = null; + for (int i = queueRouteDataList.size() - 1; i >= 0; i--) { + LogicalQueueRouteData queueRouteData = queueRouteDataList.get(i); + if (queueRouteData.isWritable()) { + writableQueueRouteData = queueRouteData; + break; + } + } + if (writableQueueRouteData == null) { + response.setCode(ResponseCode.NO_PERMISSION); + response.setRemark(String.format(Locale.ENGLISH, "broker[%s] topic[%s] queueId[%d] logicalQueueIdx[%d] not writable", AbstractSendMessageProcessor.this.brokerController.getBrokerConfig().getBrokerIP1(), topic, queueId, logicalQueueIdx)); + return CompletableFuture.completedFuture(response); + } + if ((Optional.ofNullable(requestHeader.getSysFlag()).orElse(0) & MessageSysFlag.LOGICAL_QUEUE_FLAG) > 0) { + // new client, use redirect + response.setCode(ResponseCode.NO_PERMISSION); + response.addExtField(MessageConst.PROPERTY_REDIRECT, "1"); + response.setBody(RemotingSerializable.encode(ImmutableList.of(curQueueRouteData, writableQueueRouteData))); + return CompletableFuture.completedFuture(response); + } else { + // old client, use forward + this.logicalQueueHookForward(ctx, writableQueueRouteData, requestHeader, request, response); + } + if (response.getCode() != -1) { + return CompletableFuture.completedFuture(response); + } else if (response.getCode() == ResponseCode.ASYNC_AND_RETURN_NULL) { + return CompletableFuture.completedFuture(null); + } + return null; + } + + private void logicalQueueHookForward(ChannelHandlerContext ctx, + LogicalQueueRouteData writableQueueRouteData, + SendMessageRequestHeader requestHeader, RemotingCommand request, + RemotingCommand response) { + response.setCode(ResponseCode.SUCCESS); + requestHeader.setQueueId(writableQueueRouteData.getQueueId()); + request.writeCustomHeader(requestHeader); + String brokerName = writableQueueRouteData.getBrokerName(); + BrokerController brokerController = AbstractSendMessageProcessor.this.brokerController; + String brokerAddr = brokerController.getBrokerAddrByName(brokerName); + if (brokerAddr == null) { + log.warn("getBrokerAddrByName brokerName={} got null, fallback to queueRouteData.getBrokerAddr()", brokerName); + brokerAddr = writableQueueRouteData.getBrokerAddr(); + } + if (brokerAddr == null) { + response.setCode(ResponseCode.SYSTEM_ERROR); + String msg = String.format(Locale.ENGLISH, "unknown brokerName %s", brokerName); + response.setRemark(msg); + log.warn("logicalQueueHookForward can not look up brokerName={}: {}", brokerName, requestHeader); + return; + } + try { + String finalBrokerAddr = brokerAddr; + brokerController.getBrokerOuterAPI().forwardRequest(brokerAddr, request, brokerController.getBrokerConfig().getForwardTimeout(), responseFuture -> { + RemotingCommand forwardResponse = responseFuture.getResponseCommand(); + if (forwardResponse == null) { + forwardResponse = response; + forwardResponse.setCode(ResponseCode.SYSTEM_ERROR); + if (!responseFuture.isSendRequestOK()) { + forwardResponse.setRemark(String.format(Locale.ENGLISH, "send request failed to %s: %s", finalBrokerAddr, responseFuture.getCause())); + } else if (responseFuture.isTimeout()) { + forwardResponse.setRemark(String.format(Locale.ENGLISH, "wait response from %s timeout: %dms", finalBrokerAddr, responseFuture.getTimeoutMillis())); + } else { + forwardResponse.setRemark(String.format(Locale.ENGLISH, "unknown reason. addr: %s, timeoutMillis: %d: %s", finalBrokerAddr, responseFuture.getTimeoutMillis(), responseFuture.getCause())); + } + } else { + CommandCustomHeader customHeader = forwardResponse.readCustomHeader(); + if (customHeader instanceof SendMessageResponseHeader) { + SendMessageResponseHeader responseHeader = (SendMessageResponseHeader) customHeader; + Integer forwardQueueId = responseHeader.getQueueId(); + forwardResponse.addExtField(MessageConst.PROPERTY_FORWARD_QUEUE_ID, forwardQueueId != null ? Integer.toString(forwardQueueId) : "null"); + responseHeader.setQueueId(requestHeader.getQueueId()); + // queueOffset should not be changed since forwarded broker will add delta to it. + } + } + AbstractSendMessageProcessor.this.doResponse(ctx, request, forwardResponse); + }); + response.setCode(ResponseCode.ASYNC_AND_RETURN_NULL); + } catch (Exception e) { + response.setCode(ResponseCode.SYSTEM_ERROR); + response.setRemark("forward error"); + log.warn(String.format(Locale.ENGLISH, "logicalQueueHookForward to %s error", brokerAddr), e); + } + } + + public void hookAfterPut(CompletableFuture putMessageResult) { + Optional.ofNullable(putMessageResult).orElse(CompletableFuture.completedFuture(null)).whenComplete((result, throwable) -> { + this.inFlyWritingCounter.decrement(); + }); + } + } + + private final LogicalQueueContext noopLogicalQueueContext = new LogicalQueueContext(null, 0, null, null, null) { + @Override public CompletableFuture hookBeforePut(ChannelHandlerContext ctx, SendMessageRequestHeader requestHeader, + RemotingCommand request, RemotingCommand response) { + return null; + } + + @Override public void hookAfterPut(CompletableFuture putMessageResult) { + } + }; } 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 0a1d214b87f4288cf97e2fd3d76a37e28dd04d50..b3309e1e2e92e82f7714905879ec029682f9e61c 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 @@ -17,22 +17,49 @@ package org.apache.rocketmq.broker.processor; import com.alibaba.fastjson.JSON; +import com.alibaba.fastjson.JSONObject; +import com.google.common.collect.ImmutableList; +import com.google.common.collect.Lists; +import com.google.common.collect.Maps; import io.netty.channel.Channel; import io.netty.channel.ChannelHandlerContext; +import java.io.UnsupportedEncodingException; +import java.net.UnknownHostException; +import java.util.ArrayList; +import java.util.Collection; +import java.util.Collections; +import java.util.Comparator; +import java.util.HashMap; +import java.util.HashSet; +import java.util.Iterator; +import java.util.List; +import java.util.Locale; +import java.util.Map; +import java.util.Objects; +import java.util.Optional; +import java.util.Properties; +import java.util.Set; +import java.util.TreeMap; +import java.util.concurrent.ConcurrentMap; +import java.util.concurrent.TimeoutException; +import java.util.concurrent.atomic.LongAdder; +import java.util.stream.Collectors; import org.apache.rocketmq.acl.AccessValidator; import org.apache.rocketmq.acl.plain.PlainAccessValidator; import org.apache.rocketmq.broker.BrokerController; import org.apache.rocketmq.broker.client.ClientChannelInfo; import org.apache.rocketmq.broker.client.ConsumerGroupInfo; +import org.apache.rocketmq.broker.domain.LogicalQueuesInfoInBroker; import org.apache.rocketmq.broker.filter.ConsumerFilterData; import org.apache.rocketmq.broker.filter.ExpressionMessageFilter; -import org.apache.rocketmq.common.topic.TopicValidator; +import org.apache.rocketmq.broker.topic.TopicConfigManager; import org.apache.rocketmq.broker.transaction.queue.TransactionalMessageUtil; import org.apache.rocketmq.common.AclConfig; import org.apache.rocketmq.common.MQVersion; import org.apache.rocketmq.common.MixAll; import org.apache.rocketmq.common.PlainAccessConfig; import org.apache.rocketmq.common.TopicConfig; +import org.apache.rocketmq.common.TopicQueueId; import org.apache.rocketmq.common.UtilAll; import org.apache.rocketmq.common.admin.ConsumeStats; import org.apache.rocketmq.common.admin.OffsetWrapper; @@ -53,23 +80,29 @@ import org.apache.rocketmq.common.protocol.body.Connection; import org.apache.rocketmq.common.protocol.body.ConsumeQueueData; import org.apache.rocketmq.common.protocol.body.ConsumeStatsList; import org.apache.rocketmq.common.protocol.body.ConsumerConnection; +import org.apache.rocketmq.common.protocol.body.CreateMessageQueueForLogicalQueueRequestBody; import org.apache.rocketmq.common.protocol.body.GroupList; import org.apache.rocketmq.common.protocol.body.KVTable; import org.apache.rocketmq.common.protocol.body.LockBatchRequestBody; import org.apache.rocketmq.common.protocol.body.LockBatchResponseBody; +import org.apache.rocketmq.common.protocol.body.MigrateLogicalQueueBody; import org.apache.rocketmq.common.protocol.body.ProducerConnection; import org.apache.rocketmq.common.protocol.body.QueryConsumeQueueResponseBody; import org.apache.rocketmq.common.protocol.body.QueryConsumeTimeSpanBody; import org.apache.rocketmq.common.protocol.body.QueryCorrectionOffsetBody; import org.apache.rocketmq.common.protocol.body.QueueTimeSpan; +import org.apache.rocketmq.common.protocol.body.ReuseTopicLogicalQueueRequestBody; +import org.apache.rocketmq.common.protocol.body.SealTopicLogicalQueueRequestBody; import org.apache.rocketmq.common.protocol.body.TopicList; import org.apache.rocketmq.common.protocol.body.UnlockBatchRequestBody; +import org.apache.rocketmq.common.protocol.body.UpdateTopicLogicalQueueMappingRequestBody; import org.apache.rocketmq.common.protocol.header.CloneGroupOffsetRequestHeader; import org.apache.rocketmq.common.protocol.header.ConsumeMessageDirectlyResultRequestHeader; import org.apache.rocketmq.common.protocol.header.CreateAccessConfigRequestHeader; import org.apache.rocketmq.common.protocol.header.CreateTopicRequestHeader; import org.apache.rocketmq.common.protocol.header.DeleteAccessConfigRequestHeader; import org.apache.rocketmq.common.protocol.header.DeleteSubscriptionGroupRequestHeader; +import org.apache.rocketmq.common.protocol.header.DeleteTopicLogicalQueueRequestHeader; import org.apache.rocketmq.common.protocol.header.DeleteTopicRequestHeader; import org.apache.rocketmq.common.protocol.header.GetAllTopicConfigResponseHeader; import org.apache.rocketmq.common.protocol.header.GetBrokerAclConfigResponseHeader; @@ -88,11 +121,13 @@ import org.apache.rocketmq.common.protocol.header.GetMaxOffsetResponseHeader; import org.apache.rocketmq.common.protocol.header.GetMinOffsetRequestHeader; import org.apache.rocketmq.common.protocol.header.GetMinOffsetResponseHeader; import org.apache.rocketmq.common.protocol.header.GetProducerConnectionListRequestHeader; +import org.apache.rocketmq.common.protocol.header.GetTopicConfigRequestHeader; import org.apache.rocketmq.common.protocol.header.GetTopicStatsInfoRequestHeader; import org.apache.rocketmq.common.protocol.header.QueryConsumeQueueRequestHeader; import org.apache.rocketmq.common.protocol.header.QueryConsumeTimeSpanRequestHeader; import org.apache.rocketmq.common.protocol.header.QueryCorrectionOffsetHeader; import org.apache.rocketmq.common.protocol.header.QueryTopicConsumeByWhoRequestHeader; +import org.apache.rocketmq.common.protocol.header.QueryTopicLogicalQueueMappingRequestHeader; import org.apache.rocketmq.common.protocol.header.ResetOffsetRequestHeader; import org.apache.rocketmq.common.protocol.header.ResumeCheckHalfMessageRequestHeader; import org.apache.rocketmq.common.protocol.header.SearchOffsetRequestHeader; @@ -102,9 +137,12 @@ import org.apache.rocketmq.common.protocol.header.ViewBrokerStatsDataRequestHead import org.apache.rocketmq.common.protocol.header.filtersrv.RegisterFilterServerRequestHeader; import org.apache.rocketmq.common.protocol.header.filtersrv.RegisterFilterServerResponseHeader; import org.apache.rocketmq.common.protocol.heartbeat.SubscriptionData; +import org.apache.rocketmq.common.protocol.route.LogicalQueueRouteData; +import org.apache.rocketmq.common.protocol.route.MessageQueueRouteState; import org.apache.rocketmq.common.stats.StatsItem; import org.apache.rocketmq.common.stats.StatsSnapshot; import org.apache.rocketmq.common.subscription.SubscriptionGroupConfig; +import org.apache.rocketmq.common.topic.TopicValidator; import org.apache.rocketmq.filter.util.BitsArray; import org.apache.rocketmq.logging.InternalLogger; import org.apache.rocketmq.logging.InternalLoggerFactory; @@ -116,6 +154,7 @@ import org.apache.rocketmq.remoting.netty.NettyRequestProcessor; import org.apache.rocketmq.remoting.protocol.LanguageCode; import org.apache.rocketmq.remoting.protocol.RemotingCommand; import org.apache.rocketmq.remoting.protocol.RemotingSerializable; +import org.apache.rocketmq.srvutil.ConcurrentHashMapUtil; import org.apache.rocketmq.store.ConsumeQueue; import org.apache.rocketmq.store.ConsumeQueueExt; import org.apache.rocketmq.store.DefaultMessageStore; @@ -126,18 +165,6 @@ import org.apache.rocketmq.store.PutMessageResult; import org.apache.rocketmq.store.PutMessageStatus; import org.apache.rocketmq.store.SelectMappedBufferResult; -import java.io.UnsupportedEncodingException; -import java.net.UnknownHostException; -import java.util.ArrayList; -import java.util.HashMap; -import java.util.HashSet; -import java.util.Iterator; -import java.util.List; -import java.util.Map; -import java.util.Properties; -import java.util.Set; -import java.util.concurrent.ConcurrentMap; - public class AdminBrokerProcessor extends AsyncNettyRequestProcessor implements NettyRequestProcessor { private static final InternalLogger log = InternalLoggerFactory.getLogger(LoggerName.BROKER_LOGGER_NAME); private final BrokerController brokerController; @@ -234,6 +261,26 @@ public class AdminBrokerProcessor extends AsyncNettyRequestProcessor implements return resumeCheckHalfMessage(ctx, request); case RequestCode.GET_BROKER_CLUSTER_ACL_CONFIG: return getBrokerClusterAclConfig(ctx, request); + case RequestCode.GET_TOPIC_CONFIG: + return getTopicConfig(ctx, request); + case RequestCode.UPDATE_TOPIC_LOGICAL_QUEUE_MAPPING: + return updateTopicLogicalQueueMapping(ctx, request); + case RequestCode.DELETE_TOPIC_LOGICAL_QUEUE_MAPPING: + return deleteTopicLogicalQueueMapping(ctx, request); + case RequestCode.QUERY_TOPIC_LOGICAL_QUEUE_MAPPING: + return queryTopicLogicalQueueMapping(ctx, request); + case RequestCode.SEAL_TOPIC_LOGICAL_QUEUE: + return sealTopicLogicalQueue(ctx, request); + case RequestCode.REUSE_TOPIC_LOGICAL_QUEUE: + return reuseTopicLogicalQueue(ctx, request); + case RequestCode.CREATE_MESSAGE_QUEUE_FOR_LOGICAL_QUEUE: + return createMessageQueueForLogicalQueue(ctx, request); + case RequestCode.MIGRATE_TOPIC_LOGICAL_QUEUE_PREPARE: + return migrateTopicLogicalQueuePrepare(ctx, request); + case RequestCode.MIGRATE_TOPIC_LOGICAL_QUEUE_COMMIT: + return migrateTopicLogicalQueueCommit(ctx, request); + case RequestCode.MIGRATE_TOPIC_LOGICAL_QUEUE_NOTIFY: + return migrateTopicLogicalQueueNotify(ctx, request); default: break; } @@ -299,6 +346,7 @@ public class AdminBrokerProcessor extends AsyncNettyRequestProcessor implements if (this.brokerController.getBrokerConfig().isAutoDeleteUnusedStats()) { this.brokerController.getBrokerStatsManager().onTopicDeleted(requestHeader.getTopic()); } + this.brokerController.getTopicConfigManager().deleteQueueRouteData(requestHeader.getTopic()); response.setCode(ResponseCode.SUCCESS); response.setRemark(null); return response; @@ -573,7 +621,33 @@ public class AdminBrokerProcessor extends AsyncNettyRequestProcessor implements final GetMaxOffsetRequestHeader requestHeader = (GetMaxOffsetRequestHeader) request.decodeCommandCustomHeader(GetMaxOffsetRequestHeader.class); - long offset = this.brokerController.getMessageStore().getMaxOffsetInQueue(requestHeader.getTopic(), requestHeader.getQueueId()); + String topic = requestHeader.getTopic(); + int queueId = requestHeader.getQueueId(); + + if (requestHeader.getLogicalQueue()) { + LogicalQueuesInfoInBroker logicalQueuesInfo = this.brokerController.getTopicConfigManager().selectLogicalQueuesInfo(topic); + if (logicalQueuesInfo != null) { + // max offset must be in the queue route with largest offset + LogicalQueueRouteData requestLogicalQueueRouteData = logicalQueuesInfo.queryQueueRouteDataByQueueId(queueId, Long.MAX_VALUE); + if (requestLogicalQueueRouteData != null) { + logicalQueuesInfo.readLock().lock(); + try { + List queueRouteDataList = logicalQueuesInfo.get(requestLogicalQueueRouteData.getLogicalQueueIndex()); + if (queueRouteDataList != null && !queueRouteDataList.isEmpty()) { + LogicalQueueRouteData selectedLogicalQueueRouteData = queueRouteDataList.get(queueRouteDataList.size() - 1); + if (!Objects.equals(selectedLogicalQueueRouteData.getMessageQueue(), new MessageQueue(topic, this.brokerController.getBrokerConfig().getBrokerName(), queueId))) { + log.info("getMaxOffset topic={} queueId={} not latest, redirect: {}", topic, queueId, selectedLogicalQueueRouteData); + response.addExtField(MessageConst.PROPERTY_REDIRECT, "1"); + } + } + } finally { + logicalQueuesInfo.readLock().unlock(); + } + } + } + } + + long offset = this.brokerController.getMessageStore().getMaxOffsetInQueue(topic, queueId, requestHeader.isCommitted()); responseHeader.setOffset(offset); @@ -1622,4 +1696,616 @@ public class AdminBrokerProcessor extends AsyncNettyRequestProcessor implements inner.setWaitStoreMsgOK(false); return inner; } + + private RemotingCommand getTopicConfig(ChannelHandlerContext ctx, + RemotingCommand request) throws RemotingCommandException { + GetTopicConfigRequestHeader requestHeader = (GetTopicConfigRequestHeader) request.decodeCommandCustomHeader(GetTopicConfigRequestHeader.class); + final RemotingCommand response = RemotingCommand.createResponseCommand(null); + + TopicConfig topicConfig = this.brokerController.getTopicConfigManager().getTopicConfigTable().get(requestHeader.getTopic()); + if (topicConfig == null) { + log.error("No topic in this broker, client: {} topic: {}", ctx.channel().remoteAddress(), requestHeader.getTopic()); + response.setCode(ResponseCode.SYSTEM_ERROR); + response.setRemark("No topic in this broker. topic: " + requestHeader.getTopic()); + return response; + } + String content = JSONObject.toJSONString(topicConfig); + try { + response.setBody(content.getBytes(MixAll.DEFAULT_CHARSET)); + } catch (UnsupportedEncodingException e) { + log.error("UnsupportedEncodingException getTopicConfig: topic=" + topicConfig.getTopicName(), e); + + response.setCode(ResponseCode.SYSTEM_ERROR); + response.setRemark("UnsupportedEncodingException " + e.getMessage()); + return response; + } + response.setCode(ResponseCode.SUCCESS); + response.setRemark(null); + + return response; + } + + private RemotingCommand updateTopicLogicalQueueMapping(ChannelHandlerContext ctx, + RemotingCommand request) throws RemotingCommandException { + UpdateTopicLogicalQueueMappingRequestBody requestBody = RemotingSerializable.decode(request.getBody(), UpdateTopicLogicalQueueMappingRequestBody.class); + RemotingCommand response = RemotingCommand.createResponseCommand(null); + response.setCode(ResponseCode.SYSTEM_ERROR); + response.setRemark("unknown error"); + if (requestBody == null) { + response.setRemark("decode null"); + return response; + } + String topic = requestBody.getTopic(); + int queueId = requestBody.getQueueId(); + String brokerName = this.brokerController.getBrokerConfig().getBrokerName(); + int logicalQueueIdx = requestBody.getLogicalQueueIdx(); + log.info("updateTopicLogicalQueueMapping topic={} queueId={} logicalQueueIndex={}", topic, queueId, logicalQueueIdx); + TopicConfigManager topicConfigManager = this.brokerController.getTopicConfigManager(); + TopicConfig topicConfig = topicConfigManager.selectTopicConfig(topic); + if (topicConfig == null) { + log.warn("updateTopicLogicalQueueMapping topic={} queueId={} logicalQueueIndex={} topic not exist", topic, queueId, logicalQueueIdx); + response.setRemark("topic not exist"); + return response; + } + + LogicalQueuesInfoInBroker logicalQueuesInfo; + LogicalQueueRouteData newQueueRouteData = new LogicalQueueRouteData(); + newQueueRouteData.setBrokerAddr(this.brokerController.getBrokerAddr()); + newQueueRouteData.setMessageQueue(new MessageQueue(topic, brokerName, queueId)); + if (logicalQueueIdx >= 0) { + // add logical queue + newQueueRouteData.setLogicalQueueIndex(logicalQueueIdx); + newQueueRouteData.setLogicalQueueDelta(0L); + newQueueRouteData.setState(MessageQueueRouteState.Normal); + logicalQueuesInfo = topicConfigManager.getOrCreateLogicalQueuesInfo(topic); + logicalQueuesInfo.writeLock().lock(); + try { + // verify whether this message queue is already set up + for (List queueRouteDataList : logicalQueuesInfo.values()) { + for (Iterator iterator = queueRouteDataList.iterator(); iterator.hasNext(); ) { + LogicalQueueRouteData queueRouteData = iterator.next(); + if (Objects.equals(queueRouteData.getMessageQueue(), newQueueRouteData.getMessageQueue())) { + if (queueRouteData.getLogicalQueueIndex() == logicalQueueIdx) { + log.info("updateTopicLogicalQueueMapping topic={} queueId={} logicalQueueIndex={} already set up", topic, queueId, logicalQueueIdx); + response.setCode(ResponseCode.SUCCESS); + response.setRemark("already set up"); + return response; + } else { + log.warn("updateTopicLogicalQueueMapping topic={} queueId={} already assigned logicalQueueIdx={}, will reassign as logicalQueueIdx={}", topic, queueRouteData.getMessageQueue(), queueRouteData.getLogicalQueueIndex(), newQueueRouteData.getLogicalQueueIndex()); + iterator.remove(); + break; + } + } + } + } + List queueRouteDataList = logicalQueuesInfo.computeIfAbsent(logicalQueueIdx, ignore -> Lists.newArrayListWithExpectedSize(1)); + int idx = Collections.binarySearch(queueRouteDataList, newQueueRouteData, Comparator.comparingLong(LogicalQueueRouteData::getLogicalQueueDelta).thenComparingInt(LogicalQueueRouteData::getStateOrdinal)); + if (idx >= 0) { + log.warn("updateTopicLogicalQueueMapping topic={} queueId={} logicalQueueIdx={} found same logicalQueueOffset and will replace, exist {}, new {}", topic, queueId, logicalQueueIdx, queueRouteDataList.get(idx), newQueueRouteData); + queueRouteDataList.set(idx, newQueueRouteData); + response.setCode(ResponseCode.SYSTEM_ERROR); + response.setRemark("duplicate logicalQueueOffset found"); + } else { + idx = -idx - 1; + queueRouteDataList.add(idx, newQueueRouteData); + logicalQueuesInfo.updateQueueRouteDataByQueueId(newQueueRouteData.getQueueId(), newQueueRouteData); + response.setCode(ResponseCode.SUCCESS); + response.setRemark("set up"); + log.info("updateTopicLogicalQueueMapping topic={} queueId={} logicalQueueIdx={} added as #{}", topic, queueId, logicalQueueIdx, idx); + } + } finally { + logicalQueuesInfo.writeLock().unlock(); + } + } else { + // delete logical queue + logicalQueuesInfo = topicConfigManager.getOrCreateLogicalQueuesInfo(topic); + logicalQueuesInfo.writeLock().lock(); + try { + for (List queueRouteDataList : logicalQueuesInfo.values()) { + queueRouteDataList.removeIf(queueRouteData -> Objects.equals(queueRouteData.getMessageQueue(), newQueueRouteData.getMessageQueue())); + } + } finally { + logicalQueuesInfo.writeLock().unlock(); + } + logicalQueuesInfo.updateQueueRouteDataByQueueId(newQueueRouteData.getQueueId(), null); + this.brokerController.getSendMessageProcessor().getInFlyWritingCounterMap().remove(new TopicQueueId(topic, queueId)); + response.setCode(ResponseCode.SUCCESS); + response.setRemark("deleted"); + log.info("updateTopicLogicalQueueMapping topic={} queueId={} deleted as logical queue", topic, queueId, logicalQueueIdx); + } + + topicConfigManager.getDataVersion().nextVersion(); + topicConfigManager.persist(topic, logicalQueuesInfo); + this.brokerController.registerIncrementBrokerData(topicConfig, topicConfigManager.getDataVersion()); + + return response; + } + + private RemotingCommand deleteTopicLogicalQueueMapping(ChannelHandlerContext ctx, + RemotingCommand request) throws RemotingCommandException { + final RemotingCommand response = RemotingCommand.createResponseCommand(null); + DeleteTopicLogicalQueueRequestHeader requestHeader = + (DeleteTopicLogicalQueueRequestHeader) request.decodeCommandCustomHeader(DeleteTopicLogicalQueueRequestHeader.class); + String topic = requestHeader.getTopic(); + log.info("deleteTopicLogicalQueueMapping topic={}", topic); + + TopicConfigManager topicConfigManager = this.brokerController.getTopicConfigManager(); + LogicalQueuesInfoInBroker logicalQueuesInfo = topicConfigManager.selectLogicalQueuesInfo(topic); + if (logicalQueuesInfo == null) { + response.setCode(ResponseCode.SUCCESS); + response.setRemark("already deleted"); + log.info("deleteTopicLogicalQueueMapping topic={} already deleted", topic); + return response; + } + String brokerName = this.brokerController.getBrokerConfig().getBrokerName(); + long size = logicalQueuesInfo.values().stream().flatMap(Collection::stream).filter(v -> Objects.equals(v.getBrokerName(), brokerName)).count(); + if (size > 0) { + response.setCode(ResponseCode.SYSTEM_ERROR); + response.setRemark(String.format(Locale.ENGLISH, "still %d message queues", size)); + log.info("deleteTopicLogicalQueueMapping topic={} still {} message queues", topic, size); + return response; + } + topicConfigManager.deleteQueueRouteData(topic); + response.setCode(ResponseCode.SUCCESS); + response.setRemark("deleted"); + + topicConfigManager.getDataVersion().nextVersion(); + topicConfigManager.persist(topic, logicalQueuesInfo); + this.brokerController.registerIncrementBrokerData(topicConfigManager.selectTopicConfig(topic), topicConfigManager.getDataVersion()); + + log.info("deleteTopicLogicalQueueMapping topic={} deleted", topic); + return response; + } + + private RemotingCommand queryTopicLogicalQueueMapping(ChannelHandlerContext ctx, + RemotingCommand request) throws RemotingCommandException { + final RemotingCommand response = RemotingCommand.createResponseCommand(null); + QueryTopicLogicalQueueMappingRequestHeader requestHeader = + (QueryTopicLogicalQueueMappingRequestHeader) request.decodeCommandCustomHeader(QueryTopicLogicalQueueMappingRequestHeader.class); + String topic = requestHeader.getTopic(); + log.info("queryTopicLogicalQueueMapping topic={}", topic); + LogicalQueuesInfoInBroker logicalQueuesInfo = brokerController.getTopicConfigManager().selectLogicalQueuesInfo(topic); + TreeMap> result = null; + if (logicalQueuesInfo != null) { + result = Maps.newTreeMap(); + logicalQueuesInfo.readLock().lock(); + try { + for (Map.Entry> entry : logicalQueuesInfo.entrySet()) { + Integer k = entry.getKey(); + List v = entry.getValue(); + result.put(k, ImmutableList.copyOf(v)); + } + } finally { + logicalQueuesInfo.readLock().unlock(); + } + } + response.setCode(ResponseCode.SUCCESS); + response.setBody(RemotingSerializable.encode(result)); + return response; + } + + private void sealLogicalQueueRouteData(LogicalQueueRouteData queueRouteData, + MessageStore messageStore) throws TimeoutException, InterruptedException { + queueRouteData.setState(MessageQueueRouteState.ReadOnly); + + String topic = queueRouteData.getTopic(); + int queueId = queueRouteData.getQueueId(); + + // busy wait for all in-fly messages to be finished + TopicQueueId key = new TopicQueueId(topic, queueId); + long startTime = System.currentTimeMillis(); + while (true) { + LongAdder counter = this.brokerController.getSendMessageProcessor().getInFlyWritingCounterMap().get(key); + if (counter == null || counter.sum() == 0) { + break; + } + if (System.currentTimeMillis() - startTime > 10_000) { + throw new TimeoutException(); + } + Thread.sleep(100); + } + // busy wait for all CQ to be finished + while (messageStore.getMaxOffsetInQueue(topic, queueId, true) != messageStore.getMaxOffsetInQueue(topic, queueId, false)) { + if (System.currentTimeMillis() - startTime > 10_000) { + throw new TimeoutException(); + } + Thread.sleep(100); + } + + long firstMsgQueueOffset = messageStore.getMinOffsetInQueue(topic, queueId); + long lastMsgQueueOffset = messageStore.getMaxOffsetInQueue(topic, queueId, false) - 1; + long firstMsgTimeMillis = 0L; + long lastMsgTimeMillis = 0L; + boolean expired = false; + if (firstMsgQueueOffset == lastMsgQueueOffset) { + // no message at all + expired = true; + } else { + long minPhyOffset = messageStore.getMinPhyOffset(); + long lastMsgCommitLogOffset = lastMsgQueueOffset >= 0 ? messageStore.getCommitLogOffsetInQueue(topic, queueId, lastMsgQueueOffset) : -1; + if (lastMsgCommitLogOffset < minPhyOffset) { + // commitLog already cleaned + expired = true; + } else { + long firstMsgCommitLogOffset = firstMsgQueueOffset >= 0 ? messageStore.getCommitLogOffsetInQueue(topic, queueId, firstMsgQueueOffset) : -1; + MessageExt firstMsg = firstMsgCommitLogOffset >= 0 ? messageStore.lookMessageByOffset(firstMsgCommitLogOffset) : null; + firstMsgTimeMillis = firstMsg != null ? firstMsg.getStoreTimestamp() : 0L; + + MessageExt lastMsg = lastMsgCommitLogOffset >= 0 ? messageStore.lookMessageByOffset(lastMsgCommitLogOffset) : null; + lastMsgTimeMillis = lastMsg != null ? lastMsg.getStoreTimestamp() : 0L; + } + } + + queueRouteData.setOffsetMax(lastMsgQueueOffset + 1); + queueRouteData.setFirstMsgTimeMillis(firstMsgTimeMillis); + queueRouteData.setLastMsgTimeMillis(lastMsgTimeMillis); + + if (expired) { + queueRouteData.setState(MessageQueueRouteState.Expired); + } + } + + private RemotingCommand sealTopicLogicalQueue(ChannelHandlerContext ctx, RemotingCommand request) { + SealTopicLogicalQueueRequestBody requestBody = RemotingSerializable.decode(request.getBody(), SealTopicLogicalQueueRequestBody.class); + if (requestBody == null) { + return RemotingCommand.createResponseCommand(ResponseCode.SYSTEM_ERROR, "decode error"); + } + String topic = requestBody.getTopic(); + int queueId = requestBody.getQueueId(); + int logicalQueueIdx = requestBody.getLogicalQueueIndex(); + log.info("sealTopicLogicalQueue topic={} queueId={}", topic, queueId); + TopicConfigManager topicConfigManager = this.brokerController.getTopicConfigManager(); + TopicConfig topicConfig = topicConfigManager.selectTopicConfig(topic); + if (topicConfig == null) { + log.warn("sealTopicLogicalQueue topic={} queueId={} topic not exist", topic, queueId); + return RemotingCommand.createResponseCommand(ResponseCode.SYSTEM_ERROR, "topic not exist"); + } + final RemotingCommand response = RemotingCommand.createResponseCommand(null); + response.setCode(ResponseCode.SUCCESS); + + LogicalQueueRouteData resultQueueRouteData; + + LogicalQueuesInfoInBroker logicalQueuesInfo = topicConfigManager.getOrCreateLogicalQueuesInfo(topic); + logicalQueuesInfo.readLock().lock(); + try { + List queueRouteDataList = logicalQueuesInfo.get(logicalQueueIdx); + if (queueRouteDataList == null) { + log.info("sealTopicLogicalQueue topic={} queueId={} logicalQueueIdx={} not exist", topic, queueId, logicalQueueIdx); + response.setRemark("logical queue not exist"); + return response; + } + List foundQueues = queueRouteDataList.stream() + .filter(queueRouteData -> queueId == queueRouteData.getQueueId()).collect(Collectors.toList()); + if (foundQueues.isEmpty()) { + log.info("sealTopicLogicalQueue topic={} queueId={} logicalQueueIdx={} queueId={} not exist", topic, queueId, logicalQueueIdx, queueId); + response.setRemark("message queue not exist"); + return response; + } + Optional firstMainQueueRouteDataOptional = foundQueues.stream().filter(LogicalQueueRouteData::isWritable).findFirst(); + if (!firstMainQueueRouteDataOptional.isPresent()) { + log.info("sealTopicLogicalQueue topic={} queueId={} logicalQueueIdx={} queueId={} already sealed", topic, queueId, logicalQueueIdx, queueId); + response.setRemark("message queue already sealed"); + return response; + } + resultQueueRouteData = firstMainQueueRouteDataOptional.get(); + } finally { + logicalQueuesInfo.readLock().unlock(); + } + try { + sealLogicalQueueRouteData(resultQueueRouteData, brokerController.getMessageStore()); + } catch (InterruptedException e) { + return RemotingCommand.createResponseCommand(ResponseCode.SYSTEM_ERROR, "thread interrupted"); + } catch (TimeoutException e) { + return RemotingCommand.createResponseCommand(ResponseCode.SYSTEM_ERROR, "seal timeout"); + } + + topicConfigManager.getDataVersion().nextVersion(); + topicConfigManager.persist(topic, logicalQueuesInfo); + this.brokerController.registerIncrementBrokerData(topicConfigManager.selectTopicConfig(topic), topicConfigManager.getDataVersion()); + + response.setBody(RemotingSerializable.encode(resultQueueRouteData)); + return response; + } + + private RemotingCommand reuseTopicLogicalQueue(ChannelHandlerContext ctx, RemotingCommand request) { + ReuseTopicLogicalQueueRequestBody requestBody = RemotingSerializable.decode(request.getBody(), ReuseTopicLogicalQueueRequestBody.class); + if (requestBody == null) { + return RemotingCommand.createResponseCommand(ResponseCode.SYSTEM_ERROR, "decode error"); + } + String topic = requestBody.getTopic(); + int queueId = requestBody.getQueueId(); + String brokerName = this.brokerController.getBrokerConfig().getBrokerName(); + MessageQueue mq = new MessageQueue(topic, brokerName, queueId); + int logicalQueueIdx = requestBody.getLogicalQueueIndex(); + MessageQueueRouteState messageQueueRouteState = requestBody.getMessageQueueRouteState(); + log.info("reuseTopicLogicalQueue topic={} queueId={} logicalQueueIdx={} messageQueueRouteState={}", topic, queueId, logicalQueueIdx, messageQueueRouteState); + TopicConfigManager topicConfigManager = this.brokerController.getTopicConfigManager(); + LogicalQueuesInfoInBroker logicalQueuesInfo = topicConfigManager.selectLogicalQueuesInfo(topic); + if (logicalQueuesInfo == null) { + return RemotingCommand.createResponseCommand(ResponseCode.SYSTEM_ERROR, "queue route data not found"); + } + if (queueId >= topicConfigManager.selectTopicConfig(topic).getWriteQueueNums()) { + return RemotingCommand.createResponseCommand(ResponseCode.SYSTEM_ERROR, "found no queue"); + } + logicalQueuesInfo.writeLock().lock(); + LogicalQueueRouteData queueRouteData = new LogicalQueueRouteData( + logicalQueueIdx, + MessageQueueRouteState.WriteOnly.equals(messageQueueRouteState) ? -1 : 0, + new MessageQueue(topic, brokerName, queueId), + messageQueueRouteState, + this.brokerController.getMessageStore().getMaxOffsetInQueue(topic, queueId, false), + -1, + -1, + -1, + this.brokerController.getBrokerAddr() + ); + try { + if (logicalQueuesInfo.values().stream().flatMap(Collection::stream).filter(v -> Objects.equals(v.getMessageQueue(), mq)).anyMatch(LogicalQueueRouteData::isWritable)) { + return RemotingCommand.createResponseCommand(ResponseCode.SYSTEM_ERROR, "queue writable"); + } + logicalQueuesInfo.computeIfAbsent(logicalQueueIdx, ignore -> Lists.newArrayListWithExpectedSize(1)).add(queueRouteData); + logicalQueuesInfo.updateQueueRouteDataByQueueId(queueId, queueRouteData); + this.brokerController.getSendMessageProcessor().getInFlyWritingCounterMap().remove(new TopicQueueId(topic, queueId)); + } finally { + logicalQueuesInfo.writeLock().unlock(); + } + + topicConfigManager.getDataVersion().nextVersion(); + topicConfigManager.persist(topic, logicalQueuesInfo); + this.brokerController.registerIncrementBrokerData(topicConfigManager.selectTopicConfig(topic), topicConfigManager.getDataVersion()); + + final RemotingCommand response = RemotingCommand.createResponseCommand(null); + response.setCode(ResponseCode.SUCCESS); + response.setBody(RemotingSerializable.encode(queueRouteData)); + return response; + } + + private RemotingCommand createMessageQueueForLogicalQueue(ChannelHandlerContext ctx, RemotingCommand request) { + CreateMessageQueueForLogicalQueueRequestBody requestBody = RemotingSerializable.decode(request.getBody(), CreateMessageQueueForLogicalQueueRequestBody.class); + if (requestBody == null) { + return RemotingCommand.createResponseCommand(ResponseCode.SYSTEM_ERROR, "decode error"); + } + String topic = requestBody.getTopic(); + int logicalQueueIdx = requestBody.getLogicalQueueIndex(); + MessageQueueRouteState messageQueueStatus = requestBody.getMessageQueueStatus(); + log.info("createMessageQueueForLogicalQueue topic={} logicalQueueIdx={}", topic, logicalQueueIdx); + + TopicConfigManager topicConfigManager = this.brokerController.getTopicConfigManager(); + LogicalQueuesInfoInBroker logicalQueuesInfo; + LogicalQueueRouteData queueRouteData; + TopicConfig topicConfig; + while (true) { + topicConfig = topicConfigManager.selectTopicConfig(topic); + if (topicConfig == null || topicConfig.getWriteQueueNums() == 0) { + // create topic if not exist + topicConfig = ConcurrentHashMapUtil.computeIfAbsent(topicConfigManager.getTopicConfigTable(), topic, s -> new TopicConfig(topic, 0, 0, this.brokerController.getBrokerConfig().getBrokerPermission())); + logicalQueuesInfo = topicConfigManager.getOrCreateLogicalQueuesInfo(topic); + } else { + logicalQueuesInfo = topicConfigManager.selectLogicalQueuesInfo(topic); + } + if (topicConfig.getWriteQueueNums() > 0 && logicalQueuesInfo == null) { + return RemotingCommand.createResponseCommand(ResponseCode.SYSTEM_ERROR, "topic not enable logical queue"); + } + TopicConfig newTopicConfig = new TopicConfig(topicConfig); + newTopicConfig.setWriteQueueNums(newTopicConfig.getWriteQueueNums() + 1); + newTopicConfig.setReadQueueNums(newTopicConfig.getReadQueueNums() + 1); + + int queueId = newTopicConfig.getWriteQueueNums() - 1; + queueRouteData = new LogicalQueueRouteData(); + queueRouteData.setLogicalQueueIndex(logicalQueueIdx); + queueRouteData.setBrokerAddr(this.brokerController.getBrokerAddr()); + queueRouteData.setMessageQueue(new MessageQueue(topic, this.brokerController.getBrokerConfig().getBrokerName(), queueId)); + if (messageQueueStatus != null) { + queueRouteData.setState(messageQueueStatus); + switch (messageQueueStatus) { + case WriteOnly: + case Expired: + queueRouteData.setLogicalQueueDelta(-1L); + break; + default: + queueRouteData.setLogicalQueueDelta(0L); + } + } + + logicalQueuesInfo.writeLock().lock(); + try { + List l = logicalQueuesInfo.computeIfAbsent(logicalQueueIdx, i -> Lists.newArrayListWithExpectedSize(1)); + if (MessageQueueRouteState.WriteOnly.equals(messageQueueStatus) && l.stream().anyMatch(LogicalQueueRouteData::isWriteOnly)) { + return RemotingCommand.createResponseCommand(ResponseCode.SYSTEM_ERROR, "multiple WriteOnly queue"); + } else if (MessageQueueRouteState.Normal.equals(messageQueueStatus) && l.stream().anyMatch(LogicalQueueRouteData::isWritable)) { + return RemotingCommand.createResponseCommand(ResponseCode.SYSTEM_ERROR, "multiple writable queue"); + } + if (topicConfigManager.replaceTopicConfig(topic, topicConfig, newTopicConfig)) { + l.add(queueRouteData); + logicalQueuesInfo.updateQueueRouteDataByQueueId(queueId, queueRouteData); + break; + } + } finally { + logicalQueuesInfo.writeLock().unlock(); + } + } + + topicConfigManager.getDataVersion().nextVersion(); + topicConfigManager.persist(topic, logicalQueuesInfo); + this.brokerController.registerIncrementBrokerData(topicConfig, topicConfigManager.getDataVersion()); + + final RemotingCommand response = RemotingCommand.createResponseCommand(null); + response.setCode(ResponseCode.SUCCESS); + response.setBody(RemotingSerializable.encode(queueRouteData)); + return response; + } + + private RemotingCommand migrateTopicLogicalQueuePrepare(ChannelHandlerContext ctx, RemotingCommand request) { + MigrateLogicalQueueBody reqRespBody = RemotingSerializable.decode(request.getBody(), MigrateLogicalQueueBody.class); + if (reqRespBody == null) { + return RemotingCommand.createResponseCommand(ResponseCode.SYSTEM_ERROR, "decode error"); + } + LogicalQueueRouteData fromQueueRouteData = reqRespBody.getFromQueueRouteData(); + LogicalQueueRouteData toQueueRouteData = reqRespBody.getToQueueRouteData(); + log.info("migrateTopicLogicalQueuePrepare fromQueueRouteData={} toQueueRouteData={}", fromQueueRouteData, toQueueRouteData); + final MessageQueue fromMessageQueue = fromQueueRouteData.getMessageQueue(); + final int fromQueueId = fromQueueRouteData.getQueueId(); + final long fromOffsetDelta = fromQueueRouteData.getOffsetDelta(); + int logicalQueueIndex = fromQueueRouteData.getLogicalQueueIndex(); + TopicConfigManager topicConfigManager = this.brokerController.getTopicConfigManager(); + String topic = fromQueueRouteData.getTopic(); + if (!topicConfigManager.getTopicConfigTable().containsKey(topic)) { + return RemotingCommand.createResponseCommand(ResponseCode.SYSTEM_ERROR, "topic not exist"); + } + LogicalQueuesInfoInBroker logicalQueuesInfo = topicConfigManager.selectLogicalQueuesInfo(topic); + if (logicalQueuesInfo == null) { + return RemotingCommand.createResponseCommand(ResponseCode.SYSTEM_ERROR, "topic not enable logical queue"); + } + logicalQueuesInfo.writeLock().lock(); + try { + List queueRouteDataList = logicalQueuesInfo.get(logicalQueueIndex); + if (queueRouteDataList == null) { + return RemotingCommand.createResponseCommand(ResponseCode.SYSTEM_ERROR, String.format(Locale.ENGLISH, "logical queue %d not exist", logicalQueueIndex)); + } + fromQueueRouteData = null; + for (LogicalQueueRouteData v : queueRouteDataList) { + if (v.isSameTo(fromMessageQueue, fromOffsetDelta)) { + fromQueueRouteData = v; + break; + } + } + if (fromQueueRouteData == null) { + return RemotingCommand.createResponseCommand(ResponseCode.SYSTEM_ERROR, String.format(Locale.ENGLISH, "message queue %d not exist", fromQueueId)); + } + if (!MessageQueueRouteState.Normal.equals(fromQueueRouteData.getState())) { + return RemotingCommand.createResponseCommand(ResponseCode.SYSTEM_ERROR, String.format(Locale.ENGLISH, "message queue %d not normal state", fromQueueId)); + } + reqRespBody.setFromQueueRouteData(fromQueueRouteData); + if (fromQueueRouteData.isWritable()) { + sealLogicalQueueRouteData(fromQueueRouteData, brokerController.getMessageStore()); + } + toQueueRouteData.setLogicalQueueDelta(fromQueueRouteData.getLogicalQueueDelta() + fromQueueRouteData.getMessagesCount()); + queueRouteDataList.add(toQueueRouteData); + } catch (InterruptedException e) { + return RemotingCommand.createResponseCommand(ResponseCode.SYSTEM_ERROR, "thread interrupted"); + } catch (TimeoutException e) { + return RemotingCommand.createResponseCommand(ResponseCode.SYSTEM_ERROR, "seal timeout"); + } finally { + logicalQueuesInfo.writeLock().unlock(); + } + + topicConfigManager.getDataVersion().nextVersion(); + topicConfigManager.persist(topic, logicalQueuesInfo); + this.brokerController.registerIncrementBrokerData(topicConfigManager.selectTopicConfig(topic), topicConfigManager.getDataVersion()); + + final RemotingCommand response = RemotingCommand.createResponseCommand(null); + response.setCode(ResponseCode.SUCCESS); + response.setBody(RemotingSerializable.encode(reqRespBody)); + return response; + } + + private RemotingCommand migrateTopicLogicalQueueCommit(ChannelHandlerContext ctx, RemotingCommand request) { + MigrateLogicalQueueBody reqRespBody = RemotingSerializable.decode(request.getBody(), MigrateLogicalQueueBody.class); + if (reqRespBody == null) { + return RemotingCommand.createResponseCommand(ResponseCode.SYSTEM_ERROR, "decode error"); + } + LogicalQueueRouteData fromQueueRouteData = reqRespBody.getFromQueueRouteData(); + LogicalQueueRouteData toQueueRouteData = reqRespBody.getToQueueRouteData(); + log.info("migrateTopicLogicalQueueCommit toQueueRouteData={}", toQueueRouteData); + TopicConfigManager topicConfigManager = this.brokerController.getTopicConfigManager(); + String topic = toQueueRouteData.getTopic(); + int toQueueId = toQueueRouteData.getQueueId(); + if (!topicConfigManager.getTopicConfigTable().containsKey(topic)) { + return RemotingCommand.createResponseCommand(ResponseCode.SYSTEM_ERROR, "topic not exist"); + } + LogicalQueuesInfoInBroker logicalQueuesInfo = topicConfigManager.selectLogicalQueuesInfo(topic); + if (logicalQueuesInfo == null) { + return RemotingCommand.createResponseCommand(ResponseCode.SYSTEM_ERROR, "topic not enable logical queue"); + } + LogicalQueueRouteData queueRouteData; + logicalQueuesInfo.writeLock().lock(); + try { + List queueRouteDataList = logicalQueuesInfo.get(toQueueRouteData.getLogicalQueueIndex()); + if (queueRouteDataList == null) { + return RemotingCommand.createResponseCommand(ResponseCode.SYSTEM_ERROR, String.format(Locale.ENGLISH, "logical queue %d not exist", toQueueRouteData.getLogicalQueueIndex())); + } + queueRouteDataList.stream().filter(fromQueueRouteData::isSameTo).forEach(d -> { + d.copyFrom(fromQueueRouteData); + }); + queueRouteData = queueRouteDataList.stream().filter(toQueueRouteData::isSameTo).findFirst().orElse(null); + if (queueRouteData == null) { + return RemotingCommand.createResponseCommand(ResponseCode.SYSTEM_ERROR, String.format(Locale.ENGLISH, "message queue %d-%d not exist", toQueueId, toQueueRouteData.getOffsetDelta())); + } + queueRouteData.setLogicalQueueDelta(toQueueRouteData.getLogicalQueueDelta()); + queueRouteData.setState(MessageQueueRouteState.Normal); + toQueueRouteData.setState(MessageQueueRouteState.Normal); + if (toQueueRouteData.getBrokerAddr() != null) { + queueRouteData.setBrokerAddr(toQueueRouteData.getBrokerAddr()); + } + } finally { + logicalQueuesInfo.writeLock().unlock(); + } + + topicConfigManager.getDataVersion().nextVersion(); + topicConfigManager.persist(topic, logicalQueuesInfo); + this.brokerController.registerIncrementBrokerData(topicConfigManager.selectTopicConfig(topic), topicConfigManager.getDataVersion()); + + final RemotingCommand response = RemotingCommand.createResponseCommand(null); + response.setCode(ResponseCode.SUCCESS); + response.setBody(RemotingSerializable.encode(reqRespBody)); + return response; + } + + private RemotingCommand migrateTopicLogicalQueueNotify(ChannelHandlerContext ctx, RemotingCommand request) { + MigrateLogicalQueueBody requestBody = RemotingSerializable.decode(request.getBody(), MigrateLogicalQueueBody.class); + if (requestBody == null) { + return RemotingCommand.createResponseCommand(ResponseCode.SYSTEM_ERROR, "decode error"); + } + LogicalQueueRouteData fromQueueRouteData = requestBody.getFromQueueRouteData(); + LogicalQueueRouteData toQueueRouteData = requestBody.getToQueueRouteData(); + log.info("migrateTopicLogicalQueueNotify fromQueueRouteData={} toQueueRouteData={}", fromQueueRouteData, toQueueRouteData); + TopicConfigManager topicConfigManager = this.brokerController.getTopicConfigManager(); + String topic = toQueueRouteData.getTopic(); + if (!topicConfigManager.getTopicConfigTable().containsKey(topic)) { + return RemotingCommand.createResponseCommand(ResponseCode.SYSTEM_ERROR, "topic not exist"); + } + LogicalQueuesInfoInBroker logicalQueuesInfo = topicConfigManager.selectLogicalQueuesInfo(topic); + if (logicalQueuesInfo == null) { + return RemotingCommand.createResponseCommand(ResponseCode.SYSTEM_ERROR, "topic not enable logical queue"); + } + List requestQueueRouteDataList = Lists.newArrayList(fromQueueRouteData, toQueueRouteData); + boolean toQueueRouteDataFound = false; + logicalQueuesInfo.writeLock().lock(); + try { + List queueRouteDataList = logicalQueuesInfo.computeIfAbsent(toQueueRouteData.getLogicalQueueIndex(), ignore -> Lists.newArrayListWithExpectedSize(1)); + for (LogicalQueueRouteData v : queueRouteDataList) { + for (Iterator iterator = requestQueueRouteDataList.iterator(); iterator.hasNext(); ) { + LogicalQueueRouteData queueRouteData = iterator.next(); + if (queueRouteData.isSameTo(v)) { + v.copyFrom(queueRouteData); + if (queueRouteData.getBrokerAddr() != null) { + v.setBrokerAddr(queueRouteData.getBrokerAddr()); + } + if (!toQueueRouteDataFound && toQueueRouteData.isSameTo(v)) { + toQueueRouteDataFound = true; + } + iterator.remove(); + break; + } + } + if (requestQueueRouteDataList.isEmpty()) { + break; + } + } + if (!queueRouteDataList.isEmpty() && !toQueueRouteDataFound) { + // if this broker has this logical queue before, it should add latest writable route here, so that SendMessage request can be proxied + queueRouteDataList.add(toQueueRouteData); + } + } finally { + logicalQueuesInfo.writeLock().unlock(); + } + + topicConfigManager.getDataVersion().nextVersion(); + topicConfigManager.persist(topic, logicalQueuesInfo); + this.brokerController.registerIncrementBrokerData(topicConfigManager.selectTopicConfig(topic), topicConfigManager.getDataVersion()); + + final RemotingCommand response = RemotingCommand.createResponseCommand(null); + response.setCode(ResponseCode.SUCCESS); + return response; + } } diff --git a/broker/src/main/java/org/apache/rocketmq/broker/processor/PullMessageProcessor.java b/broker/src/main/java/org/apache/rocketmq/broker/processor/PullMessageProcessor.java index 8879a722fcf1967fd013aea6730c48bda6cf5c5c..e61ef11309e93ddb701bbe0adfc6e293cf354e9f 100644 --- a/broker/src/main/java/org/apache/rocketmq/broker/processor/PullMessageProcessor.java +++ b/broker/src/main/java/org/apache/rocketmq/broker/processor/PullMessageProcessor.java @@ -16,6 +16,7 @@ */ package org.apache.rocketmq.broker.processor; +import com.google.common.collect.ImmutableList; import io.netty.channel.Channel; import io.netty.channel.ChannelFuture; import io.netty.channel.ChannelFutureListener; @@ -25,6 +26,7 @@ import java.nio.ByteBuffer; import java.util.List; import org.apache.rocketmq.broker.BrokerController; import org.apache.rocketmq.broker.client.ConsumerGroupInfo; +import org.apache.rocketmq.broker.domain.LogicalQueuesInfoInBroker; import org.apache.rocketmq.broker.filter.ConsumerFilterData; import org.apache.rocketmq.broker.filter.ConsumerFilterManager; import org.apache.rocketmq.broker.filter.ExpressionForRetryMessageFilter; @@ -41,6 +43,7 @@ import org.apache.rocketmq.common.constant.PermName; import org.apache.rocketmq.common.filter.ExpressionType; import org.apache.rocketmq.common.filter.FilterAPI; import org.apache.rocketmq.common.help.FAQUrl; +import org.apache.rocketmq.common.message.MessageConst; import org.apache.rocketmq.common.message.MessageDecoder; import org.apache.rocketmq.common.message.MessageQueue; import org.apache.rocketmq.common.protocol.ResponseCode; @@ -48,6 +51,8 @@ import org.apache.rocketmq.common.protocol.header.PullMessageRequestHeader; import org.apache.rocketmq.common.protocol.header.PullMessageResponseHeader; import org.apache.rocketmq.common.protocol.heartbeat.MessageModel; import org.apache.rocketmq.common.protocol.heartbeat.SubscriptionData; +import org.apache.rocketmq.common.protocol.route.LogicalQueueRouteData; +import org.apache.rocketmq.common.protocol.route.MessageQueueRouteState; import org.apache.rocketmq.common.protocol.topic.OffsetMovedEvent; import org.apache.rocketmq.common.subscription.SubscriptionGroupConfig; import org.apache.rocketmq.common.sysflag.MessageSysFlag; @@ -62,6 +67,7 @@ import org.apache.rocketmq.remoting.netty.AsyncNettyRequestProcessor; import org.apache.rocketmq.remoting.netty.NettyRequestProcessor; import org.apache.rocketmq.remoting.netty.RequestTask; import org.apache.rocketmq.remoting.protocol.RemotingCommand; +import org.apache.rocketmq.remoting.protocol.RemotingSerializable; import org.apache.rocketmq.store.GetMessageResult; import org.apache.rocketmq.store.MessageExtBrokerInner; import org.apache.rocketmq.store.MessageFilter; @@ -126,23 +132,25 @@ public class PullMessageProcessor extends AsyncNettyRequestProcessor implements final long suspendTimeoutMillisLong = hasSuspendFlag ? requestHeader.getSuspendTimeoutMillis() : 0; - TopicConfig topicConfig = this.brokerController.getTopicConfigManager().selectTopicConfig(requestHeader.getTopic()); + String topic = requestHeader.getTopic(); + TopicConfig topicConfig = this.brokerController.getTopicConfigManager().selectTopicConfig(topic); if (null == topicConfig) { - log.error("the topic {} not exist, consumer: {}", requestHeader.getTopic(), RemotingHelper.parseChannelRemoteAddr(channel)); + log.error("the topic {} not exist, consumer: {}", topic, RemotingHelper.parseChannelRemoteAddr(channel)); response.setCode(ResponseCode.TOPIC_NOT_EXIST); - response.setRemark(String.format("topic[%s] not exist, apply first please! %s", requestHeader.getTopic(), FAQUrl.suggestTodo(FAQUrl.APPLY_TOPIC_URL))); + response.setRemark(String.format("topic[%s] not exist, apply first please! %s", topic, FAQUrl.suggestTodo(FAQUrl.APPLY_TOPIC_URL))); return response; } if (!PermName.isReadable(topicConfig.getPerm())) { response.setCode(ResponseCode.NO_PERMISSION); - response.setRemark("the topic[" + requestHeader.getTopic() + "] pulling message is forbidden"); + response.setRemark("the topic[" + topic + "] pulling message is forbidden"); return response; } - if (requestHeader.getQueueId() < 0 || requestHeader.getQueueId() >= topicConfig.getReadQueueNums()) { + int queueId = requestHeader.getQueueId(); + if (queueId < 0 || queueId >= topicConfig.getReadQueueNums()) { String errorInfo = String.format("queueId[%d] is illegal, topic:[%s] topicConfig.readQueueNums:[%d] consumer:[%s]", - requestHeader.getQueueId(), requestHeader.getTopic(), topicConfig.getReadQueueNums(), channel.remoteAddress()); + queueId, topic, topicConfig.getReadQueueNums(), channel.remoteAddress()); log.warn(errorInfo); response.setCode(ResponseCode.SYSTEM_ERROR); response.setRemark(errorInfo); @@ -154,11 +162,11 @@ public class PullMessageProcessor extends AsyncNettyRequestProcessor implements if (hasSubscriptionFlag) { try { subscriptionData = FilterAPI.build( - requestHeader.getTopic(), requestHeader.getSubscription(), requestHeader.getExpressionType() + topic, requestHeader.getSubscription(), requestHeader.getExpressionType() ); if (!ExpressionType.isTagType(subscriptionData.getExpressionType())) { consumerFilterData = ConsumerFilterManager.build( - requestHeader.getTopic(), requestHeader.getConsumerGroup(), requestHeader.getSubscription(), + topic, requestHeader.getConsumerGroup(), requestHeader.getSubscription(), requestHeader.getExpressionType(), requestHeader.getSubVersion() ); assert consumerFilterData != null; @@ -187,9 +195,9 @@ public class PullMessageProcessor extends AsyncNettyRequestProcessor implements return response; } - subscriptionData = consumerGroupInfo.findSubscriptionData(requestHeader.getTopic()); + subscriptionData = consumerGroupInfo.findSubscriptionData(topic); if (null == subscriptionData) { - log.warn("the consumer's subscription not exist, group: {}, topic:{}", requestHeader.getConsumerGroup(), requestHeader.getTopic()); + log.warn("the consumer's subscription not exist, group: {}, topic:{}", requestHeader.getConsumerGroup(), topic); response.setCode(ResponseCode.SUBSCRIPTION_NOT_EXIST); response.setRemark("the consumer's subscription not exist" + FAQUrl.suggestTodo(FAQUrl.SAME_GROUP_DIFFERENT_TOPIC)); return response; @@ -203,7 +211,7 @@ public class PullMessageProcessor extends AsyncNettyRequestProcessor implements return response; } if (!ExpressionType.isTagType(subscriptionData.getExpressionType())) { - consumerFilterData = this.brokerController.getConsumerFilterManager().get(requestHeader.getTopic(), + consumerFilterData = this.brokerController.getConsumerFilterManager().get(topic, requestHeader.getConsumerGroup()); if (consumerFilterData == null) { response.setCode(ResponseCode.FILTER_DATA_NOT_EXIST); @@ -212,7 +220,7 @@ public class PullMessageProcessor extends AsyncNettyRequestProcessor implements } if (consumerFilterData.getClientVersion() < requestHeader.getSubVersion()) { log.warn("The broker's consumer filter data is not latest, group: {}, topic: {}, serverV: {}, clientV: {}", - requestHeader.getConsumerGroup(), requestHeader.getTopic(), consumerFilterData.getClientVersion(), requestHeader.getSubVersion()); + requestHeader.getConsumerGroup(), topic, consumerFilterData.getClientVersion(), requestHeader.getSubVersion()); response.setCode(ResponseCode.FILTER_DATA_NOT_LATEST); response.setRemark("the consumer's consumer filter data not latest"); return response; @@ -236,13 +244,74 @@ public class PullMessageProcessor extends AsyncNettyRequestProcessor implements this.brokerController.getConsumerFilterManager()); } + long offset = requestHeader.getQueueOffset(); + int maxMsgNums = requestHeader.getMaxMsgNums(); + + LogicalQueuesInfoInBroker logicalQueuesInfo = this.brokerController.getTopicConfigManager().selectLogicalQueuesInfo(topic); + LogicalQueueRouteData queueRouteData = null; + if (logicalQueuesInfo != null) { + int responseErrorCode = ResponseCode.SUCCESS; + queueRouteData = logicalQueuesInfo.queryQueueRouteDataByQueueId(queueId, offset); + if (queueRouteData != null) { + if (queueRouteData.isWriteOnly()) { + responseErrorCode = ResponseCode.PULL_NOT_FOUND; + response.setRemark("logical queue write only"); + } else if (queueRouteData.isExpired()) { + responseErrorCode = ResponseCode.PULL_RETRY_IMMEDIATELY; + response.setRemark("logical queue expired"); + prepareRedirectResponse(response, logicalQueuesInfo, queueRouteData); + } else if (MessageQueueRouteState.ReadOnly.equals(queueRouteData.getState()) && queueRouteData.getOffsetMax() >= 0) { + if (offset >= queueRouteData.getOffsetMax()) { + responseErrorCode = ResponseCode.PULL_RETRY_IMMEDIATELY; + response.setRemark("queue offset exceed offsetMax"); + prepareRedirectResponse(response, logicalQueuesInfo, queueRouteData); + } else if (offset + maxMsgNums > queueRouteData.getOffsetMax()) { + if ((queueRouteData.getOffsetMax() - 1 <= this.brokerController.getMessageStore().getMaxOffsetInQueue(topic, queueId)) && + (this.brokerController.getMessageStore().getCommitLogOffsetInQueue(topic, queueId, queueRouteData.getOffsetMax() - 1) < this.brokerController.getMessageStore().getMinPhyOffset())) { + responseErrorCode = ResponseCode.PULL_RETRY_IMMEDIATELY; + response.setRemark("queue offset removed"); + prepareRedirectResponse(response, logicalQueuesInfo, queueRouteData); + } else { + maxMsgNums = (int) (queueRouteData.getOffsetMax() - offset); + if (maxMsgNums <= 0) { + responseErrorCode = ResponseCode.PULL_RETRY_IMMEDIATELY; + response.setRemark("queue offset out of range"); + prepareRedirectResponse(response, logicalQueuesInfo, queueRouteData); + } + } + } + } + } else { + responseErrorCode = ResponseCode.PULL_RETRY_IMMEDIATELY; + response.setRemark("no suitable queue"); + response.addExtField(MessageConst.PROPERTY_REDIRECT, "1"); + // instruct client to refresh all + response.setBody(null); + queueRouteData = logicalQueuesInfo.queryQueueRouteDataByQueueId(queueId, 0L); + } + if (responseErrorCode != ResponseCode.SUCCESS) { + response.setCode(responseErrorCode); + responseHeader.setMinOffset(offset); + responseHeader.setMaxOffset(queueRouteData != null ? queueRouteData.getOffsetMax() : offset); + responseHeader.setNextBeginOffset(queueRouteData != null ? queueRouteData.getOffsetMax() : offset); + responseHeader.setSuggestWhichBrokerId(MixAll.MASTER_ID); + return response; + } + } + final GetMessageResult getMessageResult = - this.brokerController.getMessageStore().getMessage(requestHeader.getConsumerGroup(), requestHeader.getTopic(), - requestHeader.getQueueId(), requestHeader.getQueueOffset(), requestHeader.getMaxMsgNums(), messageFilter); + this.brokerController.getMessageStore().getMessage(requestHeader.getConsumerGroup(), topic, + queueId, offset, maxMsgNums, messageFilter); if (getMessageResult != null) { response.setRemark(getMessageResult.getStatus().name()); - responseHeader.setNextBeginOffset(getMessageResult.getNextBeginOffset()); + long nextBeginOffset = getMessageResult.getNextBeginOffset(); + if (queueRouteData != null && queueRouteData.getOffsetMax() >= 0 && nextBeginOffset > queueRouteData.getOffsetMax()) { + // prevent from pulling messages from next logical queue route data + nextBeginOffset = queueRouteData.getOffsetMax(); + } + responseHeader.setNextBeginOffset(nextBeginOffset); responseHeader.setMinOffset(getMessageResult.getMinOffset()); + // this does not need to be modified since it's not an accurate value under logical queue. responseHeader.setMaxOffset(getMessageResult.getMaxOffset()); if (getMessageResult.isSuggestPullingFromSlave()) { @@ -291,9 +360,9 @@ public class PullMessageProcessor extends AsyncNettyRequestProcessor implements // XXX: warn and notify me log.info("the broker store no queue data, fix the request offset {} to {}, Topic: {} QueueId: {} Consumer Group: {}", requestHeader.getQueueOffset(), - getMessageResult.getNextBeginOffset(), - requestHeader.getTopic(), - requestHeader.getQueueId(), + nextBeginOffset, + topic, + queueId, requestHeader.getConsumerGroup() ); } else { @@ -318,7 +387,7 @@ public class PullMessageProcessor extends AsyncNettyRequestProcessor implements case OFFSET_TOO_SMALL: response.setCode(ResponseCode.PULL_OFFSET_MOVED); log.info("the request offset too small. group={}, topic={}, requestOffset={}, brokerMinOffset={}, clientIp={}", - requestHeader.getConsumerGroup(), requestHeader.getTopic(), requestHeader.getQueueOffset(), + requestHeader.getConsumerGroup(), topic, requestHeader.getQueueOffset(), getMessageResult.getMinOffset(), channel.remoteAddress()); break; default: @@ -329,8 +398,8 @@ public class PullMessageProcessor extends AsyncNettyRequestProcessor implements if (this.hasConsumeMessageHook()) { ConsumeMessageContext context = new ConsumeMessageContext(); context.setConsumerGroup(requestHeader.getConsumerGroup()); - context.setTopic(requestHeader.getTopic()); - context.setQueueId(requestHeader.getQueueId()); + context.setTopic(topic); + context.setQueueId(queueId); String owner = request.getExtFields().get(BrokerStatsManager.COMMERCIAL_OWNER); @@ -414,9 +483,6 @@ public class PullMessageProcessor extends AsyncNettyRequestProcessor implements pollingTimeMills = this.brokerController.getBrokerConfig().getShortPollingTimeMills(); } - String topic = requestHeader.getTopic(); - long offset = requestHeader.getQueueOffset(); - int queueId = requestHeader.getQueueId(); PullRequest pullRequest = new PullRequest(request, channel, pollingTimeMills, this.brokerController.getMessageStore().now(), offset, subscriptionData, messageFilter); this.brokerController.getPullRequestHoldService().suspendPullRequest(topic, queueId, pullRequest); @@ -424,6 +490,20 @@ public class PullMessageProcessor extends AsyncNettyRequestProcessor implements break; } + if (queueRouteData != null) { + logicalQueuesInfo.readLock().lock(); + try { + List queueRouteDataList = logicalQueuesInfo.get(queueRouteData.getLogicalQueueIndex()); + MessageQueue latestMessageQueue = queueRouteDataList.get(queueRouteDataList.size() - 1).getMessageQueue(); + if (!latestMessageQueue.getBrokerName().equals(brokerController.getBrokerConfig().getBrokerName()) || latestMessageQueue.getQueueId() != queueId) { + // There are other newer message queue, instruct client to refresh meta-data to access these + prepareRedirectResponse(response, logicalQueuesInfo, queueRouteData); + } + } finally { + logicalQueuesInfo.readLock().unlock(); + } + } + case ResponseCode.PULL_RETRY_IMMEDIATELY: break; case ResponseCode.PULL_OFFSET_MOVED: @@ -438,7 +518,7 @@ public class PullMessageProcessor extends AsyncNettyRequestProcessor implements event.setConsumerGroup(requestHeader.getConsumerGroup()); event.setMessageQueue(mq); event.setOffsetRequest(requestHeader.getQueueOffset()); - event.setOffsetNew(getMessageResult.getNextBeginOffset()); + event.setOffsetNew(nextBeginOffset); this.generateOffsetMovedEvent(event); log.warn( "PULL_OFFSET_MOVED:correction offset. topic={}, groupId={}, requestOffset={}, newOffset={}, suggestBrokerId={}", @@ -467,11 +547,20 @@ public class PullMessageProcessor extends AsyncNettyRequestProcessor implements && this.brokerController.getMessageStoreConfig().getBrokerRole() != BrokerRole.SLAVE; if (storeOffsetEnable) { this.brokerController.getConsumerOffsetManager().commitOffset(RemotingHelper.parseChannelRemoteAddr(channel), - requestHeader.getConsumerGroup(), requestHeader.getTopic(), requestHeader.getQueueId(), requestHeader.getCommitOffset()); + requestHeader.getConsumerGroup(), topic, queueId, requestHeader.getCommitOffset()); } return response; } + private void prepareRedirectResponse(RemotingCommand response, LogicalQueuesInfoInBroker logicalQueuesInfo, + LogicalQueueRouteData queueRouteData) { + LogicalQueueRouteData nextReadableLogicalQueueRouteData = logicalQueuesInfo.nextAvailableLogicalRouteData(queueRouteData, LogicalQueueRouteData::isReadable); + if (nextReadableLogicalQueueRouteData != null) { + response.addExtField(MessageConst.PROPERTY_REDIRECT, "1"); + response.setBody(RemotingSerializable.encode(ImmutableList.of(queueRouteData, nextReadableLogicalQueueRouteData))); + } + } + public boolean hasConsumeMessageHook() { return consumeMessageHookList != null && !this.consumeMessageHookList.isEmpty(); } diff --git a/broker/src/main/java/org/apache/rocketmq/broker/processor/SendMessageProcessor.java b/broker/src/main/java/org/apache/rocketmq/broker/processor/SendMessageProcessor.java index 724cf54c8137c57d1dd8a5d434e20e582674d7a7..97b7e62850dc5ddaa039b4568b92427e6ab8c489 100644 --- a/broker/src/main/java/org/apache/rocketmq/broker/processor/SendMessageProcessor.java +++ b/broker/src/main/java/org/apache/rocketmq/broker/processor/SendMessageProcessor.java @@ -287,6 +287,12 @@ public class SendMessageProcessor extends AbstractSendMessageProcessor implement MessageAccessor.putProperty(msgInner, MessageConst.PROPERTY_CLUSTER, clusterName); msgInner.setPropertiesString(MessageDecoder.messageProperties2String(msgInner.getProperties())); + LogicalQueueContext logicalQueueContext = super.buildLogicalQueueContext(msgInner.getTopic(), msgInner.getQueueId(), response); + CompletableFuture future = logicalQueueContext.hookBeforePut(ctx, requestHeader, request, response); + if (future != null) { + return future; + } + CompletableFuture putMessageResult = null; Map origProps = MessageDecoder.string2messageProperties(requestHeader.getProperties()); String transFlag = origProps.get(MessageConst.PROPERTY_TRANSACTION_PREPARED); @@ -296,12 +302,14 @@ public class SendMessageProcessor extends AbstractSendMessageProcessor implement response.setRemark( "the broker[" + this.brokerController.getBrokerConfig().getBrokerIP1() + "] sending transaction message is forbidden"); + logicalQueueContext.hookAfterPut(null); return CompletableFuture.completedFuture(response); } putMessageResult = this.brokerController.getTransactionalMessageService().asyncPrepareMessage(msgInner); } else { putMessageResult = this.brokerController.getMessageStore().asyncPutMessage(msgInner); } + logicalQueueContext.hookAfterPut(putMessageResult); return handlePutMessageResultFuture(putMessageResult, response, request, msgInner, responseHeader, mqtraceContext, ctx, queueIdInt); } @@ -362,82 +370,6 @@ public class SendMessageProcessor extends AbstractSendMessageProcessor implement return true; } - private RemotingCommand sendMessage(final ChannelHandlerContext ctx, - final RemotingCommand request, - final SendMessageContext sendMessageContext, - final SendMessageRequestHeader requestHeader) throws RemotingCommandException { - - final RemotingCommand response = RemotingCommand.createResponseCommand(SendMessageResponseHeader.class); - final SendMessageResponseHeader responseHeader = (SendMessageResponseHeader)response.readCustomHeader(); - - response.setOpaque(request.getOpaque()); - - response.addExtField(MessageConst.PROPERTY_MSG_REGION, this.brokerController.getBrokerConfig().getRegionId()); - response.addExtField(MessageConst.PROPERTY_TRACE_SWITCH, String.valueOf(this.brokerController.getBrokerConfig().isTraceOn())); - - log.debug("receive SendMessage request command, {}", request); - - final long startTimstamp = this.brokerController.getBrokerConfig().getStartAcceptSendRequestTimeStamp(); - if (this.brokerController.getMessageStore().now() < startTimstamp) { - response.setCode(ResponseCode.SYSTEM_ERROR); - response.setRemark(String.format("broker unable to service, until %s", UtilAll.timeMillisToHumanString2(startTimstamp))); - return response; - } - - response.setCode(-1); - super.msgCheck(ctx, requestHeader, response); - if (response.getCode() != -1) { - return response; - } - - final byte[] body = request.getBody(); - - int queueIdInt = requestHeader.getQueueId(); - TopicConfig topicConfig = this.brokerController.getTopicConfigManager().selectTopicConfig(requestHeader.getTopic()); - - if (queueIdInt < 0) { - queueIdInt = Math.abs(this.random.nextInt() % 99999999) % topicConfig.getWriteQueueNums(); - } - - MessageExtBrokerInner msgInner = new MessageExtBrokerInner(); - msgInner.setTopic(requestHeader.getTopic()); - msgInner.setQueueId(queueIdInt); - - if (!handleRetryAndDLQ(requestHeader, response, request, msgInner, topicConfig)) { - return response; - } - - msgInner.setBody(body); - msgInner.setFlag(requestHeader.getFlag()); - MessageAccessor.setProperties(msgInner, MessageDecoder.string2messageProperties(requestHeader.getProperties())); - msgInner.setBornTimestamp(requestHeader.getBornTimestamp()); - msgInner.setBornHost(ctx.channel().remoteAddress()); - msgInner.setStoreHost(this.getStoreHost()); - msgInner.setReconsumeTimes(requestHeader.getReconsumeTimes() == null ? 0 : requestHeader.getReconsumeTimes()); - String clusterName = this.brokerController.getBrokerConfig().getBrokerClusterName(); - MessageAccessor.putProperty(msgInner, MessageConst.PROPERTY_CLUSTER, clusterName); - msgInner.setPropertiesString(MessageDecoder.messageProperties2String(msgInner.getProperties())); - PutMessageResult putMessageResult = null; - Map oriProps = MessageDecoder.string2messageProperties(requestHeader.getProperties()); - String traFlag = oriProps.get(MessageConst.PROPERTY_TRANSACTION_PREPARED); - if (traFlag != null && Boolean.parseBoolean(traFlag) - && !(msgInner.getReconsumeTimes() > 0 && msgInner.getDelayTimeLevel() > 0)) { //For client under version 4.6.1 - if (this.brokerController.getBrokerConfig().isRejectTransactionMessage()) { - response.setCode(ResponseCode.NO_PERMISSION); - response.setRemark( - "the broker[" + this.brokerController.getBrokerConfig().getBrokerIP1() - + "] sending transaction message is forbidden"); - return response; - } - putMessageResult = this.brokerController.getTransactionalMessageService().prepareMessage(msgInner); - } else { - putMessageResult = this.brokerController.getMessageStore().putMessage(msgInner); - } - - return handlePutMessageResult(putMessageResult, response, request, msgInner, responseHeader, sendMessageContext, ctx, queueIdInt); - - } - private RemotingCommand handlePutMessageResult(PutMessageResult putMessageResult, RemotingCommand response, RemotingCommand request, MessageExt msg, SendMessageResponseHeader responseHeader, SendMessageContext sendMessageContext, ChannelHandlerContext ctx, @@ -587,7 +519,16 @@ public class SendMessageProcessor extends AbstractSendMessageProcessor implement String clusterName = this.brokerController.getBrokerConfig().getBrokerClusterName(); MessageAccessor.putProperty(messageExtBatch, MessageConst.PROPERTY_CLUSTER, clusterName); + LogicalQueueContext logicalQueueContext = super.buildLogicalQueueContext(messageExtBatch.getTopic(), messageExtBatch.getQueueId(), response); + CompletableFuture future = logicalQueueContext.hookBeforePut(ctx, requestHeader, request, response); + if (future != null) { + return future; + } + CompletableFuture putMessageResult = this.brokerController.getMessageStore().asyncPutMessages(messageExtBatch); + + logicalQueueContext.hookAfterPut(putMessageResult); + return handlePutMessageResultFuture(putMessageResult, response, request, messageExtBatch, responseHeader, mqtraceContext, ctx, queueIdInt); } 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 1f2bb4d8ba49c00a4b10a0f674bd05aaf9aaa7a2..bf690057c6c64b387246584ec4c82fc479f40cbb 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 @@ -17,16 +17,20 @@ package org.apache.rocketmq.broker.topic; import java.util.Iterator; +import java.util.List; import java.util.Map; import java.util.Map.Entry; +import java.util.Objects; import java.util.Set; import java.util.concurrent.ConcurrentHashMap; import java.util.concurrent.ConcurrentMap; import java.util.concurrent.TimeUnit; import java.util.concurrent.locks.Lock; import java.util.concurrent.locks.ReentrantLock; +import java.util.stream.Collectors; import org.apache.rocketmq.broker.BrokerController; import org.apache.rocketmq.broker.BrokerPathConfigHelper; +import org.apache.rocketmq.broker.domain.LogicalQueuesInfoInBroker; import org.apache.rocketmq.common.ConfigManager; import org.apache.rocketmq.common.DataVersion; import org.apache.rocketmq.common.MixAll; @@ -35,10 +39,15 @@ import org.apache.rocketmq.common.constant.LoggerName; import org.apache.rocketmq.common.constant.PermName; import org.apache.rocketmq.common.protocol.body.KVTable; import org.apache.rocketmq.common.protocol.body.TopicConfigSerializeWrapper; +import org.apache.rocketmq.common.protocol.route.LogicalQueueRouteData; import org.apache.rocketmq.common.sysflag.TopicSysFlag; import org.apache.rocketmq.common.topic.TopicValidator; import org.apache.rocketmq.logging.InternalLogger; import org.apache.rocketmq.logging.InternalLoggerFactory; +import org.apache.rocketmq.srvutil.ConcurrentHashMapUtil; +import org.apache.rocketmq.store.CleanFilesHook; +import org.apache.rocketmq.store.DefaultMessageStore; +import org.apache.rocketmq.store.MessageStore; public class TopicConfigManager extends ConfigManager { private static final InternalLogger log = InternalLoggerFactory.getLogger(LoggerName.BROKER_LOGGER_NAME); @@ -52,8 +61,19 @@ public class TopicConfigManager extends ConfigManager { private final DataVersion dataVersion = new DataVersion(); private transient BrokerController brokerController; - public TopicConfigManager() { - } + private final ConcurrentMap logicalQueuesInfoTable = new ConcurrentHashMap<>(); + private final CleanFilesHook logicalQueueCleanHook = new CleanFilesHook() { + @Override public void execute(DefaultMessageStore defaultMessageStore, long deleteCount) { + if (deleteCount == 0) { + return; + } + TopicConfigManager.this.logicalQueueClean(); + } + + @Override public String getName() { + return TopicConfigManager.class.getSimpleName() + ".logicalQueueCleanHook"; + } + }; public TopicConfigManager(BrokerController brokerController) { this.brokerController = brokerController; @@ -362,7 +382,7 @@ public class TopicConfigManager extends ConfigManager { this.dataVersion.nextVersion(); - this.persist(); + this.persist(topicConfig.getTopicName(), topicConfig); } public void updateOrderTopicConfig(final KVTable orderKVTableFromNs) { @@ -421,6 +441,8 @@ public class TopicConfigManager extends ConfigManager { public TopicConfigSerializeWrapper buildTopicConfigSerializeWrapper() { TopicConfigSerializeWrapper topicConfigSerializeWrapper = new TopicConfigSerializeWrapper(); topicConfigSerializeWrapper.setTopicConfigTable(this.topicConfigTable); + String brokerName = this.brokerController.getBrokerConfig().getBrokerName(); + topicConfigSerializeWrapper.setLogicalQueuesInfoMap(this.logicalQueuesInfoTable.entrySet().stream().collect(Collectors.toMap(Map.Entry::getKey, e -> new LogicalQueuesInfoInBroker(e.getValue(), data -> Objects.equals(data.getBrokerName(), brokerName))))); topicConfigSerializeWrapper.setDataVersion(this.dataVersion); return topicConfigSerializeWrapper; } @@ -452,6 +474,7 @@ public class TopicConfigManager extends ConfigManager { public String encode(final boolean prettyFormat) { TopicConfigSerializeWrapper topicConfigSerializeWrapper = new TopicConfigSerializeWrapper(); topicConfigSerializeWrapper.setTopicConfigTable(this.topicConfigTable); + topicConfigSerializeWrapper.setLogicalQueuesInfoMap(this.logicalQueuesInfoTable.entrySet().stream().collect(Collectors.toMap(Map.Entry::getKey, e -> new LogicalQueuesInfoInBroker(e.getValue())))); topicConfigSerializeWrapper.setDataVersion(this.dataVersion); return topicConfigSerializeWrapper.toJson(prettyFormat); } @@ -471,4 +494,92 @@ public class TopicConfigManager extends ConfigManager { public ConcurrentMap getTopicConfigTable() { return topicConfigTable; } + + public LogicalQueuesInfoInBroker selectLogicalQueuesInfo(String topicName) { + return this.logicalQueuesInfoTable.get(topicName); + } + + public LogicalQueuesInfoInBroker getOrCreateLogicalQueuesInfo(String topicName) { + return ConcurrentHashMapUtil.computeIfAbsent(this.logicalQueuesInfoTable, topicName, ignored -> new LogicalQueuesInfoInBroker()); + } + + public boolean replaceTopicConfig(String topic, TopicConfig oldTopicConfig, TopicConfig newTopicConfig) { + boolean ok = this.topicConfigTable.replace(topic, oldTopicConfig, newTopicConfig); + if (ok) { + this.dataVersion.nextVersion(); + persist(topic, newTopicConfig); + } + return ok; + } + + public CleanFilesHook getLogicalQueueCleanHook() { + return logicalQueueCleanHook; + } + + void logicalQueueClean() { + String brokerName = this.brokerController.getBrokerConfig().getBrokerName(); + MessageStore messageStore = this.brokerController.getMessageStore(); + for (Entry entry : this.logicalQueuesInfoTable.entrySet()) { + String topic = entry.getKey(); + LogicalQueuesInfoInBroker logicalQueuesInfo = entry.getValue(); + Lock readLock = logicalQueuesInfo.readLock(); + Lock writeLock = logicalQueuesInfo.writeLock(); + boolean changed = false; + readLock.lock(); + try { + for (List list : logicalQueuesInfo.values()) { + while (!list.isEmpty()) { + LogicalQueueRouteData logicalQueueRouteData = list.get(0); + String brokerBelongs; + if (brokerName.equals(logicalQueueRouteData.getBrokerName())) { + if (logicalQueueRouteData.isWritable()) { + break; + } + boolean canRemove = logicalQueueRouteData.isExpired() || logicalQueueRouteData.getMessagesCount() == 0; + if (!canRemove) { + // do not use getMinOffsetInQueue method, since it is using ConsumeQueue data, but not CommitLog, CQ data is not accurate after CommitLog cleaning. + long commitLogOffset = messageStore.getCommitLogOffsetInQueue(topic, logicalQueueRouteData.getQueueId(), logicalQueueRouteData.getOffsetMax() - 1); + canRemove = commitLogOffset == 0 || messageStore.getMinPhyOffset() > commitLogOffset; + } + if (!canRemove) { + break; + } + brokerBelongs = "self"; + } else { + brokerBelongs = "other"; + } + readLock.unlock(); + writeLock.lock(); + try { + list.remove(0); + } finally { + readLock.lock(); + writeLock.unlock(); + } + log.info("logicalQueueClean remove {} broker {}", brokerBelongs, logicalQueueRouteData); + changed = true; + } + } + if (changed) { + logicalQueuesInfo = new LogicalQueuesInfoInBroker(logicalQueuesInfo); + } + } finally { + readLock.unlock(); + } + if (changed) { + this.dataVersion.nextVersion(); + this.persist(topic, logicalQueuesInfo); + this.brokerController.registerIncrementBrokerData(this.selectTopicConfig(topic), this.dataVersion); + log.info("registerIncrementBrokerData because logicalQueueClean: {}", topic); + } + } + } + + public void deleteQueueRouteData(String topic) { + if (this.logicalQueuesInfoTable.remove(topic) != null) { + log.info("delete queueRouteData config OK, topic: {}", topic); + this.dataVersion.nextVersion(); + persist(topic, (LogicalQueuesInfoInBroker) null); + } + } } diff --git a/broker/src/test/java/org/apache/rocketmq/broker/BrokerOuterAPITest.java b/broker/src/test/java/org/apache/rocketmq/broker/BrokerOuterAPITest.java index 9ea1eeee31b9995f35890a8b06e8ff765690b8ef..339ed110c8597da6eb515f33f65b25798c4c9875 100644 --- a/broker/src/test/java/org/apache/rocketmq/broker/BrokerOuterAPITest.java +++ b/broker/src/test/java/org/apache/rocketmq/broker/BrokerOuterAPITest.java @@ -22,19 +22,26 @@ import com.google.common.collect.Iterables; import com.google.common.collect.Lists; import io.netty.channel.ChannelHandlerContext; import java.lang.reflect.Field; +import java.util.Collections; +import java.util.HashMap; import java.util.List; import java.util.concurrent.TimeUnit; import org.apache.rocketmq.broker.out.BrokerOuterAPI; import org.apache.rocketmq.common.BrokerConfig; +import org.apache.rocketmq.common.MixAll; import org.apache.rocketmq.common.namesrv.RegisterBrokerResult; +import org.apache.rocketmq.common.protocol.RequestCode; import org.apache.rocketmq.common.protocol.ResponseCode; +import org.apache.rocketmq.common.protocol.body.ClusterInfo; import org.apache.rocketmq.common.protocol.body.TopicConfigSerializeWrapper; import org.apache.rocketmq.common.protocol.header.namesrv.QueryDataVersionResponseHeader; import org.apache.rocketmq.common.protocol.header.namesrv.RegisterBrokerResponseHeader; +import org.apache.rocketmq.common.protocol.route.BrokerData; import org.apache.rocketmq.remoting.netty.NettyClientConfig; import org.apache.rocketmq.remoting.netty.NettyRemotingClient; import org.apache.rocketmq.remoting.netty.NettyServerConfig; import org.apache.rocketmq.remoting.protocol.RemotingCommand; +import org.apache.rocketmq.remoting.protocol.RemotingSerializable; import org.apache.rocketmq.store.MessageStore; import org.apache.rocketmq.store.config.MessageStoreConfig; import org.junit.Test; @@ -50,6 +57,8 @@ import static org.junit.Assert.assertTrue; import static org.mockito.ArgumentMatchers.any; import static org.mockito.ArgumentMatchers.anyLong; import static org.mockito.ArgumentMatchers.anyString; +import static org.mockito.ArgumentMatchers.argThat; +import static org.mockito.ArgumentMatchers.isNull; import static org.mockito.Mockito.when; @RunWith(MockitoJUnitRunner.class) @@ -182,6 +191,26 @@ public class BrokerOuterAPITest { assertEquals(2, registerBrokerResultList.size()); } + @Test + public void testGetBrokerClusterInfo() throws Exception { + init(); + brokerOuterAPI.start(); + + final RemotingCommand response = RemotingCommand.createResponseCommand(null); + response.setCode(ResponseCode.SUCCESS); + response.setRemark(null); + + ClusterInfo want = new ClusterInfo(); + want.setBrokerAddrTable(new HashMap<>(Collections.singletonMap("key", new BrokerData("cluster", "broker", new HashMap<>(Collections.singletonMap(MixAll.MASTER_ID, "127.0.0.1:10911")))))); + response.setBody(RemotingSerializable.encode(want)); + + when(nettyRemotingClient.invokeSync(isNull(), argThat(argument -> argument.getCode() == RequestCode.GET_BROKER_CLUSTER_INFO), anyLong())).thenReturn(response); + ClusterInfo got = brokerOuterAPI.getBrokerClusterInfo(); + + assertEquals(want, got); + } + + private RemotingCommand buildResponse(Boolean changed) { final RemotingCommand response = RemotingCommand.createResponseCommand(QueryDataVersionResponseHeader.class); final QueryDataVersionResponseHeader responseHeader = (QueryDataVersionResponseHeader) response.readCustomHeader(); diff --git a/broker/src/test/java/org/apache/rocketmq/broker/processor/AdminBrokerProcessorTest.java b/broker/src/test/java/org/apache/rocketmq/broker/processor/AdminBrokerProcessorTest.java index 2764761d39f68fa081a961479a02ce25825a4f2d..d6cc8f9bd8a188870b5bda68988ac009261af889 100644 --- a/broker/src/test/java/org/apache/rocketmq/broker/processor/AdminBrokerProcessorTest.java +++ b/broker/src/test/java/org/apache/rocketmq/broker/processor/AdminBrokerProcessorTest.java @@ -16,26 +16,54 @@ */ package org.apache.rocketmq.broker.processor; +import com.alibaba.fastjson.JSON; +import com.alibaba.fastjson.TypeReference; import com.google.common.collect.Sets; +import io.netty.channel.Channel; import io.netty.channel.ChannelHandlerContext; +import java.net.InetSocketAddress; +import java.net.UnknownHostException; +import java.nio.ByteBuffer; +import java.util.Arrays; +import java.util.List; +import java.util.Map; +import java.util.Set; +import java.util.concurrent.ConcurrentMap; +import java.util.concurrent.atomic.LongAdder; import org.apache.rocketmq.broker.BrokerController; +import org.apache.rocketmq.broker.domain.LogicalQueuesInfoInBroker; +import org.apache.rocketmq.broker.topic.TopicConfigManager; import org.apache.rocketmq.common.BrokerConfig; import org.apache.rocketmq.common.MixAll; +import org.apache.rocketmq.common.TopicConfig; import org.apache.rocketmq.common.TopicFilterType; +import org.apache.rocketmq.common.TopicQueueId; import org.apache.rocketmq.common.constant.PermName; import org.apache.rocketmq.common.message.MessageAccessor; import org.apache.rocketmq.common.message.MessageConst; import org.apache.rocketmq.common.message.MessageExt; +import org.apache.rocketmq.common.message.MessageQueue; import org.apache.rocketmq.common.protocol.RequestCode; import org.apache.rocketmq.common.protocol.ResponseCode; +import org.apache.rocketmq.common.protocol.body.CreateMessageQueueForLogicalQueueRequestBody; +import org.apache.rocketmq.common.protocol.body.MigrateLogicalQueueBody; +import org.apache.rocketmq.common.protocol.body.ReuseTopicLogicalQueueRequestBody; +import org.apache.rocketmq.common.protocol.body.SealTopicLogicalQueueRequestBody; +import org.apache.rocketmq.common.protocol.body.UpdateTopicLogicalQueueMappingRequestBody; import org.apache.rocketmq.common.protocol.header.CreateTopicRequestHeader; +import org.apache.rocketmq.common.protocol.header.DeleteTopicLogicalQueueRequestHeader; import org.apache.rocketmq.common.protocol.header.DeleteTopicRequestHeader; +import org.apache.rocketmq.common.protocol.header.GetTopicConfigRequestHeader; +import org.apache.rocketmq.common.protocol.header.QueryTopicLogicalQueueMappingRequestHeader; import org.apache.rocketmq.common.protocol.header.ResumeCheckHalfMessageRequestHeader; +import org.apache.rocketmq.common.protocol.route.LogicalQueueRouteData; +import org.apache.rocketmq.common.protocol.route.MessageQueueRouteState; import org.apache.rocketmq.common.topic.TopicValidator; import org.apache.rocketmq.remoting.exception.RemotingCommandException; import org.apache.rocketmq.remoting.netty.NettyClientConfig; import org.apache.rocketmq.remoting.netty.NettyServerConfig; import org.apache.rocketmq.remoting.protocol.RemotingCommand; +import org.apache.rocketmq.remoting.protocol.RemotingSerializable; import org.apache.rocketmq.store.AppendMessageResult; import org.apache.rocketmq.store.AppendMessageStatus; import org.apache.rocketmq.store.MappedFile; @@ -45,6 +73,7 @@ import org.apache.rocketmq.store.PutMessageResult; import org.apache.rocketmq.store.PutMessageStatus; import org.apache.rocketmq.store.SelectMappedBufferResult; import org.apache.rocketmq.store.config.MessageStoreConfig; +import org.assertj.core.util.Lists; import org.junit.Before; import org.junit.Test; import org.junit.runner.RunWith; @@ -52,12 +81,13 @@ import org.mockito.Mock; import org.mockito.Spy; import org.mockito.junit.MockitoJUnitRunner; -import java.net.UnknownHostException; -import java.nio.ByteBuffer; -import java.util.Set; - import static org.assertj.core.api.Assertions.assertThat; import static org.mockito.ArgumentMatchers.any; +import static org.mockito.ArgumentMatchers.anyBoolean; +import static org.mockito.ArgumentMatchers.eq; +import static org.mockito.Mockito.doReturn; +import static org.mockito.Mockito.mock; +import static org.mockito.Mockito.verify; import static org.mockito.Mockito.when; @RunWith(MockitoJUnitRunner.class) @@ -68,32 +98,52 @@ public class AdminBrokerProcessorTest { @Mock private ChannelHandlerContext handlerContext; + @Mock + private Channel channel; + @Spy private BrokerController - brokerController = new BrokerController(new BrokerConfig(), new NettyServerConfig(), new NettyClientConfig(), - new MessageStoreConfig()); + brokerController = new BrokerController(new BrokerConfig(), new NettyServerConfig(), new NettyClientConfig(), + new MessageStoreConfig()); @Mock private MessageStore messageStore; + @Mock + private SendMessageProcessor sendMessageProcessor; + + @Mock + private ConcurrentMap inFlyWritingCouterMap; + private Set systemTopicSet; + private String topic; @Before - public void init() { + public void init() throws Exception { brokerController.setMessageStore(messageStore); + + doReturn(sendMessageProcessor).when(brokerController).getSendMessageProcessor(); + when(sendMessageProcessor.getInFlyWritingCounterMap()).thenReturn(inFlyWritingCouterMap); + adminBrokerProcessor = new AdminBrokerProcessor(brokerController); systemTopicSet = Sets.newHashSet( - TopicValidator.RMQ_SYS_SELF_TEST_TOPIC, - TopicValidator.RMQ_SYS_BENCHMARK_TOPIC, - TopicValidator.RMQ_SYS_SCHEDULE_TOPIC, - TopicValidator.RMQ_SYS_OFFSET_MOVED_EVENT, - TopicValidator.AUTO_CREATE_TOPIC_KEY_TOPIC, - this.brokerController.getBrokerConfig().getBrokerClusterName(), - this.brokerController.getBrokerConfig().getBrokerClusterName() + "_" + MixAll.REPLY_TOPIC_POSTFIX); + TopicValidator.RMQ_SYS_SELF_TEST_TOPIC, + TopicValidator.RMQ_SYS_BENCHMARK_TOPIC, + TopicValidator.RMQ_SYS_SCHEDULE_TOPIC, + TopicValidator.RMQ_SYS_OFFSET_MOVED_EVENT, + TopicValidator.AUTO_CREATE_TOPIC_KEY_TOPIC, + this.brokerController.getBrokerConfig().getBrokerClusterName(), + this.brokerController.getBrokerConfig().getBrokerClusterName() + "_" + MixAll.REPLY_TOPIC_POSTFIX); if (this.brokerController.getBrokerConfig().isTraceTopicEnable()) { systemTopicSet.add(this.brokerController.getBrokerConfig().getMsgTraceTopicName()); } + when(handlerContext.channel()).thenReturn(channel); + when(channel.remoteAddress()).thenReturn(new InetSocketAddress("127.0.0.1", 12345)); + + topic = "FooBar" + System.nanoTime(); + TopicConfigManager topicConfigManager = brokerController.getTopicConfigManager(); + topicConfigManager.updateTopicConfig(new TopicConfig(topic)); } @Test @@ -101,7 +151,7 @@ public class AdminBrokerProcessorTest { RemotingCommand request = createResumeCheckHalfMessageCommand(); when(messageStore.selectOneMessageByOffset(any(Long.class))).thenReturn(createSelectMappedBufferResult()); when(messageStore.putMessage(any(MessageExtBrokerInner.class))).thenReturn(new PutMessageResult - (PutMessageStatus.PUT_OK, new AppendMessageResult(AppendMessageStatus.PUT_OK))); + (PutMessageStatus.PUT_OK, new AppendMessageResult(AppendMessageStatus.PUT_OK))); RemotingCommand response = adminBrokerProcessor.processRequest(handlerContext, request); assertThat(response.getCode()).isEqualTo(ResponseCode.SUCCESS); } @@ -111,7 +161,7 @@ public class AdminBrokerProcessorTest { RemotingCommand request = createResumeCheckHalfMessageCommand(); when(messageStore.selectOneMessageByOffset(any(Long.class))).thenReturn(createSelectMappedBufferResult()); when(messageStore.putMessage(any(MessageExtBrokerInner.class))).thenReturn(new PutMessageResult - (PutMessageStatus.UNKNOWN_ERROR, new AppendMessageResult(AppendMessageStatus.UNKNOWN_ERROR))); + (PutMessageStatus.UNKNOWN_ERROR, new AppendMessageResult(AppendMessageStatus.UNKNOWN_ERROR))); RemotingCommand response = adminBrokerProcessor.processRequest(handlerContext, request); assertThat(response.getCode()).isEqualTo(ResponseCode.SYSTEM_ERROR); } @@ -155,6 +205,292 @@ public class AdminBrokerProcessorTest { assertThat(response.getCode()).isEqualTo(ResponseCode.SUCCESS); } + @Test + public void testGetTopicConfig() throws Exception { + String topic = "foobar"; + brokerController.getTopicConfigManager().updateTopicConfig(new TopicConfig(topic)); + + { + GetTopicConfigRequestHeader requestHeader = new GetTopicConfigRequestHeader(); + requestHeader.setTopic(topic); + RemotingCommand request = RemotingCommand.createRequestCommand(RequestCode.GET_TOPIC_CONFIG, requestHeader); + request.makeCustomHeaderToNet(); + RemotingCommand response = adminBrokerProcessor.processRequest(handlerContext, request); + assertThat(response.getCode()).isEqualTo(ResponseCode.SUCCESS); + assertThat(response.getBody()).isNotEmpty(); + } + { + GetTopicConfigRequestHeader requestHeader = new GetTopicConfigRequestHeader(); + requestHeader.setTopic("aaaaaaa"); + RemotingCommand request = RemotingCommand.createRequestCommand(RequestCode.GET_TOPIC_CONFIG, requestHeader); + request.makeCustomHeaderToNet(); + RemotingCommand response = adminBrokerProcessor.processRequest(handlerContext, request); + assertThat(response.getCode()).isEqualTo(ResponseCode.SYSTEM_ERROR); + assertThat(response.getRemark()).contains("No topic in this broker."); + } + } + + @Test + public void testUpdateTopicLogicalQueueMapping() throws Exception { + int queueId = 0; + int logicalQueueIndex = 0; + + TopicConfigManager topicConfigManager = brokerController.getTopicConfigManager(); + RemotingCommand request = RemotingCommand.createRequestCommand(RequestCode.UPDATE_TOPIC_LOGICAL_QUEUE_MAPPING, null); + UpdateTopicLogicalQueueMappingRequestBody requestBody = new UpdateTopicLogicalQueueMappingRequestBody(); + requestBody.setTopic(topic); + requestBody.setQueueId(queueId); + requestBody.setLogicalQueueIdx(logicalQueueIndex); + request.setBody(requestBody.encode()); + RemotingCommand response; + response = adminBrokerProcessor.processRequest(handlerContext, request); + assertThat(response.getCode()).isEqualTo(ResponseCode.SUCCESS); + assertThat(topicConfigManager.getOrCreateLogicalQueuesInfo(topic).get(logicalQueueIndex).get(0)).isEqualTo(new LogicalQueueRouteData(logicalQueueIndex, 0L, new MessageQueue(topic, brokerController.getBrokerConfig().getBrokerName(), queueId), MessageQueueRouteState.Normal, 0L, -1, -1, -1, brokerController.getBrokerAddr())); + + // delete + requestBody.setLogicalQueueIdx(-1); + request.setBody(requestBody.encode()); + response = adminBrokerProcessor.processRequest(handlerContext, request); + assertThat(response.getCode()).isEqualTo(ResponseCode.SUCCESS); + assertThat(topicConfigManager.getOrCreateLogicalQueuesInfo(topic).get(logicalQueueIndex)).isEmpty(); + verify(inFlyWritingCouterMap).remove(new TopicQueueId(topic, queueId)); + } + + @Test + public void testDeleteTopicLogicalQueueMapping() throws Exception { + int queueId = 0; + int logicalQueueIndex = 0; + TopicConfigManager topicConfigManager = brokerController.getTopicConfigManager(); + LogicalQueuesInfoInBroker logicalQueuesInfo = topicConfigManager.getOrCreateLogicalQueuesInfo(topic); + logicalQueuesInfo.put(logicalQueueIndex, Lists.newArrayList(new LogicalQueueRouteData(logicalQueueIndex, 0L, new MessageQueue(topic, brokerController.getBrokerConfig().getBrokerName(), queueId), MessageQueueRouteState.Normal, 0L, -1, -1, -1, brokerController.getBrokerAddr()))); + + DeleteTopicLogicalQueueRequestHeader requestHeader = new DeleteTopicLogicalQueueRequestHeader(); + requestHeader.setTopic(topic); + RemotingCommand request = RemotingCommand.createRequestCommand(RequestCode.DELETE_TOPIC_LOGICAL_QUEUE_MAPPING, requestHeader); + request.makeCustomHeaderToNet(); + RemotingCommand response = adminBrokerProcessor.processRequest(handlerContext, request); + assertThat(response.getCode()).isEqualTo(ResponseCode.SYSTEM_ERROR); + assertThat(response.getRemark()).isEqualTo("still 1 message queues"); + + logicalQueuesInfo.remove(logicalQueueIndex); + response = adminBrokerProcessor.processRequest(handlerContext, request); + assertThat(response.getCode()).isEqualTo(ResponseCode.SUCCESS); + assertThat(topicConfigManager.selectLogicalQueuesInfo(topic)).isNull(); + } + + @Test + public void testQueryTopicLogicalQueueMapping() throws Exception { + int queueId = 0; + int logicalQueueIndex = 0; + TopicConfigManager topicConfigManager = brokerController.getTopicConfigManager(); + LogicalQueuesInfoInBroker logicalQueuesInfo = topicConfigManager.getOrCreateLogicalQueuesInfo(topic); + logicalQueuesInfo.put(logicalQueueIndex, Lists.newArrayList(new LogicalQueueRouteData(logicalQueueIndex, 0L, new MessageQueue(topic, brokerController.getBrokerConfig().getBrokerName(), queueId), MessageQueueRouteState.Normal, 0L, -1, -1, -1, brokerController.getBrokerAddr()))); + + QueryTopicLogicalQueueMappingRequestHeader requestHeader = new QueryTopicLogicalQueueMappingRequestHeader(); + requestHeader.setTopic(topic); + RemotingCommand request = RemotingCommand.createRequestCommand(RequestCode.QUERY_TOPIC_LOGICAL_QUEUE_MAPPING, requestHeader); + request.makeCustomHeaderToNet(); + RemotingCommand response = adminBrokerProcessor.processRequest(handlerContext, request); + assertThat(response.getCode()).isEqualTo(ResponseCode.SUCCESS); + Map> m = JSON.parseObject(response.getBody(), new TypeReference>>() { + }.getType()); + assertThat(m.get(logicalQueueIndex)).isEqualTo(logicalQueuesInfo.get(logicalQueueIndex)); + } + + @Test + public void testSealTopicLogicalQueue() throws Exception { + int queueId = 0; + int logicalQueueIndex = 0; + TopicConfigManager topicConfigManager = brokerController.getTopicConfigManager(); + LogicalQueuesInfoInBroker logicalQueuesInfo = topicConfigManager.getOrCreateLogicalQueuesInfo(topic); + MessageQueue mq = new MessageQueue(topic, brokerController.getBrokerConfig().getBrokerName(), queueId); + logicalQueuesInfo.put(logicalQueueIndex, Lists.newArrayList(new LogicalQueueRouteData(logicalQueueIndex, 0L, mq, MessageQueueRouteState.Normal, 0L, -1, -1, -1, brokerController.getBrokerAddr()))); + + when(messageStore.getMaxOffsetInQueue(eq(topic), eq(queueId), anyBoolean())).thenReturn(100L); + when(messageStore.getMinOffsetInQueue(eq(topic), eq(queueId))).thenReturn(0L); + when(messageStore.getMinPhyOffset()).thenReturn(1000L); + when(messageStore.getCommitLogOffsetInQueue(eq(topic), eq(queueId), eq(0L))).thenReturn(2000L); + when(messageStore.getCommitLogOffsetInQueue(eq(topic), eq(queueId), eq(99L))).thenReturn(3000L); + MessageExt firstMsg = mock(MessageExt.class); + when(firstMsg.getStoreTimestamp()).thenReturn(200L); + when(messageStore.lookMessageByOffset(eq(2000L))).thenReturn(firstMsg); + MessageExt lastMsg = mock(MessageExt.class); + when(lastMsg.getStoreTimestamp()).thenReturn(300L); + when(messageStore.lookMessageByOffset(eq(3000L))).thenReturn(lastMsg); + + RemotingCommand request = RemotingCommand.createRequestCommand(RequestCode.SEAL_TOPIC_LOGICAL_QUEUE, null); + SealTopicLogicalQueueRequestBody requestBody = new SealTopicLogicalQueueRequestBody(); + requestBody.setTopic(topic); + requestBody.setQueueId(queueId); + requestBody.setLogicalQueueIndex(logicalQueueIndex); + request.setBody(requestBody.encode()); + RemotingCommand response = adminBrokerProcessor.processRequest(handlerContext, request); + assertThat(response.getCode()).isEqualTo(ResponseCode.SUCCESS); + LogicalQueueRouteData wantLogicalQueueRouteData = new LogicalQueueRouteData(logicalQueueIndex, 0L, mq, MessageQueueRouteState.ReadOnly, 0, 100, 200, 300, brokerController.getBrokerAddr()); + assertThat(logicalQueuesInfo.get(logicalQueueIndex).get(0)).isEqualTo(wantLogicalQueueRouteData); + assertThat((LogicalQueueRouteData) JSON.parseObject(response.getBody(), LogicalQueueRouteData.class)).isEqualTo(wantLogicalQueueRouteData); + + // expired + logicalQueuesInfo.put(logicalQueueIndex, Lists.newArrayList(new LogicalQueueRouteData(logicalQueueIndex, 0L, mq, MessageQueueRouteState.Normal, 0L, -1, -1, -1, brokerController.getBrokerAddr()))); + when(messageStore.getMinPhyOffset()).thenReturn(10000L); + response = adminBrokerProcessor.processRequest(handlerContext, request); + assertThat(response.getCode()).isEqualTo(ResponseCode.SUCCESS); + wantLogicalQueueRouteData = new LogicalQueueRouteData(logicalQueueIndex, 0L, mq, MessageQueueRouteState.Expired, 0, 100, 0, 0, brokerController.getBrokerAddr()); + assertThat(logicalQueuesInfo.get(logicalQueueIndex).get(0)).isEqualTo(wantLogicalQueueRouteData); + assertThat((LogicalQueueRouteData) JSON.parseObject(response.getBody(), LogicalQueueRouteData.class)).isEqualTo(wantLogicalQueueRouteData); + + // expired and empty + logicalQueuesInfo.put(logicalQueueIndex, Lists.newArrayList(new LogicalQueueRouteData(logicalQueueIndex, 0L, mq, MessageQueueRouteState.Normal, 0L, -1, -1, -1, brokerController.getBrokerAddr()))); + when(messageStore.getMinOffsetInQueue(eq(topic), eq(queueId))).thenReturn(100L); + response = adminBrokerProcessor.processRequest(handlerContext, request); + assertThat(response.getCode()).isEqualTo(ResponseCode.SUCCESS); + wantLogicalQueueRouteData = new LogicalQueueRouteData(logicalQueueIndex, 0L, mq, MessageQueueRouteState.Expired, 0, 100, 0, 0, brokerController.getBrokerAddr()); + assertThat(logicalQueuesInfo.get(logicalQueueIndex).get(0)).isEqualTo(wantLogicalQueueRouteData); + assertThat((LogicalQueueRouteData) JSON.parseObject(response.getBody(), LogicalQueueRouteData.class)).isEqualTo(wantLogicalQueueRouteData); + } + + @Test + public void testReuseTopicLogicalQueue() throws Exception { + int queueId = 0; + int logicalQueueIndex = 0; + TopicConfigManager topicConfigManager = brokerController.getTopicConfigManager(); + LogicalQueuesInfoInBroker logicalQueuesInfo = topicConfigManager.getOrCreateLogicalQueuesInfo(topic); + MessageQueue mq = new MessageQueue(topic, brokerController.getBrokerConfig().getBrokerName(), queueId); + LogicalQueueRouteData logicalQueueRouteData = new LogicalQueueRouteData(logicalQueueIndex, 500L, mq, MessageQueueRouteState.Expired, 100L, 200L, 300L, 400L, brokerController.getBrokerAddr()); + logicalQueuesInfo.put(logicalQueueIndex, Lists.newArrayList(logicalQueueRouteData)); + LogicalQueueRouteData wantData0 = new LogicalQueueRouteData(logicalQueueRouteData); + + when(messageStore.getMaxOffsetInQueue(eq(topic), eq(queueId), anyBoolean())).thenReturn(600L); + + RemotingCommand request = RemotingCommand.createRequestCommand(RequestCode.REUSE_TOPIC_LOGICAL_QUEUE, null); + ReuseTopicLogicalQueueRequestBody requestBody = new ReuseTopicLogicalQueueRequestBody(); + requestBody.setTopic(topic); + requestBody.setQueueId(queueId); + requestBody.setLogicalQueueIndex(logicalQueueIndex); + requestBody.setMessageQueueRouteState(MessageQueueRouteState.WriteOnly); + request.setBody(requestBody.encode()); + + RemotingCommand response = adminBrokerProcessor.processRequest(handlerContext, request); + assertThat(response.getCode()).isEqualTo(ResponseCode.SUCCESS); + LogicalQueueRouteData wantData1 = new LogicalQueueRouteData(logicalQueueIndex, -1L, mq, MessageQueueRouteState.WriteOnly, 600L, -1, -1, -1, brokerController.getBrokerAddr()); + assertThat((LogicalQueueRouteData) JSON.parseObject(response.getBody(), LogicalQueueRouteData.class)).isEqualTo(wantData1); + assertThat(logicalQueuesInfo.get(logicalQueueIndex)).isEqualTo(Arrays.asList(wantData0, wantData1)); + verify(inFlyWritingCouterMap).remove(new TopicQueueId(topic, queueId)); + } + + @Test + public void testCreateMessageQueueForLogicalQueue() throws Exception { + int logicalQueueIndex = 0; + TopicConfigManager topicConfigManager = brokerController.getTopicConfigManager(); + TopicConfig topicConfig = topicConfigManager.selectTopicConfig(topic); + topicConfig.setWriteQueueNums(0); + topicConfig.setReadQueueNums(0); + int queueId = 0; + assertThat(topicConfigManager.selectLogicalQueuesInfo(topic)).isNull(); + RemotingCommand request = RemotingCommand.createRequestCommand(RequestCode.CREATE_MESSAGE_QUEUE_FOR_LOGICAL_QUEUE, null); + CreateMessageQueueForLogicalQueueRequestBody requestBody = new CreateMessageQueueForLogicalQueueRequestBody(); + requestBody.setTopic(topic); + requestBody.setLogicalQueueIndex(logicalQueueIndex); + requestBody.setMessageQueueStatus(MessageQueueRouteState.WriteOnly); + request.setBody(requestBody.encode()); + + RemotingCommand response = adminBrokerProcessor.processRequest(handlerContext, request); + assertThat(response.getCode()).withFailMessage("remark: %s", response.getRemark()).isEqualTo(ResponseCode.SUCCESS); + LogicalQueueRouteData wantLogicalQueueRouteData = new LogicalQueueRouteData(logicalQueueIndex, -1L, new MessageQueue(topic, brokerController.getBrokerConfig().getBrokerName(), queueId), MessageQueueRouteState.WriteOnly, 0L, -1, -1, -1, brokerController.getBrokerAddr()); + assertThat((LogicalQueueRouteData) JSON.parseObject(response.getBody(), LogicalQueueRouteData.class)).isEqualTo(wantLogicalQueueRouteData); + assertThat(topicConfigManager.selectLogicalQueuesInfo(topic).get(logicalQueueIndex).get(0)).isEqualTo(wantLogicalQueueRouteData); + } + + @Test + public void testMigrateTopicLogicalQueuePrepare() throws Exception { + int queueId = 0; + int logicalQueueIndex = 0; + TopicConfigManager topicConfigManager = brokerController.getTopicConfigManager(); + LogicalQueuesInfoInBroker logicalQueuesInfo = topicConfigManager.getOrCreateLogicalQueuesInfo(topic); + MessageQueue mq = new MessageQueue(topic, brokerController.getBrokerConfig().getBrokerName(), queueId); + LogicalQueueRouteData fromQueueRouteData = new LogicalQueueRouteData(logicalQueueIndex, 500L, mq, MessageQueueRouteState.Normal, 10L, -1L, -1L, -1L, brokerController.getBrokerAddr()); + logicalQueuesInfo.put(logicalQueueIndex, Lists.newArrayList(new LogicalQueueRouteData(fromQueueRouteData))); + + when(messageStore.getMaxOffsetInQueue(eq(topic), eq(queueId), anyBoolean())).thenReturn(100L); + when(messageStore.getMinOffsetInQueue(eq(topic), eq(queueId))).thenReturn(10L); + when(messageStore.getMinPhyOffset()).thenReturn(1000L); + when(messageStore.getCommitLogOffsetInQueue(eq(topic), eq(queueId), eq(10L))).thenReturn(2000L); + when(messageStore.getCommitLogOffsetInQueue(eq(topic), eq(queueId), eq(99L))).thenReturn(3000L); + MessageExt firstMsg = mock(MessageExt.class); + when(firstMsg.getStoreTimestamp()).thenReturn(200L); + when(messageStore.lookMessageByOffset(eq(2000L))).thenReturn(firstMsg); + MessageExt lastMsg = mock(MessageExt.class); + when(lastMsg.getStoreTimestamp()).thenReturn(300L); + when(messageStore.lookMessageByOffset(eq(3000L))).thenReturn(lastMsg); + + RemotingCommand request = RemotingCommand.createRequestCommand(RequestCode.MIGRATE_TOPIC_LOGICAL_QUEUE_PREPARE, null); + MigrateLogicalQueueBody requestBody = new MigrateLogicalQueueBody(); + requestBody.setFromQueueRouteData(fromQueueRouteData); + LogicalQueueRouteData toQueueRouteData = new LogicalQueueRouteData(); + toQueueRouteData.setMessageQueue(new MessageQueue(topic, "toBroker", 1)); + requestBody.setToQueueRouteData(toQueueRouteData); + request.setBody(requestBody.encode()); + + RemotingCommand response = adminBrokerProcessor.processRequest(handlerContext, request); + assertThat(response.getCode()).withFailMessage("remark: %s", response.getRemark()).isEqualTo(ResponseCode.SUCCESS); + fromQueueRouteData.setState(MessageQueueRouteState.ReadOnly); + fromQueueRouteData.setOffsetMax(100L); + fromQueueRouteData.setFirstMsgTimeMillis(200L); + fromQueueRouteData.setLastMsgTimeMillis(300L); + toQueueRouteData.setLogicalQueueDelta(590L); + MigrateLogicalQueueBody responseBody = RemotingSerializable.decode(response.getBody(), MigrateLogicalQueueBody.class); + assertThat(responseBody.getFromQueueRouteData()).isEqualTo(fromQueueRouteData); + assertThat(responseBody.getToQueueRouteData()).isEqualTo(toQueueRouteData); + assertThat(logicalQueuesInfo.get(logicalQueueIndex)).isEqualTo(Lists.newArrayList(fromQueueRouteData, toQueueRouteData)); + } + + @Test + public void testMigrateTopicLogicalQueueCommit() throws Exception { + int queueId = 0; + int logicalQueueIndex = 0; + TopicConfigManager topicConfigManager = brokerController.getTopicConfigManager(); + LogicalQueuesInfoInBroker logicalQueuesInfo = topicConfigManager.getOrCreateLogicalQueuesInfo(topic); + MessageQueue mq = new MessageQueue(topic, brokerController.getBrokerConfig().getBrokerName(), queueId); + LogicalQueueRouteData fromQueueRouteData = new LogicalQueueRouteData(); + fromQueueRouteData.setMessageQueue(new MessageQueue(topic, "fromBroker", 0)); + LogicalQueueRouteData toQueueRouteData = new LogicalQueueRouteData(logicalQueueIndex, 500L, mq, MessageQueueRouteState.Normal, 500L, -1L, -1L, -1L, brokerController.getBrokerAddr()); + logicalQueuesInfo.put(logicalQueueIndex, Lists.newArrayList(new LogicalQueueRouteData(toQueueRouteData))); + + RemotingCommand request = RemotingCommand.createRequestCommand(RequestCode.MIGRATE_TOPIC_LOGICAL_QUEUE_COMMIT, null); + MigrateLogicalQueueBody requestBody = new MigrateLogicalQueueBody(); + requestBody.setFromQueueRouteData(fromQueueRouteData); + requestBody.setToQueueRouteData(toQueueRouteData); + request.setBody(requestBody.encode()); + + RemotingCommand response = adminBrokerProcessor.processRequest(handlerContext, request); + assertThat(response.getCode()).withFailMessage("remark: %s", response.getRemark()).isEqualTo(ResponseCode.SUCCESS); + MigrateLogicalQueueBody responseBody = RemotingSerializable.decode(response.getBody(), MigrateLogicalQueueBody.class); + assertThat(responseBody.getFromQueueRouteData()).isEqualTo(fromQueueRouteData); + assertThat(responseBody.getToQueueRouteData()).isEqualTo(toQueueRouteData); + assertThat(logicalQueuesInfo.get(logicalQueueIndex)).isEqualTo(Lists.newArrayList(toQueueRouteData)); + } + + @Test + public void testMigrateTopicLogicalQueueNotify() throws Exception { + int queueId = 0; + int logicalQueueIndex = 0; + TopicConfigManager topicConfigManager = brokerController.getTopicConfigManager(); + LogicalQueuesInfoInBroker logicalQueuesInfo = topicConfigManager.getOrCreateLogicalQueuesInfo(topic); + LogicalQueueRouteData fromQueueRouteData = new LogicalQueueRouteData(logicalQueueIndex, 100L, new MessageQueue(topic, "fromBroker", queueId), MessageQueueRouteState.ReadOnly, 10L, 410L, 200L, 300L, brokerController.getBrokerAddr()); + LogicalQueueRouteData toQueueRouteData = new LogicalQueueRouteData(logicalQueueIndex, 500L, new MessageQueue(topic, "toBroker", queueId), MessageQueueRouteState.Normal, 500L, -1L, -1L, -1L, brokerController.getBrokerAddr()); + logicalQueuesInfo.put(logicalQueueIndex, Lists.newArrayList(new LogicalQueueRouteData(fromQueueRouteData))); + + RemotingCommand request = RemotingCommand.createRequestCommand(RequestCode.MIGRATE_TOPIC_LOGICAL_QUEUE_NOTIFY, null); + MigrateLogicalQueueBody requestBody = new MigrateLogicalQueueBody(); + requestBody.setFromQueueRouteData(fromQueueRouteData); + requestBody.setToQueueRouteData(toQueueRouteData); + request.setBody(requestBody.encode()); + + RemotingCommand response = adminBrokerProcessor.processRequest(handlerContext, request); + assertThat(response.getCode()).withFailMessage("remark: %s", response.getRemark()).isEqualTo(ResponseCode.SUCCESS); + assertThat(logicalQueuesInfo.get(logicalQueueIndex)).isEqualTo(Lists.newArrayList(fromQueueRouteData, toQueueRouteData)); + } + private RemotingCommand buildCreateTopicRequest(String topic) { CreateTopicRequestHeader requestHeader = new CreateTopicRequestHeader(); requestHeader.setTopic(topic); diff --git a/broker/src/test/java/org/apache/rocketmq/broker/processor/PullMessageProcessorTest.java b/broker/src/test/java/org/apache/rocketmq/broker/processor/PullMessageProcessorTest.java index c96f708e85463937ada305d8930c37dc4d83de9d..b6377553d7afd994473317d9ab46631af3ab8371 100644 --- a/broker/src/test/java/org/apache/rocketmq/broker/processor/PullMessageProcessorTest.java +++ b/broker/src/test/java/org/apache/rocketmq/broker/processor/PullMessageProcessorTest.java @@ -20,24 +20,31 @@ import io.netty.channel.Channel; import io.netty.channel.ChannelHandlerContext; import java.net.InetSocketAddress; import java.util.ArrayList; +import java.util.HashMap; import java.util.HashSet; import java.util.List; import java.util.Set; import org.apache.rocketmq.broker.BrokerController; import org.apache.rocketmq.broker.client.ClientChannelInfo; +import org.apache.rocketmq.broker.domain.LogicalQueuesInfoInBroker; import org.apache.rocketmq.broker.filter.ExpressionMessageFilter; import org.apache.rocketmq.broker.mqtrace.ConsumeMessageContext; import org.apache.rocketmq.broker.mqtrace.ConsumeMessageHook; import org.apache.rocketmq.common.BrokerConfig; import org.apache.rocketmq.common.TopicConfig; import org.apache.rocketmq.common.consumer.ConsumeFromWhere; +import org.apache.rocketmq.common.message.MessageConst; +import org.apache.rocketmq.common.message.MessageQueue; import org.apache.rocketmq.common.protocol.RequestCode; import org.apache.rocketmq.common.protocol.ResponseCode; import org.apache.rocketmq.common.protocol.header.PullMessageRequestHeader; +import org.apache.rocketmq.common.protocol.header.PullMessageResponseHeader; import org.apache.rocketmq.common.protocol.heartbeat.ConsumeType; import org.apache.rocketmq.common.protocol.heartbeat.ConsumerData; import org.apache.rocketmq.common.protocol.heartbeat.MessageModel; import org.apache.rocketmq.common.protocol.heartbeat.SubscriptionData; +import org.apache.rocketmq.common.protocol.route.LogicalQueueRouteData; +import org.apache.rocketmq.common.protocol.route.MessageQueueRouteState; import org.apache.rocketmq.remoting.exception.RemotingCommandException; import org.apache.rocketmq.remoting.netty.NettyClientConfig; import org.apache.rocketmq.remoting.netty.NettyServerConfig; @@ -46,6 +53,7 @@ import org.apache.rocketmq.store.GetMessageResult; import org.apache.rocketmq.store.GetMessageStatus; import org.apache.rocketmq.store.MessageStore; import org.apache.rocketmq.store.config.MessageStoreConfig; +import org.assertj.core.util.Lists; import org.junit.Before; import org.junit.Test; import org.junit.runner.RunWith; @@ -53,11 +61,15 @@ import org.mockito.Mock; import org.mockito.Spy; import org.mockito.junit.MockitoJUnitRunner; +import static java.util.Optional.ofNullable; import static org.assertj.core.api.Assertions.assertThat; import static org.mockito.ArgumentMatchers.any; import static org.mockito.ArgumentMatchers.anyInt; import static org.mockito.ArgumentMatchers.anyLong; import static org.mockito.ArgumentMatchers.anyString; +import static org.mockito.ArgumentMatchers.eq; +import static org.mockito.ArgumentMatchers.intThat; +import static org.mockito.Mockito.lenient; import static org.mockito.Mockito.mock; import static org.mockito.Mockito.when; @@ -92,6 +104,7 @@ public class PullMessageProcessorTest { consumerData.getConsumeFromWhere(), consumerData.getSubscriptionDataSet(), false); + brokerController.getTopicConfigManager().updateTopicConfig(new TopicConfig(topic)); } @Test @@ -192,6 +205,94 @@ public class PullMessageProcessorTest { assertThat(response.getCode()).isEqualTo(ResponseCode.PULL_OFFSET_MOVED); } + @Test + public void testProcessRequest_LogicalQueue() throws Exception { + String brokerName = brokerController.getBrokerConfig().getBrokerName(); + int queueId = 1; + + GetMessageResult getMessageResult = createGetMessageResult(); + when(messageStore.getMessage(anyString(), eq(topic), eq(queueId), eq(456L), anyInt(), any(ExpressionMessageFilter.class))).thenReturn(getMessageResult); + when(messageStore.getMaxOffsetInQueue(eq(topic), eq(queueId))).thenReturn(2000L); + when(messageStore.getMinPhyOffset()).thenReturn(0L); + + LogicalQueuesInfoInBroker logicalQueuesInfo = brokerController.getTopicConfigManager().getOrCreateLogicalQueuesInfo(topic); + LogicalQueueRouteData queueRouteData1 = new LogicalQueueRouteData(0, 0, new MessageQueue(topic, brokerName, queueId), MessageQueueRouteState.Normal, 0, -1, -1, -1, brokerController.getBrokerAddr()); + logicalQueuesInfo.put(0, Lists.newArrayList(queueRouteData1)); + logicalQueuesInfo.updateQueueRouteDataByQueueId(queueRouteData1.getQueueId(), queueRouteData1); + + // normal + { + final RemotingCommand request = createPullMsgCommand(RequestCode.PULL_MESSAGE); + RemotingCommand response = pullMessageProcessor.processRequest(handlerContext, request); + assertThat(response).isNotNull(); + assertThat(response.getCode()).isEqualTo(ResponseCode.SUCCESS); + } + // write only + queueRouteData1.setState(MessageQueueRouteState.WriteOnly); + { + final RemotingCommand request = createPullMsgCommand(RequestCode.PULL_MESSAGE); + RemotingCommand response = pullMessageProcessor.processRequest(handlerContext, request); + assertThat(response).isNotNull(); + assertThat(response.getCode()).isEqualTo(ResponseCode.PULL_NOT_FOUND); + } + // no message and redirect + queueRouteData1.setState(MessageQueueRouteState.ReadOnly); + queueRouteData1.setOffsetMax(460); + queueRouteData1.setFirstMsgTimeMillis(100); + queueRouteData1.setLastMsgTimeMillis(200); + LogicalQueueRouteData queueRouteData2 = new LogicalQueueRouteData(0, 460, new MessageQueue(topic, "broker2", 1), MessageQueueRouteState.Normal, 0, -1, -1, -1, brokerController.getBrokerAddr()); + logicalQueuesInfo.get(0).add(queueRouteData2); + getMessageResult.setStatus(GetMessageStatus.OFFSET_FOUND_NULL); + when(messageStore.getCommitLogOffsetInQueue(eq(topic), eq(queueId), eq(460L - 1L))).thenReturn(1000L); + { + final RemotingCommand request = createPullMsgCommand(RequestCode.PULL_MESSAGE); + RemotingCommand response = pullMessageProcessor.processRequest(handlerContext, request); + assertThat(response).isNotNull(); + assertThat(response.getCode()).isEqualTo(ResponseCode.PULL_NOT_FOUND); + assertThat(response.getExtFields()).containsKey(MessageConst.PROPERTY_REDIRECT); + } + // same message queue has two routes + queueRouteData2.setState(MessageQueueRouteState.ReadOnly); + queueRouteData2.setOffsetMax(50); + queueRouteData2.setFirstMsgTimeMillis(300); + queueRouteData2.setLastMsgTimeMillis(400); + LogicalQueueRouteData queueRouteData3 = new LogicalQueueRouteData(0, 510, new MessageQueue(topic, queueRouteData2.getBrokerName(), queueId), MessageQueueRouteState.Normal, 460, -1, -1, -1, queueRouteData1.getBrokerAddr()); + logicalQueuesInfo.get(0).add(queueRouteData3); + logicalQueuesInfo.updateQueueRouteDataByQueueId(queueRouteData3.getQueueId(), queueRouteData3); + { + GetMessageResult getMessageResult2 = createGetMessageResult(); + getMessageResult2.setStatus(GetMessageStatus.FOUND); + getMessageResult2.setNextBeginOffset(460); + when(messageStore.getMessage(anyString(), eq(queueRouteData1.getTopic()), eq(queueRouteData1.getQueueId()), eq(456L), eq(4), any(ExpressionMessageFilter.class))).thenReturn(getMessageResult2); + } + { + GetMessageResult getMessageResult2 = createGetMessageResult(); + getMessageResult2.setStatus(GetMessageStatus.FOUND); + getMessageResult2.setNextBeginOffset(470); + lenient().when(messageStore.getMessage(anyString(), eq(queueRouteData1.getTopic()), eq(queueRouteData1.getQueueId()), eq(456L), intThat(i -> i > 4), any(ExpressionMessageFilter.class))).thenReturn(getMessageResult2); + } + { + final RemotingCommand request = createPullMsgCommand(RequestCode.PULL_MESSAGE); + RemotingCommand response = pullMessageProcessor.processRequest(handlerContext, request); + assertThat(response).isNotNull(); + assertThat(response.getCode()).isEqualTo(ResponseCode.SUCCESS); + assertThat(ofNullable(response.getExtFields()).orElse(new HashMap<>())).doesNotContainKey(MessageConst.PROPERTY_REDIRECT); + PullMessageResponseHeader header = (PullMessageResponseHeader) response.readCustomHeader(); + assertThat(header.getNextBeginOffset()).isEqualTo(460); + } + { + when(messageStore.getMinPhyOffset()).thenReturn(100000L); + + final RemotingCommand request = createPullMsgCommand(RequestCode.PULL_MESSAGE); + RemotingCommand response = pullMessageProcessor.processRequest(handlerContext, request); + assertThat(response).isNotNull(); + assertThat(response.getCode()).isEqualTo(ResponseCode.PULL_RETRY_IMMEDIATELY); + assertThat(ofNullable(response.getExtFields()).orElse(new HashMap<>())).containsKey(MessageConst.PROPERTY_REDIRECT); + PullMessageResponseHeader header = (PullMessageResponseHeader) response.readCustomHeader(); + assertThat(header.getNextBeginOffset()).isEqualTo(460); + } + } + private RemotingCommand createPullMsgCommand(int requestCode) { PullMessageRequestHeader requestHeader = new PullMessageRequestHeader(); requestHeader.setCommitOffset(123L); diff --git a/broker/src/test/java/org/apache/rocketmq/broker/processor/SendMessageProcessorTest.java b/broker/src/test/java/org/apache/rocketmq/broker/processor/SendMessageProcessorTest.java index b9344e90ed600f29ac72c2720b60fbbff1cda5d4..e768469f7aae4159043f4e005bcaa80e0b054cf9 100644 --- a/broker/src/test/java/org/apache/rocketmq/broker/processor/SendMessageProcessorTest.java +++ b/broker/src/test/java/org/apache/rocketmq/broker/processor/SendMessageProcessorTest.java @@ -18,18 +18,30 @@ package org.apache.rocketmq.broker.processor; import io.netty.channel.Channel; import io.netty.channel.ChannelHandlerContext; +import java.net.InetSocketAddress; +import java.util.ArrayList; +import java.util.List; +import java.util.Map; +import java.util.concurrent.CompletableFuture; +import java.util.concurrent.TimeUnit; import org.apache.rocketmq.broker.BrokerController; +import org.apache.rocketmq.broker.domain.LogicalQueuesInfoInBroker; import org.apache.rocketmq.broker.mqtrace.SendMessageContext; import org.apache.rocketmq.broker.mqtrace.SendMessageHook; import org.apache.rocketmq.broker.transaction.TransactionalMessageService; import org.apache.rocketmq.common.BrokerConfig; +import org.apache.rocketmq.common.TopicConfig; +import org.apache.rocketmq.common.constant.PermName; 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.MessageQueue; import org.apache.rocketmq.common.protocol.RequestCode; import org.apache.rocketmq.common.protocol.ResponseCode; import org.apache.rocketmq.common.protocol.header.ConsumerSendMsgBackRequestHeader; import org.apache.rocketmq.common.protocol.header.SendMessageRequestHeader; +import org.apache.rocketmq.common.protocol.route.LogicalQueueRouteData; +import org.apache.rocketmq.common.protocol.route.MessageQueueRouteState; import org.apache.rocketmq.common.sysflag.MessageSysFlag; import org.apache.rocketmq.common.topic.TopicValidator; import org.apache.rocketmq.remoting.exception.RemotingCommandException; @@ -43,6 +55,7 @@ import org.apache.rocketmq.store.MessageStore; import org.apache.rocketmq.store.PutMessageResult; import org.apache.rocketmq.store.PutMessageStatus; import org.apache.rocketmq.store.config.MessageStoreConfig; +import org.assertj.core.util.Lists; import org.junit.Before; import org.junit.Test; import org.junit.runner.RunWith; @@ -52,12 +65,6 @@ import org.mockito.invocation.InvocationOnMock; import org.mockito.junit.MockitoJUnitRunner; import org.mockito.stubbing.Answer; -import java.net.InetSocketAddress; -import java.util.ArrayList; -import java.util.List; -import java.util.Map; -import java.util.concurrent.CompletableFuture; - import static org.assertj.core.api.Assertions.assertThat; import static org.mockito.ArgumentMatchers.any; import static org.mockito.ArgumentMatchers.anyLong; @@ -90,6 +97,8 @@ public class SendMessageProcessorTest { when(handlerContext.channel()).thenReturn(mockChannel); when(messageStore.lookMessageByOffset(anyLong())).thenReturn(new MessageExt()); sendMessageProcessor = new SendMessageProcessor(brokerController); + + brokerController.getTopicConfigManager().updateTopicConfig(new TopicConfig(topic, 8, 8, PermName.PERM_WRITE|PermName.PERM_READ)); } @Test @@ -220,6 +229,62 @@ public class SendMessageProcessorTest { assertThat(response[0].getCode()).isEqualTo(ResponseCode.SUCCESS); } + + @Test + public void testProcessRequest_LogicalQueue() throws Exception { + when(messageStore.asyncPutMessage(any(MessageExtBrokerInner.class))) + .thenReturn(CompletableFuture.completedFuture(new PutMessageResult(PutMessageStatus.PUT_OK, new AppendMessageResult(AppendMessageStatus.PUT_OK)))); + + LogicalQueuesInfoInBroker logicalQueuesInfo = brokerController.getTopicConfigManager().getOrCreateLogicalQueuesInfo(topic); + LogicalQueueRouteData queueRouteData1 = new LogicalQueueRouteData(0, 0, new MessageQueue(topic, brokerController.getBrokerConfig().getBrokerName(), 1), MessageQueueRouteState.Normal, 0, -1, -1, -1, brokerController.getBrokerAddr()); + logicalQueuesInfo.put(0, Lists.newArrayList(queueRouteData1)); + logicalQueuesInfo.updateQueueRouteDataByQueueId(queueRouteData1.getQueueId(), queueRouteData1); + + SendMessageRequestHeader requestHeader = createSendMsgRequestHeader(); + RemotingCommand request = RemotingCommand.createRequestCommand(RequestCode.SEND_MESSAGE, requestHeader); + request.setBody(new byte[] {'a'}); + request.makeCustomHeaderToNet(); + + // normal + RemotingCommand responseToReturn; + { + CompletableFuture responseFuture = new CompletableFuture<>(); + doAnswer(invocation -> { + responseFuture.complete(invocation.getArgument(0)); + return null; + }).when(handlerContext).writeAndFlush(any(Object.class)); + + responseToReturn = sendMessageProcessor.processRequest(handlerContext, request); + if (responseToReturn == null) { + responseToReturn = responseFuture.get(3, TimeUnit.SECONDS); + } + } + + assertThat(responseToReturn.getCode()).isEqualTo(ResponseCode.SUCCESS); + assertThat(responseToReturn.getOpaque()).isEqualTo(request.getOpaque()); + + // read only + queueRouteData1.setState(MessageQueueRouteState.ReadOnly); + responseToReturn = sendMessageProcessor.processRequest(handlerContext, request); + assertThat(responseToReturn.getCode()).isEqualTo(ResponseCode.NO_PERMISSION); + assertThat(responseToReturn.getRemark()).contains("not writable"); + + // read only and forward + logicalQueuesInfo.get(0).add(new LogicalQueueRouteData(0, 100, new MessageQueue(topic, "broker2", 1), MessageQueueRouteState.Normal, 0, -1, -1, -1, brokerController.getBrokerAddr())); + + responseToReturn = sendMessageProcessor.processRequest(handlerContext, request); + assertThat(responseToReturn.getCode()).isEqualTo(ResponseCode.SYSTEM_ERROR); + assertThat(responseToReturn.getRemark()).contains("forward error"); + + // read only and redirect + requestHeader = (SendMessageRequestHeader) request.readCustomHeader(); + requestHeader.setSysFlag(MessageSysFlag.LOGICAL_QUEUE_FLAG); + request.makeCustomHeaderToNet(); + responseToReturn = sendMessageProcessor.processRequest(handlerContext, request); + assertThat(responseToReturn.getCode()).isEqualTo(ResponseCode.NO_PERMISSION); + assertThat(responseToReturn.getExtFields()).containsKey(MessageConst.PROPERTY_REDIRECT); + } + private RemotingCommand createSendTransactionMsgCommand(int requestCode) { SendMessageRequestHeader header = createSendMsgRequestHeader(); int sysFlag = header.getSysFlag(); diff --git a/broker/src/test/java/org/apache/rocketmq/broker/topic/TopicConfigManagerTest.java b/broker/src/test/java/org/apache/rocketmq/broker/topic/TopicConfigManagerTest.java new file mode 100644 index 0000000000000000000000000000000000000000..f504965fada079273d8e2991a400defab9223c10 --- /dev/null +++ b/broker/src/test/java/org/apache/rocketmq/broker/topic/TopicConfigManagerTest.java @@ -0,0 +1,138 @@ +/* + * 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.topic; + +import com.google.common.collect.Lists; +import java.nio.file.Files; +import java.nio.file.Paths; +import java.util.Arrays; +import java.util.Collections; +import java.util.List; +import org.apache.rocketmq.broker.BrokerController; +import org.apache.rocketmq.broker.domain.LogicalQueuesInfoInBroker; +import org.apache.rocketmq.common.BrokerConfig; +import org.apache.rocketmq.common.DataVersion; +import org.apache.rocketmq.common.TopicConfig; +import org.apache.rocketmq.common.message.MessageQueue; +import org.apache.rocketmq.common.protocol.route.LogicalQueueRouteData; +import org.apache.rocketmq.common.protocol.route.MessageQueueRouteState; +import org.apache.rocketmq.store.DefaultMessageStore; +import org.apache.rocketmq.store.config.MessageStoreConfig; +import org.junit.After; +import org.junit.Before; +import org.junit.Test; +import org.junit.runner.RunWith; +import org.mockito.ArgumentMatchers; +import org.mockito.Mock; +import org.mockito.junit.MockitoJUnitRunner; + +import static org.assertj.core.api.Assertions.assertThat; +import static org.mockito.ArgumentMatchers.any; +import static org.mockito.ArgumentMatchers.anyLong; +import static org.mockito.ArgumentMatchers.eq; +import static org.mockito.Mockito.never; +import static org.mockito.Mockito.verify; +import static org.mockito.Mockito.when; + +@RunWith(MockitoJUnitRunner.class) +public class TopicConfigManagerTest { + @Mock + private DefaultMessageStore messageStore; + @Mock + private BrokerController brokerController; + + private TopicConfigManager topicConfigManager; + + private static final String topic = "FooBar"; + private static final String broker1Name = "broker1"; + private static final String broker1Addr = "127.0.0.1:12345"; + private static final int queueId1 = 1; + private static final String broker2Name = "broker2"; + private static final String broker2Addr = "127.0.0.2:12345"; + private static final int queueId2 = 2; + + @Before + public void before() { + BrokerConfig brokerConfig = new BrokerConfig(); + brokerConfig.setBrokerName(broker1Name); + when(brokerController.getBrokerConfig()).thenReturn(brokerConfig); + + when(brokerController.getMessageStore()).thenReturn(messageStore); + + MessageStoreConfig messageStoreConfig = new MessageStoreConfig(); + messageStoreConfig.setStorePathRootDir(System.getProperty("java.io.tmpdir")); + when(brokerController.getMessageStoreConfig()).thenReturn(messageStoreConfig); + + this.topicConfigManager = new TopicConfigManager(brokerController); + this.topicConfigManager.getTopicConfigTable().put(topic, new TopicConfig(topic)); + } + + @After + public void after() throws Exception { + if (topicConfigManager != null) { + Files.deleteIfExists(Paths.get(topicConfigManager.configFilePath())); + } + } + + @Test + public void logicalQueueCleanTest() { + LogicalQueuesInfoInBroker info = this.topicConfigManager.getOrCreateLogicalQueuesInfo(topic); + topicConfigManager.logicalQueueClean(); + assertThat(info).isEmpty(); + + final int logicalQueueIndex = 0; + LogicalQueueRouteData queueRouteData1 = new LogicalQueueRouteData(logicalQueueIndex, 0, new MessageQueue(topic, broker1Name, queueId1), MessageQueueRouteState.Normal, 0, -1, -1, -1, broker1Addr); + List l = Lists.newArrayList(new LogicalQueueRouteData(queueRouteData1)); + info.put(logicalQueueIndex, l); + + topicConfigManager.logicalQueueClean(); + assertThat(info.get(logicalQueueIndex)).isEqualTo(Collections.singletonList(queueRouteData1)); + verify(messageStore, never()).getCommitLogOffsetInQueue(eq(topic), eq(queueId1), anyLong()); + verify(messageStore, never()).getMinPhyOffset(); + verify(brokerController, never()).registerIncrementBrokerData(ArgumentMatchers.argThat(arg -> topic.equals(arg.getTopicName())), any(DataVersion.class)); + + LogicalQueueRouteData queueRouteData2 = new LogicalQueueRouteData(logicalQueueIndex, 100, new MessageQueue(topic, broker2Name, queueId2), MessageQueueRouteState.Normal, 0, -1, -1, -1, broker2Addr); + l.add(new LogicalQueueRouteData(queueRouteData2)); + queueRouteData1 = l.get(0); + queueRouteData1.setState(MessageQueueRouteState.ReadOnly); + queueRouteData1.setOffsetMax(100); + queueRouteData1.setFirstMsgTimeMillis(200); + queueRouteData1.setLastMsgTimeMillis(300); + queueRouteData1 = new LogicalQueueRouteData(queueRouteData1); + LogicalQueueRouteData queueRouteData3 = new LogicalQueueRouteData(logicalQueueIndex, 200, new MessageQueue(topic, broker1Name, queueId1), MessageQueueRouteState.Normal, 100, -1, -1, -1, broker1Addr); + l.add(new LogicalQueueRouteData(queueRouteData3)); + queueRouteData2 = l.get(1); + queueRouteData2.setState(MessageQueueRouteState.ReadOnly); + queueRouteData2.setOffsetMax(100); + queueRouteData2.setFirstMsgTimeMillis(400); + queueRouteData2.setLastMsgTimeMillis(500); + queueRouteData2 = new LogicalQueueRouteData(queueRouteData2); + when(messageStore.getCommitLogOffsetInQueue(eq(topic), eq(queueId1), eq(queueRouteData1.getOffsetMax() - 1))).thenReturn(1000L); + when(messageStore.getMinPhyOffset()).thenReturn(0L); + topicConfigManager.logicalQueueClean(); + assertThat(info.get(logicalQueueIndex)).isEqualTo(Arrays.asList(queueRouteData1, queueRouteData2, queueRouteData3)); + verify(messageStore).getCommitLogOffsetInQueue(eq(topic), eq(queueId1), eq(queueRouteData1.getOffsetMax() - 1)); + verify(messageStore).getMinPhyOffset(); + verify(brokerController, never()).registerIncrementBrokerData(ArgumentMatchers.argThat(arg -> topic.equals(arg.getTopicName())), any(DataVersion.class)); + + when(messageStore.getMinPhyOffset()).thenReturn(2000L); + topicConfigManager.logicalQueueClean(); + assertThat(info.get(logicalQueueIndex)).isEqualTo(Collections.singletonList(queueRouteData3)); + verify(brokerController).registerIncrementBrokerData(ArgumentMatchers.argThat(arg -> topic.equals(arg.getTopicName())), any(DataVersion.class)); + } +} \ No newline at end of file diff --git a/srvutil/src/main/java/org/apache/rocketmq/srvutil/ConcurrentHashMapUtil.java b/srvutil/src/main/java/org/apache/rocketmq/srvutil/ConcurrentHashMapUtil.java new file mode 100644 index 0000000000000000000000000000000000000000..cc98eb5bcb5eaade41cf034bcd7f0ebfd37f78f4 --- /dev/null +++ b/srvutil/src/main/java/org/apache/rocketmq/srvutil/ConcurrentHashMapUtil.java @@ -0,0 +1,50 @@ +/* + * 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.srvutil; + +import java.util.concurrent.ConcurrentMap; +import java.util.function.Function; + +public class ConcurrentHashMapUtil { + private static final boolean IS_JDK8; + + static { + // Java 8 or lower: 1.6.0_23, 1.7.0, 1.7.0_80, 1.8.0_211 + // Java 9 or higher: 9.0.1, 11.0.4, 12, 12.0.1 + IS_JDK8 = System.getProperty("java.version").startsWith("1.8."); + } + + private ConcurrentHashMapUtil() { + } + + /** + * A temporary workaround for Java 8 specific performance issue JDK-8161372 .
Use implementation of + * ConcurrentMap.computeIfAbsent instead. + * + * @see https://bugs.openjdk.java.net/browse/JDK-8161372 + */ + public static V computeIfAbsent(ConcurrentMap map, K key, Function func) { + if (IS_JDK8) { + V v, newValue; + return ((v = map.get(key)) == null && + (newValue = func.apply(key)) != null && + (v = map.putIfAbsent(key, newValue)) == null) ? newValue : v; + } else { + return map.computeIfAbsent(key, func); + } + } +} diff --git a/store/src/main/java/org/apache/rocketmq/store/CleanFilesHook.java b/store/src/main/java/org/apache/rocketmq/store/CleanFilesHook.java new file mode 100644 index 0000000000000000000000000000000000000000..ef6e22a8f887d2d1dd009aa005b203df2efa0b95 --- /dev/null +++ b/store/src/main/java/org/apache/rocketmq/store/CleanFilesHook.java @@ -0,0 +1,23 @@ +/* + * 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; + +public interface CleanFilesHook { + void execute(DefaultMessageStore defaultMessageStore, long deleteCount); + + String getName(); +} 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 cce6481b8daa119def971b3c4c2f95d4b3e6502a..0492aa95e79d2c43bf3208c5ec2a38da9e5a2fb7 100644 --- a/store/src/main/java/org/apache/rocketmq/store/CommitLog.java +++ b/store/src/main/java/org/apache/rocketmq/store/CommitLog.java @@ -140,7 +140,17 @@ public class CommitLog { final long intervalForcibly, final boolean cleanImmediately ) { - return this.mappedFileQueue.deleteExpiredFileByTime(expiredTime, deleteFilesInterval, intervalForcibly, cleanImmediately); + return deleteExpiredFile(expiredTime, deleteFilesInterval, intervalForcibly, cleanImmediately, 0); + } + + public int deleteExpiredFile( + final long expiredTime, + final int deleteFilesInterval, + final long intervalForcibly, + final boolean cleanImmediately, + final int deleteFileBatchMax + ) { + return this.mappedFileQueue.deleteExpiredFileByTime(expiredTime, deleteFilesInterval, intervalForcibly, cleanImmediately, deleteFileBatchMax); } /** 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 c25a1147328644bf2ea23c69755c49b0036a7028..b11eb49047c0f8e79f51773ec548a491259af56d 100644 --- a/store/src/main/java/org/apache/rocketmq/store/DefaultMessageStore.java +++ b/store/src/main/java/org/apache/rocketmq/store/DefaultMessageStore.java @@ -28,12 +28,14 @@ import java.util.Collections; import java.util.HashMap; import java.util.Iterator; import java.util.LinkedList; +import java.util.List; import java.util.Map; import java.util.Map.Entry; import java.util.Set; import java.util.concurrent.CompletableFuture; import java.util.concurrent.ConcurrentHashMap; import java.util.concurrent.ConcurrentMap; +import java.util.concurrent.CopyOnWriteArrayList; import java.util.concurrent.Executors; import java.util.concurrent.ScheduledExecutorService; import java.util.concurrent.TimeUnit; @@ -118,6 +120,8 @@ public class DefaultMessageStore implements MessageStore { private final ScheduledExecutorService diskCheckScheduledExecutorService = Executors.newSingleThreadScheduledExecutor(new ThreadFactoryImpl("DiskCheckScheduledThread")); + private final List cleanFilesHooks = new CopyOnWriteArrayList<>(); + public DefaultMessageStore(final MessageStoreConfig messageStoreConfig, final BrokerStatsManager brokerStatsManager, final MessageArrivingListener messageArrivingListener, final BrokerConfig brokerConfig) throws IOException { this.messageArrivingListener = messageArrivingListener; @@ -720,10 +724,20 @@ public class DefaultMessageStore implements MessageStore { } public long getMaxOffsetInQueue(String topic, int queueId) { - ConsumeQueue logic = this.findConsumeQueue(topic, queueId); - if (logic != null) { - long offset = logic.getMaxOffsetInQueue(); - return offset; + return getMaxOffsetInQueue(topic, queueId, true); + } + + public long getMaxOffsetInQueue(String topic, int queueId, boolean committed) { + if (committed) { + ConsumeQueue logic = this.findConsumeQueue(topic, queueId); + if (logic != null) { + return logic.getMaxOffsetInQueue(); + } + } else { + Long offset = this.commitLog.getTopicQueueTable().get(topic + "-" + queueId); + if (offset != null) { + return offset; + } } return 0; @@ -1301,12 +1315,23 @@ public class DefaultMessageStore implements MessageStore { log.info(fileName + (result ? " create OK" : " already exists")); } + public void registerCleanFileHook(CleanFilesHook hook) { + this.cleanFilesHooks.add(hook); + } + private void addScheduleTask() { this.scheduledExecutorService.scheduleAtFixedRate(new Runnable() { @Override public void run() { - DefaultMessageStore.this.cleanFilesPeriodically(); + long deleteCount = DefaultMessageStore.this.cleanFilesPeriodically(); + DefaultMessageStore.this.cleanFilesHooks.forEach(hook -> { + try { + hook.execute(DefaultMessageStore.this, deleteCount); + } catch (Throwable t) { + log.error("execute CleanFilesHook[{}] error", hook.getName(), t); + } + }); } }, 1000 * 60, this.messageStoreConfig.getCleanResourceInterval(), TimeUnit.MILLISECONDS); @@ -1351,9 +1376,11 @@ public class DefaultMessageStore implements MessageStore { }, 1000L, 10000L, TimeUnit.MILLISECONDS); } - private void cleanFilesPeriodically() { - this.cleanCommitLogService.run(); - this.cleanConsumeQueueService.run(); + private long cleanFilesPeriodically() { + long deleteCount = 0L; + deleteCount += this.cleanCommitLogService.run(); + deleteCount += this.cleanConsumeQueueService.run(); + return deleteCount; } private void checkSelf() { @@ -1611,17 +1638,19 @@ public class DefaultMessageStore implements MessageStore { DefaultMessageStore.log.info("executeDeleteFilesManually was invoked"); } - public void run() { + public long run() { + int deleteCount = 0; try { - this.deleteExpiredFiles(); + deleteCount = this.deleteExpiredFiles(); this.redeleteHangedFile(); } catch (Throwable e) { DefaultMessageStore.log.warn(this.getServiceName() + " service has exception. ", e); } + return deleteCount; } - private void deleteExpiredFiles() { + private int deleteExpiredFiles() { int deleteCount = 0; long fileReservedTime = DefaultMessageStore.this.getMessageStoreConfig().getFileReservedTime(); int deletePhysicFilesInterval = DefaultMessageStore.this.getMessageStoreConfig().getDeleteCommitLogFilesInterval(); @@ -1654,6 +1683,7 @@ public class DefaultMessageStore implements MessageStore { log.warn("disk space will be full soon, but delete file failed."); } } + return deleteCount; } private void redeleteHangedFile() { @@ -1775,17 +1805,20 @@ public class DefaultMessageStore implements MessageStore { class CleanConsumeQueueService { private long lastPhysicalMinOffset = 0; - public void run() { + public long run() { + long deleteCount = 0; try { - this.deleteExpiredFiles(); + deleteCount = this.deleteExpiredFiles(); } catch (Throwable e) { DefaultMessageStore.log.warn(this.getServiceName() + " service has exception. ", e); } + return deleteCount; } - private void deleteExpiredFiles() { + private long deleteExpiredFiles() { int deleteLogicsFilesInterval = DefaultMessageStore.this.getMessageStoreConfig().getDeleteConsumeQueueFilesInterval(); + long deleteCountSum = 0L; long minOffset = DefaultMessageStore.this.commitLog.getMinOffset(); if (minOffset > this.lastPhysicalMinOffset) { this.lastPhysicalMinOffset = minOffset; @@ -1795,7 +1828,7 @@ public class DefaultMessageStore implements MessageStore { for (ConcurrentMap maps : tables.values()) { for (ConsumeQueue logic : maps.values()) { int deleteCount = logic.deleteExpiredFile(minOffset); - + deleteCountSum += deleteCount; if (deleteCount > 0 && deleteLogicsFilesInterval > 0) { try { Thread.sleep(deleteLogicsFilesInterval); @@ -1807,6 +1840,7 @@ public class DefaultMessageStore implements MessageStore { DefaultMessageStore.this.indexService.deleteExpiredFile(minOffset); } + return deleteCountSum; } public String getServiceName() { 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 cc145921cef90239ca13dc5ff9b3714e2f4a27c2..ac7894d2dc285f7889cc8fcd3513699e8c448fd2 100644 --- a/store/src/main/java/org/apache/rocketmq/store/MappedFileQueue.java +++ b/store/src/main/java/org/apache/rocketmq/store/MappedFileQueue.java @@ -56,8 +56,8 @@ public class MappedFileQueue { } public void checkSelf() { - - if (!this.mappedFiles.isEmpty()) { + List mappedFiles = new ArrayList<>(this.mappedFiles); + if (!mappedFiles.isEmpty()) { Iterator iterator = mappedFiles.iterator(); MappedFile pre = null; while (iterator.hasNext()) { @@ -238,21 +238,8 @@ public class MappedFileQueue { } public MappedFile getLastMappedFile() { - MappedFile mappedFileLast = null; - - while (!this.mappedFiles.isEmpty()) { - try { - mappedFileLast = this.mappedFiles.get(this.mappedFiles.size() - 1); - break; - } catch (IndexOutOfBoundsException e) { - //continue; - } catch (Exception e) { - log.error("getLastMappedFile has exception.", e); - break; - } - } - - return mappedFileLast; + MappedFile[] mappedFiles = this.mappedFiles.toArray(new MappedFile[0]); + return mappedFiles.length == 0 ? null : mappedFiles[mappedFiles.length - 1]; } public boolean resetOffset(long offset) { @@ -336,7 +323,11 @@ public class MappedFileQueue { public int deleteExpiredFileByTime(final long expiredTime, final int deleteFilesInterval, final long intervalForcibly, - final boolean cleanImmediately) { + final boolean cleanImmediately, + int deleteFileBatchMax) { + if (deleteFileBatchMax == 0) { + deleteFileBatchMax = DELETE_FILES_BATCH_MAX; + } Object[] mfs = this.copyMappedFiles(0); if (null == mfs) @@ -354,7 +345,7 @@ public class MappedFileQueue { files.add(mappedFile); deleteCount++; - if (files.size() >= DELETE_FILES_BATCH_MAX) { + if (files.size() >= deleteFileBatchMax) { break; } 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 0cea607677d4200e77885710168018357c7608f0..74fa4f45286bec345fd16d470eb648f2a0ce4176 100644 --- a/store/src/main/java/org/apache/rocketmq/store/MessageStore.java +++ b/store/src/main/java/org/apache/rocketmq/store/MessageStore.java @@ -115,6 +115,16 @@ public interface MessageStore { */ long getMaxOffsetInQueue(final String topic, final int queueId); + /** + * Get maximum offset of the topic queue. + * + * @param topic Topic name. + * @param queueId Queue ID. + * @param committed If only count committed + * @return Maximum offset at present. + */ + long getMaxOffsetInQueue(final String topic, final int queueId, final boolean committed); + /** * Get the minimum offset of the topic queue. * diff --git a/store/src/test/java/org/apache/rocketmq/store/MappedFileQueueTest.java b/store/src/test/java/org/apache/rocketmq/store/MappedFileQueueTest.java index 8f76051d1f849bd382ad998af91dee6a07ace65f..0a736f92799613f671eb5712340c4b1533beb767 100644 --- a/store/src/test/java/org/apache/rocketmq/store/MappedFileQueueTest.java +++ b/store/src/test/java/org/apache/rocketmq/store/MappedFileQueueTest.java @@ -225,7 +225,7 @@ public class MappedFileQueueTest { mappedFile.getFile().setLastModified(System.currentTimeMillis() - expiredTime * 2); } } - mappedFileQueue.deleteExpiredFileByTime(expiredTime, 0, 0, false); + mappedFileQueue.deleteExpiredFileByTime(expiredTime, 0, 0, false, Integer.MAX_VALUE); assertThat(mappedFileQueue.getMappedFiles().size()).isEqualTo(45); }