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 e83bea23975..5eb916922f6 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.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.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.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.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 boolean initialize() throws CloneNotSupportedException { 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 void run() { }, 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 @@ private void initialRpcHooks() { } } + 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 void run() { } 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 synchronized void registerBrokerAll(final boolean checkOrderConfig, boole 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 00000000000..a6728c828b0 --- /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 252201a058e..4d33663bb90 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.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.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 TopicRouteData getTopicRouteInfoFromNameServer(final String topic, final 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 1db019bec81..db1b6267b09 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 long getMaxOffsetInQueue(String topic, int queueId) { 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 ced7c2014f4..4daa832dd0a 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.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 void executeSendMessageHookAfter(final RemotingCommand response, final Se 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 0a1d214b87f..b3309e1e2e9 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.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.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.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.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.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 RemotingCommand processRequest(ChannelHandlerContext ctx, 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 @@ private synchronized RemotingCommand deleteTopic(ChannelHandlerContext ctx, 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 @@ private RemotingCommand getMaxOffset(ChannelHandlerContext ctx, 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 @@ private MessageExtBrokerInner toMessageExtBrokerInner(MessageExt msgExt) { 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 8879a722fcf..e61ef11309e 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.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.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.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.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 @@ private RemotingCommand processRequest(final Channel channel, RemotingCommand re 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 @@ private RemotingCommand processRequest(final Channel channel, RemotingCommand re 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 @@ private RemotingCommand processRequest(final Channel channel, RemotingCommand re 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 @@ private RemotingCommand processRequest(final Channel channel, RemotingCommand re 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 @@ private RemotingCommand processRequest(final Channel channel, RemotingCommand re } 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 @@ private RemotingCommand processRequest(final Channel channel, RemotingCommand re 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 @@ private RemotingCommand processRequest(final Channel channel, RemotingCommand re // 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 @@ private RemotingCommand processRequest(final Channel channel, RemotingCommand re 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 @@ private RemotingCommand processRequest(final Channel channel, RemotingCommand re 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 void operationComplete(ChannelFuture future) throws Exception { 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 void operationComplete(ChannelFuture future) throws Exception { 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 void operationComplete(ChannelFuture future) throws Exception { 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 void operationComplete(ChannelFuture future) throws Exception { && 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 724cf54c813..97b7e62850d 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 @@ private CompletableFuture asyncSendMessage(ChannelHandlerContex 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 @@ private CompletableFuture asyncSendMessage(ChannelHandlerContex 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 @@ private boolean handleRetryAndDLQ(SendMessageRequestHeader requestHeader, Remoti 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 @@ private CompletableFuture asyncSendBatchMessage(ChannelHandlerC 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 1f2bb4d8ba4..bf690057c6c 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.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 void updateTopicConfig(final TopicConfig topicConfig) { this.dataVersion.nextVersion(); - this.persist(); + this.persist(topicConfig.getTopicName(), topicConfig); } public void updateOrderTopicConfig(final KVTable orderKVTableFromNs) { @@ -421,6 +441,8 @@ public void deleteTopicConfig(final String topic) { 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 void decode(String jsonString) { 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 DataVersion getDataVersion() { 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 9ea1eeee31b..339ed110c85 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.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.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 RemotingCommand answer(InvocationOnMock invocation) throws Throwable { 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 2764761d39f..d6cc8f9bd8a 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.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.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 void testProcessRequest_success() throws RemotingCommandException, Unknow 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 void testProcessRequest_fail() throws RemotingCommandException, UnknownHo 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 void testDeleteTopic() throws Exception { 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 c96f708e854..b6377553d7a 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.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.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.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 void init() { consumerData.getConsumeFromWhere(), consumerData.getSubscriptionDataSet(), false); + brokerController.getTopicConfigManager().updateTopicConfig(new TopicConfig(topic)); } @Test @@ -192,6 +205,94 @@ public void testProcessRequest_NoMsgInQueue() throws RemotingCommandException { 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 b9344e90ed6..e768469f7aa 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 @@ 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.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.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 void init() { 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 Object answer(InvocationOnMock invocation) throws Throwable { 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 00000000000..f504965fada --- /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/distribution/conf/logback_tools.xml b/distribution/conf/logback_tools.xml index 28283ad1d1d..d4bd5c27714 100644 --- a/distribution/conf/logback_tools.xml +++ b/distribution/conf/logback_tools.xml @@ -66,6 +66,11 @@ + + + + + diff --git a/namesrv/pom.xml b/namesrv/pom.xml index 01ef5b60fe9..42da143f03a 100644 --- a/namesrv/pom.xml +++ b/namesrv/pom.xml @@ -48,5 +48,10 @@ org.slf4j slf4j-api + + com.google.guava + guava + test + diff --git a/namesrv/src/main/java/org/apache/rocketmq/namesrv/processor/ClusterTestRequestProcessor.java b/namesrv/src/main/java/org/apache/rocketmq/namesrv/processor/ClusterTestRequestProcessor.java index a58a3b97fc2..dd152888aa7 100644 --- a/namesrv/src/main/java/org/apache/rocketmq/namesrv/processor/ClusterTestRequestProcessor.java +++ b/namesrv/src/main/java/org/apache/rocketmq/namesrv/processor/ClusterTestRequestProcessor.java @@ -56,7 +56,7 @@ public RemotingCommand getRouteInfoByTopic(ChannelHandlerContext ctx, final GetRouteInfoRequestHeader requestHeader = (GetRouteInfoRequestHeader) request.decodeCommandCustomHeader(GetRouteInfoRequestHeader.class); - TopicRouteData topicRouteData = this.namesrvController.getRouteInfoManager().pickupTopicRouteData(requestHeader.getTopic()); + TopicRouteData topicRouteData = this.namesrvController.getRouteInfoManager().pickupTopicRouteData(requestHeader.getTopic(), false); if (topicRouteData != null) { String orderTopicConf = this.namesrvController.getKvConfigManager().getKVConfig(NamesrvUtil.NAMESPACE_ORDER_TOPIC_CONFIG, diff --git a/namesrv/src/main/java/org/apache/rocketmq/namesrv/processor/DefaultRequestProcessor.java b/namesrv/src/main/java/org/apache/rocketmq/namesrv/processor/DefaultRequestProcessor.java index f8bc55e7aab..11cfcd2dcbe 100644 --- a/namesrv/src/main/java/org/apache/rocketmq/namesrv/processor/DefaultRequestProcessor.java +++ b/namesrv/src/main/java/org/apache/rocketmq/namesrv/processor/DefaultRequestProcessor.java @@ -19,6 +19,7 @@ import io.netty.channel.ChannelHandlerContext; import java.io.UnsupportedEncodingException; import java.util.Properties; +import java.util.Set; import java.util.concurrent.atomic.AtomicLong; import org.apache.rocketmq.common.DataVersion; import org.apache.rocketmq.common.MQVersion; @@ -27,8 +28,6 @@ import org.apache.rocketmq.common.UtilAll; import org.apache.rocketmq.common.constant.LoggerName; import org.apache.rocketmq.common.help.FAQUrl; -import org.apache.rocketmq.logging.InternalLogger; -import org.apache.rocketmq.logging.InternalLoggerFactory; import org.apache.rocketmq.common.namesrv.NamesrvUtil; import org.apache.rocketmq.common.namesrv.RegisterBrokerResult; import org.apache.rocketmq.common.protocol.RequestCode; @@ -50,7 +49,11 @@ import org.apache.rocketmq.common.protocol.header.namesrv.UnRegisterBrokerRequestHeader; import org.apache.rocketmq.common.protocol.header.namesrv.WipeWritePermOfBrokerRequestHeader; import org.apache.rocketmq.common.protocol.header.namesrv.WipeWritePermOfBrokerResponseHeader; -import org.apache.rocketmq.common.protocol.route.TopicRouteData; +import org.apache.rocketmq.common.protocol.route.LogicalQueuesInfoUnordered; +import org.apache.rocketmq.common.protocol.route.TopicRouteDataNameSrv; +import org.apache.rocketmq.common.sysflag.MessageSysFlag; +import org.apache.rocketmq.logging.InternalLogger; +import org.apache.rocketmq.logging.InternalLoggerFactory; import org.apache.rocketmq.namesrv.NamesrvController; import org.apache.rocketmq.remoting.common.RemotingHelper; import org.apache.rocketmq.remoting.exception.RemotingCommandException; @@ -346,7 +349,9 @@ public RemotingCommand getRouteInfoByTopic(ChannelHandlerContext ctx, final GetRouteInfoRequestHeader requestHeader = (GetRouteInfoRequestHeader) request.decodeCommandCustomHeader(GetRouteInfoRequestHeader.class); - TopicRouteData topicRouteData = this.namesrvController.getRouteInfoManager().pickupTopicRouteData(requestHeader.getTopic()); + boolean includeLogicalQueuesInfo = (requestHeader.getSysFlag() & MessageSysFlag.LOGICAL_QUEUE_FLAG) > 0; + + TopicRouteDataNameSrv topicRouteData = this.namesrvController.getRouteInfoManager().pickupTopicRouteData(requestHeader.getTopic(), includeLogicalQueuesInfo); if (topicRouteData != null) { if (this.namesrvController.getNamesrvConfig().isOrderMessageEnable()) { @@ -356,6 +361,16 @@ public RemotingCommand getRouteInfoByTopic(ChannelHandlerContext ctx, topicRouteData.setOrderTopicConf(orderTopicConf); } + Set logicalQueueIdsFilter = requestHeader.getLogicalQueueIdsFilter(); + if (logicalQueueIdsFilter != null) { + LogicalQueuesInfoUnordered logicalQueuesInfo = topicRouteData.getLogicalQueuesInfoUnordered(); + if (logicalQueuesInfo != null) { + LogicalQueuesInfoUnordered filtered = new LogicalQueuesInfoUnordered(logicalQueueIdsFilter.size()); + logicalQueueIdsFilter.forEach(integer -> filtered.put(integer, logicalQueuesInfo.get(integer))); + topicRouteData.setLogicalQueuesInfoUnordered(filtered); + } + } + byte[] content = topicRouteData.encode(); response.setBody(content); response.setCode(ResponseCode.SUCCESS); diff --git a/namesrv/src/main/java/org/apache/rocketmq/namesrv/routeinfo/RouteInfoManager.java b/namesrv/src/main/java/org/apache/rocketmq/namesrv/routeinfo/RouteInfoManager.java index edef87ce2d7..476e92bc338 100644 --- a/namesrv/src/main/java/org/apache/rocketmq/namesrv/routeinfo/RouteInfoManager.java +++ b/namesrv/src/main/java/org/apache/rocketmq/namesrv/routeinfo/RouteInfoManager.java @@ -17,6 +17,7 @@ package org.apache.rocketmq.namesrv.routeinfo; import io.netty.channel.Channel; +import java.util.Collection; import java.util.HashMap; import java.util.HashSet; import java.util.Iterator; @@ -24,26 +25,32 @@ 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.locks.ReadWriteLock; import java.util.concurrent.locks.ReentrantReadWriteLock; +import java.util.stream.Collectors; import org.apache.rocketmq.common.DataVersion; import org.apache.rocketmq.common.MixAll; import org.apache.rocketmq.common.TopicConfig; import org.apache.rocketmq.common.constant.LoggerName; import org.apache.rocketmq.common.constant.PermName; -import org.apache.rocketmq.logging.InternalLogger; -import org.apache.rocketmq.logging.InternalLoggerFactory; import org.apache.rocketmq.common.namesrv.RegisterBrokerResult; import org.apache.rocketmq.common.protocol.body.ClusterInfo; import org.apache.rocketmq.common.protocol.body.TopicConfigSerializeWrapper; import org.apache.rocketmq.common.protocol.body.TopicList; import org.apache.rocketmq.common.protocol.route.BrokerData; +import org.apache.rocketmq.common.protocol.route.LogicalQueuesInfo; +import org.apache.rocketmq.common.protocol.route.LogicalQueuesInfoUnordered; import org.apache.rocketmq.common.protocol.route.QueueData; -import org.apache.rocketmq.common.protocol.route.TopicRouteData; +import org.apache.rocketmq.common.protocol.route.TopicRouteDataNameSrv; import org.apache.rocketmq.common.sysflag.TopicSysFlag; +import org.apache.rocketmq.logging.InternalLogger; +import org.apache.rocketmq.logging.InternalLoggerFactory; import org.apache.rocketmq.remoting.common.RemotingUtil; +import org.apache.rocketmq.srvutil.ConcurrentHashMapUtil; public class RouteInfoManager { private static final InternalLogger log = InternalLoggerFactory.getLogger(LoggerName.NAMESRV_LOGGER_NAME); @@ -54,6 +61,7 @@ public class RouteInfoManager { private final HashMap> clusterAddrTable; private final HashMap brokerLiveTable; private final HashMap/* Filter Server */> filterServerTable; + private final ConcurrentMap logicalQueuesInfoTable; public RouteInfoManager() { this.topicQueueTable = new HashMap>(1024); @@ -61,6 +69,7 @@ public RouteInfoManager() { this.clusterAddrTable = new HashMap>(32); this.brokerLiveTable = new HashMap(256); this.filterServerTable = new HashMap>(256); + this.logicalQueuesInfoTable = new ConcurrentHashMap<>(1024); } public byte[] getAllClusterInfo() { @@ -148,18 +157,28 @@ public RegisterBrokerResult registerBroker( || registerFirst) { ConcurrentMap tcTable = topicConfigWrapper.getTopicConfigTable(); + Map logicalQueuesInfoMap = topicConfigWrapper.getLogicalQueuesInfoMap(); if (tcTable != null) { for (Map.Entry entry : tcTable.entrySet()) { this.createAndUpdateQueueData(brokerName, entry.getValue()); } } + if (logicalQueuesInfoMap != null) { + long startTime = System.nanoTime(); + for (Map.Entry entry : logicalQueuesInfoMap.entrySet()) { + String topicName = entry.getKey(); + LogicalQueuesInfoUnordered logicalQueuesInfo = ConcurrentHashMapUtil.computeIfAbsent(this.logicalQueuesInfoTable, topicName, ignore -> new LogicalQueuesInfoUnordered()); + mergeLogicalQueuesInfo(brokerName, topicName, logicalQueuesInfo, entry.getValue()); + } + log.debug("mergeQueueRouteDataTable topic={} time={}ns", System.nanoTime() - startTime); + } } } BrokerLiveInfo prevBrokerLiveInfo = this.brokerLiveTable.put(brokerAddr, new BrokerLiveInfo( System.currentTimeMillis(), - topicConfigWrapper.getDataVersion(), + topicConfigWrapper != null ? topicConfigWrapper.getDataVersion() : new DataVersion(), channel, haServerAddr)); if (null == prevBrokerLiveInfo) { @@ -371,8 +390,12 @@ private void removeTopicByBrokerName(final String brokerName) { } } - public TopicRouteData pickupTopicRouteData(final String topic) { - TopicRouteData topicRouteData = new TopicRouteData(); + public TopicRouteDataNameSrv pickupTopicRouteData(final String topic) { + return pickupTopicRouteData(topic, false); + } + + public TopicRouteDataNameSrv pickupTopicRouteData(final String topic, boolean includeLogicalQueuesInfo) { + TopicRouteDataNameSrv topicRouteData = new TopicRouteDataNameSrv(); boolean foundQueueData = false; boolean foundBrokerData = false; Set brokerNameSet = new HashSet(); @@ -420,6 +443,10 @@ public TopicRouteData pickupTopicRouteData(final String topic) { log.debug("pickupTopicRouteData {} {}", topic, topicRouteData); if (foundBrokerData && foundQueueData) { + if (includeLogicalQueuesInfo) { + topicRouteData.setLogicalQueuesInfoUnordered(logicalQueuesInfoTable.get(topic)); + } + return topicRouteData; } @@ -750,6 +777,34 @@ public byte[] getHasUnitSubUnUnitTopicList() { return topicList.encode(); } + + private static void mergeLogicalQueuesInfo(String brokerName, String topicName, + LogicalQueuesInfoUnordered logicalQueuesInfoInNamesrv, + LogicalQueuesInfo logicalQueuesInfoFromBroker) { + Set newKeys = logicalQueuesInfoFromBroker.values() + .stream() + .flatMap(Collection::stream) + .filter(v -> Objects.equals(brokerName, v.getBrokerName())) + .map(v -> new LogicalQueuesInfoUnordered.Key(null, v.getQueueId(), v.getOffsetDelta())) + .collect(Collectors.toSet()); + logicalQueuesInfoInNamesrv.values().forEach(m -> + m.values().removeIf(queueRouteData -> + Objects.equals(brokerName, queueRouteData.getBrokerName()) && + !newKeys.contains(new LogicalQueuesInfoUnordered.Key(null, queueRouteData.getQueueId(), queueRouteData.getOffsetDelta())))); + logicalQueuesInfoFromBroker.forEach((logicalQueueId, queueRouteDataListFromBroker) -> { + if (logicalQueueId == null) { + log.warn("queueRouteDataTable topic {} contains null logicalQueueId: {}", topicName, logicalQueuesInfoFromBroker); + return; + } + queueRouteDataListFromBroker.stream() + .filter(queueRouteDataFromBroker -> Objects.equals(brokerName, queueRouteDataFromBroker.getBrokerName())) + .forEach(queueRouteDataFromBroker -> + ConcurrentHashMapUtil.computeIfAbsent(logicalQueuesInfoInNamesrv, logicalQueueId, ignored -> new ConcurrentHashMap<>(queueRouteDataListFromBroker.size())) + .put(new LogicalQueuesInfoUnordered.Key(brokerName, queueRouteDataFromBroker.getQueueId(), queueRouteDataFromBroker.getOffsetDelta()), + queueRouteDataFromBroker) + ); + }); + } } class BrokerLiveInfo { diff --git a/namesrv/src/test/java/org/apache/rocketmq/namesrv/processor/DefaultRequestProcessorTest.java b/namesrv/src/test/java/org/apache/rocketmq/namesrv/processor/DefaultRequestProcessorTest.java index d4a2f66f99f..ff477a3e012 100644 --- a/namesrv/src/test/java/org/apache/rocketmq/namesrv/processor/DefaultRequestProcessorTest.java +++ b/namesrv/src/test/java/org/apache/rocketmq/namesrv/processor/DefaultRequestProcessorTest.java @@ -16,16 +16,22 @@ */ package org.apache.rocketmq.namesrv.processor; +import com.alibaba.fastjson.JSON; +import com.google.common.collect.ImmutableMap; +import com.google.common.collect.Lists; import io.netty.channel.Channel; import io.netty.channel.ChannelHandlerContext; import java.lang.reflect.Field; import java.lang.reflect.Modifier; import java.util.ArrayList; +import java.util.Collections; import java.util.HashMap; import java.util.Map; import java.util.concurrent.ConcurrentHashMap; +import org.apache.rocketmq.common.DataVersion; +import org.apache.rocketmq.common.MixAll; import org.apache.rocketmq.common.TopicConfig; -import org.apache.rocketmq.logging.InternalLogger; +import org.apache.rocketmq.common.message.MessageQueue; import org.apache.rocketmq.common.namesrv.NamesrvConfig; import org.apache.rocketmq.common.namesrv.RegisterBrokerResult; import org.apache.rocketmq.common.protocol.RequestCode; @@ -34,14 +40,23 @@ import org.apache.rocketmq.common.protocol.header.namesrv.DeleteKVConfigRequestHeader; import org.apache.rocketmq.common.protocol.header.namesrv.GetKVConfigRequestHeader; import org.apache.rocketmq.common.protocol.header.namesrv.GetKVConfigResponseHeader; +import org.apache.rocketmq.common.protocol.header.namesrv.GetRouteInfoRequestHeader; import org.apache.rocketmq.common.protocol.header.namesrv.PutKVConfigRequestHeader; import org.apache.rocketmq.common.protocol.header.namesrv.RegisterBrokerRequestHeader; import org.apache.rocketmq.common.protocol.route.BrokerData; +import org.apache.rocketmq.common.protocol.route.LogicalQueueRouteData; +import org.apache.rocketmq.common.protocol.route.LogicalQueuesInfo; +import org.apache.rocketmq.common.protocol.route.LogicalQueuesInfoUnordered; +import org.apache.rocketmq.common.protocol.route.MessageQueueRouteState; +import org.apache.rocketmq.common.protocol.route.TopicRouteDataNameSrv; +import org.apache.rocketmq.common.sysflag.MessageSysFlag; +import org.apache.rocketmq.logging.InternalLogger; import org.apache.rocketmq.namesrv.NamesrvController; import org.apache.rocketmq.namesrv.routeinfo.RouteInfoManager; import org.apache.rocketmq.remoting.exception.RemotingCommandException; import org.apache.rocketmq.remoting.netty.NettyServerConfig; import org.apache.rocketmq.remoting.protocol.RemotingCommand; +import org.apache.rocketmq.remoting.protocol.RemotingSerializable; import org.assertj.core.util.Maps; import org.junit.Before; import org.junit.Test; @@ -184,6 +199,98 @@ public void testProcessRequest_RegisterBroker() throws RemotingCommandException, .contains(new HashMap.SimpleEntry("broker", broker)); } + @Test + public void testProcessRequest_RegisterBrokerLogicalQueue() throws Exception { + String cluster = "cluster"; + String broker1Name = "broker1"; + String broker1Addr = "10.10.1.1"; + String broker2Name = "broker2"; + String broker2Addr = "10.10.1.2"; + String topic = "foobar"; + + LogicalQueueRouteData queueRouteData1 = new LogicalQueueRouteData(0, 0, new MessageQueue(topic, broker1Name, 0), MessageQueueRouteState.ReadOnly, 0, 10, 100, 100, broker1Addr); + { + RegisterBrokerRequestHeader header = new RegisterBrokerRequestHeader(); + header.setBrokerName(broker1Name); + RemotingCommand request = RemotingCommand.createRequestCommand( + RequestCode.REGISTER_BROKER, header); + request.addExtField("brokerName", broker1Name); + request.addExtField("brokerAddr", broker1Addr); + request.addExtField("clusterName", cluster); + request.addExtField("haServerAddr", "10.10.2.1"); + request.addExtField("brokerId", String.valueOf(MixAll.MASTER_ID)); + TopicConfigSerializeWrapper topicConfigSerializeWrapper = new TopicConfigSerializeWrapper(); + topicConfigSerializeWrapper.setTopicConfigTable(new ConcurrentHashMap<>(Collections.singletonMap(topic, new TopicConfig(topic)))); + topicConfigSerializeWrapper.setLogicalQueuesInfoMap(Maps.newHashMap(topic, new LogicalQueuesInfo(Collections.singletonMap(0, Lists.newArrayList( + queueRouteData1 + ))))); + topicConfigSerializeWrapper.setDataVersion(new DataVersion()); + request.setBody(RemotingSerializable.encode(topicConfigSerializeWrapper)); + + ChannelHandlerContext ctx = mock(ChannelHandlerContext.class); + when(ctx.channel()).thenReturn(null); + + RemotingCommand response = defaultRequestProcessor.processRequest(ctx, request); + + assertThat(response.getCode()).isEqualTo(ResponseCode.SUCCESS); + assertThat(response.getRemark()).isNull(); + } + LogicalQueueRouteData queueRouteData2 = new LogicalQueueRouteData(0, 100, new MessageQueue(topic, broker2Name, 0), MessageQueueRouteState.Normal, 0, -1, -1, -1, broker2Addr); + LogicalQueueRouteData queueRouteData3 = new LogicalQueueRouteData(1, 100, new MessageQueue(topic, broker2Name, 0), MessageQueueRouteState.Normal, 0, -1, -1, -1, broker2Addr); + { + RegisterBrokerRequestHeader header = new RegisterBrokerRequestHeader(); + header.setBrokerName(broker2Name); + RemotingCommand request = RemotingCommand.createRequestCommand( + RequestCode.REGISTER_BROKER, header); + request.addExtField("brokerName", broker2Name); + request.addExtField("brokerAddr", broker2Addr); + request.addExtField("clusterName", cluster); + request.addExtField("haServerAddr", "10.10.2.1"); + request.addExtField("brokerId", String.valueOf(MixAll.MASTER_ID)); + TopicConfigSerializeWrapper topicConfigSerializeWrapper = new TopicConfigSerializeWrapper(); + topicConfigSerializeWrapper.setTopicConfigTable(new ConcurrentHashMap<>(Collections.singletonMap(topic, new TopicConfig(topic)))); + topicConfigSerializeWrapper.setLogicalQueuesInfoMap(Maps.newHashMap(topic, new LogicalQueuesInfo(ImmutableMap.of( + 0, Collections.singletonList(queueRouteData2), + 1, Collections.singletonList(queueRouteData3) + )))); + topicConfigSerializeWrapper.setDataVersion(new DataVersion()); + request.setBody(RemotingSerializable.encode(topicConfigSerializeWrapper)); + + ChannelHandlerContext ctx = mock(ChannelHandlerContext.class); + when(ctx.channel()).thenReturn(null); + + RemotingCommand response = defaultRequestProcessor.processRequest(ctx, request); + + assertThat(response.getCode()).isEqualTo(ResponseCode.SUCCESS); + assertThat(response.getRemark()).isNull(); + } + + { + GetRouteInfoRequestHeader header = new GetRouteInfoRequestHeader(); + header.setTopic(topic); + header.setSysFlag(MessageSysFlag.LOGICAL_QUEUE_FLAG); + RemotingCommand request = RemotingCommand.createRequestCommand(RequestCode.GET_ROUTEINFO_BY_TOPIC, header); + request.makeCustomHeaderToNet(); + + ChannelHandlerContext ctx = mock(ChannelHandlerContext.class); + when(ctx.channel()).thenReturn(null); + + RemotingCommand response = defaultRequestProcessor.processRequest(ctx, request); + + assertThat(response.getCode()).isEqualTo(ResponseCode.SUCCESS); + + TopicRouteDataNameSrv topicRouteDataNameSrv = JSON.parseObject(response.getBody(), TopicRouteDataNameSrv.class); + assertThat(topicRouteDataNameSrv).isNotNull(); + LogicalQueuesInfoUnordered logicalQueuesInfoUnordered = new LogicalQueuesInfoUnordered(); + logicalQueuesInfoUnordered.put(0, ImmutableMap.of( + new LogicalQueuesInfoUnordered.Key(queueRouteData1.getBrokerName(), queueRouteData1.getQueueId(), queueRouteData1.getOffsetDelta()), queueRouteData1, + new LogicalQueuesInfoUnordered.Key(queueRouteData2.getBrokerName(), queueRouteData2.getQueueId(), queueRouteData2.getOffsetDelta()), queueRouteData2 + )); + logicalQueuesInfoUnordered.put(1, ImmutableMap.of(new LogicalQueuesInfoUnordered.Key(queueRouteData3.getBrokerName(), queueRouteData3.getQueueId(), queueRouteData3.getOffsetDelta()), queueRouteData3)); + assertThat(topicRouteDataNameSrv.getLogicalQueuesInfoUnordered()).isEqualTo(logicalQueuesInfoUnordered); + } + } + @Test public void testProcessRequest_RegisterBrokerWithFilterServer() throws RemotingCommandException, NoSuchFieldException, IllegalAccessException { 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 00000000000..cc98eb5bcb5 --- /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 00000000000..ef6e22a8f88 --- /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 cce6481b8da..0492aa95e79 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 int deleteExpiredFile( 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 c25a1147328..b11eb49047c 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.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 GetMessageResult getMessage(final String group, final String topic, final } 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 @@ private void createTempFile() throws IOException { 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 void run() { }, 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 void excuteDeleteFilesManualy() { 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 @@ private void deleteExpiredFiles() { log.warn("disk space will be full soon, but delete file failed."); } } + return deleteCount; } private void redeleteHangedFile() { @@ -1775,17 +1805,20 @@ public boolean isSpaceFull() { 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 @@ private void deleteExpiredFiles() { 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 @@ private void deleteExpiredFiles() { 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 cc145921cef..ac7894d2dc2 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 MappedFileQueue(final String storePath, int mappedFileSize, } 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 MappedFile getLastMappedFile(final long startOffset) { } 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 void deleteLastMappedFile() { 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 int deleteExpiredFileByTime(final long expiredTime, 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 0cea607677d..74fa4f45286 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 @@ GetMessageResult getMessage(final String group, final String topic, final int qu */ 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 8f76051d1f8..0a736f92799 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 void testDeleteExpiredFileByTime() throws Exception { 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); } diff --git a/test/src/main/java/org/apache/rocketmq/test/util/MQAdmin.java b/test/src/main/java/org/apache/rocketmq/test/util/MQAdmin.java index 8863ee3e52d..8c0caa6a271 100644 --- a/test/src/main/java/org/apache/rocketmq/test/util/MQAdmin.java +++ b/test/src/main/java/org/apache/rocketmq/test/util/MQAdmin.java @@ -20,6 +20,7 @@ import java.util.HashMap; import java.util.Set; import java.util.UUID; +import java.util.concurrent.ForkJoinPool; import org.apache.log4j.Logger; import org.apache.rocketmq.common.admin.TopicStatsTable; import org.apache.rocketmq.common.protocol.body.ClusterInfo; @@ -60,7 +61,7 @@ public static boolean createTopic(String nameSrvAddr, String clusterName, String } } - mqAdminExt.shutdown(); + ForkJoinPool.commonPool().execute(mqAdminExt::shutdown); return createResult; } @@ -99,7 +100,7 @@ public static boolean createSub(String nameSrvAddr, String clusterName, String c createResult = false; e.printStackTrace(); } - mqAdminExt.shutdown(); + ForkJoinPool.commonPool().execute(mqAdminExt::shutdown); return createResult; } @@ -113,7 +114,7 @@ public static ClusterInfo getCluster(String nameSrvAddr) { } catch (Exception e) { e.printStackTrace(); } - mqAdminExt.shutdown(); + ForkJoinPool.commonPool().execute(mqAdminExt::shutdown); return clusterInfo; } @@ -159,7 +160,7 @@ public void getSubConnection(String nameSrvAddr, String clusterName, String cons createResult = false; e.printStackTrace(); } - mqAdminExt.shutdown(); + ForkJoinPool.commonPool().execute(mqAdminExt::shutdown); } } diff --git a/test/src/test/java/org/apache/rocketmq/test/base/BaseConf.java b/test/src/test/java/org/apache/rocketmq/test/base/BaseConf.java index c6a835fd67f..79469e1f3b0 100644 --- a/test/src/test/java/org/apache/rocketmq/test/base/BaseConf.java +++ b/test/src/test/java/org/apache/rocketmq/test/base/BaseConf.java @@ -17,13 +17,23 @@ package org.apache.rocketmq.test.base; +import com.google.common.collect.ImmutableList; import java.util.ArrayList; import java.util.List; - +import java.util.Map; +import java.util.concurrent.ForkJoinPool; +import java.util.concurrent.TimeUnit; +import java.util.function.Function; +import java.util.stream.Collectors; import org.apache.log4j.Logger; import org.apache.rocketmq.broker.BrokerController; +import org.apache.rocketmq.client.consumer.MQPullConsumer; +import org.apache.rocketmq.client.consumer.MQPushConsumer; +import org.apache.rocketmq.client.exception.MQClientException; +import org.apache.rocketmq.client.producer.MQProducer; import org.apache.rocketmq.client.producer.TransactionListener; import org.apache.rocketmq.common.MQVersion; +import org.apache.rocketmq.common.protocol.route.BrokerData; import org.apache.rocketmq.namesrv.NamesrvController; import org.apache.rocketmq.remoting.protocol.RemotingCommand; import org.apache.rocketmq.test.client.rmq.RMQAsyncSendProducer; @@ -36,21 +46,28 @@ import org.apache.rocketmq.test.listener.AbstractListener; import org.apache.rocketmq.test.util.MQAdmin; import org.apache.rocketmq.test.util.MQRandomUtils; +import org.apache.rocketmq.tools.admin.DefaultMQAdminExt; +import org.apache.rocketmq.tools.admin.MQAdminExt; + +import static org.awaitility.Awaitility.await; public class BaseConf { - public static String nsAddr; - protected static String broker1Name; - protected static String broker2Name; - protected static String clusterName; - protected static int brokerNum; - protected static int waitTime = 5; - protected static int consumeTime = 2 * 60 * 1000; - protected static NamesrvController namesrvController; - protected static BrokerController brokerController1; - protected static BrokerController brokerController2; - protected static List mqClients = new ArrayList(); - protected static boolean debug = false; - private static Logger log = Logger.getLogger(BaseConf.class); + public final static String nsAddr; + protected final static String broker1Name; + protected final static String broker2Name; + protected final static String clusterName; + protected final static int brokerNum; + protected final static int waitTime = 5; + protected final static int consumeTime = 2 * 60 * 1000; + protected final static int QUEUE_NUMBERS = 8; + protected final static NamesrvController namesrvController; + protected final static BrokerController brokerController1; + protected final static BrokerController brokerController2; + protected final static List brokerControllerList; + protected final static Map brokerControllerMap; + protected final static List mqClients = new ArrayList(); + protected final static boolean debug = false; + private final static Logger log = Logger.getLogger(BaseConf.class); static { System.setProperty(RemotingCommand.REMOTING_VERSION_KEY, Integer.toString(MQVersion.CURRENT_VERSION)); @@ -62,14 +79,32 @@ public class BaseConf { broker1Name = brokerController1.getBrokerConfig().getBrokerName(); broker2Name = brokerController2.getBrokerConfig().getBrokerName(); brokerNum = 2; + brokerControllerList = ImmutableList.of(brokerController1, brokerController2); + brokerControllerMap = brokerControllerList.stream().collect(Collectors.toMap(input -> input.getBrokerConfig().getBrokerName(), Function.identity())); } public BaseConf() { } + // This method can't be placed in the static block of BaseConf, which seems to lead to a strange dead lock. + public static void waitBrokerRegistered(final String nsAddr, final String clusterName) { + final DefaultMQAdminExt mqAdminExt = new DefaultMQAdminExt(500); + mqAdminExt.setNamesrvAddr(nsAddr); + try { + mqAdminExt.start(); + await().atMost(30, TimeUnit.SECONDS).until(() -> { + List brokerDatas = mqAdminExt.examineTopicRouteInfo(clusterName).getBrokerDatas(); + return brokerDatas.size() == brokerNum; + }); + } catch (MQClientException e) { + log.error("init failed, please check BaseConf"); + } + ForkJoinPool.commonPool().execute(mqAdminExt::shutdown); + } + public static String initTopic() { - String topic = MQRandomUtils.getRandomTopic(); + String topic = "tt-" + MQRandomUtils.getRandomTopic(); IntegrationTestBase.initTopic(topic, nsAddr, clusterName); return topic; @@ -157,18 +192,26 @@ public static RMQNormalConsumer getConsumer(String nsAddr, String consumerGroup, } public static void shutdown() { - try { - for (Object mqClient : mqClients) { - if (mqClient instanceof AbstractMQProducer) { - ((AbstractMQProducer) mqClient).shutdown(); + ImmutableList mqClients = ImmutableList.copyOf(BaseConf.mqClients); + BaseConf.mqClients.clear(); + shutdown(mqClients); + } - } else { - ((AbstractMQConsumer) mqClient).shutdown(); - } + public static void shutdown(List mqClients) { + mqClients.forEach(mqClient -> ForkJoinPool.commonPool().execute(() -> { + if (mqClient instanceof AbstractMQProducer) { + ((AbstractMQProducer) mqClient).shutdown(); + } else if (mqClient instanceof AbstractMQConsumer) { + ((AbstractMQConsumer) mqClient).shutdown(); + } else if (mqClient instanceof MQAdminExt) { + ((MQAdminExt) mqClient).shutdown(); + } else if (mqClient instanceof MQProducer) { + ((MQProducer) mqClient).shutdown(); + } else if (mqClient instanceof MQPullConsumer) { + ((MQPullConsumer) mqClient).shutdown(); + } else if (mqClient instanceof MQPushConsumer) { + ((MQPushConsumer) mqClient).shutdown(); } - } catch (Exception e) { - e.printStackTrace(); - } - + })); } } diff --git a/test/src/test/java/org/apache/rocketmq/test/base/IntegrationTestBase.java b/test/src/test/java/org/apache/rocketmq/test/base/IntegrationTestBase.java index 82420105e07..50dc8fc64ab 100644 --- a/test/src/test/java/org/apache/rocketmq/test/base/IntegrationTestBase.java +++ b/test/src/test/java/org/apache/rocketmq/test/base/IntegrationTestBase.java @@ -47,7 +47,7 @@ public class IntegrationTestBase { protected static final List BROKER_CONTROLLERS = new ArrayList<>(); protected static final List NAMESRV_CONTROLLERS = new ArrayList<>(); protected static int topicCreateTime = 30 * 1000; - protected static final int COMMIT_LOG_SIZE = 1024 * 1024 * 100; + public static volatile int COMMIT_LOG_SIZE = 1024 * 1024 * 100; protected static final int INDEX_NUM = 1000; private static final AtomicInteger port = new AtomicInteger(40000); @@ -183,7 +183,7 @@ public static boolean initTopic(String topic, String nsAddr, String clusterName, } public static boolean initTopic(String topic, String nsAddr, String clusterName) { - return initTopic(topic, nsAddr, clusterName, 8); + return initTopic(topic, nsAddr, clusterName, BaseConf.QUEUE_NUMBERS); } public static void deleteFile(File file) { diff --git a/test/src/test/java/org/apache/rocketmq/test/smoke/LogicalQueueIT.java b/test/src/test/java/org/apache/rocketmq/test/smoke/LogicalQueueIT.java new file mode 100644 index 00000000000..b1e9013f674 --- /dev/null +++ b/test/src/test/java/org/apache/rocketmq/test/smoke/LogicalQueueIT.java @@ -0,0 +1,1170 @@ +/* + * 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.test.smoke; + +import com.google.common.collect.ImmutableList; +import com.google.common.collect.Maps; +import java.nio.charset.StandardCharsets; +import java.util.ArrayList; +import java.util.Arrays; +import java.util.Collections; +import java.util.Comparator; +import java.util.Iterator; +import java.util.LinkedList; +import java.util.List; +import java.util.Locale; +import java.util.Map; +import java.util.Queue; +import java.util.Set; +import java.util.concurrent.CompletableFuture; +import java.util.concurrent.ForkJoinPool; +import java.util.concurrent.TimeUnit; +import java.util.concurrent.atomic.AtomicInteger; +import java.util.function.Function; +import java.util.stream.Collectors; +import java.util.stream.IntStream; +import org.apache.commons.lang3.RandomStringUtils; +import org.apache.commons.lang3.reflect.FieldUtils; +import org.apache.rocketmq.broker.BrokerController; +import org.apache.rocketmq.client.consumer.DefaultMQPullConsumer; +import org.apache.rocketmq.client.consumer.PullCallback; +import org.apache.rocketmq.client.consumer.PullResult; +import org.apache.rocketmq.client.consumer.PullStatus; +import org.apache.rocketmq.client.exception.MQBrokerException; +import org.apache.rocketmq.client.exception.MQClientException; +import org.apache.rocketmq.client.producer.DefaultMQProducer; +import org.apache.rocketmq.client.producer.SendCallback; +import org.apache.rocketmq.client.producer.SendResult; +import org.apache.rocketmq.client.producer.SendResultForLogicalQueue; +import org.apache.rocketmq.common.MQVersion; +import org.apache.rocketmq.common.MixAll; +import org.apache.rocketmq.common.TopicConfig; +import org.apache.rocketmq.common.constant.PermName; +import org.apache.rocketmq.common.message.Message; +import org.apache.rocketmq.common.message.MessageExt; +import org.apache.rocketmq.common.message.MessageQueue; +import org.apache.rocketmq.common.protocol.ResponseCode; +import org.apache.rocketmq.common.protocol.route.LogicalQueueRouteData; +import org.apache.rocketmq.common.protocol.route.LogicalQueuesInfo; +import org.apache.rocketmq.common.protocol.route.MessageQueueRouteState; +import org.apache.rocketmq.common.protocol.route.TopicRouteData; +import org.apache.rocketmq.namesrv.NamesrvController; +import org.apache.rocketmq.remoting.exception.RemotingConnectException; +import org.apache.rocketmq.remoting.exception.RemotingException; +import org.apache.rocketmq.remoting.protocol.RemotingCommand; +import org.apache.rocketmq.store.CommitLog; +import org.apache.rocketmq.store.DefaultMessageStore; +import org.apache.rocketmq.store.MappedFileQueue; +import org.apache.rocketmq.test.base.BaseConf; +import org.apache.rocketmq.test.base.IntegrationTestBase; +import org.apache.rocketmq.test.util.MQRandomUtils; +import org.apache.rocketmq.tools.admin.DefaultMQAdminExt; +import org.apache.rocketmq.tools.admin.DefaultMQAdminExtImpl; +import org.apache.rocketmq.tools.command.logicalqueue.MigrateTopicLogicalQueueCommand; +import org.apache.rocketmq.tools.command.logicalqueue.UpdateTopicLogicalQueueMappingCommand; +import org.apache.rocketmq.tools.command.logicalqueue.UpdateTopicLogicalQueueNumCommand; +import org.junit.AfterClass; +import org.junit.Assert; +import org.junit.Before; +import org.junit.BeforeClass; +import org.junit.FixMethodOrder; +import org.junit.Test; +import org.junit.runners.MethodSorters; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +import static java.util.Optional.ofNullable; +import static org.assertj.core.api.Assertions.assertThat; +import static org.assertj.core.api.Assertions.assertThatThrownBy; +import static org.awaitility.Awaitility.await; +import static org.awaitility.Awaitility.waitAtMost; + +@FixMethodOrder(MethodSorters.NAME_ASCENDING) +public class LogicalQueueIT { + private static final Logger logger = LoggerFactory.getLogger(LogicalQueueIT.class); + public static String nsAddr; + private static String broker1Name; + private static String broker2Name; + private static String clusterName; + private static int brokerNum; + private final static int QUEUE_NUMBERS = 8; + private static NamesrvController namesrvController; + private static BrokerController brokerController1; + private static BrokerController brokerController2; + private static Map brokerControllerMap; + private final static List mqClients = new ArrayList<>(); + + private static DefaultMQProducer producer; + private static DefaultMQPullConsumer consumer; + private static DefaultMQAdminExt mqAdminExt; + private static volatile String topic = null; + private static final String placeholderTopic = "placeholder"; + private static final int MSG_SENT_TIMES = 3; + private static final int COMMIT_LOG_FILE_SIZE = 512 * 1024; + + @BeforeClass + public static void beforeClass() throws Exception { + System.setProperty(RemotingCommand.REMOTING_VERSION_KEY, Integer.toString(MQVersion.CURRENT_VERSION)); + namesrvController = IntegrationTestBase.createAndStartNamesrv(); + nsAddr = "127.0.0.1:" + namesrvController.getNettyServerConfig().getListenPort(); + + int oldCommitLogSize = IntegrationTestBase.COMMIT_LOG_SIZE; + IntegrationTestBase.COMMIT_LOG_SIZE = COMMIT_LOG_FILE_SIZE; + brokerController1 = IntegrationTestBase.createAndStartBroker(nsAddr); + brokerController2 = IntegrationTestBase.createAndStartBroker(nsAddr); + IntegrationTestBase.COMMIT_LOG_SIZE = oldCommitLogSize; + + clusterName = brokerController1.getBrokerConfig().getBrokerClusterName(); + broker1Name = brokerController1.getBrokerConfig().getBrokerName(); + broker2Name = brokerController2.getBrokerConfig().getBrokerName(); + brokerNum = 2; + brokerControllerMap = ImmutableList.of(brokerController1, brokerController2).stream().collect(Collectors.toMap(input -> input.getBrokerConfig().getBrokerName(), Function.identity())); + + BaseConf.waitBrokerRegistered(nsAddr, clusterName); + + producer = new DefaultMQProducer(MQRandomUtils.getRandomConsumerGroup()); + mqClients.add(producer); + producer.setNamesrvAddr(nsAddr); + producer.setCompressMsgBodyOverHowmuch(Integer.MAX_VALUE); + producer.setSendMsgTimeout(1000); + producer.start(); + + consumer = new DefaultMQPullConsumer(BaseConf.initConsumerGroup()); + mqClients.add(consumer); + consumer.setNamesrvAddr(nsAddr); + consumer.setConsumerPullTimeoutMillis(1000); + consumer.start(); + + mqAdminExt = new DefaultMQAdminExt(1000); + mqClients.add(mqAdminExt); + mqAdminExt.setNamesrvAddr(nsAddr); + mqAdminExt.start(); + + mqAdminExt.createTopic(clusterName, placeholderTopic, 1); + } + + @AfterClass + public static void afterClass() { + BaseConf.shutdown(mqClients); + brokerControllerMap.forEach((s, brokerController) -> brokerController.shutdown()); + ofNullable(namesrvController).ifPresent(obj -> ForkJoinPool.commonPool().execute(obj::shutdown)); + } + + @Before + public void setUp() throws Exception { + topic = "tt-" + MQRandomUtils.getRandomTopic(); + logger.info("use topic: {}", topic); + mqAdminExt.createTopic(clusterName, topic, QUEUE_NUMBERS); + assertThat(mqAdminExt.examineTopicRouteInfo(topic).getBrokerDatas()).hasSize(brokerNum); + await().atMost(5, TimeUnit.SECONDS).until(() -> !mqAdminExt.examineTopicStats(topic).getOffsetTable().isEmpty()); + + consumer.setRegisterTopics(Collections.singleton(topic)); + // consumer.setMessageQueueListener & consumer.registerMessageQueueListener are useless in DefaultMQPullConsumer, they will never work, so do not need to test it + + new UpdateTopicLogicalQueueMappingCommand().execute(mqAdminExt, topic, brokerControllerMap.values().stream().map(BrokerController::getBrokerAddr).collect(Collectors.toSet())); + } + + private static String getCurrentMethodName() { + // 0: getStackTrace + // 1: getCurrentMethodName + // 2: __realMethod__ + return Thread.currentThread().getStackTrace()[2].getMethodName(); + } + + @Test + public void test001_SendPullSync() throws Exception { + String methodName = getCurrentMethodName(); + + List publishMessageQueues = producer.fetchPublishMessageQueues(topic); + assertThat(publishMessageQueues).hasSize(brokerNum * QUEUE_NUMBERS); + Set queueIds = IntStream.range(0, brokerNum * QUEUE_NUMBERS).boxed().collect(Collectors.toSet()); + for (MessageQueue messageQueue : publishMessageQueues) { + assertThat(messageQueue.getBrokerName()).isEqualTo(MixAll.LOGICAL_QUEUE_MOCK_BROKER_NAME); + assertThat(queueIds.remove(messageQueue.getQueueId())).isTrue(); + for (int i = 0; i < MSG_SENT_TIMES; i++) { + SendResult sendResult = producer.send(new Message(topic, String.format(Locale.ENGLISH, "%s-sync-%d-%d", methodName, messageQueue.getQueueId(), i).getBytes(StandardCharsets.UTF_8)), messageQueue); + assertThat(sendResult.getMessageQueue().getBrokerName()).isEqualTo(messageQueue.getBrokerName()); + assertThat(sendResult.getMessageQueue().getQueueId()).isEqualTo(messageQueue.getQueueId()); + } + } + assertThat(queueIds).isEmpty(); + + List subscribeMessageQueues = consumer.fetchSubscribeMessageQueues(topic).stream().sorted().collect(Collectors.toList()); + assertThat(subscribeMessageQueues).hasSize(brokerNum * QUEUE_NUMBERS); + subscribeMessageQueues.sort(Comparator.comparingInt(MessageQueue::getQueueId)); + queueIds.addAll(IntStream.range(0, brokerNum * QUEUE_NUMBERS).boxed().collect(Collectors.toSet())); + for (MessageQueue messageQueue : subscribeMessageQueues) { + assertThat(messageQueue.getBrokerName()).isEqualTo(MixAll.LOGICAL_QUEUE_MOCK_BROKER_NAME); + assertThat(queueIds.remove(messageQueue.getQueueId())).isTrue(); + long offset = mqAdminExt.minOffset(messageQueue); + PullResult pullResult = consumer.pull(messageQueue, "*", offset, 10); + assertThat(pullResult.getPullStatus()).isEqualTo(PullStatus.FOUND); + assertThat(pullResult.getMsgFoundList()).hasSize(MSG_SENT_TIMES); + offset = -1; + for (int i = 0; i < MSG_SENT_TIMES; i++) { + MessageExt msg = pullResult.getMsgFoundList().get(i); + assertThat(msg.getBrokerName()).isEqualTo(MixAll.LOGICAL_QUEUE_MOCK_BROKER_NAME); + assertThat(msg.getQueueId()).isEqualTo(messageQueue.getQueueId()); + assertThat(new String(msg.getBody(), StandardCharsets.UTF_8)).isEqualTo(String.format(Locale.ENGLISH, "%s-sync-%d-%d", methodName, messageQueue.getQueueId(), i)); + if (i > 0) { + assertThat(msg.getQueueOffset()).isEqualTo(offset + i); + } else { + offset = msg.getQueueOffset(); + } + } + assertThat(maxOffsetUncommitted(messageQueue)).isEqualTo(offset + MSG_SENT_TIMES); + } + assertThat(queueIds).isEmpty(); + } + + @Test + public void test002_SendPullAsync() throws Exception { + String methodName = getCurrentMethodName(); + + List publishMessageQueues = producer.fetchPublishMessageQueues(topic); + for (MessageQueue messageQueue : publishMessageQueues) { + for (int i = 0; i < MSG_SENT_TIMES; i++) { + CompletableFuture future = new CompletableFuture<>(); + producer.send(new Message(topic, String.format(Locale.ENGLISH, "%s-async-%d-%d", methodName, messageQueue.getQueueId(), i).getBytes(StandardCharsets.UTF_8)), messageQueue, new SendCallback() { + @Override public void onSuccess(SendResult sendResult) { + future.complete(sendResult); + } + + @Override public void onException(Throwable e) { + future.completeExceptionally(e); + } + }); + SendResult sendResult = future.get(); + assertThat(sendResult.getMessageQueue().getBrokerName()).isEqualTo(messageQueue.getBrokerName()); + assertThat(sendResult.getMessageQueue().getQueueId()).isEqualTo(messageQueue.getQueueId()); + } + } + + List subscribeMessageQueues = consumer.fetchSubscribeMessageQueues(topic).stream().sorted().collect(Collectors.toList()); + for (MessageQueue messageQueue : subscribeMessageQueues) { + long offset = mqAdminExt.minOffset(messageQueue); + CompletableFuture future = new CompletableFuture<>(); + consumer.pull(messageQueue, "*", offset, 10, new PullCallback() { + @Override public void onSuccess(PullResult pullResult) { + future.complete(pullResult); + } + + @Override public void onException(Throwable e) { + future.completeExceptionally(e); + } + }); + PullResult pullResult = future.get(); + assertThat(pullResult.getPullStatus()).isEqualTo(PullStatus.FOUND); + assertThat(pullResult.getMsgFoundList()).hasSize(MSG_SENT_TIMES); + offset = -1; + Iterator it = pullResult.getMsgFoundList().iterator(); + for (int i = 0; i < MSG_SENT_TIMES; i++) { + MessageExt msg = it.next(); + assertThat(msg.getBrokerName()).isEqualTo(MixAll.LOGICAL_QUEUE_MOCK_BROKER_NAME); + assertThat(msg.getQueueId()).isEqualTo(messageQueue.getQueueId()); + assertThat(new String(msg.getBody(), StandardCharsets.UTF_8)).isEqualTo(String.format(Locale.ENGLISH, "%s-async-%d-%d", methodName, messageQueue.getQueueId(), i)); + if (i > 0) { + assertThat(msg.getQueueOffset()).isEqualTo(offset + i); + } else { + offset = msg.getQueueOffset(); + } + } + } + } + + @Test + public void test003_MigrateOnceWithoutData() throws Exception { + final String methodName = getCurrentMethodName(); + + final int logicalQueueIdx = 1; + + TopicRouteData topicRouteInfo = mqAdminExt.examineTopicRouteInfo(topic); + List logicalQueueRouteDataList1 = topicRouteInfo.getLogicalQueuesInfo().get(logicalQueueIdx); + LogicalQueueRouteData lastLogicalQueueRouteData1 = logicalQueueRouteDataList1.get(logicalQueueRouteDataList1.size() - 1); + String newBrokerName; + if (lastLogicalQueueRouteData1.getBrokerName().equals(broker1Name)) { + newBrokerName = broker2Name; + } else { + newBrokerName = broker1Name; + } + + MessageQueue migratedMessageQueue = new MessageQueue(topic, MixAll.LOGICAL_QUEUE_MOCK_BROKER_NAME, logicalQueueIdx); + + new MigrateTopicLogicalQueueCommand().execute(mqAdminExt, topic, logicalQueueIdx, newBrokerName, null); + + topicRouteInfo = mqAdminExt.examineTopicRouteInfo(topic); + assertThat(topicRouteInfo.getLogicalQueuesInfo()).isNotNull(); + for (Map.Entry> entry : topicRouteInfo.getLogicalQueuesInfo().entrySet()) { + List logicalQueueRouteDataList2 = entry.getValue(); + if (entry.getKey() == logicalQueueIdx) { + assertThat(logicalQueueRouteDataList2).hasSize(logicalQueueRouteDataList1.size() + 1); + LogicalQueueRouteData lastLogicalQueueRouteData2 = logicalQueueRouteDataList2.get(logicalQueueRouteDataList2.size() - 2); + assertThat(lastLogicalQueueRouteData2.getMessageQueue()).isEqualTo(lastLogicalQueueRouteData1.getMessageQueue()); + assertThat(lastLogicalQueueRouteData2.getOffsetMax()).isGreaterThanOrEqualTo(0L); + assertThat(lastLogicalQueueRouteData2.getMessagesCount()).isEqualTo(0L); + assertThat(lastLogicalQueueRouteData2.isWritable()).isFalse(); + assertThat(lastLogicalQueueRouteData2.isReadable()).isFalse(); + assertThat(lastLogicalQueueRouteData2.isExpired()).isTrue(); + assertThat(lastLogicalQueueRouteData2.getLogicalQueueDelta()).isEqualTo(0L); + + LogicalQueueRouteData lastLogicalQueueRouteData3 = logicalQueueRouteDataList2.get(logicalQueueRouteDataList2.size() - 1); + assertThat(lastLogicalQueueRouteData3.getBrokerName()).isEqualTo(newBrokerName); + assertThat(lastLogicalQueueRouteData3.getOffsetMax()).isLessThan(0L); + assertThat(lastLogicalQueueRouteData3.isWritable()).isTrue(); + assertThat(lastLogicalQueueRouteData3.isReadable()).isTrue(); + assertThat(lastLogicalQueueRouteData3.isExpired()).isFalse(); + assertThat(lastLogicalQueueRouteData3.getLogicalQueueDelta()).isEqualTo(0L); + } else { + assertThat(logicalQueueRouteDataList2).hasSize(1); + LogicalQueueRouteData logicalQueueRouteData = logicalQueueRouteDataList2.get(0); + assertThat(logicalQueueRouteData.getOffsetMax()).isLessThan(0L); + assertThat(logicalQueueRouteData.isWritable()).isTrue(); + assertThat(logicalQueueRouteData.isReadable()).isTrue(); + assertThat(logicalQueueRouteData.isExpired()).isFalse(); + assertThat(logicalQueueRouteData.getLogicalQueueDelta()).isEqualTo(0L); + } + } + + List subscribeMessageQueues = consumer.fetchSubscribeMessageQueues(topic).stream().sorted().collect(Collectors.toList()); + assertThat(subscribeMessageQueues).hasSize(brokerNum * QUEUE_NUMBERS); + for (MessageQueue mq : subscribeMessageQueues) { + assertThat(mqAdminExt.minOffset(mq)).isEqualTo(0L); + } + + for (int i = 0; i < MSG_SENT_TIMES; i++) { + SendResult sendResult = producer.send(new Message(topic, String.format(Locale.ENGLISH, "%s-sync-%d-%d", methodName, migratedMessageQueue.getQueueId(), i).getBytes(StandardCharsets.UTF_8)), migratedMessageQueue); + assertThat(sendResult.getMessageQueue().getBrokerName()).isEqualTo(migratedMessageQueue.getBrokerName()); + assertThat(sendResult.getMessageQueue().getQueueId()).isEqualTo(migratedMessageQueue.getQueueId()); + SendResultForLogicalQueue sendResult2 = (SendResultForLogicalQueue) sendResult; + assertThat(sendResult2.getOrigBrokerName()).isEqualTo(newBrokerName); + assertThat(sendResult2.getOrigQueueId()).isEqualTo(QUEUE_NUMBERS); + } + + for (int i = 0; i < MSG_SENT_TIMES; i++) { + CompletableFuture future = new CompletableFuture<>(); + producer.send(new Message(topic, String.format(Locale.ENGLISH, "%s-async-%d-%d", methodName, migratedMessageQueue.getQueueId(), i).getBytes(StandardCharsets.UTF_8)), migratedMessageQueue, new SendCallback() { + @Override public void onSuccess(SendResult sendResult) { + future.complete(sendResult); + } + + @Override public void onException(Throwable e) { + future.completeExceptionally(e); + } + }); + SendResult sendResult = future.get(); + assertThat(sendResult.getMessageQueue().getBrokerName()).isEqualTo(migratedMessageQueue.getBrokerName()); + assertThat(sendResult.getMessageQueue().getQueueId()).isEqualTo(migratedMessageQueue.getQueueId()); + SendResultForLogicalQueue sendResult2 = (SendResultForLogicalQueue) sendResult; + assertThat(sendResult2.getOrigBrokerName()).isEqualTo(newBrokerName); + assertThat(sendResult2.getOrigQueueId()).isEqualTo(QUEUE_NUMBERS); + } + + assertThat(maxOffsetUncommitted(migratedMessageQueue)).isEqualTo(2 * MSG_SENT_TIMES); + + waitAtMost(5, TimeUnit.SECONDS).until(() -> mqAdminExt.maxOffset(migratedMessageQueue) == 2 * MSG_SENT_TIMES); + + PullResult pullResult = consumer.pull(migratedMessageQueue, "*", 0L, 2 * MSG_SENT_TIMES); + assertThat(pullResult.getPullStatus()).isEqualTo(PullStatus.FOUND); + assertThat(pullResult.getMinOffset()).isEqualTo(0); + assertThat(pullResult.getMaxOffset()).isEqualTo(2 * MSG_SENT_TIMES); + assertThat(pullResult.getNextBeginOffset()).isEqualTo(2 * MSG_SENT_TIMES); + List msgFoundList = pullResult.getMsgFoundList(); + assertThat(msgFoundList).hasSize(2 * MSG_SENT_TIMES); + Iterator it = pullResult.getMsgFoundList().iterator(); + long offset = 0L; + for (String prefix : new String[] {"sync", "async"}) { + for (int i = 0; i < MSG_SENT_TIMES; i++) { + MessageExt msg = it.next(); + assertThat(msg.getBrokerName()).isEqualTo(MixAll.LOGICAL_QUEUE_MOCK_BROKER_NAME); + assertThat(msg.getQueueId()).isEqualTo(migratedMessageQueue.getQueueId()); + assertThat(new String(msg.getBody(), StandardCharsets.UTF_8)).isEqualTo(String.format(Locale.ENGLISH, "%s-%s-%d-%d", methodName, prefix, migratedMessageQueue.getQueueId(), i)); + assertThat(msg.getQueueOffset()).isEqualTo(offset); + offset++; + } + } + + offset = pullResult.getNextBeginOffset(); + CompletableFuture future = new CompletableFuture<>(); + consumer.pull(migratedMessageQueue, "*", offset, 10, new PullCallback() { + @Override public void onSuccess(PullResult pullResult) { + future.complete(pullResult); + } + + @Override public void onException(Throwable e) { + future.completeExceptionally(e); + } + }); + pullResult = future.get(); + assertThat(pullResult.getPullStatus()).isEqualTo(PullStatus.NO_NEW_MSG); + assertThat(pullResult.getMinOffset()).isEqualTo(0); + assertThat(pullResult.getMaxOffset()).isEqualTo(2 * MSG_SENT_TIMES); + assertThat(pullResult.getNextBeginOffset()).isEqualTo(2 * MSG_SENT_TIMES); + assertThat(pullResult.getMsgFoundList()).isNull(); + } + + @Test + public void test004_MigrateOnceWithData() throws Exception { + final String methodName = getCurrentMethodName(); + + final int logicalQueueIdx = 1; + + TopicRouteData topicRouteInfo = mqAdminExt.examineTopicRouteInfo(topic); + List logicalQueueRouteDataList1 = topicRouteInfo.getLogicalQueuesInfo().get(logicalQueueIdx); + LogicalQueueRouteData lastLogicalQueueRouteData1 = logicalQueueRouteDataList1.get(logicalQueueRouteDataList1.size() - 1); + String newBrokerName; + if (lastLogicalQueueRouteData1.getBrokerName().equals(broker1Name)) { + newBrokerName = broker2Name; + } else { + newBrokerName = broker1Name; + } + + MessageQueue migratedMessageQueue = new MessageQueue(topic, MixAll.LOGICAL_QUEUE_MOCK_BROKER_NAME, logicalQueueIdx); + + for (int i = 0; i < MSG_SENT_TIMES; i++) { + SendResult sendResult = producer.send(new Message(topic, String.format(Locale.ENGLISH, "%s-sync-%d-%d", methodName, migratedMessageQueue.getQueueId(), i).getBytes(StandardCharsets.UTF_8)), migratedMessageQueue); + assertThat(sendResult.getMessageQueue().getBrokerName()).isEqualTo(migratedMessageQueue.getBrokerName()); + assertThat(sendResult.getMessageQueue().getQueueId()).isEqualTo(migratedMessageQueue.getQueueId()); + } + assertThat(maxOffsetUncommitted(migratedMessageQueue)).isEqualTo(MSG_SENT_TIMES); + + waitAtMost(5, TimeUnit.SECONDS).until(() -> mqAdminExt.maxOffset(migratedMessageQueue) == MSG_SENT_TIMES); + + { + long offset = 0L; + PullResult pullResult = consumer.pull(migratedMessageQueue, "*", offset, 2 * MSG_SENT_TIMES); + assertThat(pullResult.getPullStatus()).isEqualTo(PullStatus.FOUND); + assertThat(pullResult.getMinOffset()).isEqualTo(0); + assertThat(pullResult.getMaxOffset()).isEqualTo(MSG_SENT_TIMES); + assertThat(pullResult.getNextBeginOffset()).isEqualTo(MSG_SENT_TIMES); + List msgFoundList = pullResult.getMsgFoundList(); + assertThat(msgFoundList).hasSize(MSG_SENT_TIMES); + Iterator it = pullResult.getMsgFoundList().iterator(); + for (int i = 0; i < MSG_SENT_TIMES; i++) { + MessageExt msg = it.next(); + assertThat(msg.getBrokerName()).isEqualTo(MixAll.LOGICAL_QUEUE_MOCK_BROKER_NAME); + assertThat(msg.getQueueId()).isEqualTo(migratedMessageQueue.getQueueId()); + assertThat(new String(msg.getBody(), StandardCharsets.UTF_8)).isEqualTo(String.format(Locale.ENGLISH, "%s-sync-%d-%d", methodName, migratedMessageQueue.getQueueId(), i)); + assertThat(msg.getQueueOffset()).isEqualTo(offset); + offset++; + } + } + + new MigrateTopicLogicalQueueCommand().execute(mqAdminExt, topic, logicalQueueIdx, newBrokerName, null); + + topicRouteInfo = mqAdminExt.examineTopicRouteInfo(topic); + assertThat(topicRouteInfo.getLogicalQueuesInfo()).isNotNull(); + for (Map.Entry> entry : topicRouteInfo.getLogicalQueuesInfo().entrySet()) { + List logicalQueueRouteDataList2 = entry.getValue(); + if (entry.getKey() == logicalQueueIdx) { + assertThat(logicalQueueRouteDataList2).hasSize(logicalQueueRouteDataList1.size() + 1); + LogicalQueueRouteData lastLogicalQueueRouteData2 = logicalQueueRouteDataList2.get(logicalQueueRouteDataList2.size() - 2); + assertThat(lastLogicalQueueRouteData2.getMessageQueue()).isEqualTo(lastLogicalQueueRouteData1.getMessageQueue()); + assertThat(lastLogicalQueueRouteData2.getOffsetMax()).isGreaterThanOrEqualTo(0L); + assertThat(lastLogicalQueueRouteData2.getMessagesCount()).isEqualTo(MSG_SENT_TIMES); + assertThat(lastLogicalQueueRouteData2.isWritable()).isFalse(); + assertThat(lastLogicalQueueRouteData2.isReadable()).isTrue(); + assertThat(lastLogicalQueueRouteData2.isExpired()).isFalse(); + assertThat(lastLogicalQueueRouteData2.getLogicalQueueDelta()).isEqualTo(0L); + + LogicalQueueRouteData lastLogicalQueueRouteData3 = logicalQueueRouteDataList2.get(logicalQueueRouteDataList2.size() - 1); + assertThat(lastLogicalQueueRouteData3.getBrokerName()).isEqualTo(newBrokerName); + assertThat(lastLogicalQueueRouteData3.getOffsetMax()).isLessThan(0L); + assertThat(lastLogicalQueueRouteData3.isWritable()).isTrue(); + assertThat(lastLogicalQueueRouteData3.isReadable()).isTrue(); + assertThat(lastLogicalQueueRouteData3.isExpired()).isFalse(); + assertThat(lastLogicalQueueRouteData3.getLogicalQueueDelta()).isEqualTo(MSG_SENT_TIMES); + } else { + assertThat(logicalQueueRouteDataList2).hasSize(1); + LogicalQueueRouteData logicalQueueRouteData = logicalQueueRouteDataList2.get(0); + assertThat(logicalQueueRouteData.getOffsetMax()).isLessThan(0L); + assertThat(logicalQueueRouteData.isWritable()).isTrue(); + assertThat(logicalQueueRouteData.isReadable()).isTrue(); + assertThat(logicalQueueRouteData.isExpired()).isFalse(); + assertThat(logicalQueueRouteData.getLogicalQueueDelta()).isEqualTo(0L); + } + } + assertThat(migratedMessageQueue).isNotNull(); + + List subscribeMessageQueues = consumer.fetchSubscribeMessageQueues(topic).stream().sorted().collect(Collectors.toList()); + assertThat(subscribeMessageQueues).hasSize(brokerNum * QUEUE_NUMBERS); + for (MessageQueue mq : subscribeMessageQueues) { + assertThat(mqAdminExt.minOffset(mq)).isEqualTo(0L); + } + + for (int i = 0; i < MSG_SENT_TIMES; i++) { + CompletableFuture future = new CompletableFuture<>(); + producer.send(new Message(topic, String.format(Locale.ENGLISH, "%s-async-%d-%d", methodName, migratedMessageQueue.getQueueId(), i).getBytes(StandardCharsets.UTF_8)), migratedMessageQueue, new SendCallback() { + @Override public void onSuccess(SendResult sendResult) { + future.complete(sendResult); + } + + @Override public void onException(Throwable e) { + future.completeExceptionally(e); + } + }); + SendResult sendResult = future.get(); + assertThat(sendResult.getMessageQueue().getBrokerName()).isEqualTo(migratedMessageQueue.getBrokerName()); + assertThat(sendResult.getMessageQueue().getQueueId()).isEqualTo(migratedMessageQueue.getQueueId()); + SendResultForLogicalQueue sendResult2 = (SendResultForLogicalQueue) sendResult; + assertThat(sendResult2.getOrigBrokerName()).isEqualTo(newBrokerName); + assertThat(sendResult2.getOrigQueueId()).isEqualTo(QUEUE_NUMBERS); + } + + assertThat(maxOffsetUncommitted(migratedMessageQueue)).isEqualTo(2 * MSG_SENT_TIMES); + + waitAtMost(5, TimeUnit.SECONDS).until(() -> mqAdminExt.maxOffset(migratedMessageQueue) == 2 * MSG_SENT_TIMES); + + long offset = 0L; + PullResult pullResult = consumer.pull(migratedMessageQueue, "*", offset, 2 * MSG_SENT_TIMES); + assertThat(pullResult.getPullStatus()).isEqualTo(PullStatus.FOUND); + assertThat(pullResult.getMinOffset()).isEqualTo(0); + assertThat(pullResult.getMaxOffset()).isEqualTo(MSG_SENT_TIMES); + assertThat(pullResult.getNextBeginOffset()).isEqualTo(MSG_SENT_TIMES); + List msgFoundList = pullResult.getMsgFoundList(); + assertThat(msgFoundList).hasSize(MSG_SENT_TIMES); + Iterator it = pullResult.getMsgFoundList().iterator(); + for (int i = 0; i < MSG_SENT_TIMES; i++) { + MessageExt msg = it.next(); + assertThat(msg.getBrokerName()).isEqualTo(MixAll.LOGICAL_QUEUE_MOCK_BROKER_NAME); + assertThat(msg.getQueueId()).isEqualTo(migratedMessageQueue.getQueueId()); + assertThat(new String(msg.getBody(), StandardCharsets.UTF_8)).isEqualTo(String.format(Locale.ENGLISH, "%s-sync-%d-%d", methodName, migratedMessageQueue.getQueueId(), i)); + assertThat(msg.getQueueOffset()).isEqualTo(offset); + offset++; + } + + offset = pullResult.getNextBeginOffset(); + CompletableFuture pullResultFuture = new CompletableFuture<>(); + consumer.pull(migratedMessageQueue, "*", offset, 2 * MSG_SENT_TIMES, new PullCallback() { + @Override public void onSuccess(PullResult pullResult) { + pullResultFuture.complete(pullResult); + } + + @Override public void onException(Throwable e) { + pullResultFuture.completeExceptionally(e); + } + }); + pullResult = pullResultFuture.get(); + assertThat(pullResult.getPullStatus()).isEqualTo(PullStatus.FOUND); + assertThat(pullResult.getMinOffset()).isEqualTo(MSG_SENT_TIMES); + assertThat(pullResult.getMaxOffset()).isEqualTo(2 * MSG_SENT_TIMES); + assertThat(pullResult.getNextBeginOffset()).isEqualTo(2 * MSG_SENT_TIMES); + msgFoundList = pullResult.getMsgFoundList(); + assertThat(msgFoundList).hasSize(MSG_SENT_TIMES); + it = pullResult.getMsgFoundList().iterator(); + for (int i = 0; i < MSG_SENT_TIMES; i++) { + MessageExt msg = it.next(); + assertThat(msg.getBrokerName()).isEqualTo(MixAll.LOGICAL_QUEUE_MOCK_BROKER_NAME); + assertThat(msg.getQueueId()).isEqualTo(migratedMessageQueue.getQueueId()); + assertThat(new String(msg.getBody(), StandardCharsets.UTF_8)).isEqualTo(String.format(Locale.ENGLISH, "%s-async-%d-%d", methodName, migratedMessageQueue.getQueueId(), i)); + assertThat(msg.getQueueOffset()).isEqualTo(offset); + offset++; + } + + offset = pullResult.getNextBeginOffset(); + pullResult = consumer.pull(migratedMessageQueue, "*", offset, 10); + assertThat(pullResult.getPullStatus()).isEqualTo(PullStatus.NO_NEW_MSG); + assertThat(pullResult.getMinOffset()).isEqualTo(MSG_SENT_TIMES); + assertThat(pullResult.getMaxOffset()).isEqualTo(2 * MSG_SENT_TIMES); + assertThat(pullResult.getNextBeginOffset()).isEqualTo(2 * MSG_SENT_TIMES); + assertThat(pullResult.getMsgFoundList()).isNull(); + } + + @Test + public void test005_MigrateWithDataBackAndForth() throws Exception { + final String methodName = getCurrentMethodName(); + + final int logicalQueueIdx = 1; + + MessageQueue migratedMessageQueue = new MessageQueue(topic, MixAll.LOGICAL_QUEUE_MOCK_BROKER_NAME, logicalQueueIdx); + + BrokerController brokerController; + + TopicRouteData topicRouteInfo = mqAdminExt.examineTopicRouteInfo(topic); + LogicalQueueRouteData lastLogicalQueueRouteData; + { + List logicalQueueRouteDataList = topicRouteInfo.getLogicalQueuesInfo().get(logicalQueueIdx); + lastLogicalQueueRouteData = logicalQueueRouteDataList.get(logicalQueueRouteDataList.size() - 1); + } + final String fromBrokerName, toBrokerName, fromBrokerAddr, toBrokerAddr; + if (lastLogicalQueueRouteData.getBrokerName().equals(broker1Name)) { + fromBrokerName = broker1Name; + fromBrokerAddr = brokerController1.getBrokerAddr(); + toBrokerName = broker2Name; + toBrokerAddr = brokerController2.getBrokerAddr(); + } else { + fromBrokerName = broker2Name; + fromBrokerAddr = brokerController2.getBrokerAddr(); + toBrokerName = broker1Name; + toBrokerAddr = brokerController1.getBrokerAddr(); + } + + int msgIdx = 0; + + for (int i = 0; i < MSG_SENT_TIMES; i++) { + SendResult sendResult = producer.send(new Message(topic, String.format(Locale.ENGLISH, "%s-%d-%d", methodName, logicalQueueIdx, msgIdx++).getBytes(StandardCharsets.UTF_8)), migratedMessageQueue); + SendResultForLogicalQueue sendResult2 = (SendResultForLogicalQueue) sendResult; + assertThat(sendResult2.getOrigBrokerName()).isEqualTo(fromBrokerName); + assertThat(sendResult2.getOrigQueueId()).isEqualTo(logicalQueueIdx); + } + + rotateBrokerCommitLog(brokerControllerMap.get(fromBrokerName)); + + new MigrateTopicLogicalQueueCommand().execute(mqAdminExt, topic, logicalQueueIdx, toBrokerName, null); + + { + LogicalQueuesInfo info; + List logicalQueueRouteDataList; + info = mqAdminExt.queryTopicLogicalQueueMapping(fromBrokerAddr, topic); + logicalQueueRouteDataList = info.get(logicalQueueIdx); + assertThat(logicalQueueRouteDataList).hasSize(2); + info = mqAdminExt.queryTopicLogicalQueueMapping(toBrokerAddr, topic); + logicalQueueRouteDataList = info.get(logicalQueueIdx); + assertThat(logicalQueueRouteDataList).hasSize(1); + } + + for (int i = 0; i < MSG_SENT_TIMES; i++) { + SendResult sendResult = producer.send(new Message(topic, String.format(Locale.ENGLISH, "%s-%d-%d", methodName, logicalQueueIdx, msgIdx++).getBytes(StandardCharsets.UTF_8)), migratedMessageQueue); + SendResultForLogicalQueue sendResult2 = (SendResultForLogicalQueue) sendResult; + assertThat(sendResult2.getOrigBrokerName()).isEqualTo(toBrokerName); + assertThat(sendResult2.getOrigQueueId()).isEqualTo(QUEUE_NUMBERS); + } + + new MigrateTopicLogicalQueueCommand().execute(mqAdminExt, topic, logicalQueueIdx, fromBrokerName, null); + // now will reuse queue with a ReadOnly one + + { + LogicalQueuesInfo info; + List logicalQueueRouteDataList; + info = mqAdminExt.queryTopicLogicalQueueMapping(fromBrokerAddr, topic); + logicalQueueRouteDataList = info.get(logicalQueueIdx); + assertThat(logicalQueueRouteDataList).hasSize(3); + info = mqAdminExt.queryTopicLogicalQueueMapping(toBrokerAddr, topic); + logicalQueueRouteDataList = info.get(logicalQueueIdx); + assertThat(logicalQueueRouteDataList).hasSize(2); + } + + for (int i = 0; i < MSG_SENT_TIMES; i++) { + SendResult sendResult = producer.send(new Message(topic, String.format(Locale.ENGLISH, "%s-%d-%d", methodName, logicalQueueIdx, msgIdx++).getBytes(StandardCharsets.UTF_8)), migratedMessageQueue); + SendResultForLogicalQueue sendResult2 = (SendResultForLogicalQueue) sendResult; + assertThat(sendResult2.getOrigBrokerName()).isEqualTo(fromBrokerName); + assertThat(sendResult2.getOrigQueueId()).isEqualTo(logicalQueueIdx); + } + + LogicalQueueRouteData logicalQueueRouteData1; + LogicalQueueRouteData logicalQueueRouteData2; + { + List logicalQueueRouteDataList; + topicRouteInfo = mqAdminExt.examineTopicRouteInfo(topic); + logicalQueueRouteDataList = topicRouteInfo.getLogicalQueuesInfo().get(logicalQueueIdx); + assertThat(logicalQueueRouteDataList).hasSize(3); + logicalQueueRouteData1 = logicalQueueRouteDataList.get(0); + assertThat(logicalQueueRouteData1.getLogicalQueueDelta()).isEqualTo(0); + assertThat(logicalQueueRouteData1.isReadable()).isTrue(); + assertThat(logicalQueueRouteData1.isWritable()).isFalse(); + assertThat(logicalQueueRouteData1.isExpired()).isFalse(); + assertThat(logicalQueueRouteData1.isWriteOnly()).isFalse(); + assertThat(logicalQueueRouteData1.getBrokerName()).isEqualTo(fromBrokerName); + assertThat(logicalQueueRouteData1.getOffsetMax()).isGreaterThanOrEqualTo(0L); + assertThat(logicalQueueRouteData1.getMessagesCount()).isEqualTo(MSG_SENT_TIMES); + assertThat(logicalQueueRouteData1.getFirstMsgTimeMillis()).isGreaterThan(0L); + assertThat(logicalQueueRouteData1.getLastMsgTimeMillis()).isGreaterThan(0L); + logicalQueueRouteData2 = logicalQueueRouteDataList.get(1); + assertThat(logicalQueueRouteData2.getLogicalQueueDelta()).isEqualTo(MSG_SENT_TIMES); + assertThat(logicalQueueRouteData2.isReadable()).isTrue(); + assertThat(logicalQueueRouteData2.isWritable()).isFalse(); + assertThat(logicalQueueRouteData2.isExpired()).isFalse(); + assertThat(logicalQueueRouteData2.isWriteOnly()).isFalse(); + assertThat(logicalQueueRouteData2.getBrokerName()).isEqualTo(toBrokerName); + assertThat(logicalQueueRouteData2.getOffsetMax()).isGreaterThanOrEqualTo(0L); + assertThat(logicalQueueRouteData2.getMessagesCount()).isEqualTo(MSG_SENT_TIMES); + assertThat(logicalQueueRouteData2.getFirstMsgTimeMillis()).isGreaterThan(0L); + assertThat(logicalQueueRouteData2.getLastMsgTimeMillis()).isGreaterThan(0L); + LogicalQueueRouteData logicalQueueRouteData3 = logicalQueueRouteDataList.get(2); + assertThat(logicalQueueRouteData3.getLogicalQueueDelta()).isEqualTo(2 * MSG_SENT_TIMES); + assertThat(logicalQueueRouteData3.isReadable()).isTrue(); + assertThat(logicalQueueRouteData3.isWritable()).isTrue(); + assertThat(logicalQueueRouteData3.isExpired()).isFalse(); + assertThat(logicalQueueRouteData3.isWriteOnly()).isFalse(); + assertThat(logicalQueueRouteData3.getBrokerName()).isEqualTo(fromBrokerName); + assertThat(logicalQueueRouteData3.getOffsetMax()).isLessThan(0L); + } + + msgIdx = 0; + forLoop: + for (long offset = 0L; ; ) { + PullResult pullResult = consumer.pull(migratedMessageQueue, "*", offset, 3 * MSG_SENT_TIMES); + switch (pullResult.getPullStatus()) { + case NO_NEW_MSG: + assertThat(offset).isGreaterThanOrEqualTo(3L * MSG_SENT_TIMES); + break forLoop; + case OFFSET_ILLEGAL: + offset = pullResult.getNextBeginOffset(); + break; + default: + assertThat(pullResult.getPullStatus()).isEqualTo(PullStatus.FOUND); + assertThat(pullResult.getMsgFoundList()).isNotNull(); + assertThat(pullResult.getMsgFoundList()).hasSize(MSG_SENT_TIMES); + for (MessageExt msg : pullResult.getMsgFoundList()) { + assertThat(new String(msg.getBody(), StandardCharsets.UTF_8)).isEqualTo(String.format(Locale.ENGLISH, "%s-%d-%d", methodName, logicalQueueIdx, msgIdx)); + msgIdx++; + assertThat(msg.getQueueOffset()).isEqualTo(offset); + offset++; + } + offset = pullResult.getNextBeginOffset(); + break; + } + } + + waitAtMost(5, TimeUnit.SECONDS).until(() -> maxOffsetUncommitted(logicalQueueRouteData1.getMessageQueue()) == mqAdminExt.maxOffset(logicalQueueRouteData1.getMessageQueue())); + waitAtMost(5, TimeUnit.SECONDS).until(() -> maxOffsetUncommitted(logicalQueueRouteData2.getMessageQueue()) == mqAdminExt.maxOffset(logicalQueueRouteData2.getMessageQueue())); + + // now verify after commit log cleaned, toBroker's first queue route data will be expired too + brokerController = brokerControllerMap.get(logicalQueueRouteData2.getBrokerName()); + rotateBrokerCommitLog(brokerController); + deleteCommitLogFiles(brokerController, 1); + + { + topicRouteInfo = mqAdminExt.examineTopicRouteInfo(topic); + List logicalQueueRouteDataList = topicRouteInfo.getLogicalQueuesInfo().get(logicalQueueIdx); + assertThat(logicalQueueRouteDataList).hasSize(2); + assertThat(logicalQueueRouteDataList.get(0)).isEqualToIgnoringGivenFields(new LogicalQueueRouteData(logicalQueueIdx, 0, new MessageQueue(topic, fromBrokerName, logicalQueueIdx), MessageQueueRouteState.ReadOnly, 0, 3, -1, -1, fromBrokerAddr), "firstMsgTimeMillis", "lastMsgTimeMillis"); + assertThat(logicalQueueRouteDataList.get(1)).isEqualToComparingFieldByField(new LogicalQueueRouteData(logicalQueueIdx, 2 * MSG_SENT_TIMES, new MessageQueue(topic, fromBrokerName, logicalQueueIdx), MessageQueueRouteState.Normal, MSG_SENT_TIMES, -1, -1, -1, fromBrokerAddr)); + } + + // try pull again, since there is an expired queue route in the middle. + { + int msgCount = 0; + Queue wantMsgIdx = new LinkedList<>(); + wantMsgIdx.addAll(IntStream.range(0, MSG_SENT_TIMES).boxed().collect(Collectors.toList())); + wantMsgIdx.addAll(IntStream.range(2 * MSG_SENT_TIMES, 3 * MSG_SENT_TIMES).boxed().collect(Collectors.toList())); + forLoop: + for (long offset = mqAdminExt.minOffset(migratedMessageQueue); ; ) { + PullResult pullResult = consumer.pull(migratedMessageQueue, "*", offset, 3 * MSG_SENT_TIMES); + switch (pullResult.getPullStatus()) { + case NO_NEW_MSG: + assertThat(msgCount).as("offset=%d", offset).isEqualTo(2 * MSG_SENT_TIMES); + break forLoop; + case OFFSET_ILLEGAL: + offset = pullResult.getNextBeginOffset(); + break; + case FOUND: + msgCount += pullResult.getMsgFoundList().size(); + boolean first = true; + for (MessageExt msg : pullResult.getMsgFoundList()) { + assertThat(new String(msg.getBody(), StandardCharsets.UTF_8)).as("offset=%d", offset).isEqualTo(String.format(Locale.ENGLISH, "%s-%d-%d", methodName, logicalQueueIdx, wantMsgIdx.poll())); + if (first) { + assertThat(msg.getQueueOffset()).isGreaterThanOrEqualTo(offset); + first = false; + } else { + assertThat(msg.getQueueOffset()).isGreaterThan(offset); + } + offset = msg.getQueueOffset(); + } + offset = pullResult.getNextBeginOffset(); + break; + default: + Assert.fail(String.format(Locale.ENGLISH, "unexpected pull offset=%d status: %s", offset, pullResult)); + } + } + } + + // rotate first queue route to expired, and pull it + brokerController = brokerControllerMap.get(logicalQueueRouteData1.getBrokerName()); + rotateBrokerCommitLog(brokerController); + deleteCommitLogFiles(brokerController, 2); + + { + List logicalQueueRouteDataList; + topicRouteInfo = mqAdminExt.examineTopicRouteInfo(topic); + logicalQueueRouteDataList = topicRouteInfo.getLogicalQueuesInfo().get(logicalQueueIdx); + assertThat(logicalQueueRouteDataList).isEqualTo(Collections.singletonList(new LogicalQueueRouteData(logicalQueueIdx, 2 * MSG_SENT_TIMES, new MessageQueue(topic, fromBrokerName, logicalQueueIdx), MessageQueueRouteState.Normal, MSG_SENT_TIMES, -1, -1, -1, fromBrokerAddr))); + } + + { + int msgCount = 0; + Queue wantMsgIdx = new LinkedList<>(); + wantMsgIdx.addAll(IntStream.range(2 * MSG_SENT_TIMES, 3 * MSG_SENT_TIMES).boxed().collect(Collectors.toList())); + forLoop: + for (long offset = mqAdminExt.minOffset(migratedMessageQueue); ; ) { + PullResult pullResult = consumer.pull(migratedMessageQueue, "*", offset, 3 * MSG_SENT_TIMES); + switch (pullResult.getPullStatus()) { + case NO_NEW_MSG: + if (msgCount != MSG_SENT_TIMES) { + Assert.fail(String.format(Locale.ENGLISH, "want %d msg but got %d", MSG_SENT_TIMES, msgCount)); + } + break forLoop; + case OFFSET_ILLEGAL: + offset = pullResult.getNextBeginOffset(); + break; + case FOUND: + msgCount += pullResult.getMsgFoundList().size(); + boolean first = true; + for (MessageExt msg : pullResult.getMsgFoundList()) { + assertThat(new String(msg.getBody(), StandardCharsets.UTF_8)).as("offset=%d", offset).isEqualTo(String.format(Locale.ENGLISH, "%s-%d-%d", methodName, logicalQueueIdx, wantMsgIdx.poll())); + if (first) { + assertThat(msg.getQueueOffset()).isGreaterThanOrEqualTo(offset); + first = false; + } else { + assertThat(msg.getQueueOffset()).isGreaterThan(offset); + } + offset = msg.getQueueOffset(); + } + offset = pullResult.getNextBeginOffset(); + break; + default: + Assert.fail(String.format(Locale.ENGLISH, "unexpected pull offset=%d status: %s", offset, pullResult)); + } + } + } + + brokerController = brokerControllerMap.get(fromBrokerName); + rotateBrokerCommitLog(brokerController); + deleteCommitLogFiles(brokerController, 1); + + { + forLoop: + for (long offset = mqAdminExt.minOffset(migratedMessageQueue); ; ) { + PullResult pullResult = consumer.pull(migratedMessageQueue, "*", offset, 3 * MSG_SENT_TIMES); + // commit log rotate and cleaned, so there is no message. + switch (pullResult.getPullStatus()) { + case NO_MATCHED_MSG: + case NO_NEW_MSG: + assertThat(pullResult.getNextBeginOffset()).isEqualTo(3 * MSG_SENT_TIMES); + break forLoop; + case OFFSET_ILLEGAL: + offset = pullResult.getNextBeginOffset(); + break; + default: + Assert.fail(String.format(Locale.ENGLISH, "unexpected pull offset=%d status: %s", offset, pullResult)); + } + } + } + + { + LogicalQueuesInfo logicalQueuesInfo = mqAdminExt.queryTopicLogicalQueueMapping(brokerController.getBrokerAddr(), topic); + List logicalQueueRouteDataList = logicalQueuesInfo.get(logicalQueueIdx); + assertThat(logicalQueueRouteDataList).isEqualTo(Collections.singletonList(new LogicalQueueRouteData(logicalQueueIdx, 2 * MSG_SENT_TIMES, new MessageQueue(topic, fromBrokerName, logicalQueueIdx), MessageQueueRouteState.Normal, MSG_SENT_TIMES, -1, -1, -1, fromBrokerAddr))); + } + + // try migrate to this broker which has a expired queue, expect it will reuse the expired one, pull it to verify if delta works well + new MigrateTopicLogicalQueueCommand().execute(mqAdminExt, topic, logicalQueueIdx, toBrokerName, null); + + { + List logicalQueueRouteDataList; + topicRouteInfo = mqAdminExt.examineTopicRouteInfo(topic); + logicalQueueRouteDataList = topicRouteInfo.getLogicalQueuesInfo().get(logicalQueueIdx); + assertThat(logicalQueueRouteDataList).isEqualTo(Arrays.asList( + new LogicalQueueRouteData(logicalQueueIdx, 2 * MSG_SENT_TIMES, new MessageQueue(topic, fromBrokerName, logicalQueueIdx), MessageQueueRouteState.Expired, MSG_SENT_TIMES, 2 * MSG_SENT_TIMES, 0, 0, fromBrokerAddr) + , new LogicalQueueRouteData(logicalQueueIdx, 3 * MSG_SENT_TIMES, new MessageQueue(topic, toBrokerName, QUEUE_NUMBERS), MessageQueueRouteState.Normal, MSG_SENT_TIMES, -1, -1, -1, toBrokerAddr) + )); + + LogicalQueuesInfo info; + info = mqAdminExt.queryTopicLogicalQueueMapping(fromBrokerAddr, topic); + logicalQueueRouteDataList = info.get(logicalQueueIdx); + assertThat(logicalQueueRouteDataList).isEqualTo(Arrays.asList( + new LogicalQueueRouteData(logicalQueueIdx, 2 * MSG_SENT_TIMES, new MessageQueue(topic, fromBrokerName, logicalQueueIdx), MessageQueueRouteState.Expired, MSG_SENT_TIMES, 2 * MSG_SENT_TIMES, 0, 0, fromBrokerAddr) + , new LogicalQueueRouteData(logicalQueueIdx, 3 * MSG_SENT_TIMES, new MessageQueue(topic, toBrokerName, QUEUE_NUMBERS), MessageQueueRouteState.Normal, MSG_SENT_TIMES, -1, -1, -1, toBrokerAddr) + )); + info = mqAdminExt.queryTopicLogicalQueueMapping(toBrokerAddr, topic); + logicalQueueRouteDataList = info.get(logicalQueueIdx); + assertThat(logicalQueueRouteDataList).isEqualTo(Collections.singletonList(new LogicalQueueRouteData(logicalQueueIdx, 3 * MSG_SENT_TIMES, new MessageQueue(topic, toBrokerName, QUEUE_NUMBERS), MessageQueueRouteState.Normal, MSG_SENT_TIMES, -1, -1, -1, toBrokerAddr))); + } + + msgIdx = 3 * MSG_SENT_TIMES; + for (int i = 0; i < MSG_SENT_TIMES; i++) { + SendResult sendResult = producer.send(new Message(topic, String.format(Locale.ENGLISH, "%s-%d-%d", methodName, logicalQueueIdx, msgIdx++).getBytes(StandardCharsets.UTF_8)), migratedMessageQueue); + SendResultForLogicalQueue sendResult2 = (SendResultForLogicalQueue) sendResult; + assertThat(sendResult2.getOrigBrokerName()).isEqualTo(toBrokerName); + assertThat(sendResult2.getOrigQueueId()).isEqualTo(QUEUE_NUMBERS); + } + + { + int msgCount = 0; + Queue wantMsgIdx = new LinkedList<>(); + wantMsgIdx.addAll(IntStream.range(3 * MSG_SENT_TIMES, 4 * MSG_SENT_TIMES).boxed().collect(Collectors.toList())); + LOOP: + for (long offset = 0L; ; ) { + PullResult pullResult = consumer.pull(migratedMessageQueue, "*", offset, 3 * MSG_SENT_TIMES); + switch (pullResult.getPullStatus()) { + case NO_NEW_MSG: + assertThat(msgCount).as("msgCount with offset=%d", offset).isEqualTo(MSG_SENT_TIMES); + break LOOP; + case OFFSET_ILLEGAL: + assertThat(pullResult.getNextBeginOffset()).isNotEqualTo(Long.MIN_VALUE); + offset = pullResult.getNextBeginOffset(); + break; + case FOUND: + msgCount += pullResult.getMsgFoundList().size(); + boolean first = true; + for (MessageExt msg : pullResult.getMsgFoundList()) { + assertThat(new String(msg.getBody(), StandardCharsets.UTF_8)).as("offset=%d", offset).isEqualTo(String.format(Locale.ENGLISH, "%s-%d-%d", methodName, logicalQueueIdx, wantMsgIdx.poll())); + if (first) { + assertThat(msg.getQueueOffset()).isGreaterThanOrEqualTo(offset); + first = false; + } else { + assertThat(msg.getQueueOffset()).isGreaterThan(offset); + } + offset = msg.getQueueOffset(); + } + offset = pullResult.getNextBeginOffset(); + break; + default: + Assert.fail(String.format(Locale.ENGLISH, "unexpected pull offset=%d status: %s", offset, pullResult)); + } + } + } + } + + @Test + public void test006_LogicalQueueNumChanged() throws Exception { + String methodName = getCurrentMethodName(); + int logicalQueueNum = brokerNum * QUEUE_NUMBERS; + + List publishMessageQueues; + publishMessageQueues = producer.fetchPublishMessageQueues(topic); + assertThat(publishMessageQueues).hasSize(logicalQueueNum); + List subscribeMessageQueues; + subscribeMessageQueues = consumer.fetchSubscribeMessageQueues(topic).stream().sorted().collect(Collectors.toList()); + assertThat(subscribeMessageQueues).hasSize(logicalQueueNum); + + logicalQueueNum++; + new UpdateTopicLogicalQueueNumCommand().execute(mqAdminExt, clusterName, topic, logicalQueueNum); + + int newAddLogicalQueueIdx = logicalQueueNum - 1; + MessageQueue newAddLogicalQueue = new MessageQueue(topic, MixAll.LOGICAL_QUEUE_MOCK_BROKER_NAME, newAddLogicalQueueIdx); + String newAddLogicalQueueBrokerName; + { + TopicRouteData topicRouteInfo = mqAdminExt.examineTopicRouteInfo(topic); + LogicalQueuesInfo info = topicRouteInfo.getLogicalQueuesInfo(); + assertThat(info).isNotNull(); + List queueRouteDataList = info.get(newAddLogicalQueueIdx); + assertThat(queueRouteDataList).isNotNull(); + assertThat(queueRouteDataList).hasSize(1); + LogicalQueueRouteData queueRouteData = queueRouteDataList.get(0); + newAddLogicalQueueBrokerName = queueRouteData.getBrokerName(); + assertThat(queueRouteData.getState()).isEqualTo(MessageQueueRouteState.Normal); + assertThat(queueRouteData.getLogicalQueueDelta()).isEqualTo(0); + assertThat(queueRouteData.getLogicalQueueIndex()).isEqualTo(newAddLogicalQueueIdx); + } + + publishMessageQueues = producer.fetchPublishMessageQueues(topic); + assertThat(publishMessageQueues).hasSize(logicalQueueNum); + Set logicalQueueIds = IntStream.range(0, logicalQueueNum).boxed().collect(Collectors.toSet()); + Map> queueIds = Maps.newHashMap(); + for (String brokerName : Arrays.asList(broker1Name, broker2Name)) { + queueIds.put(brokerName, IntStream.range(0, QUEUE_NUMBERS).boxed().collect(Collectors.toSet())); + } + queueIds.get(newAddLogicalQueueBrokerName).add(QUEUE_NUMBERS); + for (MessageQueue messageQueue : publishMessageQueues) { + assertThat(messageQueue.getBrokerName()).isEqualTo(MixAll.LOGICAL_QUEUE_MOCK_BROKER_NAME); + assertThat(logicalQueueIds.remove(messageQueue.getQueueId())).isTrue(); + for (int i = 0; i < MSG_SENT_TIMES; i++) { + SendResult sendResult = producer.send(new Message(topic, String.format(Locale.ENGLISH, "%s-%d-%d", methodName, messageQueue.getQueueId(), i).getBytes(StandardCharsets.UTF_8)), messageQueue); + assertThat(sendResult.getMessageQueue().getBrokerName()).isEqualTo(messageQueue.getBrokerName()); + assertThat(sendResult.getMessageQueue().getQueueId()).isEqualTo(messageQueue.getQueueId()); + if (i == 0) { + SendResultForLogicalQueue sendResult2 = (SendResultForLogicalQueue) sendResult; + assertThat(queueIds.get(sendResult2.getOrigBrokerName()).remove(sendResult2.getOrigQueueId())).as("brokerName %s queueId %d", sendResult2.getOrigBrokerName(), sendResult2.getOrigQueueId()).isTrue(); + } + } + } + assertThat(logicalQueueIds).isEmpty(); + + subscribeMessageQueues = consumer.fetchSubscribeMessageQueues(topic).stream().sorted().collect(Collectors.toList()); + assertThat(subscribeMessageQueues).hasSize(logicalQueueNum); + subscribeMessageQueues.sort(Comparator.comparingInt(MessageQueue::getQueueId)); + logicalQueueIds.addAll(IntStream.range(0, logicalQueueNum).boxed().collect(Collectors.toSet())); + for (MessageQueue messageQueue : subscribeMessageQueues) { + assertThat(messageQueue.getBrokerName()).isEqualTo(MixAll.LOGICAL_QUEUE_MOCK_BROKER_NAME); + assertThat(logicalQueueIds.remove(messageQueue.getQueueId())).isTrue(); + long offset = mqAdminExt.minOffset(messageQueue); + assertThat(offset).isEqualTo(0); + PullResult pullResult = consumer.pull(messageQueue, "*", offset, 10); + assertThat(pullResult.getPullStatus()).isEqualTo(PullStatus.FOUND); + assertThat(pullResult.getMsgFoundList()).hasSize(MSG_SENT_TIMES); + for (int i = 0; i < MSG_SENT_TIMES; i++) { + MessageExt msg = pullResult.getMsgFoundList().get(i); + assertThat(msg.getBrokerName()).isEqualTo(MixAll.LOGICAL_QUEUE_MOCK_BROKER_NAME); + assertThat(msg.getQueueId()).isEqualTo(messageQueue.getQueueId()); + assertThat(new String(msg.getBody(), StandardCharsets.UTF_8)).isEqualTo(String.format(Locale.ENGLISH, "%s-%d-%d", methodName, messageQueue.getQueueId(), i)); + assertThat(msg.getQueueOffset()).isEqualTo(offset + i); + } + assertThat(maxOffsetUncommitted(messageQueue)).isEqualTo(offset + MSG_SENT_TIMES); + } + assertThat(logicalQueueIds).isEmpty(); + + // increase TopicConfig write queue first then increase logical queue, expect to reuse + String broker2Addr = brokerController2.getBrokerAddr(); + TopicConfig topicConfig = mqAdminExt.examineTopicConfig(broker2Addr, topic); + topicConfig.setWriteQueueNums(topicConfig.getWriteQueueNums() + 1); + topicConfig.setReadQueueNums(topicConfig.getReadQueueNums() + 1); + mqAdminExt.createAndUpdateTopicConfig(broker2Addr, topicConfig); + logicalQueueNum++; + new UpdateTopicLogicalQueueNumCommand().execute(mqAdminExt, clusterName, topic, logicalQueueNum); + { + newAddLogicalQueueIdx = logicalQueueNum -1; + TopicRouteData topicRouteInfo = mqAdminExt.examineTopicRouteInfo(topic); + LogicalQueuesInfo info = topicRouteInfo.getLogicalQueuesInfo(); + assertThat(info).isNotNull(); + List queueRouteDataList = info.get(newAddLogicalQueueIdx); + assertThat(queueRouteDataList).isNotNull(); + assertThat(queueRouteDataList).hasSize(1); + LogicalQueueRouteData queueRouteData = queueRouteDataList.get(0); + assertThat(queueRouteData.getState()).isEqualTo(MessageQueueRouteState.Normal); + assertThat(queueRouteData.getLogicalQueueDelta()).isEqualTo(0); + assertThat(queueRouteData.getLogicalQueueIndex()).isEqualTo(newAddLogicalQueueIdx); + assertThat(queueRouteData.getBrokerName()).isEqualTo(broker2Name); + assertThat(queueRouteData.getQueueId()).isEqualTo(topicConfig.getWriteQueueNums() -1); + } + + logicalQueueNum-=2; + new UpdateTopicLogicalQueueNumCommand().execute(mqAdminExt, clusterName, topic, logicalQueueNum); + + try { + producer.send(new Message(topic, "aaa".getBytes(StandardCharsets.UTF_8)), newAddLogicalQueue); + Assert.fail("write to decreased logical queue success, want it failed"); + } catch (MQBrokerException e) { + assertThat(e.getResponseCode()).isEqualTo(ResponseCode.NO_PERMISSION); + } + { + int offset = 0; + PullResult pullResult = consumer.pull(newAddLogicalQueue, "*", offset, 10); + assertThat(pullResult.getPullStatus()).isEqualTo(PullStatus.FOUND); + assertThat(pullResult.getMsgFoundList()).hasSize(MSG_SENT_TIMES); + for (int i = 0; i < MSG_SENT_TIMES; i++) { + MessageExt msg = pullResult.getMsgFoundList().get(i); + assertThat(msg.getBrokerName()).isEqualTo(MixAll.LOGICAL_QUEUE_MOCK_BROKER_NAME); + assertThat(msg.getQueueId()).isEqualTo(newAddLogicalQueue.getQueueId()); + assertThat(new String(msg.getBody(), StandardCharsets.UTF_8)).isEqualTo(String.format(Locale.ENGLISH, "%s-%d-%d", methodName, newAddLogicalQueue.getQueueId(), i)); + assertThat(msg.getQueueOffset()).isEqualTo(offset + i); + } + } + + // rotate to remove new add queue's data, and try pull again + { + BrokerController brokerController = brokerControllerMap.get(newAddLogicalQueueBrokerName); + rotateBrokerCommitLog(brokerController); + deleteCommitLogFiles(brokerController, 1); + } + { + int offset = 0; + PullResult pullResult = consumer.pull(newAddLogicalQueue, "*", offset, 10); + assertThat(pullResult.getPullStatus()).isIn(PullStatus.NO_NEW_MSG, PullStatus.NO_MATCHED_MSG); + } + } + + @Test + public void test007_LogicalQueueWritableEvenBrokerDown() throws Exception { + final String methodName = getCurrentMethodName(); + + final int logicalQueueIdx = 1; + + BrokerController brokerController3 = IntegrationTestBase.createAndStartBroker(nsAddr); + String broker3Name = brokerController3.getBrokerConfig().getBrokerName(); + brokerControllerMap.put(broker3Name, brokerController3); + await().atMost(30, TimeUnit.SECONDS).until(() -> mqAdminExt.examineBrokerClusterInfo().getBrokerAddrTable().containsKey(broker3Name)); + mqAdminExt.createAndUpdateTopicConfig(brokerController3.getBrokerAddr(), new TopicConfig(topic, 0, 0, PermName.PERM_READ | PermName.PERM_WRITE)); + + new MigrateTopicLogicalQueueCommand().execute(mqAdminExt, topic, logicalQueueIdx, brokerController3.getBrokerConfig().getBrokerName(), null); + + MessageQueue migrateMessageQueue = new MessageQueue(topic, MixAll.LOGICAL_QUEUE_MOCK_BROKER_NAME, logicalQueueIdx); + { + for (int i = 0; i < MSG_SENT_TIMES; i++) { + SendResult sendResult = producer.send(new Message(topic, String.format(Locale.ENGLISH, "%s-%d-%d", methodName, migrateMessageQueue.getQueueId(), i).getBytes(StandardCharsets.UTF_8)), migrateMessageQueue); + assertThat(sendResult.getMessageQueue().getBrokerName()).isEqualTo(migrateMessageQueue.getBrokerName()); + assertThat(sendResult.getMessageQueue().getQueueId()).isEqualTo(migrateMessageQueue.getQueueId()); + SendResultForLogicalQueue sendResult2 = (SendResultForLogicalQueue) sendResult; + assertThat(sendResult2.getOrigBrokerName()).isEqualTo(broker3Name); + assertThat(sendResult2.getOrigQueueId()).isEqualTo(0); + } + } + brokerController3.shutdown(); + brokerControllerMap.remove(broker3Name); + + assertThatThrownBy(() -> { + SendResult sendResult = producer.send(new Message(topic, "aaa".getBytes(StandardCharsets.UTF_8)), migrateMessageQueue); + logger.error("send should fail but got {}", sendResult); + }).isInstanceOf(RemotingException.class).hasMessageMatching("connect to [0-9.:]+ failed"); + + assertThatThrownBy(() -> { + new MigrateTopicLogicalQueueCommand().execute(mqAdminExt, topic, logicalQueueIdx, broker1Name, null); + }).hasRootCauseInstanceOf(RemotingConnectException.class).hasMessageContaining("migrateTopicLogicalQueuePrepare"); + + { + SendResult sendResult = producer.send(new Message(topic, "aaa".getBytes(StandardCharsets.UTF_8)), migrateMessageQueue); + assertThat(sendResult.getMessageQueue().getBrokerName()).isEqualTo(migrateMessageQueue.getBrokerName()); + assertThat(sendResult.getMessageQueue().getQueueId()).isEqualTo(migrateMessageQueue.getQueueId()); + assertThat(sendResult.getQueueOffset()).isEqualTo(-1); + SendResultForLogicalQueue sendResult2 = (SendResultForLogicalQueue) sendResult; + assertThat(sendResult2.getOrigBrokerName()).isEqualTo(broker1Name); + assertThat(sendResult2.getOrigQueueId()).isIn( + /* CommitLog not rotated, will not reuse */QUEUE_NUMBERS, + /* CommitLog rotated in other test cases, will reuse */logicalQueueIdx + ); + } + } + + private static String getBrokerCommitLogFileName(BrokerController brokerController) throws IllegalAccessException { + DefaultMessageStore defaultMessageStore = (DefaultMessageStore) brokerController.getMessageStore(); + MappedFileQueue mfq = (MappedFileQueue) FieldUtils.readDeclaredField(defaultMessageStore.getCommitLog(), "mappedFileQueue", true); + return mfq.getLastMappedFile().getFileName(); + } + + private static void deleteCommitLogFiles(BrokerController brokerController, + int keepNum) throws IllegalAccessException { + CommitLog commitLog = ((DefaultMessageStore) brokerController.getMessageStore()).getCommitLog(); + commitLog.flush(); + MappedFileQueue mfq = (MappedFileQueue) FieldUtils.readDeclaredField(commitLog, "mappedFileQueue", true); + AtomicInteger count = new AtomicInteger(); + waitAtMost(5, TimeUnit.SECONDS).until(() -> { + count.getAndAdd(commitLog.deleteExpiredFile(0, 0, 5000, true, 1)); + return mfq.getMappedFiles().size() <= keepNum; + }); + brokerController.getTopicConfigManager().getLogicalQueueCleanHook().execute((DefaultMessageStore) brokerController.getMessageStore(), count.get()); + logger.info("deleteCommitLogFiles {} count {}", brokerController.getBrokerConfig().getBrokerName(), count.get()); + } + + private static void rotateBrokerCommitLog(BrokerController brokerController) throws IllegalAccessException { + CommitLog commitLog = ((DefaultMessageStore) brokerController.getMessageStore()).getCommitLog(); + commitLog.flush(); + String brokerName = brokerController.getBrokerConfig().getBrokerName(); + String fileName1 = getBrokerCommitLogFileName(brokerController); + logger.info("rotateBrokerCommitLog {} first {}", brokerName, fileName1); + int msgSize = 4 * 1024; + byte[] data = RandomStringUtils.randomAscii(msgSize).getBytes(StandardCharsets.UTF_8); + Message msg = new Message(placeholderTopic, data); + MessageQueue mq = new MessageQueue(placeholderTopic, brokerName, 0); + waitAtMost(5, TimeUnit.SECONDS).until(() -> { + for (int i = 0; i < 128; i++) { + producer.send(msg, mq); + } + commitLog.flush(); + String fileName2 = getBrokerCommitLogFileName(brokerController); + if (!fileName1.equals(fileName2)) { + logger.info("rotateBrokerCommitLog {} 4K msg last {}", brokerName, fileName2); + return true; + } + return false; + }); + } + + private long maxOffsetUncommitted(MessageQueue mq) throws IllegalAccessException, MQClientException { + DefaultMQAdminExtImpl defaultMQAdminExtImpl = (DefaultMQAdminExtImpl) FieldUtils.readDeclaredField(mqAdminExt, "defaultMQAdminExtImpl", true); + return defaultMQAdminExtImpl.maxOffset(mq, false); + } +} diff --git a/tools/pom.xml b/tools/pom.xml index abe8197de39..096d531fa75 100644 --- a/tools/pom.xml +++ b/tools/pom.xml @@ -64,5 +64,9 @@ org.yaml snakeyaml + + com.google.guava + guava + diff --git a/tools/src/main/java/org/apache/rocketmq/tools/admin/DefaultMQAdminExt.java b/tools/src/main/java/org/apache/rocketmq/tools/admin/DefaultMQAdminExt.java index c27c85c66de..1c20324d1fa 100644 --- a/tools/src/main/java/org/apache/rocketmq/tools/admin/DefaultMQAdminExt.java +++ b/tools/src/main/java/org/apache/rocketmq/tools/admin/DefaultMQAdminExt.java @@ -43,12 +43,16 @@ import org.apache.rocketmq.common.protocol.body.ConsumerRunningInfo; import org.apache.rocketmq.common.protocol.body.GroupList; import org.apache.rocketmq.common.protocol.body.KVTable; +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.QueueTimeSpan; import org.apache.rocketmq.common.protocol.body.SubscriptionGroupWrapper; import org.apache.rocketmq.common.protocol.body.TopicConfigSerializeWrapper; import org.apache.rocketmq.common.protocol.body.TopicList; +import org.apache.rocketmq.common.protocol.route.LogicalQueueRouteData; +import org.apache.rocketmq.common.protocol.route.LogicalQueuesInfo; +import org.apache.rocketmq.common.protocol.route.MessageQueueRouteState; import org.apache.rocketmq.common.protocol.route.TopicRouteData; import org.apache.rocketmq.common.subscription.SubscriptionGroupConfig; import org.apache.rocketmq.common.topic.TopicValidator; @@ -209,7 +213,8 @@ public SubscriptionGroupConfig examineSubscriptionGroupConfig(String addr, Strin } @Override - public TopicConfig examineTopicConfig(String addr, String topic) { + public TopicConfig examineTopicConfig(String addr, + String topic) throws RemotingSendRequestException, RemotingConnectException, RemotingTimeoutException, MQBrokerException, InterruptedException { return defaultMQAdminExtImpl.examineTopicConfig(addr, topic); } @@ -578,4 +583,52 @@ public void setMessageRequestMode(final String brokerAddr, final String topic, f RemotingConnectException, MQClientException { this.defaultMQAdminExtImpl.setMessageRequestMode(brokerAddr, topic, consumerGroup, mode, popShareQueueNum, timeoutMillis); } + + @Override + public void updateTopicLogicalQueueMapping(String brokerAddr, String topic, int queueId, int logicalQueueIndex) throws InterruptedException, RemotingTimeoutException, RemotingSendRequestException, RemotingConnectException, MQBrokerException { + this.defaultMQAdminExtImpl.updateTopicLogicalQueueMapping(brokerAddr, topic, queueId, logicalQueueIndex); + } + + @Override + public LogicalQueuesInfo queryTopicLogicalQueueMapping(String brokerAddr, String topic) throws InterruptedException, RemotingConnectException, RemotingTimeoutException, RemotingSendRequestException, MQBrokerException { + return this.defaultMQAdminExtImpl.queryTopicLogicalQueueMapping(brokerAddr, topic); + } + + @Override + public void deleteTopicLogicalQueueMapping(String brokerAddr, String topic) throws InterruptedException, MQBrokerException, RemotingTimeoutException, RemotingSendRequestException, RemotingConnectException { + this.defaultMQAdminExtImpl.deleteTopicLogicalQueueMapping(brokerAddr, topic); + } + + @Override + public LogicalQueueRouteData sealTopicLogicalQueue(String brokerAddr, LogicalQueueRouteData queueRouteData) throws InterruptedException, RemotingConnectException, RemotingTimeoutException, RemotingSendRequestException, MQBrokerException { + return this.defaultMQAdminExtImpl.sealTopicLogicalQueue(brokerAddr, queueRouteData); + } + + @Override public LogicalQueueRouteData reuseTopicLogicalQueue(String brokerAddr, String topic, int queueId, + int logicalQueueIdx, MessageQueueRouteState messageQueueRouteState) throws InterruptedException, MQBrokerException, RemotingTimeoutException, RemotingSendRequestException, RemotingConnectException { + return this.defaultMQAdminExtImpl.reuseTopicLogicalQueue(brokerAddr, topic, queueId, logicalQueueIdx, messageQueueRouteState); + } + + @Override public LogicalQueueRouteData createMessageQueueForLogicalQueue(String brokerAddr, String topic, + int logicalQueueIdx, MessageQueueRouteState messageQueueStatus) throws InterruptedException, MQBrokerException, RemotingTimeoutException, RemotingSendRequestException, RemotingConnectException { + return this.defaultMQAdminExtImpl.createMessageQueueForLogicalQueue(brokerAddr, topic, logicalQueueIdx, messageQueueStatus); + } + + @Override public MigrateLogicalQueueBody migrateTopicLogicalQueuePrepare( + LogicalQueueRouteData fromQueueRouteData, + LogicalQueueRouteData toQueueRouteData) throws InterruptedException, MQBrokerException, RemotingTimeoutException, RemotingSendRequestException, RemotingConnectException { + return this.defaultMQAdminExtImpl.migrateTopicLogicalQueuePrepare(fromQueueRouteData, toQueueRouteData); + } + + @Override public MigrateLogicalQueueBody migrateTopicLogicalQueueCommit( + LogicalQueueRouteData fromQueueRouteData, + LogicalQueueRouteData toQueueRouteData) throws InterruptedException, MQBrokerException, RemotingTimeoutException, RemotingSendRequestException, RemotingConnectException { + return this.defaultMQAdminExtImpl.migrateTopicLogicalQueueCommit(fromQueueRouteData, toQueueRouteData); + } + + @Override public void migrateTopicLogicalQueueNotify(String brokerAddr, + LogicalQueueRouteData fromQueueRouteData, + LogicalQueueRouteData toQueueRouteData) throws InterruptedException, MQBrokerException, RemotingTimeoutException, RemotingSendRequestException, RemotingConnectException { + this.defaultMQAdminExtImpl.migrateTopicLogicalQueueNotify(brokerAddr, fromQueueRouteData, toQueueRouteData); + } } diff --git a/tools/src/main/java/org/apache/rocketmq/tools/admin/DefaultMQAdminExtImpl.java b/tools/src/main/java/org/apache/rocketmq/tools/admin/DefaultMQAdminExtImpl.java index 8ae68cdecbe..ac936561e96 100644 --- a/tools/src/main/java/org/apache/rocketmq/tools/admin/DefaultMQAdminExtImpl.java +++ b/tools/src/main/java/org/apache/rocketmq/tools/admin/DefaultMQAdminExtImpl.java @@ -66,6 +66,7 @@ import org.apache.rocketmq.common.protocol.body.ConsumerRunningInfo; import org.apache.rocketmq.common.protocol.body.GroupList; import org.apache.rocketmq.common.protocol.body.KVTable; +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.QueueTimeSpan; @@ -75,6 +76,9 @@ import org.apache.rocketmq.common.protocol.header.UpdateConsumerOffsetRequestHeader; import org.apache.rocketmq.common.protocol.heartbeat.SubscriptionData; import org.apache.rocketmq.common.protocol.route.BrokerData; +import org.apache.rocketmq.common.protocol.route.LogicalQueueRouteData; +import org.apache.rocketmq.common.protocol.route.LogicalQueuesInfo; +import org.apache.rocketmq.common.protocol.route.MessageQueueRouteState; import org.apache.rocketmq.common.protocol.route.QueueData; import org.apache.rocketmq.common.protocol.route.TopicRouteData; import org.apache.rocketmq.common.subscription.SubscriptionGroupConfig; @@ -223,8 +227,8 @@ public SubscriptionGroupConfig examineSubscriptionGroupConfig(String addr, Strin } @Override - public TopicConfig examineTopicConfig(String addr, String topic) { - return null; + public TopicConfig examineTopicConfig(String addr, String topic) throws InterruptedException, MQBrokerException, RemotingTimeoutException, RemotingSendRequestException, RemotingConnectException { + return this.mqClientInstance.getMQClientAPIImpl().getTopicConfig(addr, topic, timeoutMillis); } @Override @@ -325,6 +329,54 @@ public MessageExt viewMessage(String topic, return this.mqClientInstance.getMQAdminImpl().queryMessageByUniqKey(topic, msgId); } + @Override + public void updateTopicLogicalQueueMapping(String brokerAddr, String topic, int queueId, int logicalQueueIndex) throws InterruptedException, RemotingTimeoutException, RemotingSendRequestException, RemotingConnectException, MQBrokerException { + this.mqClientInstance.getMQClientAPIImpl().updateTopicLogicalQueue(brokerAddr, topic, queueId, logicalQueueIndex, timeoutMillis); + } + + @Override + public LogicalQueuesInfo queryTopicLogicalQueueMapping(String brokerAddr, String topic) throws InterruptedException, MQBrokerException, RemotingTimeoutException, RemotingSendRequestException, RemotingConnectException { + return this.mqClientInstance.getMQClientAPIImpl().queryTopicLogicalQueue(brokerAddr, topic, timeoutMillis); + } + + @Override + public void deleteTopicLogicalQueueMapping(String brokerAddr, String topic) throws InterruptedException, RemotingConnectException, RemotingTimeoutException, RemotingSendRequestException, MQBrokerException { + this.mqClientInstance.getMQClientAPIImpl().deleteTopicLogicalQueueMapping(brokerAddr, topic, timeoutMillis); + } + + @Override + public LogicalQueueRouteData sealTopicLogicalQueue(String brokerAddr, LogicalQueueRouteData queueRouteData) throws InterruptedException, MQBrokerException, RemotingTimeoutException, RemotingSendRequestException, RemotingConnectException { + return this.mqClientInstance.getMQClientAPIImpl().sealTopicLogicalQueue(brokerAddr, queueRouteData, timeoutMillis); + } + + @Override public LogicalQueueRouteData reuseTopicLogicalQueue(String brokerAddr, String topic, int queueId, + int logicalQueueIdx, MessageQueueRouteState messageQueueRouteState) throws InterruptedException, MQBrokerException, RemotingTimeoutException, RemotingSendRequestException, RemotingConnectException { + return this.mqClientInstance.getMQClientAPIImpl().reuseTopicLogicalQueue(brokerAddr, topic, queueId, logicalQueueIdx, messageQueueRouteState, timeoutMillis); + } + + @Override public LogicalQueueRouteData createMessageQueueForLogicalQueue(String brokerAddr, String topic, + int logicalQueueIdx, MessageQueueRouteState messageQueueStatus) throws InterruptedException, MQBrokerException, RemotingTimeoutException, RemotingSendRequestException, RemotingConnectException { + return this.mqClientInstance.getMQClientAPIImpl().createMessageQueueForLogicalQueue(brokerAddr, topic, logicalQueueIdx, messageQueueStatus, timeoutMillis); + } + + @Override public MigrateLogicalQueueBody migrateTopicLogicalQueuePrepare( + LogicalQueueRouteData fromQueueRouteData, + LogicalQueueRouteData toQueueRouteData) throws InterruptedException, MQBrokerException, RemotingTimeoutException, RemotingSendRequestException, RemotingConnectException { + return this.mqClientInstance.getMQClientAPIImpl().migrateTopicLogicalQueuePrepare(fromQueueRouteData.getBrokerAddr(), fromQueueRouteData, toQueueRouteData, timeoutMillis); + } + + @Override public MigrateLogicalQueueBody migrateTopicLogicalQueueCommit( + LogicalQueueRouteData fromQueueRouteData, + LogicalQueueRouteData toQueueRouteData) throws InterruptedException, MQBrokerException, RemotingTimeoutException, RemotingSendRequestException, RemotingConnectException { + return this.mqClientInstance.getMQClientAPIImpl().migrateTopicLogicalQueueCommit(toQueueRouteData.getBrokerAddr(), fromQueueRouteData, toQueueRouteData, timeoutMillis); + } + + @Override public void migrateTopicLogicalQueueNotify(String brokerAddr, + LogicalQueueRouteData fromQueueRouteData, + LogicalQueueRouteData toQueueRouteData) throws InterruptedException, RemotingConnectException, RemotingTimeoutException, RemotingSendRequestException, MQBrokerException { + this.mqClientInstance.getMQClientAPIImpl().migrateTopicLogicalQueueNotify(brokerAddr, fromQueueRouteData, toQueueRouteData, timeoutMillis); + } + @Override public ConsumerConnection examineConsumerConnectionInfo( String consumerGroup) throws InterruptedException, MQBrokerException, @@ -982,6 +1034,10 @@ public long maxOffset(MessageQueue mq) throws MQClientException { return this.mqClientInstance.getMQAdminImpl().maxOffset(mq); } + public long maxOffset(MessageQueue mq, boolean committed) throws MQClientException { + return this.mqClientInstance.getMQAdminImpl().maxOffset(mq, committed); + } + @Override public long minOffset(MessageQueue mq) throws MQClientException { return this.mqClientInstance.getMQAdminImpl().minOffset(mq); diff --git a/tools/src/main/java/org/apache/rocketmq/tools/admin/MQAdminExt.java b/tools/src/main/java/org/apache/rocketmq/tools/admin/MQAdminExt.java index 5ce6db18997..c1b42f5aeb4 100644 --- a/tools/src/main/java/org/apache/rocketmq/tools/admin/MQAdminExt.java +++ b/tools/src/main/java/org/apache/rocketmq/tools/admin/MQAdminExt.java @@ -42,6 +42,7 @@ import org.apache.rocketmq.common.protocol.body.ConsumerRunningInfo; import org.apache.rocketmq.common.protocol.body.GroupList; import org.apache.rocketmq.common.protocol.body.KVTable; +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.QueueTimeSpan; @@ -49,6 +50,9 @@ import org.apache.rocketmq.common.protocol.body.TopicConfigSerializeWrapper; import org.apache.rocketmq.common.protocol.body.TopicList; import org.apache.rocketmq.common.protocol.route.TopicRouteData; +import org.apache.rocketmq.common.protocol.route.LogicalQueueRouteData; +import org.apache.rocketmq.common.protocol.route.LogicalQueuesInfo; +import org.apache.rocketmq.common.protocol.route.MessageQueueRouteState; import org.apache.rocketmq.common.subscription.SubscriptionGroupConfig; import org.apache.rocketmq.remoting.exception.RemotingCommandException; import org.apache.rocketmq.remoting.exception.RemotingConnectException; @@ -96,7 +100,8 @@ void createAndUpdateSubscriptionGroupConfig(final String addr, SubscriptionGroupConfig examineSubscriptionGroupConfig(final String addr, final String group); - TopicConfig examineTopicConfig(final String addr, final String topic); + TopicConfig examineTopicConfig(final String addr, + final String topic) throws InterruptedException, MQBrokerException, RemotingTimeoutException, RemotingSendRequestException, RemotingConnectException; TopicStatsTable examineTopicStats( final String topic) throws RemotingException, MQClientException, InterruptedException, @@ -295,4 +300,28 @@ void setMessageRequestMode(final String brokerAddr, final String topic, final St MessageRequestMode mode, final int popWorkGroupSize, final long timeoutMillis) throws InterruptedException, RemotingTimeoutException, RemotingSendRequestException, RemotingConnectException, MQClientException; + + void updateTopicLogicalQueueMapping(String brokerAddr, String topic, int queueId, int logicalQueueIndex) throws InterruptedException, RemotingTimeoutException, RemotingSendRequestException, RemotingConnectException, MQBrokerException; + + LogicalQueuesInfo queryTopicLogicalQueueMapping(String brokerAddr, String topic) throws InterruptedException, MQBrokerException, RemotingTimeoutException, RemotingSendRequestException, RemotingConnectException; + + void deleteTopicLogicalQueueMapping(String brokerAddr, String topic) throws InterruptedException, RemotingConnectException, RemotingTimeoutException, RemotingSendRequestException, MQBrokerException; + + LogicalQueueRouteData sealTopicLogicalQueue(String brokerAddr, LogicalQueueRouteData queueRouteData) throws InterruptedException, MQBrokerException, RemotingTimeoutException, RemotingSendRequestException, RemotingConnectException; + + LogicalQueueRouteData reuseTopicLogicalQueue(String brokerAddr, String topic, int queueId, int logicalQueueIdx, + MessageQueueRouteState messageQueueRouteState) throws InterruptedException, MQBrokerException, RemotingTimeoutException, RemotingSendRequestException, RemotingConnectException; + + LogicalQueueRouteData createMessageQueueForLogicalQueue(String brokerAddr, String topic, int logicalQueueIdx, + MessageQueueRouteState messageQueueStatus) throws InterruptedException, MQBrokerException, RemotingTimeoutException, RemotingSendRequestException, RemotingConnectException; + + MigrateLogicalQueueBody migrateTopicLogicalQueuePrepare(LogicalQueueRouteData fromQueueRouteData, LogicalQueueRouteData toQueueRouteData) throws InterruptedException, MQBrokerException, RemotingTimeoutException, RemotingSendRequestException, RemotingConnectException; + + MigrateLogicalQueueBody migrateTopicLogicalQueueCommit( + LogicalQueueRouteData fromQueueRouteData, + LogicalQueueRouteData toQueueRouteData) throws InterruptedException, MQBrokerException, RemotingTimeoutException, RemotingSendRequestException, RemotingConnectException; + + void migrateTopicLogicalQueueNotify(String brokerAddr, + LogicalQueueRouteData fromQueueRouteData, + LogicalQueueRouteData toQueueRouteData) throws InterruptedException, RemotingConnectException, RemotingTimeoutException, RemotingSendRequestException, MQBrokerException; } diff --git a/tools/src/main/java/org/apache/rocketmq/tools/command/MQAdminStartup.java b/tools/src/main/java/org/apache/rocketmq/tools/command/MQAdminStartup.java index 6e075f79595..3e989391f80 100644 --- a/tools/src/main/java/org/apache/rocketmq/tools/command/MQAdminStartup.java +++ b/tools/src/main/java/org/apache/rocketmq/tools/command/MQAdminStartup.java @@ -52,6 +52,11 @@ import org.apache.rocketmq.tools.command.consumer.SetConsumeModeSubCommand; import org.apache.rocketmq.tools.command.consumer.StartMonitoringSubCommand; import org.apache.rocketmq.tools.command.consumer.UpdateSubGroupSubCommand; +import org.apache.rocketmq.tools.command.logicalqueue.DeleteTopicLogicalQueueMappingCommand; +import org.apache.rocketmq.tools.command.logicalqueue.MigrateTopicLogicalQueueCommand; +import org.apache.rocketmq.tools.command.logicalqueue.QueryTopicLogicalQueueMappingCommand; +import org.apache.rocketmq.tools.command.logicalqueue.UpdateTopicLogicalQueueMappingCommand; +import org.apache.rocketmq.tools.command.logicalqueue.UpdateTopicLogicalQueueNumCommand; import org.apache.rocketmq.tools.command.message.CheckMsgSendRTCommand; import org.apache.rocketmq.tools.command.message.ConsumeMessageCommand; import org.apache.rocketmq.tools.command.message.PrintMessageByQueueCommand; @@ -215,6 +220,12 @@ public static void initCommand() { initCommand(new ClusterAclConfigVersionListSubCommand()); initCommand(new UpdateGlobalWhiteAddrSubCommand()); initCommand(new GetAccessConfigSubCommand()); + + initCommand(new UpdateTopicLogicalQueueMappingCommand()); + initCommand(new DeleteTopicLogicalQueueMappingCommand()); + initCommand(new QueryTopicLogicalQueueMappingCommand()); + initCommand(new MigrateTopicLogicalQueueCommand()); + initCommand(new UpdateTopicLogicalQueueNumCommand()); } private static void initLogback() throws JoranException { diff --git a/tools/src/main/java/org/apache/rocketmq/tools/command/SubCommandException.java b/tools/src/main/java/org/apache/rocketmq/tools/command/SubCommandException.java index fadd8532ef1..d907059c03a 100644 --- a/tools/src/main/java/org/apache/rocketmq/tools/command/SubCommandException.java +++ b/tools/src/main/java/org/apache/rocketmq/tools/command/SubCommandException.java @@ -26,6 +26,10 @@ public SubCommandException(String msg) { super(msg); } + public SubCommandException(String format, Object... args) { + super(String.format(format, args)); + } + /** * @param msg Message. * @param cause Cause. diff --git a/tools/src/main/java/org/apache/rocketmq/tools/command/logicalqueue/DeleteTopicLogicalQueueMappingCommand.java b/tools/src/main/java/org/apache/rocketmq/tools/command/logicalqueue/DeleteTopicLogicalQueueMappingCommand.java new file mode 100644 index 00000000000..77eb918b0ee --- /dev/null +++ b/tools/src/main/java/org/apache/rocketmq/tools/command/logicalqueue/DeleteTopicLogicalQueueMappingCommand.java @@ -0,0 +1,91 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package org.apache.rocketmq.tools.command.logicalqueue; + +import java.util.Collections; +import java.util.List; +import java.util.stream.Collectors; +import org.apache.commons.cli.CommandLine; +import org.apache.commons.cli.Option; +import org.apache.commons.cli.Options; +import org.apache.rocketmq.common.constant.LoggerName; +import org.apache.rocketmq.remoting.RPCHook; +import org.apache.rocketmq.srvutil.ServerUtil; +import org.apache.rocketmq.tools.admin.DefaultMQAdminExt; +import org.apache.rocketmq.tools.command.CommandUtil; +import org.apache.rocketmq.tools.command.SubCommand; +import org.apache.rocketmq.tools.command.SubCommandException; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +public class DeleteTopicLogicalQueueMappingCommand implements SubCommand { + private static final Logger log = LoggerFactory.getLogger(LoggerName.STDOUT_LOGGER_NAME); + + @Override public String commandName() { + return "deleteTopicLogicalQueueMapping"; + } + + @Override public String commandDesc() { + return "delete logical queue mapping info of a topic"; + } + + @Override public Options buildCommandlineOptions(Options options) { + Option opt; + + opt = new Option("t", "topic", true, "topic name."); + opt.setRequired(true); + options.addOption(opt); + + opt = new Option("c", "clusterName", true, "cluster name."); + opt.setRequired(false); + options.addOption(opt); + + opt = new Option("b", "brokerAddr", true, "broker addr."); + opt.setRequired(false); + options.addOption(opt); + + return options; + } + + @Override public void execute(CommandLine commandLine, Options options, + RPCHook rpcHook) throws SubCommandException { + DefaultMQAdminExt defaultMQAdminExt = new DefaultMQAdminExt(rpcHook); + defaultMQAdminExt.setInstanceName(Long.toString(System.currentTimeMillis())); + + try { + String topic = commandLine.getOptionValue("t").trim(); + List brokerAddrs; + if (commandLine.hasOption("b")) { + brokerAddrs = Collections.singletonList(commandLine.getOptionValue("c").trim()); + } else if (commandLine.hasOption("c")) { + String clusterName = commandLine.getOptionValue("c").trim(); + brokerAddrs = CommandUtil.fetchMasterAddrByClusterName(defaultMQAdminExt, clusterName).stream().sorted().collect(Collectors.toList()); + } else { + ServerUtil.printCommandLineHelp("mqadmin " + this.commandName(), options); + return; + } + for (String brokerAddr : brokerAddrs) { + log.info("deleteTopicLogicalQueueMapping {} {}", brokerAddr, topic); + defaultMQAdminExt.deleteTopicLogicalQueueMapping(brokerAddr, topic); + } + } catch (Exception e) { + throw new SubCommandException(this.getClass().getSimpleName() + " command failed", e); + } finally { + defaultMQAdminExt.shutdown(); + } + } +} diff --git a/tools/src/main/java/org/apache/rocketmq/tools/command/logicalqueue/MigrateTopicLogicalQueueCommand.java b/tools/src/main/java/org/apache/rocketmq/tools/command/logicalqueue/MigrateTopicLogicalQueueCommand.java new file mode 100644 index 00000000000..5da8b0d4808 --- /dev/null +++ b/tools/src/main/java/org/apache/rocketmq/tools/command/logicalqueue/MigrateTopicLogicalQueueCommand.java @@ -0,0 +1,210 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package org.apache.rocketmq.tools.command.logicalqueue; + +import com.google.common.collect.Multimap; +import com.google.common.collect.Multimaps; +import java.util.Arrays; +import java.util.Collection; +import java.util.Collections; +import java.util.List; +import java.util.Map; +import java.util.Objects; +import java.util.Set; +import java.util.stream.Collectors; +import org.apache.commons.cli.CommandLine; +import org.apache.commons.cli.Option; +import org.apache.commons.cli.Options; +import org.apache.rocketmq.client.exception.MQBrokerException; +import org.apache.rocketmq.client.exception.MQClientException; +import org.apache.rocketmq.common.TopicConfig; +import org.apache.rocketmq.common.constant.LoggerName; +import org.apache.rocketmq.common.protocol.body.MigrateLogicalQueueBody; +import org.apache.rocketmq.common.protocol.route.BrokerData; +import org.apache.rocketmq.common.protocol.route.LogicalQueueRouteData; +import org.apache.rocketmq.common.protocol.route.LogicalQueuesInfo; +import org.apache.rocketmq.common.protocol.route.MessageQueueRouteState; +import org.apache.rocketmq.common.protocol.route.TopicRouteData; +import org.apache.rocketmq.remoting.RPCHook; +import org.apache.rocketmq.remoting.exception.RemotingConnectException; +import org.apache.rocketmq.remoting.exception.RemotingException; +import org.apache.rocketmq.tools.admin.DefaultMQAdminExt; +import org.apache.rocketmq.tools.command.SubCommand; +import org.apache.rocketmq.tools.command.SubCommandException; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +import static java.util.Optional.ofNullable; + +public class MigrateTopicLogicalQueueCommand implements SubCommand { + private static final Logger log = LoggerFactory.getLogger(LoggerName.STDOUT_LOGGER_NAME); + + @Override public String commandName() { + return "migrateTopicLogicalQueue"; + } + + @Override public String commandDesc() { + return "migrate a logical queue of a topic from one broker to another."; + } + + @Override public Options buildCommandlineOptions(Options options) { + Option opt; + + opt = new Option("t", "topic", true, "topic name."); + opt.setRequired(true); + options.addOption(opt); + + opt = new Option("i", "index", true, "logical queue index."); + opt.setRequired(true); + options.addOption(opt); + + opt = new Option("b", "brokerAddr", true, "new broker name"); + opt.setRequired(true); + options.addOption(opt); + + opt = new Option("fd", "forceDelta", true, "assume fromBroker down, force migrate"); + opt.setRequired(false); + options.addOption(opt); + + return options; + } + + public void execute(DefaultMQAdminExt mqAdminExt, String topic, int logicalQueueIdx, + String toBrokerName, + Long forceDelta) throws RemotingException, MQBrokerException, InterruptedException, SubCommandException, MQClientException { + TopicRouteData topicRouteInfo = mqAdminExt.examineTopicRouteInfo(topic); + LogicalQueuesInfo logicalQueuesInfo = topicRouteInfo.getLogicalQueuesInfo(); + if (logicalQueuesInfo == null) { + throw new SubCommandException("topic not enabled logical queue"); + } + List queueRouteDataList = logicalQueuesInfo.get(logicalQueueIdx); + if (queueRouteDataList == null) { + throw new SubCommandException("logical queue %d not exist", logicalQueueIdx); + } + Map brokerAddrTable = mqAdminExt.examineBrokerClusterInfo().getBrokerAddrTable(); + String toBrokerAddr = lookupBrokerMasterAddr(brokerAddrTable, toBrokerName); + if (toBrokerAddr == null) { + throw new SubCommandException("destination broker %s not found", toBrokerName); + } + LogicalQueueRouteData fromQueueRouteData = queueRouteDataList.stream().filter(LogicalQueueRouteData::isWritable).reduce((first, second) -> second).orElse(null); + if (fromQueueRouteData == null) { + throw new SubCommandException("logical queue %d not writable, can not migrate", logicalQueueIdx); + } + String fromBrokerName = fromQueueRouteData.getBrokerName(); + String fromBrokerAddr = ofNullable(lookupBrokerMasterAddr(brokerAddrTable, fromBrokerName)).orElse(fromQueueRouteData.getBrokerAddr()); + if (fromBrokerAddr == null) { + throw new SubCommandException("unexpected source broker name %s not found", fromBrokerName); + } + LogicalQueueRouteData toQueueRouteData; + RETRY: + while (true) { + TopicConfig topicConfig = mqAdminExt.examineTopicConfig(toBrokerAddr, topic); + LogicalQueuesInfo logicalQueuesInfoInBroker = ofNullable(mqAdminExt.queryTopicLogicalQueueMapping(toBrokerAddr, topic)).orElse(new LogicalQueuesInfo()); + toQueueRouteData = logicalQueuesInfoInBroker.getOrDefault(logicalQueueIdx, Collections.emptyList()).stream().filter(queueRouteData -> Objects.equals(toBrokerName, queueRouteData.getBrokerName()) && queueRouteData.isWriteOnly()).findFirst().orElse(null); + if (toQueueRouteData == null) { + Multimap m = Multimaps.index(logicalQueuesInfoInBroker.values().stream().flatMap(Collection::stream).filter(queueRouteData -> Objects.equals(toBrokerName, queueRouteData.getBrokerName())).iterator(), LogicalQueueRouteData::getQueueId); + for (int queueId = 0, writeQueueNums = topicConfig.getWriteQueueNums(); queueId < writeQueueNums; queueId++) { + if (m.get(queueId).stream().anyMatch(LogicalQueueRouteData::isWritable)) { + continue; + } + try { + toQueueRouteData = mqAdminExt.reuseTopicLogicalQueue(toBrokerAddr, topic, queueId, logicalQueueIdx, MessageQueueRouteState.WriteOnly); + log.info("reuseTopicLogicalQueue brokerName={} brokerAddr={} queueId={} logicalQueueIdx={} ok: {}", toBrokerName, toBrokerAddr, queueId, logicalQueueIdx, toQueueRouteData); + break; + } catch (MQBrokerException e) { + if ("queue writable".equals(e.getErrorMessage())) { + log.info("reuseTopicLogicalQueue brokerName={} brokerAddr={} queueId={} logicalQueueIdx={} writable, try again.", toBrokerName, toBrokerAddr, queueId, logicalQueueIdx); + continue RETRY; + } else { + throw e; + } + } + } + } + break; + } + if (toQueueRouteData == null) { + toQueueRouteData = mqAdminExt.createMessageQueueForLogicalQueue(toBrokerAddr, topic, logicalQueueIdx, MessageQueueRouteState.WriteOnly); + log.info("createMessageQueueForLogicalQueue brokerName={} brokerAddr={} logicalQueueIdx={} ok: {}", toBrokerName, toBrokerAddr, logicalQueueIdx, toQueueRouteData); + } + MigrateLogicalQueueBody migrateLogicalQueueBody; + if (forceDelta == null) { + try { + migrateLogicalQueueBody = mqAdminExt.migrateTopicLogicalQueuePrepare(fromQueueRouteData, toQueueRouteData); + } catch (RemotingConnectException e) { + throw new SubCommandException("migrateTopicLogicalQueuePrepare", e); + } + fromQueueRouteData = migrateLogicalQueueBody.getFromQueueRouteData(); + toQueueRouteData = migrateLogicalQueueBody.getToQueueRouteData(); + log.info("migrateTopicLogicalQueuePrepare from {} to {}", fromQueueRouteData, toQueueRouteData); + } else { + toQueueRouteData.setLogicalQueueDelta(forceDelta); + log.warn("migrateTopicLogicalQueuePrepare skip with forceDelta={}", forceDelta); + } + migrateLogicalQueueBody = mqAdminExt.migrateTopicLogicalQueueCommit(fromQueueRouteData, toQueueRouteData); + toQueueRouteData = migrateLogicalQueueBody.getToQueueRouteData(); + log.info("migrateTopicLogicalQueueCommit got: {}", toQueueRouteData); + if (forceDelta == null) { + try { + mqAdminExt.migrateTopicLogicalQueueNotify(fromBrokerAddr, fromQueueRouteData, toQueueRouteData); + } finally { + log.info("migrateTopicLogicalQueueNotify fromBroker {} {}", fromQueueRouteData.getBrokerName(), fromBrokerAddr); + } + } + Collection ignoreBrokerNames = Arrays.asList(fromBrokerName, toBrokerName); + Set brokerNames = queueRouteDataList.stream() + .map(LogicalQueueRouteData::getBrokerName) + .filter(v -> !ignoreBrokerNames.contains(v)) + .map(v -> lookupBrokerMasterAddr(brokerAddrTable, v)) + .collect(Collectors.toSet()); + int i = 1; + for (String brokerName : brokerNames) { + String brokerAddr = null; + try { + brokerAddr = lookupBrokerMasterAddr(brokerAddrTable, brokerName); + mqAdminExt.migrateTopicLogicalQueueNotify(brokerAddr, fromQueueRouteData, toQueueRouteData); + } finally { + log.info("migrateTopicLogicalQueueNotify otherBroker {}({}}) ({}/{})", brokerName, brokerAddr, i, brokerNames.size()); + } + } + } + + @Override public void execute(CommandLine commandLine, Options options, + RPCHook rpcHook) throws SubCommandException { + DefaultMQAdminExt mqAdminExt = new DefaultMQAdminExt(rpcHook); + mqAdminExt.setInstanceName(Long.toString(System.currentTimeMillis())); + + try { + String topic = commandLine.getOptionValue("t").trim(); + String newBrokerName = commandLine.getOptionValue("b").trim(); + int logicalQueueIdx = Integer.parseInt(commandLine.getOptionValue("i").trim()); + Long forceDelta = null; + if (commandLine.hasOption("fd")) { + forceDelta = Long.parseLong(commandLine.getOptionValue("fd").trim()); + } + execute(mqAdminExt, topic, logicalQueueIdx, newBrokerName, forceDelta); + } catch (Exception e) { + throw new SubCommandException(this.getClass().getSimpleName() + " command failed", e); + } finally { + mqAdminExt.shutdown(); + } + } + + private static String lookupBrokerMasterAddr(Map brokerAddrTable, String brokerName) { + return ofNullable(brokerAddrTable.get(brokerName)).map(BrokerData::selectBrokerAddr).orElse(null); + } +} diff --git a/tools/src/main/java/org/apache/rocketmq/tools/command/logicalqueue/QueryTopicLogicalQueueMappingCommand.java b/tools/src/main/java/org/apache/rocketmq/tools/command/logicalqueue/QueryTopicLogicalQueueMappingCommand.java new file mode 100644 index 00000000000..6ce4bac35bc --- /dev/null +++ b/tools/src/main/java/org/apache/rocketmq/tools/command/logicalqueue/QueryTopicLogicalQueueMappingCommand.java @@ -0,0 +1,123 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package org.apache.rocketmq.tools.command.logicalqueue; + +import com.alibaba.fastjson.JSON; +import com.google.common.collect.ImmutableList; +import com.google.common.collect.ImmutableSortedMap; +import com.google.common.collect.Lists; +import com.google.common.collect.Maps; +import java.util.Collection; +import java.util.Collections; +import java.util.Comparator; +import java.util.List; +import java.util.SortedMap; +import java.util.stream.Collectors; +import org.apache.commons.cli.CommandLine; +import org.apache.commons.cli.Option; +import org.apache.commons.cli.Options; +import org.apache.rocketmq.common.constant.LoggerName; +import org.apache.rocketmq.common.protocol.route.LogicalQueueRouteData; +import org.apache.rocketmq.common.protocol.route.LogicalQueuesInfo; +import org.apache.rocketmq.remoting.RPCHook; +import org.apache.rocketmq.srvutil.ServerUtil; +import org.apache.rocketmq.tools.admin.DefaultMQAdminExt; +import org.apache.rocketmq.tools.command.CommandUtil; +import org.apache.rocketmq.tools.command.SubCommand; +import org.apache.rocketmq.tools.command.SubCommandException; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +public class QueryTopicLogicalQueueMappingCommand implements SubCommand { + private static final Logger log = LoggerFactory.getLogger(LoggerName.STDOUT_LOGGER_NAME); + + @Override public String commandName() { + return "queryTopicLogicalQueueMapping"; + } + + @Override public String commandDesc() { + return "query logical queue mapping info of a topic"; + } + + @Override public Options buildCommandlineOptions(Options options) { + Option opt; + + opt = new Option("t", "topic", true, "topic name."); + opt.setRequired(true); + options.addOption(opt); + + opt = new Option("b", "brokerAddr", true, "broker address."); + opt.setRequired(false); + options.addOption(opt); + + opt = new Option("c", "clusterName", true, "cluster name."); + opt.setRequired(false); + options.addOption(opt); + + opt = new Option("m", "merge", false, "merge all brokers' result into one."); + opt.setRequired(false); + options.addOption(opt); + + return options; + } + + @Override public void execute(CommandLine commandLine, Options options, + RPCHook rpcHook) throws SubCommandException { + DefaultMQAdminExt defaultMQAdminExt = new DefaultMQAdminExt(rpcHook); + defaultMQAdminExt.setInstanceName(Long.toString(System.currentTimeMillis())); + + try { + String topic = commandLine.getOptionValue("t").trim(); + List brokerAddrs; + if (commandLine.hasOption("b")) { + brokerAddrs = Collections.singletonList(commandLine.getOptionValue("c").trim()); + } else if (commandLine.hasOption("c")) { + String clusterName = commandLine.getOptionValue("c").trim(); + brokerAddrs = CommandUtil.fetchMasterAddrByClusterName(defaultMQAdminExt, clusterName).stream().sorted().collect(Collectors.toList()); + } else { + ServerUtil.printCommandLineHelp("mqadmin " + this.commandName(), options); + return; + } + SortedMap>> result = Maps.newTreeMap(); + for (String brokerAddr : brokerAddrs) { + LogicalQueuesInfo one = defaultMQAdminExt.queryTopicLogicalQueueMapping(brokerAddr, topic); + result.put(brokerAddr, ImmutableSortedMap.copyOf(one)); + } + if (commandLine.hasOption("m")) { + SortedMap> mergedResultMap = Maps.newTreeMap(); + result.values().stream().flatMap(map -> map.values().stream()).flatMap(Collection::stream).forEach(queueRouteData -> { + List sortedQueueRouteDataList = mergedResultMap.computeIfAbsent(queueRouteData.getLogicalQueueIndex(), ignore -> Lists.newArrayList()); + int idx = Collections.binarySearch(sortedQueueRouteDataList, queueRouteData, + Comparator.comparingLong(LogicalQueueRouteData::getLogicalQueueDelta) + .thenComparing(LogicalQueueRouteData::getMessageQueue) + .thenComparingInt(LogicalQueueRouteData::getStateOrdinal)); + if (idx < 0) { + idx = -idx - 1; + } + sortedQueueRouteDataList.add(idx, queueRouteData); + }); + System.out.printf("%s%n", JSON.toJSONString(ImmutableList.copyOf(mergedResultMap.values()))); + } else { + System.out.printf("%s%n", JSON.toJSONString(result)); + } + } catch (Exception e) { + throw new SubCommandException(this.getClass().getSimpleName() + " command failed", e); + } finally { + defaultMQAdminExt.shutdown(); + } + } +} diff --git a/tools/src/main/java/org/apache/rocketmq/tools/command/logicalqueue/UpdateTopicLogicalQueueMappingCommand.java b/tools/src/main/java/org/apache/rocketmq/tools/command/logicalqueue/UpdateTopicLogicalQueueMappingCommand.java new file mode 100644 index 00000000000..46276a31cde --- /dev/null +++ b/tools/src/main/java/org/apache/rocketmq/tools/command/logicalqueue/UpdateTopicLogicalQueueMappingCommand.java @@ -0,0 +1,159 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package org.apache.rocketmq.tools.command.logicalqueue; + +import com.google.common.collect.Maps; +import java.util.BitSet; +import java.util.Collection; +import java.util.List; +import java.util.Map; +import java.util.Optional; +import java.util.stream.Collectors; +import org.apache.commons.cli.CommandLine; +import org.apache.commons.cli.Option; +import org.apache.commons.cli.Options; +import org.apache.rocketmq.client.exception.MQBrokerException; +import org.apache.rocketmq.common.TopicConfig; +import org.apache.rocketmq.common.constant.LoggerName; +import org.apache.rocketmq.common.protocol.route.LogicalQueueRouteData; +import org.apache.rocketmq.remoting.RPCHook; +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.srvutil.ServerUtil; +import org.apache.rocketmq.tools.admin.DefaultMQAdminExt; +import org.apache.rocketmq.tools.command.CommandUtil; +import org.apache.rocketmq.tools.command.SubCommand; +import org.apache.rocketmq.tools.command.SubCommandException; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +public class UpdateTopicLogicalQueueMappingCommand implements SubCommand { + private static final Logger log = LoggerFactory.getLogger(LoggerName.STDOUT_LOGGER_NAME); + + @Override public String commandName() { + return "updateTopicLogicalQueueMapping"; + } + + @Override public String commandDesc() { + return "update logical queue mapping info of a topic"; + } + + @Override public Options buildCommandlineOptions(Options options) { + Option opt; + + opt = new Option("t", "topic", true, "topic name."); + opt.setRequired(true); + options.addOption(opt); + + opt = new Option("q", "queue", true, "message queue id."); + opt.setRequired(false); + options.addOption(opt); + + opt = new Option("i", "index", true, "logical queue index."); + opt.setRequired(false); + options.addOption(opt); + + opt = new Option("b", "broker", true, "broker addr."); + opt.setRequired(false); + options.addOption(opt); + + opt = new Option("c", "clusterName", true, "cluster name."); + opt.setRequired(false); + options.addOption(opt); + + return options; + } + + public void execute(DefaultMQAdminExt defaultMQAdminExt, String topic, Collection brokerAddrs) throws InterruptedException, MQBrokerException, RemotingTimeoutException, RemotingSendRequestException, RemotingConnectException { + Map topicConfigMap = Maps.newHashMapWithExpectedSize(brokerAddrs.size()); + Map allocatedMessageQueueMap = Maps.newHashMap(); + BitSet allocatedLogicalQueueIndices = new BitSet(); + brokerAddrs = brokerAddrs.stream().sorted().collect(Collectors.toList()); + for (String brokerAddr : brokerAddrs) { + TopicConfig topicConfig = defaultMQAdminExt.examineTopicConfig(brokerAddr, topic); + if (topicConfig == null) { + log.warn("examineTopicConfig brokerAddr={} topic={} not exist, skip!", brokerAddr, topic); + continue; + } + topicConfigMap.put(brokerAddr, topicConfig); + + BitSet allocatedQueueIds = new BitSet(); + Optional.ofNullable(defaultMQAdminExt.queryTopicLogicalQueueMapping(brokerAddr, topic)) + .ifPresent(queueRouteData -> queueRouteData.forEach((idx, value) -> { + allocatedLogicalQueueIndices.set(idx); + value.stream().mapToInt(LogicalQueueRouteData::getQueueId).forEach(allocatedQueueIds::set); + })); + allocatedMessageQueueMap.put(brokerAddr, allocatedQueueIds); + } + int unallocatedLogicalQueueIdx = -1; + for (Map.Entry entry : topicConfigMap.entrySet()) { + String brokerAddr = entry.getKey(); + TopicConfig topicConfig = entry.getValue(); + int queueNums = Integer.max(topicConfig.getReadQueueNums(), topicConfig.getWriteQueueNums()); + BitSet allocatedQueueIds = allocatedMessageQueueMap.get(brokerAddr); + for (int unallocatedQueueId = allocatedQueueIds.nextClearBit(0); unallocatedQueueId < queueNums; unallocatedQueueId = allocatedQueueIds.nextClearBit(unallocatedQueueId + 1)) { + unallocatedLogicalQueueIdx = allocatedLogicalQueueIndices.nextClearBit(unallocatedLogicalQueueIdx + 1); + log.info("updateTopicLogicalQueueMapping brokerAddr={} topic={} queueId={} to {}", brokerAddr, topic, unallocatedQueueId, unallocatedLogicalQueueIdx); + defaultMQAdminExt.updateTopicLogicalQueueMapping(brokerAddr, topic, unallocatedQueueId, unallocatedLogicalQueueIdx); + allocatedQueueIds.set(unallocatedQueueId); + allocatedLogicalQueueIndices.set(unallocatedLogicalQueueIdx); + } + } + } + + @Override public void execute(CommandLine commandLine, Options options, + RPCHook rpcHook) throws SubCommandException { + DefaultMQAdminExt defaultMQAdminExt = new DefaultMQAdminExt(rpcHook); + defaultMQAdminExt.setInstanceName(Long.toString(System.currentTimeMillis())); + + try { + defaultMQAdminExt.start(); + String topic = commandLine.getOptionValue("t").trim(); + List brokerAddrs; + if (commandLine.hasOption("b")) { + String brokerAddr = commandLine.getOptionValue("b").trim(); + boolean hasQueueId = commandLine.hasOption("q"); + boolean hasLogicalQueueIndex = commandLine.hasOption("i"); + if (hasQueueId && hasLogicalQueueIndex) { + int queueId = Integer.parseInt(commandLine.getOptionValue("q").trim()); + int logicalQueueIndex = Integer.parseInt(commandLine.getOptionValue("i").trim()); + defaultMQAdminExt.updateTopicLogicalQueueMapping(brokerAddr, topic, queueId, logicalQueueIndex); + log.info("updateTopicLogicalQueueMapping brokerAddr={} topic={} queueId={} to {}", brokerAddr, topic, queueId, logicalQueueIndex); + return; + } else if (hasQueueId || hasLogicalQueueIndex) { + log.error("logicalQueueIndex and queueId must be specified together."); + return; + } else { + log.error("brokerAddr specified but no logicalQueueIndex and queueId found"); + return; + } + } else if (commandLine.hasOption("c")) { + String clusterName = commandLine.getOptionValue("c").trim(); + brokerAddrs = CommandUtil.fetchMasterAddrByClusterName(defaultMQAdminExt, clusterName).stream().sorted().collect(Collectors.toList()); + } else { + ServerUtil.printCommandLineHelp("mqadmin " + this.commandName(), options); + return; + } + this.execute(defaultMQAdminExt, topic, brokerAddrs); + } catch (Exception e) { + throw new SubCommandException(this.getClass().getSimpleName() + " command failed", e); + } finally { + defaultMQAdminExt.shutdown(); + } + } +} diff --git a/tools/src/main/java/org/apache/rocketmq/tools/command/logicalqueue/UpdateTopicLogicalQueueNumCommand.java b/tools/src/main/java/org/apache/rocketmq/tools/command/logicalqueue/UpdateTopicLogicalQueueNumCommand.java new file mode 100644 index 00000000000..6b9dd2fdf82 --- /dev/null +++ b/tools/src/main/java/org/apache/rocketmq/tools/command/logicalqueue/UpdateTopicLogicalQueueNumCommand.java @@ -0,0 +1,285 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package org.apache.rocketmq.tools.command.logicalqueue; + +import com.google.common.collect.ListMultimap; +import com.google.common.collect.Lists; +import com.google.common.collect.Maps; +import com.google.common.collect.Multimaps; +import java.util.Collection; +import java.util.Collections; +import java.util.Comparator; +import java.util.Iterator; +import java.util.List; +import java.util.Locale; +import java.util.Map; +import java.util.NavigableMap; +import java.util.Objects; +import java.util.stream.Collectors; +import java.util.stream.IntStream; +import org.apache.commons.cli.CommandLine; +import org.apache.commons.cli.Option; +import org.apache.commons.cli.Options; +import org.apache.rocketmq.client.exception.MQBrokerException; +import org.apache.rocketmq.common.TopicConfig; +import org.apache.rocketmq.common.constant.LoggerName; +import org.apache.rocketmq.common.protocol.route.LogicalQueueRouteData; +import org.apache.rocketmq.common.protocol.route.LogicalQueuesInfo; +import org.apache.rocketmq.common.protocol.route.MessageQueueRouteState; +import org.apache.rocketmq.remoting.RPCHook; +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.tools.admin.DefaultMQAdminExt; +import org.apache.rocketmq.tools.command.CommandUtil; +import org.apache.rocketmq.tools.command.SubCommand; +import org.apache.rocketmq.tools.command.SubCommandException; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +public class UpdateTopicLogicalQueueNumCommand implements SubCommand { + private static final Logger log = LoggerFactory.getLogger(LoggerName.STDOUT_LOGGER_NAME); + + @Override public String commandName() { + return "updateTopicLogicalQueueNum"; + } + + @Override public String commandDesc() { + return "change logical queue num (increase or decrease) of a topic."; + } + + @Override public Options buildCommandlineOptions(Options options) { + Option opt; + + opt = new Option("t", "topic", true, "topic name."); + opt.setRequired(true); + options.addOption(opt); + + opt = new Option("c", "clusterName", true, "cluster name."); + opt.setRequired(true); + options.addOption(opt); + + opt = new Option("n", "num", true, "logical queue num."); + opt.setRequired(true); + options.addOption(opt); + + return options; + } + + @Override public void execute(CommandLine commandLine, Options options, + RPCHook rpcHook) throws SubCommandException { + DefaultMQAdminExt defaultMQAdminExt = new DefaultMQAdminExt(rpcHook); + defaultMQAdminExt.setInstanceName(Long.toString(System.currentTimeMillis())); + try { + defaultMQAdminExt.start(); + String clusterName = commandLine.getOptionValue("c").trim(); + String topic = commandLine.getOptionValue("t").trim(); + int newLogicalQueueNum = Integer.parseUnsignedInt(commandLine.getOptionValue("n")); + execute(defaultMQAdminExt, clusterName, topic, newLogicalQueueNum); + } catch (Exception e) { + throw new SubCommandException(this.getClass().getSimpleName() + " command failed", e); + } finally { + defaultMQAdminExt.shutdown(); + } + } + + public void execute(DefaultMQAdminExt defaultMQAdminExt, String clusterName, String topic, + int newLogicalQueueNum) throws RemotingSendRequestException, RemotingConnectException, RemotingTimeoutException, MQBrokerException, InterruptedException, SubCommandException { + List brokerAddrs = CommandUtil.fetchMasterAddrByClusterName(defaultMQAdminExt, clusterName).stream().sorted().collect(Collectors.toList()); + Map topicConfigsByBrokerAddr = Maps.newHashMapWithExpectedSize(brokerAddrs.size()); + NavigableMap> allLogicalQueueMapByIndex = Maps.newTreeMap(); + Map allLogicalQueueMapByBroker = Maps.newHashMap(); + for (String brokerAddr : brokerAddrs) { + TopicConfig topicConfig = defaultMQAdminExt.examineTopicConfig(brokerAddr, topic); + if (topicConfig == null) { + log.info("examineTopicConfig brokerAddr={} topic={} not exist, skip!", brokerAddr, topic); + continue; + } + topicConfigsByBrokerAddr.put(brokerAddr, topicConfig); + + LogicalQueuesInfo logicalQueuesInfo = defaultMQAdminExt.queryTopicLogicalQueueMapping(brokerAddr, topic); + if (logicalQueuesInfo == null) { + throw new SubCommandException(String.format(Locale.ENGLISH, "broker=%s topic=%s logical queue not enabled", brokerAddr, topic)); + } + allLogicalQueueMapByBroker.put(brokerAddr, logicalQueuesInfo); + logicalQueuesInfo.values().stream().flatMap(Collection::stream).forEach(queueRouteData -> { + List sortedQueueRouteDataList = allLogicalQueueMapByIndex.computeIfAbsent(queueRouteData.getLogicalQueueIndex(), ignore -> Lists.newArrayListWithExpectedSize(1)); + int idx = Collections.binarySearch(sortedQueueRouteDataList, queueRouteData, + Comparator.comparingLong(LogicalQueueRouteData::getLogicalQueueDelta) + .thenComparing(LogicalQueueRouteData::getMessageQueue) + .thenComparingInt(LogicalQueueRouteData::getStateOrdinal)); + if (idx < 0) { + idx = -idx - 1; + } + sortedQueueRouteDataList.add(idx, queueRouteData); + }); + } + int oldLogicalQueueNum = (int) allLogicalQueueMapByIndex.values().stream().filter(queueRouteDataList -> queueRouteDataList.stream().anyMatch(LogicalQueueRouteData::isWritable)).count(); + if (oldLogicalQueueNum == newLogicalQueueNum) { + log.info("logical queue num not changed!"); + } else if (oldLogicalQueueNum < newLogicalQueueNum) { + increaseLogicalQueueNum(defaultMQAdminExt, allLogicalQueueMapByBroker, allLogicalQueueMapByIndex, topicConfigsByBrokerAddr, oldLogicalQueueNum, newLogicalQueueNum); + } else { + decreaseLogicalQueueNum(defaultMQAdminExt, allLogicalQueueMapByIndex, oldLogicalQueueNum, newLogicalQueueNum); + } + } + + private void increaseLogicalQueueNum(DefaultMQAdminExt defaultMQAdminExt, + Map allLogicalQueuesInfoMapByBroker, + NavigableMap> allLogicalQueueMapByIndex, + Map topicConfigsByBrokerAddr, int oldLogicalQueueNum, + int newLogicalQueueNum) throws InterruptedException, MQBrokerException, RemotingTimeoutException, RemotingSendRequestException, RemotingConnectException { + int curLogicalQueueNum = oldLogicalQueueNum; + String topic = topicConfigsByBrokerAddr.values().stream().findAny().map(TopicConfig::getTopicName).get(); + // try use queue not be assigned as logical queue + for (Map.Entry e : topicConfigsByBrokerAddr.entrySet()) { + String brokerAddr = e.getKey(); + TopicConfig topicConfig = e.getValue(); + LogicalQueuesInfo logicalQueuesInfo = allLogicalQueuesInfoMapByBroker.getOrDefault(brokerAddr, new LogicalQueuesInfo()); + ListMultimap m = Multimaps.index(logicalQueuesInfo.values().stream().flatMap(Collection::stream).iterator(), LogicalQueueRouteData::getQueueId); + for (int queueId = 0, writeQueueNums = topicConfig.getWriteQueueNums(); queueId < writeQueueNums; queueId++) { + if (m.get(queueId).stream().anyMatch(LogicalQueueRouteData::isWritable)) { + continue; + } + int logicalQueueIdx = curLogicalQueueNum; + LogicalQueueRouteData queueRouteData; + try { + queueRouteData = defaultMQAdminExt.reuseTopicLogicalQueue(brokerAddr, topic, queueId, logicalQueueIdx, MessageQueueRouteState.Normal); + } finally { + log.info("updateTopicLogicalQueueMapping reuse expired message queue from sealing logical queue brokerAddr={} queueId={} logicalQueueIdx={}", brokerAddr, queueId, logicalQueueIdx); + } + curLogicalQueueNum++; + if (curLogicalQueueNum >= newLogicalQueueNum) { + return; + } + allLogicalQueueMapByIndex.computeIfAbsent(logicalQueueIdx, integer -> Lists.newArrayListWithExpectedSize(1)).add(queueRouteData); + logicalQueuesInfo.computeIfAbsent(logicalQueueIdx, integer -> Lists.newArrayListWithExpectedSize(1)).add(queueRouteData); + } + } + // try reuse still sealing logical queue + for (Map.Entry> entry : allLogicalQueueMapByIndex.entrySet()) { + List queueRouteDataList = entry.getValue(); + if (queueRouteDataList.size() == 0 || queueRouteDataList.stream().anyMatch(LogicalQueueRouteData::isWritable)) { + continue; + } + int logicalQueueIdx = entry.getKey(); + // this is a sealing logical queue + LogicalQueueRouteData queueRouteData = queueRouteDataList.get(queueRouteDataList.size() - 1); + String brokerAddr = queueRouteData.getBrokerAddr(); + List queueRouteDataListByBroker = allLogicalQueuesInfoMapByBroker.get(brokerAddr).get(logicalQueueIdx); + if (queueRouteData.isExpired()) { + int queueId = queueRouteData.getQueueId(); + try { + queueRouteData = defaultMQAdminExt.reuseTopicLogicalQueue(brokerAddr, topic, queueId, logicalQueueIdx, MessageQueueRouteState.Normal); + } finally { + log.info("updateTopicLogicalQueueMapping reuse expired message queue from sealing logical queue brokerAddr={} queueId={} logicalQueueIdx={}", brokerAddr, queueId, logicalQueueIdx); + } + queueRouteDataList.add(queueRouteData); + queueRouteDataListByBroker.add(queueRouteData); + } else { + // create a message queue in last broker + // not expired message queue can not be reused, since delta value will not be described by one `long` + int queueId = -1; + try { + queueRouteData = defaultMQAdminExt.createMessageQueueForLogicalQueue(brokerAddr, topic, logicalQueueIdx, MessageQueueRouteState.Normal); + queueId = queueRouteData.getQueueId(); + } finally { + log.info("updateTopicLogicalQueueMapping create message queue from sealing logical queue brokerAddr={} queueId={} logicalQueueIdx={}", brokerAddr, queueId, logicalQueueIdx); + } + queueRouteDataList.add(queueRouteData); + queueRouteDataListByBroker.add(queueRouteData); + topicConfigsByBrokerAddr.get(brokerAddr).setWriteQueueNums(queueId + 1); + } + curLogicalQueueNum++; + if (curLogicalQueueNum >= newLogicalQueueNum) { + return; + } + } + // try broker already with expired message queue + for (Map.Entry entry : allLogicalQueuesInfoMapByBroker.entrySet()) { + String brokerAddr = entry.getKey(); + for (Iterator it = entry.getValue().values().stream().flatMap(Collection::stream) + .filter(LogicalQueueRouteData::isExpired) + .sorted(Comparator.comparingInt(LogicalQueueRouteData::getLogicalQueueIndex).thenComparingInt(LogicalQueueRouteData::getQueueId)) + .limit(newLogicalQueueNum - curLogicalQueueNum) + .iterator(); it.hasNext(); ) { + LogicalQueueRouteData queueRouteData = it.next(); + try { + LogicalQueueRouteData result = defaultMQAdminExt.reuseTopicLogicalQueue(brokerAddr, topic, queueRouteData.getQueueId(), queueRouteData.getLogicalQueueIndex(), MessageQueueRouteState.Normal); + // modify in-place + queueRouteData.copyFrom(result); + } finally { + log.info("updateTopicLogicalQueueMapping reuse expired message queue from sealing logical queue brokerAddr={} queueId={} logicalQueueIdx={}", brokerAddr, queueRouteData.getQueueId(), queueRouteData.getLogicalQueueIndex()); + } + allLogicalQueueMapByIndex.get(queueRouteData.getLogicalQueueIndex()).stream() + .filter(LogicalQueueRouteData::isExpired) + .filter(v -> Objects.equals(brokerAddr, v.getBrokerAddr()) && queueRouteData.getQueueId() == v.getQueueId() && queueRouteData.getLogicalQueueIndex() == v.getLogicalQueueIndex()) + .forEach(v -> v.copyFrom(queueRouteData)); + curLogicalQueueNum++; + if (curLogicalQueueNum >= newLogicalQueueNum) { + return; + } + } + } + + // try broker with least amount message queue, if amount equal, random select + for (; curLogicalQueueNum < newLogicalQueueNum; curLogicalQueueNum++) { + Map.Entry entry = allLogicalQueuesInfoMapByBroker.entrySet().stream().min(Comparator.comparingInt(value -> value.getValue().values().stream().flatMapToInt(l -> IntStream.of(l.size())).sum())).get(); + String brokerAddr = entry.getKey(); + int logicalQueueIdx = curLogicalQueueNum; + int queueId = -1; + LogicalQueueRouteData queueRouteData; + try { + queueRouteData = defaultMQAdminExt.createMessageQueueForLogicalQueue(brokerAddr, topic, logicalQueueIdx, MessageQueueRouteState.Normal); + queueId = queueRouteData.getQueueId(); + } finally { + log.info("updateTopicLogicalQueueMapping create message queue from fresh brokerAddr={} queueId={} logicalQueueIdx={}", brokerAddr, queueId, logicalQueueIdx); + } + entry.getValue().put(logicalQueueIdx, Lists.newArrayList(queueRouteData)); + } + } + + private void decreaseLogicalQueueNum(DefaultMQAdminExt defaultMQAdminExt, + NavigableMap> allLogicalQueueMapByIndex, + int oldLogicalQueueNum, + int newLogicalQueueNum) throws InterruptedException, RemotingConnectException, RemotingTimeoutException, RemotingSendRequestException, MQBrokerException, SubCommandException { + // seal logical queue from greatest index + Map.Entry> maxActiveEntry = allLogicalQueueMapByIndex.lastEntry(); + int curLogicalQueueNum = oldLogicalQueueNum; + while (curLogicalQueueNum > newLogicalQueueNum) { + boolean anyQueueSealed = false; + for (LogicalQueueRouteData queueRouteData : maxActiveEntry.getValue()) { + if (queueRouteData.isWritable()) { + anyQueueSealed = true; + LogicalQueueRouteData resultQueueRouteData = queueRouteData; + try { + resultQueueRouteData = defaultMQAdminExt.sealTopicLogicalQueue(queueRouteData.getBrokerAddr(), queueRouteData); + } finally { + log.info("seal message queue: {}", resultQueueRouteData); + } + } + } + if (anyQueueSealed) { + curLogicalQueueNum--; + } + maxActiveEntry = allLogicalQueueMapByIndex.lowerEntry(maxActiveEntry.getKey()); + if (maxActiveEntry == null) { + throw new SubCommandException(String.format(Locale.ENGLISH, "oldLogicalQueueNum=%d newLogicalQueueNum=%d curLogicalQueueNum=%d but can not find lowerEntry, unexpected situation", oldLogicalQueueNum, newLogicalQueueNum, curLogicalQueueNum)); + } + } + } +} diff --git a/tools/src/main/java/org/apache/rocketmq/tools/command/topic/UpdateTopicSubCommand.java b/tools/src/main/java/org/apache/rocketmq/tools/command/topic/UpdateTopicSubCommand.java index c33ae52dbd3..b7f5379d36e 100644 --- a/tools/src/main/java/org/apache/rocketmq/tools/command/topic/UpdateTopicSubCommand.java +++ b/tools/src/main/java/org/apache/rocketmq/tools/command/topic/UpdateTopicSubCommand.java @@ -29,6 +29,7 @@ import org.apache.rocketmq.tools.command.CommandUtil; import org.apache.rocketmq.tools.command.SubCommand; import org.apache.rocketmq.tools.command.SubCommandException; +import org.apache.rocketmq.tools.command.logicalqueue.UpdateTopicLogicalQueueMappingCommand; public class UpdateTopicSubCommand implements SubCommand { @@ -67,6 +68,10 @@ public Options buildCommandlineOptions(Options options) { opt.setRequired(false); options.addOption(opt); + opt = new Option("lq", "logicalQueue", true, "set logical queue nums"); + opt.setRequired(false); + options.addOption(opt); + opt = new Option("p", "perm", true, "set topic's permission(2|4|6), intro[2:W 4:R; 6:RW]"); opt.setRequired(false); options.addOption(opt); @@ -132,7 +137,17 @@ public void execute(final CommandLine commandLine, final Options options, } topicConfig.setOrder(isOrder); + boolean useLogicalQueue = false; + if (commandLine.hasOption("lq")) { + useLogicalQueue = Boolean.parseBoolean(commandLine.getOptionValue("lq").trim()); + } + if (commandLine.hasOption('b')) { + if (useLogicalQueue) { + System.out.printf("-lq and -b can not be used together.%n"); + return; + } + String addr = commandLine.getOptionValue('b').trim(); defaultMQAdminExt.start(); @@ -156,6 +171,7 @@ public void execute(final CommandLine commandLine, final Options options, Set masterSet = CommandUtil.fetchMasterAddrByClusterName(defaultMQAdminExt, clusterName); + for (String addr : masterSet) { defaultMQAdminExt.createAndUpdateTopicConfig(addr, topicConfig); System.out.printf("create topic to %s success.%n", addr); @@ -177,6 +193,10 @@ public void execute(final CommandLine commandLine, final Options options, } System.out.printf("%s", topicConfig); + + if (useLogicalQueue) { + new UpdateTopicLogicalQueueMappingCommand().execute(defaultMQAdminExt, topicConfig.getTopicName(), masterSet); + } return; } diff --git a/tools/src/test/java/org/apache/rocketmq/tools/admin/DefaultMQAdminExtTest.java b/tools/src/test/java/org/apache/rocketmq/tools/admin/DefaultMQAdminExtTest.java index 3146b178115..e76095480fb 100644 --- a/tools/src/test/java/org/apache/rocketmq/tools/admin/DefaultMQAdminExtTest.java +++ b/tools/src/test/java/org/apache/rocketmq/tools/admin/DefaultMQAdminExtTest.java @@ -34,6 +34,7 @@ import org.apache.rocketmq.client.impl.MQClientAPIImpl; import org.apache.rocketmq.client.impl.MQClientManager; import org.apache.rocketmq.client.impl.factory.MQClientInstance; +import org.apache.rocketmq.common.MixAll; import org.apache.rocketmq.common.admin.ConsumeStats; import org.apache.rocketmq.common.admin.OffsetWrapper; import org.apache.rocketmq.common.admin.TopicOffset; @@ -59,6 +60,9 @@ import org.apache.rocketmq.common.protocol.heartbeat.MessageModel; import org.apache.rocketmq.common.protocol.heartbeat.SubscriptionData; import org.apache.rocketmq.common.protocol.route.BrokerData; +import org.apache.rocketmq.common.protocol.route.LogicalQueueRouteData; +import org.apache.rocketmq.common.protocol.route.LogicalQueuesInfo; +import org.apache.rocketmq.common.protocol.route.MessageQueueRouteState; import org.apache.rocketmq.common.protocol.route.QueueData; import org.apache.rocketmq.common.protocol.route.TopicRouteData; import org.apache.rocketmq.common.subscription.SubscriptionGroupConfig; @@ -68,6 +72,8 @@ import org.apache.rocketmq.remoting.exception.RemotingSendRequestException; import org.apache.rocketmq.remoting.exception.RemotingTimeoutException; import org.apache.rocketmq.tools.admin.api.MessageTrack; +import org.assertj.core.util.Lists; +import org.assertj.core.util.Maps; import org.junit.AfterClass; import org.junit.BeforeClass; import org.junit.Test; @@ -75,14 +81,24 @@ import org.mockito.junit.MockitoJUnitRunner; 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.anyInt; import static org.mockito.ArgumentMatchers.anyLong; import static org.mockito.ArgumentMatchers.anyString; +import static org.mockito.ArgumentMatchers.eq; import static org.mockito.Mockito.mock; import static org.mockito.Mockito.when; @RunWith(MockitoJUnitRunner.class) public class DefaultMQAdminExtTest { + private static final String broker1Addr = "127.0.0.1:10911"; + private static final String broker1Name = "default-broker"; + private static final String cluster = "default-cluster"; + private static final String broker2Name = "broker-test"; + private static final String broker2Addr = "127.0.0.2:10911"; + private static final String topic1 = "topic_one"; + private static final String topic2 = "topic_two"; private static DefaultMQAdminExt defaultMQAdminExt; private static DefaultMQAdminExtImpl defaultMQAdminExtImpl; private static MQClientInstance mqClientInstance = MQClientManager.getInstance().getOrCreateMQClientInstance(new ClientConfig()); @@ -115,33 +131,40 @@ public static void init() throws Exception { when(mQClientAPIImpl.getBrokerConfig(anyString(), anyLong())).thenReturn(properties); Set topicSet = new HashSet<>(); - topicSet.add("topic_one"); - topicSet.add("topic_two"); + topicSet.add(topic1); + topicSet.add(topic2); topicList.setTopicList(topicSet); when(mQClientAPIImpl.getTopicListFromNameServer(anyLong())).thenReturn(topicList); List brokerDatas = new ArrayList<>(); HashMap brokerAddrs = new HashMap<>(); - brokerAddrs.put(1234l, "127.0.0.1:10911"); + brokerAddrs.put(MixAll.MASTER_ID, broker1Addr); BrokerData brokerData = new BrokerData(); - brokerData.setCluster("default-cluster"); - brokerData.setBrokerName("default-broker"); + brokerData.setCluster(cluster); + brokerData.setBrokerName(broker1Name); brokerData.setBrokerAddrs(brokerAddrs); brokerDatas.add(brokerData); + brokerDatas.add(new BrokerData(cluster, broker2Name, (HashMap) Maps.newHashMap(MixAll.MASTER_ID, broker2Addr))); topicRouteData.setBrokerDatas(brokerDatas); topicRouteData.setQueueDatas(new ArrayList()); topicRouteData.setFilterServerTable(new HashMap>()); - when(mQClientAPIImpl.getTopicRouteInfoFromNameServer(anyString(), anyLong())).thenReturn(topicRouteData); + LogicalQueuesInfo logicalQueuesInfoinfo = new LogicalQueuesInfo(); + logicalQueuesInfoinfo.put(0, Lists.newArrayList( + new LogicalQueueRouteData(0, 0, new MessageQueue(topic1, broker1Name, 0), MessageQueueRouteState.ReadOnly, 0, 1000, 2000, 3000, broker1Addr), + new LogicalQueueRouteData(0, 1000, new MessageQueue(topic1, broker2Name, 0), MessageQueueRouteState.Normal, 0, -1, -1, -1, broker2Addr) + )); + topicRouteData.setLogicalQueuesInfo(logicalQueuesInfoinfo); + when(mQClientAPIImpl.getTopicRouteInfoFromNameServer(anyString(), anyLong(), anyBoolean(), any())).thenReturn(topicRouteData); HashMap result = new HashMap<>(); - result.put("id", "1234"); - result.put("brokerName", "default-broker"); + result.put("id", String.valueOf(MixAll.MASTER_ID)); + result.put("brokerName", broker1Name); kvTable.setTable(result); when(mQClientAPIImpl.getBrokerRuntimeInfo(anyString(), anyLong())).thenReturn(kvTable); HashMap brokerAddrTable = new HashMap<>(); - brokerAddrTable.put("default-broker", brokerData); - brokerAddrTable.put("broker-test", new BrokerData()); + brokerAddrTable.put(broker1Name, brokerData); + brokerAddrTable.put(broker2Name, new BrokerData()); clusterInfo.setBrokerAddrTable(brokerAddrTable); clusterInfo.setClusterAddrTable(new HashMap>()); when(mQClientAPIImpl.getBrokerClusterInfo(anyLong())).thenReturn(clusterInfo); @@ -251,7 +274,7 @@ public void testFetchAllTopicList() throws RemotingException, MQClientException, @Test public void testFetchBrokerRuntimeStats() throws InterruptedException, MQBrokerException, RemotingTimeoutException, RemotingSendRequestException, RemotingConnectException { KVTable brokerStats = defaultMQAdminExt.fetchBrokerRuntimeStats("127.0.0.1:10911"); - assertThat(brokerStats.getTable().get("id")).isEqualTo("1234"); + assertThat(brokerStats.getTable().get("id")).isEqualTo(String.valueOf(MixAll.MASTER_ID)); assertThat(brokerStats.getTable().get("brokerName")).isEqualTo("default-broker"); } @@ -277,7 +300,7 @@ public void testExamineBrokerClusterInfo() throws InterruptedException, MQBroker @Test public void testExamineConsumeStats() throws InterruptedException, RemotingException, MQClientException, MQBrokerException { ConsumeStats consumeStats = defaultMQAdminExt.examineConsumeStats("default-consumer-group", "unit-test"); - assertThat(consumeStats.getConsumeTps()).isEqualTo(1234); + assertThat(consumeStats.getConsumeTps()).isGreaterThanOrEqualTo(1234); } @Test @@ -406,4 +429,32 @@ public void testGetAllSubscriptionGroup() throws InterruptedException, MQBrokerE assertThat(subscriptionGroupWrapper.getSubscriptionGroupTable().get("Consumer-group-one").getGroupName()).isEqualTo("Consumer-group-one"); assertThat(subscriptionGroupWrapper.getSubscriptionGroupTable().get("Consumer-group-one").isConsumeBroadcastEnable()).isTrue(); } + + @Test + public void testMaxOffset() throws Exception { + when(mQClientAPIImpl.getMaxOffset(anyString(), anyString(), anyInt(), anyBoolean(), anyBoolean(), anyLong())).thenReturn(100L); + + assertThat(defaultMQAdminExt.maxOffset(new MessageQueue(topic1, broker1Name, 0))).isEqualTo(100L); + } + + @Test + public void testSearchOffset() throws Exception { + when(mQClientAPIImpl.searchOffset(anyString(), anyString(), anyInt(), anyLong(), anyLong())).thenReturn(101L); + + assertThat(defaultMQAdminExt.searchOffset(new MessageQueue(topic1, broker1Name, 0), System.currentTimeMillis())).isEqualTo(101L); + } + + @Test + public void testMaxOffset_LogicalQueue() throws Exception { + when(mQClientAPIImpl.getMaxOffset(eq(broker2Addr), anyString(), anyInt(), anyBoolean(), anyBoolean(), anyLong())).thenReturn(10L); + + assertThat(defaultMQAdminExt.maxOffset(new MessageQueue(topic1, MixAll.LOGICAL_QUEUE_MOCK_BROKER_NAME, 0))).isEqualTo(1010L); + } + + @Test + public void testSearchOffset_LogicalQueue() throws Exception { + when(mQClientAPIImpl.searchOffset(eq(broker2Addr), anyString(), anyInt(), anyLong(), anyLong())).thenReturn(11L); + + assertThat(defaultMQAdminExt.searchOffset(new MessageQueue(topic1, MixAll.LOGICAL_QUEUE_MOCK_BROKER_NAME, 0), System.currentTimeMillis())).isEqualTo(1011L); + } } \ No newline at end of file