diff --git a/auth/src/main/java/org/apache/rocketmq/auth/authorization/builder/DefaultAuthorizationContextBuilder.java b/auth/src/main/java/org/apache/rocketmq/auth/authorization/builder/DefaultAuthorizationContextBuilder.java index 7134c6fd387..b0080084f01 100644 --- a/auth/src/main/java/org/apache/rocketmq/auth/authorization/builder/DefaultAuthorizationContextBuilder.java +++ b/auth/src/main/java/org/apache/rocketmq/auth/authorization/builder/DefaultAuthorizationContextBuilder.java @@ -31,6 +31,7 @@ import apache.rocketmq.v2.Subscription; import apache.rocketmq.v2.SubscriptionEntry; import apache.rocketmq.v2.TelemetryCommand; +import apache.rocketmq.v2.SyncLiteSubscriptionRequest; import com.google.protobuf.GeneratedMessageV3; import io.grpc.Metadata; import io.netty.channel.ChannelHandlerContext; @@ -124,6 +125,13 @@ public List build(Metadata metadata, GeneratedMessa } result = newSubContexts(metadata, request.getGroup(), request.getMessageQueue().getTopic()); } + if (message instanceof SyncLiteSubscriptionRequest) { + SyncLiteSubscriptionRequest request = (SyncLiteSubscriptionRequest) message; + if (request.getLiteTopicSetCount() <= 0) { + return null; + } + result = newSubContexts(metadata, request.getGroup(), request.getTopic()); + } if (message instanceof AckMessageRequest) { AckMessageRequest request = (AckMessageRequest) message; result = newSubContexts(metadata, request.getGroup(), request.getTopic()); 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 efc2949364d..2734e8b2463 100644 --- a/broker/src/main/java/org/apache/rocketmq/broker/BrokerController.java +++ b/broker/src/main/java/org/apache/rocketmq/broker/BrokerController.java @@ -73,6 +73,14 @@ import org.apache.rocketmq.broker.filter.CommitLogDispatcherCalcBitMap; import org.apache.rocketmq.broker.filter.ConsumerFilterManager; import org.apache.rocketmq.broker.latency.BrokerFastFailure; +import org.apache.rocketmq.broker.lite.AbstractLiteLifecycleManager; +import org.apache.rocketmq.broker.lite.LiteEventDispatcher; +import org.apache.rocketmq.broker.lite.LiteSubscriptionRegistry; +import org.apache.rocketmq.broker.lite.LiteSubscriptionRegistryImpl; +import org.apache.rocketmq.broker.lite.LiteLifecycleManager; +import org.apache.rocketmq.broker.lite.LiteSharding; +import org.apache.rocketmq.broker.lite.LiteShardingImpl; +import org.apache.rocketmq.broker.lite.RocksDBLiteLifecycleManager; import org.apache.rocketmq.broker.longpolling.LmqPullRequestHoldService; import org.apache.rocketmq.broker.longpolling.NotifyMessageArrivingListener; import org.apache.rocketmq.broker.longpolling.PullRequestHoldService; @@ -93,10 +101,13 @@ import org.apache.rocketmq.broker.processor.ClientManageProcessor; import org.apache.rocketmq.broker.processor.ConsumerManageProcessor; import org.apache.rocketmq.broker.processor.EndTransactionProcessor; +import org.apache.rocketmq.broker.processor.LiteManagerProcessor; +import org.apache.rocketmq.broker.processor.LiteSubscriptionCtlProcessor; import org.apache.rocketmq.broker.processor.NotificationProcessor; import org.apache.rocketmq.broker.processor.PeekMessageProcessor; import org.apache.rocketmq.broker.processor.PollingInfoProcessor; import org.apache.rocketmq.broker.processor.PopInflightMessageCounter; +import org.apache.rocketmq.broker.processor.PopLiteMessageProcessor; import org.apache.rocketmq.broker.processor.PopMessageProcessor; import org.apache.rocketmq.broker.processor.PullMessageProcessor; import org.apache.rocketmq.broker.processor.QueryAssignmentProcessor; @@ -206,12 +217,19 @@ public class BrokerController { protected final PullMessageProcessor pullMessageProcessor; protected final PeekMessageProcessor peekMessageProcessor; protected final PopMessageProcessor popMessageProcessor; + protected final PopLiteMessageProcessor popLiteMessageProcessor; protected final AckMessageProcessor ackMessageProcessor; protected final ChangeInvisibleTimeProcessor changeInvisibleTimeProcessor; protected final NotificationProcessor notificationProcessor; protected final PollingInfoProcessor pollingInfoProcessor; protected final QueryAssignmentProcessor queryAssignmentProcessor; protected final ClientManageProcessor clientManageProcessor; + protected final LiteSubscriptionCtlProcessor liteSubscriptionCtlProcessor; + protected final LiteSharding liteSharding; + protected final AbstractLiteLifecycleManager liteLifecycleManager; + protected final LiteSubscriptionRegistry liteSubscriptionRegistry; + protected final LiteEventDispatcher liteEventDispatcher; + protected final LiteManagerProcessor liteManagerProcessor; protected final SendMessageProcessor sendMessageProcessor; protected final RecallMessageProcessor recallMessageProcessor; protected final ReplyMessageProcessor replyMessageProcessor; @@ -376,10 +394,19 @@ public BrokerController( this.topicQueueMappingManager = new TopicQueueMappingManager(this); this.authenticationMetadataManager = AuthenticationFactory.getMetadataManager(this.authConfig); this.authorizationMetadataManager = AuthorizationFactory.getMetadataManager(this.authConfig); + this.topicRouteInfoManager = new TopicRouteInfoManager(this); + this.liteSharding = new LiteShardingImpl(this, this.topicRouteInfoManager); + this.liteLifecycleManager = this.messageStoreConfig.isEnableRocksDBStore() ? + new RocksDBLiteLifecycleManager(this, this.liteSharding) : new LiteLifecycleManager(this, this.liteSharding); + this.liteSubscriptionRegistry = new LiteSubscriptionRegistryImpl(this, liteLifecycleManager); + this.liteSubscriptionCtlProcessor = new LiteSubscriptionCtlProcessor(this, liteSubscriptionRegistry); + this.liteEventDispatcher = new LiteEventDispatcher(this, this.liteSubscriptionRegistry, this.liteLifecycleManager); + this.liteManagerProcessor = new LiteManagerProcessor(this, liteLifecycleManager, liteSharding); this.pullMessageProcessor = new PullMessageProcessor(this); this.peekMessageProcessor = new PeekMessageProcessor(this); this.pullRequestHoldService = messageStoreConfig.isEnableLmq() ? new LmqPullRequestHoldService(this) : new PullRequestHoldService(this); this.popMessageProcessor = new PopMessageProcessor(this); + this.popLiteMessageProcessor = new PopLiteMessageProcessor(this, this.liteEventDispatcher); this.notificationProcessor = new NotificationProcessor(this); this.pollingInfoProcessor = new PollingInfoProcessor(this); this.ackMessageProcessor = new AckMessageProcessor(this); @@ -387,7 +414,7 @@ public BrokerController( this.sendMessageProcessor = new SendMessageProcessor(this); this.recallMessageProcessor = new RecallMessageProcessor(this); this.replyMessageProcessor = new ReplyMessageProcessor(this); - this.messageArrivingListener = new NotifyMessageArrivingListener(this.pullRequestHoldService, this.popMessageProcessor, this.notificationProcessor); + this.messageArrivingListener = new NotifyMessageArrivingListener(this.pullRequestHoldService, this.popMessageProcessor, this.notificationProcessor, this.liteEventDispatcher); this.consumerIdsChangeListener = new DefaultConsumerIdsChangeListener(this); this.consumerManager = new ConsumerManager(this.consumerIdsChangeListener, this.brokerStatsManager, this.brokerConfig); this.producerManager = new ProducerManager(this.brokerStatsManager); @@ -466,8 +493,6 @@ public boolean online(String instanceId, String group, String topic) { this.escapeBridge = new EscapeBridge(this); - this.topicRouteInfoManager = new TopicRouteInfoManager(this); - if (this.brokerConfig.isEnableSlaveActingMaster() && !this.brokerConfig.isSkipPreOnline()) { this.brokerPreOnlineService = new BrokerPreOnlineService(this); } @@ -950,6 +975,8 @@ public boolean recoverAndInitService() throws CloneNotSupportedException { initialRequestPipeline(); + initLiteService(); + if (TlsSystemConfig.tlsMode != TlsMode.DISABLED) { // Register a listener to reload SslContext try { @@ -1045,6 +1072,21 @@ public PutMessageResult executeBeforePutMessage(MessageExt msg) { } }); + putMessageHookList.add(new PutMessageHook() { + @Override + public String hookName() { + return "handleLmqQuota"; + } + + @Override + public PutMessageResult executeBeforePutMessage(MessageExt msg) { + if (msg instanceof MessageExtBrokerInner) { + return HookUtils.handleLmqQuota(BrokerController.this, (MessageExtBrokerInner) msg); + } + return null; + } + }); + SendMessageBackHook sendMessageBackHook = new SendMessageBackHook() { @Override public boolean executeSendMessageBack(List msgList, String brokerName, String brokerAddr) { @@ -1111,6 +1153,11 @@ private void initialRequestPipeline() { } } + private void initLiteService() { + this.liteEventDispatcher.init(); + this.liteLifecycleManager.init(); + } + public void registerProcessor() { RemotingServer remotingServer = remotingServerMap.get(TCP_REMOTING_SERVER); RemotingServer fastRemotingServer = remotingServerMap.get(FAST_REMOTING_SERVER); @@ -1145,6 +1192,7 @@ public void registerProcessor() { * PopMessageProcessor */ remotingServer.registerProcessor(RequestCode.POP_MESSAGE, this.popMessageProcessor, this.pullMessageExecutor); + remotingServer.registerProcessor(RequestCode.POP_LITE_MESSAGE, this.popLiteMessageProcessor, this.pullMessageExecutor); /** * AckMessageProcessor @@ -1196,10 +1244,12 @@ public void registerProcessor() { remotingServer.registerProcessor(RequestCode.HEART_BEAT, clientManageProcessor, this.heartbeatExecutor); remotingServer.registerProcessor(RequestCode.UNREGISTER_CLIENT, clientManageProcessor, this.clientManageExecutor); remotingServer.registerProcessor(RequestCode.CHECK_CLIENT_CONFIG, clientManageProcessor, this.clientManageExecutor); + remotingServer.registerProcessor(RequestCode.LITE_SUBSCRIPTION_CTL, liteSubscriptionCtlProcessor, this.clientManageExecutor); fastRemotingServer.registerProcessor(RequestCode.HEART_BEAT, clientManageProcessor, this.heartbeatExecutor); fastRemotingServer.registerProcessor(RequestCode.UNREGISTER_CLIENT, clientManageProcessor, this.clientManageExecutor); fastRemotingServer.registerProcessor(RequestCode.CHECK_CLIENT_CONFIG, clientManageProcessor, this.clientManageExecutor); + fastRemotingServer.registerProcessor(RequestCode.LITE_SUBSCRIPTION_CTL, liteSubscriptionCtlProcessor, this.clientManageExecutor); /** * ConsumerManageProcessor @@ -1227,6 +1277,23 @@ public void registerProcessor() { remotingServer.registerProcessor(RequestCode.END_TRANSACTION, endTransactionProcessor, this.endTransactionExecutor); fastRemotingServer.registerProcessor(RequestCode.END_TRANSACTION, endTransactionProcessor, this.endTransactionExecutor); + /* + * lite admin + */ + remotingServer.registerProcessor(RequestCode.GET_BROKER_LITE_INFO, liteManagerProcessor, adminBrokerExecutor); + remotingServer.registerProcessor(RequestCode.GET_PARENT_TOPIC_INFO, liteManagerProcessor, adminBrokerExecutor); + remotingServer.registerProcessor(RequestCode.GET_LITE_TOPIC_INFO, liteManagerProcessor, adminBrokerExecutor); + remotingServer.registerProcessor(RequestCode.GET_LITE_CLIENT_INFO, liteManagerProcessor, adminBrokerExecutor); + remotingServer.registerProcessor(RequestCode.GET_LITE_GROUP_INFO, liteManagerProcessor, adminBrokerExecutor); + remotingServer.registerProcessor(RequestCode.TRIGGER_LITE_DISPATCH, liteManagerProcessor, adminBrokerExecutor); + + fastRemotingServer.registerProcessor(RequestCode.GET_BROKER_LITE_INFO, liteManagerProcessor, adminBrokerExecutor); + fastRemotingServer.registerProcessor(RequestCode.GET_PARENT_TOPIC_INFO, liteManagerProcessor, adminBrokerExecutor); + fastRemotingServer.registerProcessor(RequestCode.GET_LITE_TOPIC_INFO, liteManagerProcessor, adminBrokerExecutor); + fastRemotingServer.registerProcessor(RequestCode.GET_LITE_CLIENT_INFO, liteManagerProcessor, adminBrokerExecutor); + fastRemotingServer.registerProcessor(RequestCode.GET_LITE_GROUP_INFO, liteManagerProcessor, adminBrokerExecutor); + fastRemotingServer.registerProcessor(RequestCode.TRIGGER_LITE_DISPATCH, liteManagerProcessor, adminBrokerExecutor); + /* * Default */ @@ -1409,6 +1476,10 @@ public PopMessageProcessor getPopMessageProcessor() { return popMessageProcessor; } + public PopLiteMessageProcessor getPopLiteMessageProcessor() { + return popLiteMessageProcessor; + } + public NotificationProcessor getNotificationProcessor() { return notificationProcessor; } @@ -1437,6 +1508,14 @@ public ChangeInvisibleTimeProcessor getChangeInvisibleTimeProcessor() { return changeInvisibleTimeProcessor; } + public LiteSubscriptionRegistry getLiteSubscriptionRegistry() { + return liteSubscriptionRegistry; + } + + public AbstractLiteLifecycleManager getLiteLifecycleManager() { + return liteLifecycleManager; + } + protected void shutdownBasicService() { shutdown = true; @@ -1474,6 +1553,13 @@ protected void shutdownBasicService() { this.popMessageProcessor.getPopLongPollingService().shutdown(); } + if (this.popLiteMessageProcessor != null) { + this.popLiteMessageProcessor.stopPopLiteLockManager(); + if (this.popLiteMessageProcessor.getPopLiteLongPollingService() != null) { + this.popLiteMessageProcessor.getPopLiteLongPollingService().shutdown(); + } + } + if (this.popMessageProcessor.getQueueLockManager() != null) { this.popMessageProcessor.getQueueLockManager().shutdown(); } @@ -1637,6 +1723,18 @@ protected void shutdownBasicService() { this.coldDataCgCtrService.shutdown(); } + if (this.liteEventDispatcher != null) { + this.liteEventDispatcher.shutdown(); + } + + if (this.liteLifecycleManager != null) { + this.liteLifecycleManager.shutdown(); + } + + if (this.liteSubscriptionRegistry != null) { + this.liteSubscriptionRegistry.shutdown(); + } + shutdownScheduledExecutorService(this.syncBrokerMemberGroupExecutorService); shutdownScheduledExecutorService(this.brokerHeartbeatExecutorService); @@ -1777,6 +1875,13 @@ protected void startBasicService() throws Exception { this.popMessageProcessor.getQueueLockManager().start(); } + if (this.popLiteMessageProcessor != null) { + this.popLiteMessageProcessor.startPopLiteLockManager(); + if (this.popLiteMessageProcessor.getPopLiteLongPollingService() != null) { + this.popLiteMessageProcessor.getPopLiteLongPollingService().start(); + } + } + if (this.ackMessageProcessor != null) { if (brokerConfig.isPopConsumerFSServiceInit()) { this.ackMessageProcessor.startPopReviveService(); @@ -1838,6 +1943,18 @@ protected void startBasicService() throws Exception { if (this.coldDataCgCtrService != null) { this.coldDataCgCtrService.start(); } + + if (this.liteEventDispatcher != null) { + this.liteEventDispatcher.start(); + } + + if (this.liteLifecycleManager != null) { + this.liteLifecycleManager.start(); + } + + if (this.liteSubscriptionRegistry != null) { + this.liteSubscriptionRegistry.start(); + } } public void start() throws Exception { @@ -2699,4 +2816,8 @@ public ConfigContext getConfigContext() { public void setConfigContext(ConfigContext configContext) { this.configContext = configContext; } + + public LiteEventDispatcher getLiteEventDispatcher() { + return liteEventDispatcher; + } } diff --git a/broker/src/main/java/org/apache/rocketmq/broker/client/net/Broker2Client.java b/broker/src/main/java/org/apache/rocketmq/broker/client/net/Broker2Client.java index f8984963f94..5a6c4c94c47 100644 --- a/broker/src/main/java/org/apache/rocketmq/broker/client/net/Broker2Client.java +++ b/broker/src/main/java/org/apache/rocketmq/broker/client/net/Broker2Client.java @@ -49,6 +49,7 @@ import org.apache.rocketmq.remoting.protocol.header.CheckTransactionStateRequestHeader; import org.apache.rocketmq.remoting.protocol.header.GetConsumerStatusRequestHeader; import org.apache.rocketmq.remoting.protocol.header.NotifyConsumerIdsChangedRequestHeader; +import org.apache.rocketmq.remoting.protocol.header.NotifyUnsubscribeLiteRequestHeader; import org.apache.rocketmq.remoting.protocol.header.ResetOffsetRequestHeader; import org.apache.rocketmq.store.exception.ConsumeQueueException; @@ -60,6 +61,16 @@ public Broker2Client(BrokerController brokerController) { this.brokerController = brokerController; } + public void notifyUnsubscribeLite(Channel channel, NotifyUnsubscribeLiteRequestHeader requestHeader) { + RemotingCommand request = + RemotingCommand.createRequestCommand(RequestCode.NOTIFY_UNSUBSCRIBE_LITE, requestHeader); + try { + this.brokerController.getRemotingServer().invokeOneway(channel, request, 100); + } catch (Exception e) { + log.error("notifyUnsubscribeLite failed. header={}, error={}", requestHeader, e.toString()); + } + } + public void checkProducerTransactionState( final String group, final Channel channel, diff --git a/broker/src/main/java/org/apache/rocketmq/broker/config/v1/RocksDBConsumerOffsetManager.java b/broker/src/main/java/org/apache/rocketmq/broker/config/v1/RocksDBConsumerOffsetManager.java index 45e1a2ab235..b1d76229400 100644 --- a/broker/src/main/java/org/apache/rocketmq/broker/config/v1/RocksDBConsumerOffsetManager.java +++ b/broker/src/main/java/org/apache/rocketmq/broker/config/v1/RocksDBConsumerOffsetManager.java @@ -18,10 +18,16 @@ import com.alibaba.fastjson2.JSON; import com.alibaba.fastjson2.JSONWriter; +import java.nio.file.Path; +import java.nio.file.Paths; +import java.util.Map.Entry; +import java.util.concurrent.ConcurrentHashMap; +import java.util.concurrent.ConcurrentMap; import org.apache.commons.lang3.StringUtils; import org.apache.rocketmq.broker.BrokerController; import org.apache.rocketmq.broker.BrokerPathConfigHelper; import org.apache.rocketmq.broker.offset.ConsumerOffsetManager; +import org.apache.rocketmq.common.MixAll; import org.apache.rocketmq.common.UtilAll; import org.apache.rocketmq.common.constant.LoggerName; import org.apache.rocketmq.common.utils.DataConverter; @@ -31,11 +37,6 @@ import org.rocksdb.CompressionType; import org.rocksdb.WriteBatch; -import java.nio.file.Path; -import java.nio.file.Paths; -import java.util.Map.Entry; -import java.util.concurrent.ConcurrentMap; - public class RocksDBConsumerOffsetManager extends ConsumerOffsetManager { protected static final Logger log = LoggerFactory.getLogger(LoggerName.BROKER_LOGGER_NAME); @@ -123,7 +124,7 @@ public boolean stop() { } @Override - protected void removeConsumerOffset(String topicAtGroup) { + public void removeConsumerOffset(String topicAtGroup) { try { byte[] keyBytes = topicAtGroup.getBytes(DataConverter.CHARSET_UTF8); this.rocksDBConfigManager.delete(keyBytes); @@ -159,6 +160,11 @@ public String configFilePath() { @Override public synchronized void persist() { if (rocksDBConfigManager.isLoaded()) { + if (brokerController.getBrokerConfig().isPersistConsumerOffsetIncrementally()) { + updateDataVersion(); + this.rocksDBConfigManager.flushWAL(); + return; + } try (WriteBatch writeBatch = new WriteBatch()) { for (Entry> entry : this.offsetTable.entrySet()) { putWriteBatch(writeBatch, entry.getKey(), entry.getValue()); @@ -176,6 +182,35 @@ public synchronized void persist() { } } + @Override + public void commitOffset(String clientHost, String group, String topic, int queueId, long offset) { + String key = topic + TOPIC_GROUP_SEPARATOR + group; + ConcurrentMap map = this.offsetTable.get(key); + if (null == map) { + map = MixAll.isLmq(topic) ? new ConcurrentHashMap<>(1, 1.0F) : new ConcurrentHashMap<>(); + map.put(queueId, offset); + this.offsetTable.put(key, map); + } else { + Long storeOffset = map.put(queueId, offset); + if (storeOffset != null && offset < storeOffset) { + LOG.warn("[NOTIFYME]update consumer offset less than store. clientHost={}, key={}, queueId={}, requestOffset={}, storeOffset={}", clientHost, key, queueId, offset, storeOffset); + } + } + if (versionChangeCounter.incrementAndGet() % brokerController.getBrokerConfig().getConsumerOffsetUpdateVersionStep() == 0) { + updateDataVersion(); + } + if (!brokerController.getBrokerConfig().isPersistConsumerOffsetIncrementally()) { + return; + } + + try (WriteBatch writeBatch = new WriteBatch()) { + putWriteBatch(writeBatch, key, map); + this.rocksDBConfigManager.batchPutWithWal(writeBatch); + } catch (Exception e) { + log.error("consumer offset persist Failed", e); + } + } + public synchronized void exportToJson() { log.info("RocksDBConsumerOffsetManager export consumer offset to json file"); super.persist(); diff --git a/broker/src/main/java/org/apache/rocketmq/broker/config/v1/RocksDBOffsetSerializeWrapper.java b/broker/src/main/java/org/apache/rocketmq/broker/config/v1/RocksDBOffsetSerializeWrapper.java index 4801cfc681c..552813f0f57 100644 --- a/broker/src/main/java/org/apache/rocketmq/broker/config/v1/RocksDBOffsetSerializeWrapper.java +++ b/broker/src/main/java/org/apache/rocketmq/broker/config/v1/RocksDBOffsetSerializeWrapper.java @@ -16,13 +16,12 @@ */ package org.apache.rocketmq.broker.config.v1; -import java.util.concurrent.ConcurrentHashMap; import java.util.concurrent.ConcurrentMap; import org.apache.rocketmq.remoting.protocol.RemotingSerializable; public class RocksDBOffsetSerializeWrapper extends RemotingSerializable { - private ConcurrentMap offsetTable = new ConcurrentHashMap(16); + private ConcurrentMap offsetTable = null; public ConcurrentMap getOffsetTable() { return offsetTable; diff --git a/broker/src/main/java/org/apache/rocketmq/broker/config/v2/ConsumerOffsetManagerV2.java b/broker/src/main/java/org/apache/rocketmq/broker/config/v2/ConsumerOffsetManagerV2.java index e14ac0bb628..ce8392566ae 100644 --- a/broker/src/main/java/org/apache/rocketmq/broker/config/v2/ConsumerOffsetManagerV2.java +++ b/broker/src/main/java/org/apache/rocketmq/broker/config/v2/ConsumerOffsetManagerV2.java @@ -53,7 +53,7 @@ public ConsumerOffsetManagerV2(BrokerController brokerController, ConfigStorage } @Override - protected void removeConsumerOffset(String topicAtGroup) { + public void removeConsumerOffset(String topicAtGroup) { if (!MixAll.isLmq(topicAtGroup)) { super.removeConsumerOffset(topicAtGroup); } diff --git a/broker/src/main/java/org/apache/rocketmq/broker/lite/AbstractLiteLifecycleManager.java b/broker/src/main/java/org/apache/rocketmq/broker/lite/AbstractLiteLifecycleManager.java new file mode 100644 index 00000000000..e8fb2bde4d0 --- /dev/null +++ b/broker/src/main/java/org/apache/rocketmq/broker/lite/AbstractLiteLifecycleManager.java @@ -0,0 +1,206 @@ +/* + * 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.lite; + +import com.google.common.collect.Sets; +import org.apache.rocketmq.broker.BrokerController; +import org.apache.rocketmq.common.MixAll; +import org.apache.rocketmq.common.Pair; +import org.apache.rocketmq.common.ServiceThread; +import org.apache.rocketmq.common.constant.LoggerName; +import org.apache.rocketmq.common.lite.LiteUtil; +import org.apache.rocketmq.logging.org.slf4j.Logger; +import org.apache.rocketmq.logging.org.slf4j.LoggerFactory; +import org.apache.rocketmq.store.MessageStore; + +import java.util.Collections; +import java.util.List; +import java.util.Map; +import java.util.Set; + +import static org.apache.rocketmq.broker.offset.ConsumerOffsetManager.TOPIC_GROUP_SEPARATOR; + +/** + * Abstract class of lite lifecycle manager, which is used to manage the TTL of lite topics + * and the validity of subscription. The subclasses provide file CQ and rocksdb CQ implementations. + */ +public abstract class AbstractLiteLifecycleManager extends ServiceThread { + private static final Logger LOGGER = LoggerFactory.getLogger(LoggerName.ROCKETMQ_POP_LITE_LOGGER_NAME); + + protected final BrokerController brokerController; + protected final String brokerName; + protected final LiteSharding liteSharding; + protected MessageStore messageStore; + protected Map ttlMap = Collections.emptyMap(); + protected Map> subscriberGroupMap = Collections.emptyMap(); + + public AbstractLiteLifecycleManager(BrokerController brokerController, LiteSharding liteSharding) { + this.brokerController = brokerController; + this.brokerName = brokerController.getBrokerConfig().getBrokerName(); + this.liteSharding = liteSharding; + } + + public void init() { + this.messageStore = brokerController.getMessageStore(); + assert messageStore != null; + } + + /** + * This method actually returns NEXT slot index to use, starting from 0 + */ + public abstract long getMaxOffsetInQueue(String lmqName); + + /** + * Collect expired LMQ of lite topic, and also attach its parent topic name + * return Pair of parent topic and lmq name, not null + */ + public abstract List> collectExpiredLiteTopic(); + + /** + * Collect LMQ by parent topic + * return lmq name list, not null + */ + public abstract List collectByParentTopic(String parentTopic); + + /** + * Check if the subscription for the given LMQ is active. + * A subscription is considered active if either: + * - the current broker is responsible for this LMQ according to the sharding strategy + * - the LMQ exists (has messages) in the message store + */ + public boolean isSubscriptionActive(String parentTopic, String lmqName) { + return brokerName.equals(liteSharding.shardingByLmqName(parentTopic, lmqName)) || isLmqExist(lmqName); + } + + public int getLiteTopicCount(String parentTopic) { + if (!LiteMetadataUtil.isLiteMessageType(parentTopic, brokerController)) { + return 0; + } + return collectByParentTopic(parentTopic).size(); + } + + public boolean isLmqExist(String lmqName) { + return getMaxOffsetInQueue(lmqName) > 0; + } + + public void cleanExpiredLiteTopic() { + try { + updateMetadata(); // necessary + List> lmqToDelete = collectExpiredLiteTopic(); + LOGGER.info("collect expired topic, size:{}", lmqToDelete.size()); + lmqToDelete.forEach(pair -> deleteLmq(pair.getObject1(), pair.getObject2())); + if (!lmqToDelete.isEmpty()) { + brokerController.getMessageStore().getQueueStore().flush(); + } + } catch (Exception e) { + LOGGER.error("cleanExpiredLiteTopic error", e); + } + } + + public void cleanByParentTopic(String parentTopic) { + try { + if (!LiteMetadataUtil.isLiteMessageType(parentTopic, brokerController)) { + return; + } + updateMetadata(); // necessary + List lmqToDelete = collectByParentTopic(parentTopic); + LOGGER.info("clean by parent topic, {}, size:{}", parentTopic, lmqToDelete.size()); + lmqToDelete.forEach(lmqName -> deleteLmq(parentTopic, lmqName)); + } catch (Exception e) { + LOGGER.error("cleanByParentTopic error", e); + } + } + + @Override + public void run() { + LOGGER.info("Start checking lite ttl."); + while (!this.isStopped()) { + long runningTime = System.currentTimeMillis() - brokerController.getShouldStartTime(); + if (runningTime < brokerController.getBrokerConfig().getMinLiteTTl()) { // base protection for restart + this.waitForRunning(20 * 1000); + continue; + } + + cleanExpiredLiteTopic(); + long checkInterval = brokerController.getBrokerConfig().getLiteTtlCheckInterval(); + this.waitForRunning(checkInterval); + } + LOGGER.info("End checking lite ttl."); + } + + public void updateMetadata() { + ttlMap = LiteMetadataUtil.getTopicTtlMap(brokerController); + subscriberGroupMap = LiteMetadataUtil.getSubscriberGroupMap(brokerController); + } + + public boolean isLiteTopicExpired(String parentTopic, String lmqName, long maxOffset) { + if (!LiteUtil.isLiteTopicQueue(lmqName)) { + return false; + } + if (maxOffset <= 0) { + LOGGER.warn("unexpected condition, max offset <= 0, {}, {}", lmqName, maxOffset); + return false; + } + long latestStoreTime = + this.brokerController.getMessageStore().getMessageStoreTimeStamp(lmqName, 0, maxOffset - 1); + long inactiveTime = System.currentTimeMillis() - latestStoreTime; + if (inactiveTime < brokerController.getBrokerConfig().getMinLiteTTl()) { + return false; + } + Integer minutes = ttlMap.get(parentTopic); + if (null == minutes) { + LOGGER.warn("unexpected condition, topic ttl not found. {}", lmqName); + return false; + } + if (minutes <= 0) { + return false; + } + if (hasConsumerLag(lmqName, maxOffset, latestStoreTime, parentTopic)) { + return false; + } + return inactiveTime > minutes * 60 * 1000; + } + + public void deleteLmq(String parentTopic, String lmqName) { + try { + Set groups = subscriberGroupMap.getOrDefault(parentTopic, Collections.emptySet()); + groups.forEach(group -> { + String topicAtGroup = lmqName + TOPIC_GROUP_SEPARATOR + group; + brokerController.getConsumerOffsetManager().getOffsetTable().remove(topicAtGroup); + brokerController.getConsumerOffsetManager().removeConsumerOffset(topicAtGroup); // no iteration + brokerController.getPopLiteMessageProcessor().getConsumerOrderInfoManager().remove(lmqName, group); + }); + brokerController.getMessageStore().deleteTopics(Sets.newHashSet(lmqName)); + boolean sharding = brokerName.equals(liteSharding.shardingByLmqName(parentTopic, lmqName)); + brokerController.getLiteSubscriptionRegistry().cleanSubscription(lmqName, false); + brokerController.getConsumerOffsetManager().getPullOffsetTable().remove( + lmqName + TOPIC_GROUP_SEPARATOR + MixAll.TOOLS_CONSUMER_GROUP); + LOGGER.info("delete lmq finish. {}, sharding:{}", lmqName, sharding); + } catch (Exception e) { + LOGGER.error("delete lmq error. {}", lmqName, e); + } + } + + /** + * Maybe we can check all subscriber groups, but currently consumer lag checking is not performed. + * Only inactive time of message sending is considered for TTL expiration. + */ + public boolean hasConsumerLag(String lmqName, long maxOffset, long latestStoreTime, String parentTopic) { + return false; + } +} diff --git a/broker/src/main/java/org/apache/rocketmq/broker/lite/LiteCtlListener.java b/broker/src/main/java/org/apache/rocketmq/broker/lite/LiteCtlListener.java new file mode 100644 index 00000000000..b9b5bb35523 --- /dev/null +++ b/broker/src/main/java/org/apache/rocketmq/broker/lite/LiteCtlListener.java @@ -0,0 +1,28 @@ +/* + * 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.lite; + +public interface LiteCtlListener { + + void onRegister(String clientId, String group, String lmqName); + + void onUnregister(String clientId, String group, String lmqName); + + void onRemoveAll(String clientId, String group); + +} diff --git a/broker/src/main/java/org/apache/rocketmq/broker/lite/LiteEventDispatcher.java b/broker/src/main/java/org/apache/rocketmq/broker/lite/LiteEventDispatcher.java new file mode 100644 index 00000000000..e2b82906a35 --- /dev/null +++ b/broker/src/main/java/org/apache/rocketmq/broker/lite/LiteEventDispatcher.java @@ -0,0 +1,576 @@ +/* + * 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.lite; + +import com.google.common.annotations.VisibleForTesting; +import com.google.common.cache.Cache; +import com.google.common.cache.CacheBuilder; +import org.apache.commons.collections.CollectionUtils; +import org.apache.rocketmq.broker.BrokerController; +import org.apache.rocketmq.broker.offset.ConsumerOffsetManager; +import org.apache.rocketmq.broker.pop.orderly.ConsumerOrderInfoManager; +import org.apache.rocketmq.common.ServiceThread; +import org.apache.rocketmq.common.constant.LoggerName; +import org.apache.rocketmq.common.entity.ClientGroup; +import org.apache.rocketmq.common.lite.LiteSubscription; +import org.apache.rocketmq.common.lite.LiteUtil; +import org.apache.rocketmq.logging.org.slf4j.Logger; +import org.apache.rocketmq.logging.org.slf4j.LoggerFactory; + +import java.util.ArrayList; +import java.util.Collections; +import java.util.Comparator; +import java.util.HashMap; +import java.util.Iterator; +import java.util.List; +import java.util.Map; +import java.util.Random; +import java.util.Set; +import java.util.concurrent.BlockingQueue; +import java.util.concurrent.ConcurrentHashMap; +import java.util.concurrent.ConcurrentMap; +import java.util.concurrent.ConcurrentSkipListSet; +import java.util.concurrent.LinkedBlockingQueue; +import java.util.concurrent.ThreadLocalRandom; +import java.util.concurrent.TimeUnit; + +public class LiteEventDispatcher extends ServiceThread { + + private static final Logger LOGGER = LoggerFactory.getLogger(LoggerName.ROCKETMQ_POP_LITE_LOGGER_NAME); + private static final Object PRESENT = new Object(); + private static final long CLIENT_INACTIVE_INTERVAL = 10 * 1000; // inactive time when it has unprocessed events + private static final long CLIENT_LONG_POLLING_INTERVAL = 30 * 1000 + 5000; // at least a period of long polling as 30s + private static final long ACTIVE_CONSUMING_WINDOW = 5000; + private static final double LOW_WATER_MARK = 0.2; + private static final int BLACKLIST_EXPIRE_SECONDS = 10; + private static final int SCAN_LOG_INTERVAL = 10000; + + private final BrokerController brokerController; + private final LiteSubscriptionRegistry liteSubscriptionRegistry; + private final AbstractLiteLifecycleManager liteLifecycleManager; + private final ConsumerOffsetManager consumerOffsetManager; + private ConsumerOrderInfoManager consumerOrderInfoManager; + + private final ConcurrentMap clientEventMap = new ConcurrentHashMap<>(); + private final ConcurrentSkipListSet fullDispatchSet = new ConcurrentSkipListSet<>(COMPARATOR); + private final ConcurrentMap fullDispatchMap = new ConcurrentHashMap<>(); // deduplication + private final Cache blacklist = + CacheBuilder.newBuilder().expireAfterWrite(BLACKLIST_EXPIRE_SECONDS, TimeUnit.SECONDS).build(); + private final Random random = ThreadLocalRandom.current(); + private long lastLogTime = System.currentTimeMillis(); + + public LiteEventDispatcher(BrokerController brokerController, + LiteSubscriptionRegistry liteSubscriptionRegistry, AbstractLiteLifecycleManager liteLifecycleManager) { + this.brokerController = brokerController; + this.liteSubscriptionRegistry = liteSubscriptionRegistry; + this.liteLifecycleManager = liteLifecycleManager; + this.consumerOffsetManager = brokerController.getConsumerOffsetManager(); + } + + public void init() { + this.consumerOrderInfoManager = brokerController.getPopLiteMessageProcessor().getConsumerOrderInfoManager(); + this.liteSubscriptionRegistry.addListener(new LiteCtlListenerImpl()); + } + + /** + * If event mode is enabled, try to dispatch event to one client when message arriving or available. + * In most cases, there is only one subscriber for a LMQ under a consumer group, + * but also supports multiple clients consuming in share mode. + * When group is null, dispatch to all subscribers regardless of their group, + * when group is specified, only dispatch to subscribers belonging to this group. + *

+ * If the expected number of subscriptions by each client is small, disabling event mode can be a choice. + */ + public void dispatch(String group, String lmqName, int queueId, long offset, long msgStoreTime) { + if (!this.brokerController.getBrokerConfig().isEnableLiteEventMode()) { + return; + } + if (queueId != 0 || !LiteUtil.isLiteTopicQueue(lmqName)) { + return; + } + doDispatch(group, lmqName, null); + } + + @SuppressWarnings("unchecked") + private void doDispatch(String group, String lmqName, String excludeClientId) { + if (!this.brokerController.getBrokerConfig().isEnableLiteEventMode()) { + return; + } + Object subscribers = getAllSubscriber(group, lmqName); + if (null == subscribers) { + return; + } + if (subscribers instanceof List) { + selectAndDispatch(lmqName, (List) subscribers, excludeClientId); + } + if (subscribers instanceof Map) { + Map> map = (Map>) subscribers; + map.forEach((key, value) -> selectAndDispatch(lmqName, value, excludeClientId)); + } + } + + /** + * Select an appropriate client from the client list and try to dispatch the event to it. + * If there's only one client, dispatch directly to it. + * If there are multiple clients, randomly select one and consider fallback options + * Try to avoid dispatching to the excluded one but fallback if no other choice. + * + * @param clients all clients of one group + * @param excludeClientId the client ID to exclude from selection, probably consuming blocked. + */ + @VisibleForTesting + public void selectAndDispatch(String lmqName, List clients, String excludeClientId) { + if (!this.brokerController.getBrokerConfig().isEnableLiteEventMode()) { + return; + } + if (CollectionUtils.isEmpty(clients)) { + return; + } + + String clientId = null; // the selected one + if (clients.size() == 1) { + clientId = clients.get(0).clientId; + if (brokerController.getBrokerConfig().isEnableLitePopLog() && clientId.equals(excludeClientId)) { + LOGGER.info("no others, still dispatch to {}, {}", clientId, lmqName); + } + if (!tryDispatchToClient(lmqName, clientId, clients.get(0).group)) { + clientId = null; + } + } else { + int start = random.nextInt(clients.size()); + boolean dispatched = false; + List fallbackList = new ArrayList<>(clients.size()); + for (int i = 0; i < clients.size(); i++) { + int index = (start + i) % clients.size(); + clientId = clients.get(index).clientId; + if (clientId.equals(excludeClientId)) { + fallbackList.add(clients.get(index)); + continue; + } + if (blacklist.getIfPresent(clientId) != null) { + fallbackList.add(clients.get(index)); + continue; + } + if (tryDispatchToClient(lmqName, clientId, clients.get(index).group)) { + dispatched = true; + break; + } + } + if (!dispatched) { + clientId = null; + for (ClientGroup clientGroup : fallbackList) { + if (tryDispatchToClient(lmqName, clientGroup.clientId, clientGroup.group)) { + clientId = clientGroup.clientId; + break; + } + } + } + } + if (clientId != null) { + this.brokerController.getPopLiteMessageProcessor().getPopLiteLongPollingService() + .notifyMessageArriving(clientId, true, 0, clients.get(0).group); + } + } + + /** + * Try to dispatch an event to a selected client by adding it to the client's event queue. + * If the event queue is full, mark a full dispatch for retry later. + */ + @VisibleForTesting + public boolean tryDispatchToClient(String lmqName, String clientId, String group) { + ClientEventSet eventSet = clientEventMap.computeIfAbsent(clientId, key -> new ClientEventSet(group)); + if (eventSet.offer(lmqName)) { + return true; + } + scheduleFullDispatch(clientId, group, blacklist.getIfPresent(clientId) != null); + LOGGER.warn("client event set is full. {}", clientId); + return false; + } + + /** + * Get an iterator for iterating over events for a specific client. + * In lite event mode, returns events from the client's event queue, + * or else returns topics from the client's subscription. + */ + public Iterator getEventIterator(String clientId) { + if (this.brokerController.getBrokerConfig().isEnableLiteEventMode()) { + return new EventSetIterator(clientEventMap.get(clientId)); + } else { + LiteSubscription liteSubscription = liteSubscriptionRegistry.getLiteSubscription(clientId); + return liteSubscription != null && liteSubscription.getLiteTopicSet() != null ? + new LiteSubscriptionIterator(liteSubscription.getTopic(), liteSubscription.getLiteTopicSet().iterator()) + : Collections.emptyIterator(); + } + } + + /** + * Perform a full dispatch for a client which was previously marked for a delayed full dispatch. + * This always happens when a client's event queue is full or re-dispatching is needed. + * It iterates through all LMQ topics subscribed by the client and dispatches events for those + * with available messages. + */ + public void doFullDispatch(String clientId, String group) { + if (!this.brokerController.getBrokerConfig().isEnableLiteEventMode()) { + return; + } + LiteSubscription subscription = liteSubscriptionRegistry.getLiteSubscription(clientId); + if (null == subscription || CollectionUtils.isEmpty(subscription.getLiteTopicSet())) { + LOGGER.info("client full dispatch, but no subscription. {}", clientId); + return; + } + ClientEventSet eventSet = clientEventMap.computeIfAbsent(clientId, key -> new ClientEventSet(group)); + if (eventSet.maybeBlock()) { + LOGGER.warn("client may block for a while, wait another period. {}", clientId); + scheduleFullDispatch(clientId, group, true); + return; + } + boolean isActiveConsuming = eventSet.isActiveConsuming(); + if (!eventSet.isLowWaterMark()) { + LOGGER.warn("client event set high water mark, wait another period. {}, {}", clientId, isActiveConsuming); + scheduleFullDispatch(clientId, group, !isActiveConsuming); + return; + } + LOGGER.info("client full dispatch, {}, total:{}", clientId, subscription.getLiteTopicSet().size()); + int count = 0; + for (String lmqName : subscription.getLiteTopicSet()) { + long maxOffset = liteLifecycleManager.getMaxOffsetInQueue(lmqName); + if (maxOffset <= 0) { + continue; + } + long consumerOffset = consumerOffsetManager.queryOffset(group, lmqName, 0); + if (consumerOffset >= maxOffset) { + continue; + } + if (eventSet.offer(lmqName)) { + if (count++ % 10 == 0) { + brokerController.getPopLiteMessageProcessor().getPopLiteLongPollingService() + .notifyMessageArriving(clientId, true, 0, group); + } + } else { + LOGGER.warn("client event set full again, wait another period. {}, {}", clientId, isActiveConsuming); + scheduleFullDispatch(clientId, group, !isActiveConsuming); + break; + } + } + brokerController.getPopLiteMessageProcessor().getPopLiteLongPollingService() + .notifyMessageArriving(clientId, true, 0, group); + LOGGER.info("client full dispatch finish. {}, dispatch:{}", clientId, count); + } + + /** + * Perform a full dispatch for all clients under a specific group, only invoked by admin for now. + */ + public void doFullDispatchByGroup(String group) { + List clientIds = liteSubscriptionRegistry.getAllClientIdByGroup(group); + LOGGER.info("do full dispatch by group, {}, size:{}", group, clientIds.size()); + for (String clientId : clientIds) { + doFullDispatch(clientId, group); + } + } + + public void scheduleFullDispatch(String clientId, String group, boolean reentry) { + if (fullDispatchMap.putIfAbsent(clientId, PRESENT) != null) { + return; + } + int randomDelay = reentry ? random.nextInt(25 * 1000) : 0; + fullDispatchSet.add(new FullDispatchRequest(clientId, group, + brokerController.getBrokerConfig().getLiteEventFullDispatchDelayTime() + randomDelay)); + } + + /** + * Get all subscribers for a specific LMQ, with optional group filtering. + * To avoid unnecessary comparisons and wrapping, Object is used as the return type here. + * This method returns different types based on the subscription scenario: + * 1. When there's only one subscriber, return List + * 2. When group is specified, return List containing subscribers of that group + * 3. When group is null and multiple groups exist, return Map> + * mapping each group to its subscribers + * + * @return Object that can be either List or Map> or null if not found + */ + @VisibleForTesting + public Object getAllSubscriber(String group, String lmqName) { + Set observers = liteSubscriptionRegistry.getSubscriber(lmqName); + if (null == observers || observers.isEmpty()) { + return null; + } + if (observers.size() == 1) { + if (null == group || group.equals(observers.iterator().next().group)) { + return new ArrayList<>(observers); + } + return null; + } + if (group != null) { + List result = new ArrayList<>(4); + for (ClientGroup ele : observers) { + if (group.equals(ele.group)) { + result.add(ele); + } + } + return !result.isEmpty() ? result : null; + } + + Map> group2Clients = new HashMap<>(4); + for (ClientGroup ele : observers) { + group2Clients.computeIfAbsent(ele.group, k -> new ArrayList<>(2)).add(ele); + } + return group2Clients; + } + + /** + * Get the last access time of a client's event set. + * + * @param clientId the client id + * @return the last access time in milliseconds, or -1 if client not found + */ + public long getClientLastAccessTime(String clientId) { + ClientEventSet eventSet = clientEventMap.get(clientId); + if (eventSet != null) { + return eventSet.lastAccessTime; + } + return -1; + } + + @Override + public String getServiceName() { + if (brokerController.getBrokerConfig().isInBrokerContainer()) { + return brokerController.getBrokerIdentity().getIdentifier() + LiteEventDispatcher.class.getSimpleName(); + } + return LiteEventDispatcher.class.getSimpleName(); + } + + @Override + public void run() { + while (!this.isStopped()) { + long checkInterval = brokerController.getBrokerConfig().getLiteEventCheckInterval(); + this.waitForRunning(checkInterval); + try { + scan(); + } catch (Exception e) { + LOGGER.error("LiteEventDispatcher-scan error.", e); + } + } + } + + /** + * Due to the event pre-allocation mechanism, it is necessary to perform + * two main tasks to check inactive event queues and do full dispatch to reduce potential delivery latency. + * 1. Check client event set for inactive clients and re-dispatches their events + * 2. Process delayed full dispatch requests that are ready to be executed + */ + public void scan() { + boolean needLog = System.currentTimeMillis() - lastLogTime > SCAN_LOG_INTERVAL; + + // 1. check all client event set + if (needLog) { + LOGGER.info("Check client event set. size:{}", clientEventMap.size()); + lastLogTime = System.currentTimeMillis(); + } + Iterator> iterator = clientEventMap.entrySet().iterator(); + while (iterator.hasNext()) { + Map.Entry entry = iterator.next(); + ClientEventSet eventSet = entry.getValue(); + if (!eventSet.maybeBlock()) { + continue; + } + String clientId = entry.getKey(); + LOGGER.warn("remove inactive client and re-dispatch. {}, {}", clientId, eventSet.events.size()); + iterator.remove(); + blacklist.put(clientId, PRESENT); + String event; + while ((event = eventSet.poll()) != null) { + doDispatch(eventSet.group, event, clientId); // may still dispatch to current client + } + } + + // 2. perform full dispatch + if (needLog) { + LOGGER.info("Begin to trigger full dispatch. size:{}, mapSize:{}", fullDispatchSet.size(), fullDispatchMap.size()); + lastLogTime = System.currentTimeMillis(); + } + FullDispatchRequest request; + while ((request = fullDispatchSet.pollFirst()) != null) { + if (request.timestamp > System.currentTimeMillis()) { + fullDispatchSet.add(request); + break; + } + fullDispatchMap.remove(request.clientId); + doFullDispatch(request.clientId, request.group); + } + } + + public int getEventMapSize() { + return clientEventMap.size(); + } + + /** + * We use dual data structure to maintain the event queue for each client + * and ensure event deduplication to avoid duplicate events, although it + * has a bit more memory usage than a single concurrent set. + */ + class ClientEventSet { + private final BlockingQueue events; + private final ConcurrentMap map = new ConcurrentHashMap<>(); + private final String group; + private volatile long lastAccessTime = System.currentTimeMillis(); + private volatile long lastConsumeTime = System.currentTimeMillis(); + + public ClientEventSet(String group) { + this.group = group; + events = new LinkedBlockingQueue<>(LiteMetadataUtil.getMaxClientEventCount(group, brokerController)); + } + + // return false if and only if the queue is full, has race condition with poll(), but no side effect. + public boolean offer(String event) { + if (events.remainingCapacity() == 0) { + return false; + } + boolean rst; + if (map.putIfAbsent(event, PRESENT) == null) { + rst = events.offer(event); + if (!rst) { + map.remove(event); + } + } else { + rst = true; + } + return rst; + } + + public String poll() { + lastAccessTime = System.currentTimeMillis(); + String event = events.poll(); + if (event != null) { + map.remove(event); + lastConsumeTime = System.currentTimeMillis(); + } + return event; + } + + public boolean maybeBlock() { + long inactiveTime = System.currentTimeMillis() - lastAccessTime; + return inactiveTime > CLIENT_LONG_POLLING_INTERVAL + || !events.isEmpty() && inactiveTime > CLIENT_INACTIVE_INTERVAL; + } + + public boolean isLowWaterMark() { + int used = events.size(); + return (double) used / (used + events.remainingCapacity()) < LOW_WATER_MARK; + } + + public boolean isActiveConsuming() { + return System.currentTimeMillis() - lastAccessTime < ACTIVE_CONSUMING_WINDOW; + } + + public int size() { + return events.size(); + } + } + + class LiteCtlListenerImpl implements LiteCtlListener { + + @Override + public void onRegister(String clientId, String group, String lmqName) { + if (liteLifecycleManager.isLmqExist(lmqName)) { + doDispatch(group, lmqName, null); + } + } + + @Override + public void onUnregister(String clientId, String group, String lmqName) { + } + + /** + * Mostly triggered when client channel closed, ensure that lite subscriptions is cleared before. + */ + @Override + public void onRemoveAll(String clientId, String group) { + ClientEventSet eventSet = clientEventMap.remove(clientId); + if (null == eventSet) { + return; + } + LOGGER.warn("Maybe client offline. {}", clientId); + String event; + while ((event = eventSet.poll()) != null) { + doDispatch(eventSet.group, event, clientId); + } + } + } + + static class EventSetIterator implements Iterator { + private final ClientEventSet eventSet; + + public EventSetIterator(ClientEventSet eventSet) { + this.eventSet = eventSet; + } + + @Override + public boolean hasNext() { + return eventSet != null && !eventSet.events.isEmpty(); + } + + @Override + public String next() { + return eventSet.poll(); + } + } + + static class LiteSubscriptionIterator implements Iterator { + private final Iterator iterator; + private final String parentTopic; + public LiteSubscriptionIterator(String parentTopic, Iterator iterator) { + this.parentTopic = parentTopic; + this.iterator = iterator; + } + @Override + public boolean hasNext() { + return iterator.hasNext(); + } + + @Override + public String next() { + return iterator.next(); + } + } + + static class FullDispatchRequest { + private final String clientId; + private final String group; + private final long timestamp; + public FullDispatchRequest(String clientId, String group, long delayMillis) { + this.clientId = clientId; + this.group = group; + this.timestamp = System.currentTimeMillis() + delayMillis; + } + } + + // no need to compare group + static final Comparator COMPARATOR = (r1, r2) -> { + if (null == r1 || null == r2 || null == r1.clientId || null == r2.clientId) { + return 0; + } + if (r1.clientId.equals(r2.clientId)) { + return 0; + } + int ret = Long.compare(r1.timestamp, r2.timestamp); + if (ret != 0) { + return ret; + } + return r1.clientId.compareTo(r2.clientId); + }; +} diff --git a/broker/src/main/java/org/apache/rocketmq/broker/lite/LiteLifecycleManager.java b/broker/src/main/java/org/apache/rocketmq/broker/lite/LiteLifecycleManager.java new file mode 100644 index 00000000000..8cbf9c48e5d --- /dev/null +++ b/broker/src/main/java/org/apache/rocketmq/broker/lite/LiteLifecycleManager.java @@ -0,0 +1,89 @@ +/* + * 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.lite; + +import org.apache.commons.lang3.StringUtils; +import org.apache.rocketmq.broker.BrokerController; +import org.apache.rocketmq.common.Pair; +import org.apache.rocketmq.common.constant.LoggerName; +import org.apache.rocketmq.common.lite.LiteUtil; +import org.apache.rocketmq.logging.org.slf4j.Logger; +import org.apache.rocketmq.logging.org.slf4j.LoggerFactory; +import org.apache.rocketmq.store.queue.ConsumeQueueInterface; + +import java.util.ArrayList; +import java.util.Collections; +import java.util.Iterator; +import java.util.List; +import java.util.Map; +import java.util.concurrent.ConcurrentMap; + +public class LiteLifecycleManager extends AbstractLiteLifecycleManager { + private static final Logger LOGGER = LoggerFactory.getLogger(LoggerName.ROCKETMQ_POP_LITE_LOGGER_NAME); + + public LiteLifecycleManager(BrokerController brokerController, LiteSharding liteSharding) { + super(brokerController, liteSharding); + } + + @Override + public long getMaxOffsetInQueue(String lmqName) { + ConsumeQueueInterface consumeQueue = messageStore.getConsumeQueue(lmqName, 0); + return consumeQueue != null ? consumeQueue.getMaxOffsetInQueue() : 0L; + } + + @Override + public List collectByParentTopic(String parentTopic) { + if (StringUtils.isEmpty(parentTopic)) { + return Collections.emptyList(); + } + List resultList = new ArrayList<>(); + Iterator>> iterator = + messageStore.getQueueStore().getConsumeQueueTable().entrySet().iterator(); + while (iterator.hasNext()) { + Map.Entry> entry = iterator.next(); + if (LiteUtil.belongsTo(entry.getKey(), parentTopic)) { + resultList.add(entry.getKey()); + } + } + return resultList; + } + + @Override + public List> collectExpiredLiteTopic() { + List> lmqToDelete = new ArrayList<>(); + Iterator>> iterator = + messageStore.getQueueStore().getConsumeQueueTable().entrySet().iterator(); + while (iterator.hasNext()) { + Map.Entry> entry = iterator.next(); + String lmqName = entry.getKey(); + String parentTopic = LiteUtil.getParentTopic(lmqName); + if (null == parentTopic) { + continue; + } + Map map = entry.getValue(); + if (map.size() != 1 || null == map.get(0)) { + LOGGER.warn("unexpected lmq count. {}", lmqName); + continue; + } + if (isLiteTopicExpired(parentTopic, entry.getKey(), map.get(0).getMaxOffsetInQueue())) { + lmqToDelete.add(new Pair<>(parentTopic, lmqName)); + } + } + return lmqToDelete; + } +} diff --git a/broker/src/main/java/org/apache/rocketmq/broker/lite/LiteMetadataUtil.java b/broker/src/main/java/org/apache/rocketmq/broker/lite/LiteMetadataUtil.java new file mode 100644 index 00000000000..aa78f384a90 --- /dev/null +++ b/broker/src/main/java/org/apache/rocketmq/broker/lite/LiteMetadataUtil.java @@ -0,0 +1,135 @@ +/* + * 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.lite; + +import java.util.Collections; +import java.util.Map; +import java.util.Set; +import java.util.concurrent.ConcurrentMap; +import java.util.stream.Collectors; +import org.apache.rocketmq.broker.BrokerController; +import org.apache.rocketmq.common.TopicConfig; +import org.apache.rocketmq.common.attribute.TopicMessageType; +import org.apache.rocketmq.remoting.protocol.subscription.SubscriptionGroupConfig; + +public class LiteMetadataUtil { + + public static boolean isConsumeEnable(String group, BrokerController brokerController) { + if (null == group || null == brokerController) { + return false; + } + SubscriptionGroupConfig groupConfig = + brokerController.getSubscriptionGroupManager().findSubscriptionGroupConfig(group); + return null != groupConfig && groupConfig.isConsumeEnable(); + } + + public static boolean isLiteMessageType(String parentTopic, BrokerController brokerController) { + if (null == parentTopic || null == brokerController) { + return false; + } + TopicConfig topicConfig = brokerController.getTopicConfigManager().selectTopicConfig(parentTopic); + return topicConfig != null && TopicMessageType.LITE.equals(topicConfig.getTopicMessageType()); + } + + public static boolean isLiteGroupType(String group, BrokerController brokerController) { + if (null == group || null == brokerController) { + return false; + } + SubscriptionGroupConfig groupConfig = + brokerController.getSubscriptionGroupManager().findSubscriptionGroupConfig(group); + return null != groupConfig && groupConfig.getLiteBindTopic() != null; + } + + public static String getLiteBindTopic(String group, BrokerController brokerController) { + if (null == group || null == brokerController) { + return null; + } + SubscriptionGroupConfig groupConfig = + brokerController.getSubscriptionGroupManager().findSubscriptionGroupConfig(group); + return null != groupConfig ? groupConfig.getLiteBindTopic() : null; + } + + public static boolean isSubLiteExclusive(String group, BrokerController brokerController) { + if (null == group || null == brokerController) { + return false; + } + SubscriptionGroupConfig groupConfig = + brokerController.getSubscriptionGroupManager().findSubscriptionGroupConfig(group); + return null != groupConfig && groupConfig.isLiteSubExclusive(); + } + + public static boolean isResetOffsetInExclusiveMode(String group, BrokerController brokerController) { + if (null == group || null == brokerController) { + return false; + } + SubscriptionGroupConfig groupConfig = + brokerController.getSubscriptionGroupManager().findSubscriptionGroupConfig(group); + return null != groupConfig && groupConfig.isResetOffsetInExclusiveMode(); + } + + public static boolean isResetOffsetOnUnsubscribe(String group, BrokerController brokerController) { + if (null == group || null == brokerController) { + return false; + } + SubscriptionGroupConfig groupConfig = + brokerController.getSubscriptionGroupManager().findSubscriptionGroupConfig(group); + return null != groupConfig && groupConfig.isResetOffsetOnUnsubscribe(); + } + + public static int getMaxClientEventCount(String group, BrokerController brokerController) { + if (null == group || null == brokerController) { + return -1; + } + SubscriptionGroupConfig groupConfig = + brokerController.getSubscriptionGroupManager().findSubscriptionGroupConfig(group); + if (null == groupConfig || groupConfig.getMaxClientEventCount() <= 0) { + return brokerController.getBrokerConfig().getMaxClientEventCount(); + } + return groupConfig.getMaxClientEventCount(); + } + + public static Map getTopicTtlMap(BrokerController brokerController) { + if (null == brokerController) { + return Collections.emptyMap(); + } + ConcurrentMap topicConfigTable = + brokerController.getTopicConfigManager().getTopicConfigTable(); + + return topicConfigTable.entrySet().stream() + .filter(entry -> entry.getValue().getTopicMessageType().equals(TopicMessageType.LITE)) + .collect(Collectors.toMap( + entry -> entry.getKey(), + entry -> entry.getValue().getLiteTopicExpiration() + )); + } + + public static Map> getSubscriberGroupMap(BrokerController brokerController) { + if (null == brokerController) { + return Collections.emptyMap(); + } + ConcurrentMap groupTable = + brokerController.getSubscriptionGroupManager().getSubscriptionGroupTable(); + + return groupTable.entrySet().stream() + .filter(entry -> entry.getValue().getLiteBindTopic() != null) + .collect(Collectors.groupingBy( + entry -> entry.getValue().getLiteBindTopic(), + Collectors.mapping(Map.Entry::getKey, Collectors.toSet()) + )); + } +} diff --git a/broker/src/main/java/org/apache/rocketmq/broker/lite/LiteQuotaException.java b/broker/src/main/java/org/apache/rocketmq/broker/lite/LiteQuotaException.java new file mode 100644 index 00000000000..d6079c68579 --- /dev/null +++ b/broker/src/main/java/org/apache/rocketmq/broker/lite/LiteQuotaException.java @@ -0,0 +1,24 @@ +/* + * 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.lite; + +public class LiteQuotaException extends RuntimeException { + public LiteQuotaException(String message) { + super(message); + } +} diff --git a/broker/src/main/java/org/apache/rocketmq/broker/lite/LiteSharding.java b/broker/src/main/java/org/apache/rocketmq/broker/lite/LiteSharding.java new file mode 100644 index 00000000000..081c612522d --- /dev/null +++ b/broker/src/main/java/org/apache/rocketmq/broker/lite/LiteSharding.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.broker.lite; + +public interface LiteSharding { + + String shardingByLmqName(String parentTopic, String lmqName); +} diff --git a/broker/src/main/java/org/apache/rocketmq/broker/lite/LiteShardingImpl.java b/broker/src/main/java/org/apache/rocketmq/broker/lite/LiteShardingImpl.java new file mode 100644 index 00000000000..fec4085d1f1 --- /dev/null +++ b/broker/src/main/java/org/apache/rocketmq/broker/lite/LiteShardingImpl.java @@ -0,0 +1,60 @@ +/* + * 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.lite; + +import com.google.common.hash.Hashing; +import org.apache.commons.collections.CollectionUtils; +import org.apache.commons.lang3.StringUtils; +import org.apache.rocketmq.broker.BrokerController; +import org.apache.rocketmq.broker.topic.TopicRouteInfoManager; +import org.apache.rocketmq.client.impl.producer.TopicPublishInfo; +import org.apache.rocketmq.common.lite.LiteUtil; +import org.apache.rocketmq.common.message.MessageQueue; + +import java.util.List; + +public class LiteShardingImpl implements LiteSharding { + + private final BrokerController brokerController; + private final TopicRouteInfoManager topicRouteInfoManager; + + public LiteShardingImpl(BrokerController brokerController, TopicRouteInfoManager topicRouteInfoManager) { + this.brokerController = brokerController; + this.topicRouteInfoManager = topicRouteInfoManager; + } + + @Override + public String shardingByLmqName(String parentTopic, String lmqName) { + TopicPublishInfo topicPublishInfo = topicRouteInfoManager.tryToFindTopicPublishInfo(parentTopic); + if (topicPublishInfo == null) { + // if topic not exist, return current broker + return brokerController.getBrokerConfig().getBrokerName(); + } + List writeQueues = topicPublishInfo.getMessageQueueList(); + if (CollectionUtils.isEmpty(writeQueues)) { + return brokerController.getBrokerConfig().getBrokerName(); + } + String liteTopic = LiteUtil.getLiteTopic(lmqName); + if (StringUtils.isEmpty(liteTopic)) { + return brokerController.getBrokerConfig().getBrokerName(); + } + int bucket = Hashing.consistentHash(liteTopic.hashCode(), writeQueues.size()); + MessageQueue targetQueue = writeQueues.get(bucket); + return targetQueue.getBrokerName(); + } +} diff --git a/broker/src/main/java/org/apache/rocketmq/broker/lite/LiteSubscriptionRegistry.java b/broker/src/main/java/org/apache/rocketmq/broker/lite/LiteSubscriptionRegistry.java new file mode 100644 index 00000000000..92d6b4ea7c1 --- /dev/null +++ b/broker/src/main/java/org/apache/rocketmq/broker/lite/LiteSubscriptionRegistry.java @@ -0,0 +1,55 @@ +/* + * 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.lite; + +import io.netty.channel.Channel; + +import java.util.List; +import java.util.Set; +import org.apache.rocketmq.common.entity.ClientGroup; +import org.apache.rocketmq.common.lite.LiteSubscription; +import org.apache.rocketmq.common.lite.OffsetOption; + +public interface LiteSubscriptionRegistry { + + void updateClientChannel(String clientId, Channel channel); + + LiteSubscription getLiteSubscription(String clientId); + + int getActiveSubscriptionNum(); + + void addPartialSubscription(String clientId, String group, String topic, Set lmqNameSet, OffsetOption offsetOption); + + void removePartialSubscription(String clientId, String group, String topic, Set lmqNameSet); + + void addCompleteSubscription(String clientId, String group, String topic, Set newLmqNameSet, long version); + + void removeCompleteSubscription(String clientId); + + void addListener(LiteCtlListener listener); + + Set getSubscriber(String lmqName); + + List getAllClientIdByGroup(String group); + + void cleanSubscription(String lmqName, boolean notifyClient); + + void start(); + + void shutdown(); +} diff --git a/broker/src/main/java/org/apache/rocketmq/broker/lite/LiteSubscriptionRegistryImpl.java b/broker/src/main/java/org/apache/rocketmq/broker/lite/LiteSubscriptionRegistryImpl.java new file mode 100644 index 00000000000..dc02e6393af --- /dev/null +++ b/broker/src/main/java/org/apache/rocketmq/broker/lite/LiteSubscriptionRegistryImpl.java @@ -0,0 +1,360 @@ +/* + * 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.lite; + +import com.google.common.annotations.VisibleForTesting; +import io.netty.channel.Channel; +import java.util.ArrayList; +import java.util.List; +import java.util.Map; +import java.util.Objects; +import java.util.Set; +import java.util.concurrent.ConcurrentHashMap; +import java.util.concurrent.ConcurrentMap; +import java.util.concurrent.atomic.AtomicInteger; +import java.util.stream.Collectors; +import org.apache.commons.collections.CollectionUtils; +import org.apache.rocketmq.broker.BrokerController; +import org.apache.rocketmq.common.ServiceThread; +import org.apache.rocketmq.common.constant.LoggerName; +import org.apache.rocketmq.common.entity.ClientGroup; +import org.apache.rocketmq.common.lite.LiteSubscription; +import org.apache.rocketmq.common.lite.LiteUtil; +import org.apache.rocketmq.common.lite.OffsetOption; +import org.apache.rocketmq.common.utils.ConcurrentHashMapUtils; +import org.apache.rocketmq.logging.org.slf4j.Logger; +import org.apache.rocketmq.logging.org.slf4j.LoggerFactory; +import org.apache.rocketmq.remoting.protocol.header.NotifyUnsubscribeLiteRequestHeader; + +public class LiteSubscriptionRegistryImpl extends ServiceThread implements LiteSubscriptionRegistry { + private static final Logger LOGGER = LoggerFactory.getLogger(LoggerName.ROCKETMQ_POP_LITE_LOGGER_NAME); + + protected final ConcurrentMap clientChannels = new ConcurrentHashMap<>(); + protected final ConcurrentMap client2Subscription = new ConcurrentHashMap<>(); + protected final ConcurrentMap> liteTopic2Group = new ConcurrentHashMap<>(); + + private final List listeners = new ArrayList<>(); + private final BrokerController brokerController; + private final AbstractLiteLifecycleManager liteLifecycleManager; + + public LiteSubscriptionRegistryImpl(BrokerController brokerController, + AbstractLiteLifecycleManager liteLifecycleManager) { + this.brokerController = brokerController; + this.liteLifecycleManager = liteLifecycleManager; + } + + // Number of active liteTopic references. + // [(client1, liteTopic1), (client2, liteTopic1)] counts as two active references. + protected final AtomicInteger activeNum = new AtomicInteger(0); + + @Override + public void updateClientChannel(String clientId, Channel channel) { + clientChannels.put(clientId, channel); + } + + @Override + public void addPartialSubscription(String clientId, String group, String topic, Set lmqNameSet, + OffsetOption offsetOption) { + long maxCount = brokerController.getBrokerConfig().getMaxLiteSubscriptionCount(); + if (getActiveSubscriptionNum() >= maxCount) { + // No need to check existence, if reach here, it must be new. + throw new LiteQuotaException("lite subscription quota exceeded " + maxCount); + } + + LiteSubscription thisSub = getOrCreateLiteSubscription(clientId, group, topic); + // Utilize existing string object + final ClientGroup clientGroup = new ClientGroup(clientId, thisSub.getGroup()); + for (String lmqName : lmqNameSet) { + if (!liteLifecycleManager.isSubscriptionActive(topic, lmqName)) { + continue; + } + thisSub.addLiteTopic(lmqName); + // First remove the old subscription + if (LiteMetadataUtil.isSubLiteExclusive(group, brokerController)) { + excludeClientByLmqName(clientId, group, lmqName); + } + resetOffset(lmqName, group, clientId, offsetOption); + addTopicGroup(clientGroup, lmqName); + } + } + + @Override + public void removePartialSubscription(String clientId, String group, String topic, Set lmqNameSet) { + LiteSubscription thisSub = getOrCreateLiteSubscription(clientId, group, topic); + ClientGroup clientGroup = new ClientGroup(clientId, thisSub.getGroup()); + boolean isResetOffsetOnUnsubscribe = LiteMetadataUtil.isResetOffsetOnUnsubscribe(group, brokerController); + for (String lmqName : lmqNameSet) { + thisSub.removeLiteTopic(lmqName); + removeTopicGroup(clientGroup, lmqName, isResetOffsetOnUnsubscribe); + } + } + + @Override + public void addCompleteSubscription(String clientId, String group, String topic, Set lmqNameAll, long version) { + Set lmqNameNew = lmqNameAll.stream() + .filter(lmqName -> liteLifecycleManager.isSubscriptionActive(topic, lmqName)) + .collect(Collectors.toSet()); + + LiteSubscription thisSub = getOrCreateLiteSubscription(clientId, group, topic); + Set lmqNamePrev = thisSub.getLiteTopicSet(); + // Find topics to remove (in current set but not in new set) + Set lmqNameRemove = lmqNamePrev.stream() + .filter(lmqName -> !lmqNameNew.contains(lmqName)) + .collect(Collectors.toSet()); + + ClientGroup clientGroup = new ClientGroup(clientId, thisSub.getGroup()); + lmqNameRemove.forEach(lmqName -> { + thisSub.removeLiteTopic(lmqName); + removeTopicGroup(clientGroup, lmqName, false); + }); + lmqNameNew.forEach(lmqName -> { + thisSub.addLiteTopic(lmqName); + addTopicGroup(clientGroup, lmqName); + }); + } + + @Override + public void removeCompleteSubscription(String clientId) { + clientChannels.remove(clientId); + LiteSubscription thisSub = client2Subscription.remove(clientId); + if (thisSub == null) { + return; + } + LOGGER.info("removeCompleteSubscription, topic:{}, group:{}, clientId:{}", thisSub.getTopic(), thisSub.getGroup(), clientId); + ClientGroup clientGroup = new ClientGroup(clientId, thisSub.getGroup()); + thisSub.getLiteTopicSet().forEach(lmqName -> { + removeTopicGroup(clientGroup, lmqName, false); + }); + for (LiteCtlListener listener : listeners) { + listener.onRemoveAll(clientId, thisSub.getGroup()); + } + } + + @Override + public void addListener(LiteCtlListener listener) { + listeners.add(listener); + } + + @Override + public Set getSubscriber(String lmqName) { + return liteTopic2Group.get(lmqName); + } + + /** + * Cleans up subscription for the given LMQ name. + * Removes all related client subscriptions and notifies listeners. + * + * @param lmqName the LMQ name to clean up + */ + @Override + public void cleanSubscription(String lmqName, boolean notifyClient) { + Set topicGroupSet = liteTopic2Group.remove(lmqName); + if (CollectionUtils.isEmpty(topicGroupSet)) { + return; + } + for (ClientGroup topicGroup : topicGroupSet) { + LiteSubscription liteSubscription = client2Subscription.get(topicGroup.clientId); + if (liteSubscription == null) { + continue; + } + if (liteSubscription.removeLiteTopic(lmqName)) { + if (notifyClient) { + notifyUnsubscribeLite(topicGroup.clientId, topicGroup.group, lmqName); + } + activeNum.decrementAndGet(); + } + } + } + + protected void addTopicGroup(ClientGroup clientGroup, String lmqName) { + Set topicGroupSet = liteTopic2Group + .computeIfAbsent(lmqName, k -> ConcurrentHashMap.newKeySet()); + if (topicGroupSet.add(clientGroup)) { + activeNum.incrementAndGet(); + for (LiteCtlListener listener : listeners) { + listener.onRegister(clientGroup.clientId, clientGroup.group, lmqName); + } + } + } + + protected void removeTopicGroup(ClientGroup clientGroup, String lmqName, boolean resetOffset) { + Set topicGroupSet = liteTopic2Group.get(lmqName); + if (topicGroupSet == null) { + return; + } + if (topicGroupSet.remove(clientGroup)) { + activeNum.decrementAndGet(); + for (LiteCtlListener listener : listeners) { + listener.onUnregister(clientGroup.clientId, clientGroup.group, lmqName); + } + if (resetOffset) { + resetOffset(lmqName, clientGroup.group, clientGroup.clientId, + new OffsetOption(OffsetOption.Type.POLICY, OffsetOption.POLICY_MIN_VALUE)); + } + } + if (topicGroupSet.isEmpty()) { + liteTopic2Group.remove(lmqName); + } + } + + /** + * Remove clients that subscribe to the same liteTopic under the same group + */ + protected void excludeClientByLmqName(String newClientId, String group, String lmqName) { + Set clientSet = liteTopic2Group.get(lmqName); + if (CollectionUtils.isEmpty(clientSet)) { + return; + } + List toRemove = clientSet.stream() + .filter(clientGroup -> Objects.equals(group, clientGroup.group)) + .collect(Collectors.toList()); + + toRemove.forEach(clientGroup -> { + LiteSubscription liteSubscription = client2Subscription.get(clientGroup.clientId); + if (liteSubscription != null) { + liteSubscription.removeLiteTopic(lmqName); + } + notifyUnsubscribeLite(clientGroup.clientId, clientGroup.group, lmqName); + boolean resetOffset = LiteMetadataUtil.isResetOffsetInExclusiveMode(group, brokerController); + LOGGER.info("excludeClientByLmqName group:{}, lmqName:{}, resetOffset:{}, clientId:{} -> {}", + group, lmqName, resetOffset, clientGroup.clientId, newClientId); + removeTopicGroup(clientGroup, lmqName, resetOffset); + }); + } + + /** + * Notify the client to remove the liteTopic subscription from its local memory + */ + private void notifyUnsubscribeLite(String clientId, String group, String lmqName) { + String topic = LiteUtil.getParentTopic(lmqName); + String liteTopic = LiteUtil.getLiteTopic(lmqName); + Channel channel = clientChannels.get(clientId); + if (channel == null) { + LOGGER.warn("notifyUnsubscribeLite but channel is null, liteTopic:{}, group:{}, topic:{}, clientId:{},", + liteTopic, group, topic, clientId); + return; + } + + NotifyUnsubscribeLiteRequestHeader header = new NotifyUnsubscribeLiteRequestHeader(); + header.setClientId(clientId); + header.setConsumerGroup(group); + header.setLiteTopic(liteTopic); + brokerController.getBroker2Client().notifyUnsubscribeLite(channel, header); + LOGGER.info("notifyUnsubscribeLite liteTopic:{}, group:{}, topic:{}, clientId:{}", liteTopic, group, topic, clientId); + } + + @Override + public LiteSubscription getLiteSubscription(String clientId) { + return client2Subscription.get(clientId); + } + + @Override + public int getActiveSubscriptionNum() { + return activeNum.get(); + } + + @Override + public List getAllClientIdByGroup(String group) { + return client2Subscription.entrySet().stream() + .filter(entry -> entry.getValue().getGroup().equals(group)) + .map(Map.Entry::getKey) + .collect(Collectors.toList()); + } + + protected void resetOffset(String lmqName, String group, String clientId, OffsetOption offsetOption) { + if (null == offsetOption) { + return; + } + Long targetOffset = null; + long currentOffset = brokerController.getConsumerOffsetManager().queryOffset(group, lmqName, 0); + switch (offsetOption.getType()) { + case POLICY: + if (OffsetOption.POLICY_MIN_VALUE == offsetOption.getValue()) { + targetOffset = 0L; + } else if (OffsetOption.POLICY_MAX_VALUE == offsetOption.getValue()) { + targetOffset = liteLifecycleManager.getMaxOffsetInQueue(lmqName); + } + break; + case OFFSET: + targetOffset = offsetOption.getValue(); + break; + case TAIL_N: + if (currentOffset >= 0) { // only when consumer offset exists + targetOffset = Math.max(0L, currentOffset - offsetOption.getValue()); + } + break; + case TIMESTAMP: + // timestamp option is disabled silently for now + break; + } + + LOGGER.info("try to reset lite offset. {}, {}, {}, {}, current:{}, target:{}", + group, lmqName, clientId, offsetOption, currentOffset, targetOffset); + if (targetOffset != null && currentOffset != targetOffset) { + brokerController.getConsumerOffsetManager().assignResetOffset(lmqName, group, 0, targetOffset); + brokerController.getPopLiteMessageProcessor().getConsumerOrderInfoManager().remove(lmqName, group); + } + } + + private LiteSubscription getOrCreateLiteSubscription(String clientId, String group, String topic) { + LiteSubscription curLiteSubscription = ConcurrentHashMapUtils.computeIfAbsent(client2Subscription, clientId, + k -> new LiteSubscription().setGroup(group).setTopic(topic)); + assert curLiteSubscription != null; + return curLiteSubscription; + } + + @Override + public void run() { + LOGGER.info("Start checking lite subscription."); + while (!this.isStopped()) { + long checkInterval = brokerController.getBrokerConfig().getLiteSubscriptionCheckInterval(); + this.waitForRunning(checkInterval); + + long checkTimeout = brokerController.getBrokerConfig().getLiteSubscriptionCheckTimeoutMills(); + cleanupExpiredSubscriptions(checkTimeout); + } + LOGGER.info("End checking lite subscription."); + } + + /** + * Cleans up expired client subscriptions based on the provided timeout. + * + * @param checkTimeout the timeout in milliseconds to determine if a subscription is expired + */ + @VisibleForTesting + protected void cleanupExpiredSubscriptions(long checkTimeout) { + // Step 1: Find expired clients and their subscription information + long currentTime = System.currentTimeMillis(); + List> expiredEntries = client2Subscription.entrySet() + .stream() + .filter(entry -> currentTime - entry.getValue().getUpdateTime() > checkTimeout) + .collect(Collectors.toList()); + + // Step 2: Remove expired clients and their subscriptions + expiredEntries.forEach(expiredEntry -> { + String clientId = expiredEntry.getKey(); + LiteSubscription liteSubscription = expiredEntry.getValue(); + String group = liteSubscription.getGroup(); + String topic = liteSubscription.getTopic(); + removeCompleteSubscription(clientId); + LOGGER.info("Remove expired LiteSubscription, topic: {}, group: {}, clientId: {}, timeout: {}ms, expired: {}ms", + topic, group, clientId, checkTimeout, System.currentTimeMillis() - liteSubscription.getUpdateTime()); + }); + } + +} \ No newline at end of file diff --git a/broker/src/main/java/org/apache/rocketmq/broker/lite/RocksDBLiteLifecycleManager.java b/broker/src/main/java/org/apache/rocketmq/broker/lite/RocksDBLiteLifecycleManager.java new file mode 100644 index 00000000000..fb0eb51540c --- /dev/null +++ b/broker/src/main/java/org/apache/rocketmq/broker/lite/RocksDBLiteLifecycleManager.java @@ -0,0 +1,113 @@ +/* + * 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.lite; + +import org.apache.commons.lang3.StringUtils; +import org.apache.commons.lang3.reflect.FieldUtils; +import org.apache.rocketmq.broker.BrokerController; +import org.apache.rocketmq.common.Pair; +import org.apache.rocketmq.common.constant.LoggerName; +import org.apache.rocketmq.common.lite.LiteUtil; +import org.apache.rocketmq.logging.org.slf4j.Logger; +import org.apache.rocketmq.logging.org.slf4j.LoggerFactory; +import org.apache.rocketmq.store.RocksDBMessageStore; +import org.apache.rocketmq.store.queue.RocksDBConsumeQueueOffsetTable; +import org.apache.rocketmq.store.queue.RocksDBConsumeQueueStore; +import org.apache.rocketmq.tieredstore.TieredMessageStore; + +import java.util.ArrayList; +import java.util.Collections; +import java.util.Iterator; +import java.util.List; +import java.util.Map; +import java.util.concurrent.ConcurrentMap; + +public class RocksDBLiteLifecycleManager extends AbstractLiteLifecycleManager { + private static final Logger LOGGER = LoggerFactory.getLogger(LoggerName.ROCKETMQ_POP_LITE_LOGGER_NAME); + + private Map maxCqOffsetTable; + + public RocksDBLiteLifecycleManager(BrokerController brokerController, LiteSharding liteSharding) { + super(brokerController, liteSharding); + } + + @Override + public long getMaxOffsetInQueue(String lmqName) { + return maxCqOffsetTable.getOrDefault(lmqName + "-0", -1L) + 1; + } + + @Override + public List collectByParentTopic(String parentTopic) { + if (StringUtils.isEmpty(parentTopic)) { + return Collections.emptyList(); + } + List resultList = new ArrayList<>(); + Iterator> iterator = maxCqOffsetTable.entrySet().iterator(); + while (iterator.hasNext()) { + Map.Entry entry = iterator.next(); + String queueAndQid = entry.getKey(); + String lmqName = queueAndQid.substring(0, queueAndQid.lastIndexOf("-")); + if (LiteUtil.belongsTo(lmqName, parentTopic)) { + resultList.add(lmqName); + } + } + return resultList; + } + + @Override + public List> collectExpiredLiteTopic() { + List> lmqToDelete = new ArrayList<>(); + Iterator> iterator = maxCqOffsetTable.entrySet().iterator(); + while (iterator.hasNext()) { + Map.Entry entry = iterator.next(); + String queueAndQid = entry.getKey(); + String lmqName = queueAndQid.substring(0, queueAndQid.lastIndexOf("-")); + String parentTopic = LiteUtil.getParentTopic(lmqName); + if (null == parentTopic) { + continue; + } + if (isLiteTopicExpired(parentTopic, lmqName, entry.getValue() + 1)) { + lmqToDelete.add(new Pair<>(parentTopic, lmqName)); + } + } + return lmqToDelete; + } + + @Override + public void init() { + super.init(); + if (messageStore instanceof TieredMessageStore) { // only support TieredMessageStore plugin + messageStore = ((TieredMessageStore) messageStore).getDefaultStore(); + } + if (!(messageStore instanceof RocksDBMessageStore)) { + LOGGER.warn("init failed, not a RocksDB store. {}", messageStore.getClass()); + return; // startup with lite feature disabled + } + try { + RocksDBConsumeQueueStore queueStore = (RocksDBConsumeQueueStore) messageStore.getQueueStore(); + RocksDBConsumeQueueOffsetTable cqOffsetTable = (RocksDBConsumeQueueOffsetTable) FieldUtils.readField( + FieldUtils.getField(RocksDBConsumeQueueStore.class, "rocksDBConsumeQueueOffsetTable", true), queueStore); + @SuppressWarnings("unchecked") + ConcurrentMap innerMaxCqOffsetTable = (ConcurrentMap) FieldUtils.readField( + FieldUtils.getField(RocksDBConsumeQueueOffsetTable.class, "topicQueueMaxCqOffset", true), cqOffsetTable); + maxCqOffsetTable = Collections.unmodifiableMap(innerMaxCqOffsetTable); + } catch (Exception e) { + LOGGER.error("LiteLifecycleManager-init error", e); + } + } +} diff --git a/broker/src/main/java/org/apache/rocketmq/broker/longpolling/NotifyMessageArrivingListener.java b/broker/src/main/java/org/apache/rocketmq/broker/longpolling/NotifyMessageArrivingListener.java index 9c0ee89e4db..27d5c7c6f6a 100644 --- a/broker/src/main/java/org/apache/rocketmq/broker/longpolling/NotifyMessageArrivingListener.java +++ b/broker/src/main/java/org/apache/rocketmq/broker/longpolling/NotifyMessageArrivingListener.java @@ -18,25 +18,33 @@ package org.apache.rocketmq.broker.longpolling; import java.util.Map; + +import org.apache.rocketmq.broker.lite.LiteEventDispatcher; import org.apache.rocketmq.broker.processor.NotificationProcessor; import org.apache.rocketmq.broker.processor.PopMessageProcessor; +import org.apache.rocketmq.common.lite.LiteUtil; import org.apache.rocketmq.store.MessageArrivingListener; public class NotifyMessageArrivingListener implements MessageArrivingListener { private final PullRequestHoldService pullRequestHoldService; private final PopMessageProcessor popMessageProcessor; private final NotificationProcessor notificationProcessor; + private final LiteEventDispatcher liteEventDispatcher; - public NotifyMessageArrivingListener(final PullRequestHoldService pullRequestHoldService, final PopMessageProcessor popMessageProcessor, final NotificationProcessor notificationProcessor) { + public NotifyMessageArrivingListener(final PullRequestHoldService pullRequestHoldService, final PopMessageProcessor popMessageProcessor, final NotificationProcessor notificationProcessor, final LiteEventDispatcher liteEventDispatcher) { this.pullRequestHoldService = pullRequestHoldService; this.popMessageProcessor = popMessageProcessor; this.notificationProcessor = notificationProcessor; + this.liteEventDispatcher = liteEventDispatcher; } @Override public void arriving(String topic, int queueId, long logicOffset, long tagsCode, long msgStoreTime, byte[] filterBitMap, Map properties) { - + if (LiteUtil.isLiteTopicQueue(topic)) { + this.liteEventDispatcher.dispatch(null, topic, queueId, logicOffset, msgStoreTime); + return; + } this.pullRequestHoldService.notifyMessageArriving( topic, queueId, logicOffset, tagsCode, msgStoreTime, filterBitMap, properties); this.popMessageProcessor.notifyMessageArriving( diff --git a/broker/src/main/java/org/apache/rocketmq/broker/longpolling/PopLiteLongPollingService.java b/broker/src/main/java/org/apache/rocketmq/broker/longpolling/PopLiteLongPollingService.java new file mode 100644 index 00000000000..246583c2ee8 --- /dev/null +++ b/broker/src/main/java/org/apache/rocketmq/broker/longpolling/PopLiteLongPollingService.java @@ -0,0 +1,287 @@ +/* + * 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.longpolling; + +import com.googlecode.concurrentlinkedhashmap.ConcurrentLinkedHashMap; +import io.netty.channel.ChannelHandlerContext; +import java.util.Map; +import java.util.concurrent.ConcurrentSkipListSet; +import java.util.concurrent.atomic.AtomicLong; +import org.apache.commons.collections.CollectionUtils; +import org.apache.rocketmq.broker.BrokerController; +import org.apache.rocketmq.common.ServiceThread; +import org.apache.rocketmq.common.constant.LoggerName; +import org.apache.rocketmq.common.lite.LiteSubscription; +import org.apache.rocketmq.logging.org.slf4j.Logger; +import org.apache.rocketmq.logging.org.slf4j.LoggerFactory; +import org.apache.rocketmq.remoting.netty.NettyRemotingAbstract; +import org.apache.rocketmq.remoting.netty.NettyRequestProcessor; +import org.apache.rocketmq.remoting.netty.RequestTask; +import org.apache.rocketmq.remoting.protocol.RemotingCommand; + +import static org.apache.rocketmq.broker.longpolling.PollingResult.NOT_POLLING; +import static org.apache.rocketmq.broker.longpolling.PollingResult.POLLING_FULL; +import static org.apache.rocketmq.broker.longpolling.PollingResult.POLLING_SUC; +import static org.apache.rocketmq.broker.longpolling.PollingResult.POLLING_TIMEOUT; + +/** + * Long polling service specifically designed for lite consumption. + * Stores pending requests in memory using clientId as the key instead of topic@cid@qid. + * Notification and resource checking mechanisms are identical to those in PopLongPollingService. + */ +public class PopLiteLongPollingService extends ServiceThread { + private static final Logger LOGGER = LoggerFactory.getLogger(LoggerName.ROCKETMQ_POP_LITE_LOGGER_NAME); + + private final BrokerController brokerController; + private final NettyRequestProcessor processor; + private final ConcurrentLinkedHashMap> pollingMap; + private long lastCleanTime = 0; + + private final AtomicLong totalPollingNum = new AtomicLong(0); + private final boolean notifyLast; + + public PopLiteLongPollingService(BrokerController brokerController, NettyRequestProcessor processor, boolean notifyLast) { + this.brokerController = brokerController; + this.processor = processor; + this.pollingMap = new ConcurrentLinkedHashMap.Builder>() + .maximumWeightedCapacity(this.brokerController.getBrokerConfig().getPopPollingMapSize()).build(); + this.notifyLast = notifyLast; + } + + @Override + public String getServiceName() { + if (brokerController.getBrokerConfig().isInBrokerContainer()) { + return brokerController.getBrokerIdentity().getIdentifier() + PopLiteLongPollingService.class.getSimpleName(); + } + return PopLiteLongPollingService.class.getSimpleName(); + } + + @Override + public void run() { + int i = 0; + while (!this.stopped) { + try { + this.waitForRunning(20); + i++; + if (pollingMap.isEmpty()) { + continue; + } + long tmpTotalPollingNum = 0; + for (Map.Entry> entry : pollingMap.entrySet()) { + String key = entry.getKey(); + ConcurrentSkipListSet popQ = entry.getValue(); + if (popQ == null) { + continue; + } + PopRequest first; + do { + first = popQ.pollFirst(); + if (first == null) { + break; + } + if (!first.isTimeout()) { + if (popQ.add(first)) { + break; + } else { + LOGGER.info("lite polling, add back again but failed. {}", first); + } + } + if (brokerController.getBrokerConfig().isEnablePopLog()) { + LOGGER.info("timeout , wakeUp lite polling : {}", first); + } + totalPollingNum.decrementAndGet(); + wakeUp(first); + } + while (true); + if (i >= 100) { + long tmpPollingNum = popQ.size(); + tmpTotalPollingNum = tmpTotalPollingNum + tmpPollingNum; + if (tmpPollingNum > 20) { + LOGGER.info("lite polling queue {} , size={} ", key, tmpPollingNum); + } + } + } + + if (i >= 100) { + LOGGER.info("litePollingMapSize={}, tmpTotalSize={}, atomicTotalSize={}, diffSize={}", + pollingMap.size(), tmpTotalPollingNum, totalPollingNum.get(), + Math.abs(totalPollingNum.get() - tmpTotalPollingNum)); + totalPollingNum.set(tmpTotalPollingNum); + i = 0; + } + + // clean unused + if (lastCleanTime == 0 || System.currentTimeMillis() - lastCleanTime > 5 * 60 * 1000) { + cleanUnusedResource(); + } + } catch (Throwable e) { + LOGGER.error("checkLitePolling error", e); + } + } + // clean all; + try { + for (Map.Entry> entry : pollingMap.entrySet()) { + ConcurrentSkipListSet popQ = entry.getValue(); + PopRequest first; + while ((first = popQ.pollFirst()) != null) { + wakeUp(first); + } + } + } catch (Throwable ignored) { + } + } + + public boolean notifyMessageArriving(final String clientId, boolean force, long msgStoreTime, String group) { + String pollingKey = getPollingKey(clientId, group); + ConcurrentSkipListSet remotingCommands = pollingMap.get(pollingKey); + if (remotingCommands == null || remotingCommands.isEmpty()) { + return false; + } + PopRequest popRequest = pollRemotingCommands(remotingCommands); + if (popRequest == null) { + return false; + } + + if (brokerController.getBrokerConfig().isEnableLitePopLog()) { + LOGGER.info("notify lite polling, wakeUp: {}", popRequest); + } + return wakeUp(popRequest); + } + + public boolean wakeUp(final PopRequest request) { + if (request == null || !request.complete()) { + return false; + } + if (!request.getCtx().channel().isActive()) { + return false; + } + + Runnable run = () -> { + try { + final RemotingCommand response = processor.processRequest(request.getCtx(), request.getRemotingCommand()); + if (response != null) { + response.setOpaque(request.getRemotingCommand().getOpaque()); + response.markResponseType(); + NettyRemotingAbstract.writeResponse(request.getChannel(), request.getRemotingCommand(), response, future -> { + if (!future.isSuccess()) { + LOGGER.error("ProcessRequestWrapper response to {} failed", request.getChannel().remoteAddress(), future.cause()); + LOGGER.error(request.toString()); + LOGGER.error(response.toString()); + } + }, brokerController.getBrokerMetricsManager().getRemotingMetricsManager()); + } + } catch (Exception e) { + LOGGER.error("ExecuteRequestWhenWakeup error.", e); + } + }; + + this.brokerController.getPullMessageExecutor().submit( + new RequestTask(run, request.getChannel(), request.getRemotingCommand())); + return true; + } + + public PollingResult polling(final ChannelHandlerContext ctx, RemotingCommand remotingCommand, + long bornTime, long pollTime, String clientId, String group) { + if (pollTime <= 0 || this.isStopped()) { + return NOT_POLLING; + } + long expired = bornTime + pollTime; + final PopRequest request = new PopRequest(remotingCommand, ctx, expired, null, null); + boolean isFull = totalPollingNum.get() >= this.brokerController.getBrokerConfig().getMaxPopPollingSize(); + if (isFull) { + LOGGER.info("lite polling {}, result POLLING_FULL, total:{}", remotingCommand, totalPollingNum.get()); + return POLLING_FULL; + } + boolean isTimeout = request.isTimeout(); + if (isTimeout) { + if (brokerController.getBrokerConfig().isEnablePopLog()) { + LOGGER.info("lite polling {}, result POLLING_TIMEOUT", remotingCommand); + } + return POLLING_TIMEOUT; + } + + String pollingKey = getPollingKey(clientId, group); + ConcurrentSkipListSet queue = pollingMap.get(pollingKey); + if (queue == null) { + queue = new ConcurrentSkipListSet<>(PopRequest.COMPARATOR); + ConcurrentSkipListSet old = pollingMap.putIfAbsent(pollingKey, queue); + if (old != null) { + queue = old; + } + } else { + // check size + int size = queue.size(); + if (size > brokerController.getBrokerConfig().getPopPollingSize()) { + LOGGER.info("lite polling {}, result POLLING_FULL, singleSize:{}", remotingCommand, size); + return POLLING_FULL; + } + } + if (queue.add(request)) { + remotingCommand.setSuspended(true); + totalPollingNum.incrementAndGet(); + if (brokerController.getBrokerConfig().isEnableLitePopLog()) { + LOGGER.info("lite polling {}, result POLLING_SUC", remotingCommand); + } + return POLLING_SUC; + } else { + LOGGER.info("lite polling {}, result POLLING_FULL, add fail, {}", request, queue); + return POLLING_FULL; + } + } + + private void cleanUnusedResource() { + try { + pollingMap.entrySet().removeIf(entry -> { + String clientId = entry.getKey(); // see getPollingKey() + LiteSubscription subscription = brokerController.getLiteSubscriptionRegistry().getLiteSubscription(clientId); + if (null == subscription || CollectionUtils.isEmpty(subscription.getLiteTopicSet())) { + LOGGER.info("clean polling structure of {}", clientId); + return true; + } + return false; + }); + } catch (Throwable ignored) { + } + lastCleanTime = System.currentTimeMillis(); + } + + private PopRequest pollRemotingCommands(ConcurrentSkipListSet remotingCommands) { + if (remotingCommands == null || remotingCommands.isEmpty()) { + return null; + } + + PopRequest popRequest; + do { + if (notifyLast) { + popRequest = remotingCommands.pollLast(); + } else { + popRequest = remotingCommands.pollFirst(); + } + if (popRequest != null) { + totalPollingNum.decrementAndGet(); + } + } while (popRequest != null && !popRequest.getChannel().isActive()); + + return popRequest; + } + + // Assume that clientId is unique, so we use it as the key for now. + private String getPollingKey(String clientId, String group) { + return clientId; + } +} diff --git a/broker/src/main/java/org/apache/rocketmq/broker/metrics/BrokerMetricsManager.java b/broker/src/main/java/org/apache/rocketmq/broker/metrics/BrokerMetricsManager.java index fe6c180e452..2f8a07020c5 100644 --- a/broker/src/main/java/org/apache/rocketmq/broker/metrics/BrokerMetricsManager.java +++ b/broker/src/main/java/org/apache/rocketmq/broker/metrics/BrokerMetricsManager.java @@ -121,6 +121,7 @@ public class BrokerMetricsManager { private final MessageStore messageStore; private final BrokerController brokerController; private final ConsumerLagCalculator consumerLagCalculator; + private final LiteConsumerLagCalculator liteConsumerLagCalculator; private final Map labelMap = new HashMap<>(); private OtlpGrpcMetricExporter metricExporter; private PeriodicMetricReader periodicMetricReader; @@ -178,6 +179,7 @@ public BrokerMetricsManager(BrokerController brokerController) { this.consumerLagCalculator = new ConsumerLagCalculator(brokerController); this.remotingMetricsManager = new RemotingMetricsManager(); this.popMetricsManager = new PopMetricsManager(); + this.liteConsumerLagCalculator = new LiteConsumerLagCalculator(brokerController); init(); } @@ -673,21 +675,27 @@ private void initLagAndDlqMetrics() { consumerLagMessages = brokerMeter.gaugeBuilder(GAUGE_CONSUMER_LAG_MESSAGES) .setDescription("Consumer lag messages") .ofLongs() - .buildWithCallback(measurement -> consumerLagCalculator.calculateLag(result -> - measurement.record(result.lag, buildLagAttributes(result)))); + .buildWithCallback(measurement -> { + consumerLagCalculator.calculateLag(result -> + measurement.record(result.lag, buildLagAttributes(result)) + ); + + liteConsumerLagCalculator.calculateLiteLagCount(result -> + measurement.record(result.lag, buildLagAttributes(result)) + ); + }); consumerLagLatency = brokerMeter.gaugeBuilder(GAUGE_CONSUMER_LAG_LATENCY) .setDescription("Consumer lag time") .setUnit("milliseconds") .ofLongs() - .buildWithCallback(measurement -> consumerLagCalculator.calculateLag(result -> { - long latency = 0; - long curTimeStamp = System.currentTimeMillis(); - if (result.earliestUnconsumedTimestamp != 0) { - latency = curTimeStamp - result.earliestUnconsumedTimestamp; - } - measurement.record(latency, buildLagAttributes(result)); - })); + .buildWithCallback(measurement -> { + consumerLagCalculator.calculateLag(lagResult -> + measurement.record(lagResult.getLagLatency(), buildLagAttributes(lagResult))); + + liteConsumerLagCalculator.calculateLiteLagLatency(lagResult -> + measurement.record(lagResult.getLagLatency(), buildLagAttributes(lagResult))); + }); consumerInflightMessages = brokerMeter.gaugeBuilder(GAUGE_CONSUMER_INFLIGHT_MESSAGES) .setDescription("Consumer inflight messages") @@ -711,8 +719,14 @@ private void initLagAndDlqMetrics() { consumerReadyMessages = brokerMeter.gaugeBuilder(GAUGE_CONSUMER_READY_MESSAGES) .setDescription("Consumer ready messages") .ofLongs() - .buildWithCallback(measurement -> - consumerLagCalculator.calculateAvailable(result -> measurement.record(result.available, buildLagAttributes(result)))); + .buildWithCallback(measurement -> { + consumerLagCalculator.calculateAvailable(result -> + measurement.record(result.available, buildLagAttributes(result))); + + // for lite, ready == lag + liteConsumerLagCalculator.calculateLiteLagCount(result -> + measurement.record(result.lag, buildLagAttributes(result))); + }); sendToDlqMessages = brokerMeter.counterBuilder(COUNTER_CONSUMER_SEND_TO_DLQ_MESSAGES_TOTAL) .setDescription("Consumer send to DLQ messages") @@ -764,6 +778,10 @@ private void initOtherMetrics() { } } + public LiteConsumerLagCalculator getLiteConsumerLagCalculator() { + return liteConsumerLagCalculator; + } + public void shutdown() { if (brokerConfig.isInBrokerContainer()) { // only rto need diff --git a/broker/src/main/java/org/apache/rocketmq/broker/metrics/ConsumerLagCalculator.java b/broker/src/main/java/org/apache/rocketmq/broker/metrics/ConsumerLagCalculator.java index d42c8f0ff66..3e48a3c5bb9 100644 --- a/broker/src/main/java/org/apache/rocketmq/broker/metrics/ConsumerLagCalculator.java +++ b/broker/src/main/java/org/apache/rocketmq/broker/metrics/ConsumerLagCalculator.java @@ -25,6 +25,7 @@ import java.util.function.BiConsumer; import java.util.function.Consumer; import java.util.stream.Collectors; +import org.apache.commons.lang3.StringUtils; import org.apache.rocketmq.broker.BrokerController; import org.apache.rocketmq.broker.client.ConsumerGroupInfo; import org.apache.rocketmq.broker.client.ConsumerManager; @@ -119,6 +120,10 @@ public static class CalculateLagResult extends BaseCalculateResult { public CalculateLagResult(String group, String topic, boolean isRetry) { super(group, topic, isRetry); } + + public long getLagLatency() { + return earliestUnconsumedTimestamp == 0 ? 0 : System.currentTimeMillis() - earliestUnconsumedTimestamp; + } } public static class CalculateInflightResult extends BaseCalculateResult { @@ -141,16 +146,22 @@ public CalculateAvailableResult(String group, String topic, boolean isRetry) { private void processAllGroup(Consumer consumer) { for (Map.Entry subscriptionEntry : subscriptionGroupManager.getSubscriptionGroupTable().entrySet()) { - String group = subscriptionEntry.getKey(); + SubscriptionGroupConfig subscriptionGroupConfig = subscriptionEntry.getValue(); ConsumerGroupInfo consumerGroupInfo = consumerManager.getConsumerGroupInfo(group, true); + + boolean isLite = StringUtils.isNotEmpty(subscriptionGroupConfig.getLiteBindTopic()); + if (isLite) { + // lite consumer metrics are calculated by LiteConsumerLagCalculator + continue; + } + boolean isPop = false; if (consumerGroupInfo != null) { isPop = consumerGroupInfo.getConsumeType() == ConsumeType.CONSUME_POP; } Set topics; if (brokerConfig.isUseStaticSubscription()) { - SubscriptionGroupConfig subscriptionGroupConfig = subscriptionEntry.getValue(); if (subscriptionGroupConfig.getSubscriptionDataSet() == null || subscriptionGroupConfig.getSubscriptionDataSet().isEmpty()) { continue; diff --git a/broker/src/main/java/org/apache/rocketmq/broker/metrics/LiteConsumerLagCalculator.java b/broker/src/main/java/org/apache/rocketmq/broker/metrics/LiteConsumerLagCalculator.java new file mode 100644 index 00000000000..abde27670c0 --- /dev/null +++ b/broker/src/main/java/org/apache/rocketmq/broker/metrics/LiteConsumerLagCalculator.java @@ -0,0 +1,307 @@ +/* + * 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.metrics; + +import com.google.common.annotations.VisibleForTesting; +import java.util.ArrayList; +import java.util.Collections; +import java.util.Comparator; +import java.util.HashMap; +import java.util.List; +import java.util.Map; +import java.util.Objects; +import java.util.PriorityQueue; +import java.util.concurrent.ConcurrentHashMap; +import java.util.concurrent.ConcurrentMap; +import java.util.concurrent.PriorityBlockingQueue; +import java.util.concurrent.atomic.AtomicLong; +import java.util.function.BiConsumer; +import java.util.function.Consumer; +import org.apache.commons.collections.CollectionUtils; +import org.apache.commons.lang3.StringUtils; +import org.apache.rocketmq.broker.BrokerController; +import org.apache.rocketmq.broker.offset.ConsumerOffsetManager; +import org.apache.rocketmq.common.Pair; +import org.apache.rocketmq.common.entity.TopicGroup; +import org.apache.rocketmq.common.lite.LiteLagInfo; +import org.apache.rocketmq.common.lite.LiteUtil; + +public class LiteConsumerLagCalculator { + + protected static final long INIT_CONSUME_TIMESTAMP = -1L; + + @VisibleForTesting + protected final ConcurrentHashMap> topicGroupLagTimeMap = + new ConcurrentHashMap<>(); + + private final BrokerController brokerController; + + public LiteConsumerLagCalculator(BrokerController brokerController) { + this.brokerController = brokerController; + } + + public void removeLagInfo(String group, String bindTopic, String lmqName) { + PriorityBlockingQueue lagHeap = topicGroupLagTimeMap.get(new TopicGroup(bindTopic, group)); + if (lagHeap != null) { + lagHeap.removeIf(info -> info.getLmqName().equals(lmqName)); + } + } + + public void updateLagInfo(String group, String bindTopic, String lmqName, long storeTimestamp) { + PriorityBlockingQueue lagHeap = topicGroupLagTimeMap.computeIfAbsent( + new TopicGroup(bindTopic, group), + k -> new PriorityBlockingQueue<>(8, Comparator.comparingLong(LagTimeInfo::getLagTimestamp).reversed())); + lagHeap.removeIf(info -> info.getLmqName().equals(lmqName)); + lagHeap.offer(new LagTimeInfo(lmqName, storeTimestamp)); + int topK = brokerController.getBrokerConfig().getLiteLagLatencyTopK(); + if (lagHeap.size() > topK) { + lagHeap.remove(); + } + } + + @VisibleForTesting + protected long getStoreTimestamp(String lmqName, long offset) { + return this.brokerController.getMessageStore().getMessageStoreTimeStamp(lmqName, 0, offset); + } + + @VisibleForTesting + protected long getOffset(String group, String topic) { + return brokerController.getConsumerOffsetManager().queryOffset(group, topic, 0); + } + + @VisibleForTesting + protected long getMaxOffset(String lmqName) { + return brokerController.getLiteLifecycleManager().getMaxOffsetInQueue(lmqName); + } + + private long offsetDiff(Long offset, String lmqName) { + long consumerOffset = offset == null ? -1L : offset; + if (consumerOffset < 0) { + return 0L; + } + long maxOffset = getMaxOffset(lmqName); + return Math.max(0L, maxOffset - consumerOffset); + } + + public void calculateLiteLagCount(Consumer lagRecorder) { + if (!brokerController.getBrokerConfig().isLiteLagCountMetricsEnable()) { + return; + } + + Map counter = new HashMap<>(); + + offsetTableForEachByGroup(null, (topicGroup, consumerOffset) -> { + String lmqName = topicGroup.topic; + String group = topicGroup.group; + String parentTopic = LiteUtil.getParentTopic(lmqName); + long diff = offsetDiff(consumerOffset, lmqName); + if (diff > 0) { + TopicGroup key = new TopicGroup(parentTopic, group); + counter.merge(key, diff, Long::sum); + } + }); + + counter.forEach((topicGroup, totalCount) -> { + ConsumerLagCalculator.CalculateLagResult lagResult = + new ConsumerLagCalculator.CalculateLagResult(topicGroup.group, topicGroup.topic, false); + lagResult.lag = totalCount; + lagRecorder.accept(lagResult); + }); + } + + public void calculateLiteLagLatency(Consumer lagRecorder) { + if (!brokerController.getBrokerConfig().isLiteLagLatencyMetricsEnable()) { + return; + } + + topicGroupLagTimeMap.forEach((topicGroup, lagHeap) -> { + if (CollectionUtils.isEmpty(lagHeap)) { + return; + } + + // Find the minimum storeTimestamp in the heap + long minTimestamp = lagHeap.stream() + .mapToLong(LagTimeInfo::getLagTimestamp) + .min() + .orElse(0L); + + ConsumerLagCalculator.CalculateLagResult lagResult = + new ConsumerLagCalculator.CalculateLagResult(topicGroup.group, topicGroup.topic, false); + lagResult.earliestUnconsumedTimestamp = minTimestamp; + lagRecorder.accept(lagResult); + }); + } + + /** + * Get top K LiteLagInfo entries with the smallest lag timestamps for a topic group. + * + * @param group consumer group name + * @param parentTopic parent topic name + * @param topK max number of entries to retrieve + * @return Pair containing: + * - Left: list of at most topK LiteLagInfo entries sorted by timestamp + * - Right: minimum lag timestamp (or initial consume timestamp if no data) + */ + public Pair/*topK*/, Long/*timestamp*/> getLagTimestampTopK( + String group, + String parentTopic, + int topK + ) { + TopicGroup key = new TopicGroup(parentTopic, group); + PriorityBlockingQueue lagHeap = topicGroupLagTimeMap.get(key); + if (CollectionUtils.isEmpty(lagHeap)) { + return Pair.of(Collections.emptyList(), INIT_CONSUME_TIMESTAMP); + } + + // Evict the largest timestamp when heap is full, keeping smallest topK timestamps + PriorityQueue maxHeap = new PriorityQueue<>(topK, Comparator.comparingLong(LagTimeInfo::getLagTimestamp).reversed()); + for (LagTimeInfo lagInfo : lagHeap) { + if (maxHeap.size() < topK) { + maxHeap.offer(lagInfo); + } else if (maxHeap.peek() != null && lagInfo.getLagTimestamp() < maxHeap.peek().getLagTimestamp()) { + maxHeap.poll(); + maxHeap.offer(lagInfo); + } + } + + // Convert results to LiteLagInfo list and sort by timestamp + List topList = new ArrayList<>(maxHeap.size()); + for (LagTimeInfo lagInfo : maxHeap) { + String lmqName = lagInfo.getLmqName(); + LiteLagInfo liteLagInfo = new LiteLagInfo(); + liteLagInfo.setLiteTopic(LiteUtil.getLiteTopic(lmqName)); + liteLagInfo.setEarliestUnconsumedTimestamp(lagInfo.getLagTimestamp()); + liteLagInfo.setLagCount(offsetDiff(getOffset(group, lmqName), lmqName)); + topList.add(liteLagInfo); + } + + // Sort by timestamp in ascending order + topList.sort(Comparator.comparingLong(LiteLagInfo::getEarliestUnconsumedTimestamp)); + long minLagTimestamp = topList.isEmpty() ? INIT_CONSUME_TIMESTAMP : + topList.get(0).getEarliestUnconsumedTimestamp(); + + return Pair.of(topList, minLagTimestamp); + } + + /** + * Get top K LiteLagInfo entries with the largest lag counts for a topic group. + * + * @param group consumer group name + * @param topK max number of entries to retrieve + * @return Pair containing: + * - Left: list of at most topK LiteLagInfo entries sorted by lag count + * - Right: total lag count + */ + public Pair, Long> getLagCountTopK( + String group, + int topK + ) { + // Use a min heap to maintain the largest topK lag counts + PriorityQueue minHeap = new PriorityQueue<>(topK, Comparator.comparingLong(LiteLagInfo::getLagCount)); + AtomicLong totalLagCount = new AtomicLong(0L); + + offsetTableForEachByGroup(group, (topicGroup, consumerOffset) -> { + String topic = topicGroup.topic; + + long diff = offsetDiff(consumerOffset, topic); + if (diff > 0) { + totalLagCount.addAndGet(diff); + LiteLagInfo liteLagInfo = new LiteLagInfo(); + liteLagInfo.setLiteTopic(LiteUtil.getLiteTopic(topic)); + liteLagInfo.setLagCount(diff); + liteLagInfo.setEarliestUnconsumedTimestamp(getStoreTimestamp(topic, consumerOffset)); + + if (minHeap.size() < topK) { + minHeap.offer(liteLagInfo); + } else if (minHeap.peek() != null && liteLagInfo.getLagCount() > minHeap.peek().getLagCount()) { + minHeap.poll(); + minHeap.offer(liteLagInfo); + } + } + }); + + // Convert heap elements to list and sort by lag count in descending order + List topList = new ArrayList<>(minHeap); + topList.sort(Comparator.comparingLong(LiteLagInfo::getLagCount).reversed()); + + return Pair.of(topList, totalLagCount.get()); + } + + /** + * Filters the lite group offset by the specified group and processes each entry via BiConsumer. + * + * @param group The specified consumer group. If null, all offset information is processed. + * @param consumer The BiConsumer used to process each entry. + */ + protected void offsetTableForEachByGroup( + String group, + BiConsumer consumer + ) { + ConcurrentMap> offsetTable = + brokerController.getConsumerOffsetManager().getOffsetTable(); + offsetTable.forEach((topicAtGroup, queueOffset) -> { + String[] topicGroup = topicAtGroup.split(ConsumerOffsetManager.TOPIC_GROUP_SEPARATOR); + if (topicGroup.length == 2) { + if (!LiteUtil.isLiteTopicQueue(topicGroup[0])) { + return; + } + // If group specified, only process the matching group + if (StringUtils.isEmpty(group) || group.equals(topicGroup[1])) { + TopicGroup tg = new TopicGroup(topicGroup[0], topicGroup[1]); + Long consumerOffset = queueOffset.get(0); + if (consumerOffset == null) { + return; + } + consumer.accept(tg, consumerOffset); + } + } + }); + } + + protected static class LagTimeInfo { + private final String lmqName; + // earliest unconsumed timestamp + private final long lagTimestamp; + + public LagTimeInfo(String lmqName, long lagTimestamp) { + this.lmqName = lmqName; + this.lagTimestamp = lagTimestamp; + } + + public String getLmqName() { + return lmqName; + } + + public long getLagTimestamp() { + return lagTimestamp; + } + + @Override + public boolean equals(Object o) { + if (o == null || getClass() != o.getClass()) { + return false; + } + LagTimeInfo lagInfo = (LagTimeInfo) o; + return Objects.equals(lmqName, lagInfo.lmqName); + } + + @Override + public int hashCode() { + return Objects.hashCode(lmqName); + } + } +} diff --git a/broker/src/main/java/org/apache/rocketmq/broker/offset/ConsumerOffsetManager.java b/broker/src/main/java/org/apache/rocketmq/broker/offset/ConsumerOffsetManager.java index 3eee9fc559a..e062ceca96a 100644 --- a/broker/src/main/java/org/apache/rocketmq/broker/offset/ConsumerOffsetManager.java +++ b/broker/src/main/java/org/apache/rocketmq/broker/offset/ConsumerOffsetManager.java @@ -56,7 +56,7 @@ public class ConsumerOffsetManager extends ConfigManager { protected transient BrokerController brokerController; - private final transient AtomicLong versionChangeCounter = new AtomicLong(0); + protected final transient AtomicLong versionChangeCounter = new AtomicLong(0); public ConsumerOffsetManager() { } @@ -65,7 +65,7 @@ public ConsumerOffsetManager(BrokerController brokerController) { this.brokerController = brokerController; } - protected void removeConsumerOffset(String topicAtGroup) { + public void removeConsumerOffset(String topicAtGroup) { } @@ -205,7 +205,7 @@ public void commitOffset(final String clientHost, final String group, final Stri private void commitOffset(final String clientHost, final String key, final int queueId, final long offset) { ConcurrentMap map = this.offsetTable.get(key); if (null == map) { - map = new ConcurrentHashMap<>(32); + map = new ConcurrentHashMap<>(2); map.put(queueId, offset); this.offsetTable.put(key, map); } else { @@ -320,6 +320,10 @@ public void setOffsetTable(ConcurrentMap> o this.offsetTable = offsetTable; } + public ConcurrentMap> getPullOffsetTable() { + return pullOffsetTable; + } + public Map queryMinOffsetInAllGroup(final String topic, final String filterGroups) { Map queueMinOffset = new HashMap<>(); diff --git a/broker/src/main/java/org/apache/rocketmq/broker/offset/MemoryConsumerOrderInfoManager.java b/broker/src/main/java/org/apache/rocketmq/broker/offset/MemoryConsumerOrderInfoManager.java new file mode 100644 index 00000000000..94acc454faa --- /dev/null +++ b/broker/src/main/java/org/apache/rocketmq/broker/offset/MemoryConsumerOrderInfoManager.java @@ -0,0 +1,53 @@ +/* + * 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.offset; + +import org.apache.rocketmq.broker.BrokerController; +import org.apache.rocketmq.broker.pop.orderly.QueueLevelConsumerManager; + +/** + * Memory-based Consumer Order Information Manager for Lite Topics + * Trade-off considerations:: + * 1. Lite Topics are primarily used for lightweight consumption where + * strict ordering requirements are relatively low + * 2. Considering compatibility with traditional PushConsumer, + * a certain degree of ordering control failure is acceptable + * 3. Avoiding I/O overhead from persistence operations + *

+ * We may make structural adjustments and optimizations to reduce overhead and memory footprint. + */ +public class MemoryConsumerOrderInfoManager extends QueueLevelConsumerManager { + + public MemoryConsumerOrderInfoManager(BrokerController brokerController) { + super(brokerController); + } + + @Override + protected void updateLockFreeTimestamp(String topic, String group, int queueId, OrderInfo orderInfo) { + if (this.getConsumerOrderInfoLockManager() != null) { + // use max lock free time to prevent unexpected blocking + this.getConsumerOrderInfoLockManager().updateLockFreeTimestamp( + topic, group, queueId, orderInfo.getMaxLockFreeTimestamp()); + } + } + + @Override + public void persist() { + // MemoryConsumerOrderInfoManager persist, do nothing. + } +} diff --git a/broker/src/main/java/org/apache/rocketmq/broker/pop/PopConsumerLockService.java b/broker/src/main/java/org/apache/rocketmq/broker/pop/PopConsumerLockService.java index 33221430492..066db7192ae 100644 --- a/broker/src/main/java/org/apache/rocketmq/broker/pop/PopConsumerLockService.java +++ b/broker/src/main/java/org/apache/rocketmq/broker/pop/PopConsumerLockService.java @@ -41,18 +41,26 @@ public PopConsumerLockService(long timeout) { this.lockTable = new ConcurrentHashMap<>(); } - public boolean tryLock(String groupId, String topicId) { + public boolean tryLock(String key) { return Objects.requireNonNull(ConcurrentHashMapUtils.computeIfAbsent(lockTable, - groupId + PopAckConstants.SPLIT + topicId, s -> new TimedLock())).tryLock(); + key, s -> new TimedLock())).tryLock(); } - public void unlock(String groupId, String topicId) { - TimedLock lock = lockTable.get(groupId + PopAckConstants.SPLIT + topicId); + public boolean tryLock(String groupId, String topicId) { + return tryLock(groupId + PopAckConstants.SPLIT + topicId); + } + + public void unlock(String key) { + TimedLock lock = lockTable.get(key); if (lock != null) { lock.unlock(); } } + public void unlock(String groupId, String topicId) { + unlock(groupId + PopAckConstants.SPLIT + topicId); + } + // For retry topics, should lock origin group and topic public boolean isLockTimeout(String groupId, String topicId) { topicId = KeyBuilder.parseNormalTopic(topicId, groupId); diff --git a/broker/src/main/java/org/apache/rocketmq/broker/pop/orderly/ConsumerOrderInfoManager.java b/broker/src/main/java/org/apache/rocketmq/broker/pop/orderly/ConsumerOrderInfoManager.java index f8f56992b1a..84b0540db24 100644 --- a/broker/src/main/java/org/apache/rocketmq/broker/pop/orderly/ConsumerOrderInfoManager.java +++ b/broker/src/main/java/org/apache/rocketmq/broker/pop/orderly/ConsumerOrderInfoManager.java @@ -68,6 +68,20 @@ void update(String attemptId, boolean isRetry, String topic, String group, int q */ boolean checkBlock(String attemptId, String topic, String group, int queueId, long invisibleTime); + /** + * Remove the specified topic and group + * Usually called during topic deletion + * + * @param topic Topic name + * @param group Consumer group name + */ + void remove(String topic, String group); + + /** + * Get order info count + */ + int getOrderInfoCount(); + /** * Commit message and calculate next consumption offset * Called when consumer ACKs messages @@ -137,6 +151,7 @@ void updateNextVisibleTime(String topic, String group, int queueId, long queueOf * Get available message result * Used to retrieve messages from cache */ - CompletableFuture getAvailableMessageResult(String attemptId, long popTime, long invisibleTime, String groupId, + CompletableFuture getAvailableMessageResult(String attemptId, long popTime, long invisibleTime, + String groupId, String topicId, int queueId, int batchSize, StringBuilder orderCountInfoBuilder); } diff --git a/broker/src/main/java/org/apache/rocketmq/broker/pop/orderly/QueueLevelConsumerManager.java b/broker/src/main/java/org/apache/rocketmq/broker/pop/orderly/QueueLevelConsumerManager.java index 8a5ed6b150f..6cf5aabe44f 100644 --- a/broker/src/main/java/org/apache/rocketmq/broker/pop/orderly/QueueLevelConsumerManager.java +++ b/broker/src/main/java/org/apache/rocketmq/broker/pop/orderly/QueueLevelConsumerManager.java @@ -76,7 +76,7 @@ protected static String[] decodeKey(String key) { return key.split(TOPIC_GROUP_SEPARATOR); } - private void updateLockFreeTimestamp(String topic, String group, int queueId, OrderInfo orderInfo) { + protected void updateLockFreeTimestamp(String topic, String group, int queueId, OrderInfo orderInfo) { if (queueLevelConsumerOrderInfoLockManager != null) { queueLevelConsumerOrderInfoLockManager.updateLockFreeTimestamp(topic, group, queueId, orderInfo); } @@ -179,6 +179,16 @@ public void clearBlock(String topic, String group, int queueId) { }); } + @Override + public void remove(String topic, String group) { + table.remove(buildKey(topic, group)); + } + + @Override + public int getOrderInfoCount() { + return table.size(); + } + @Override public OrderedConsumptionLevel getOrderedConsumptionLevel() { return OrderedConsumptionLevel.QUEUE; @@ -383,7 +393,7 @@ public CompletableFuture getAvailableMessageResult(String atte } @VisibleForTesting - QueueLevelConsumerOrderInfoLockManager getConsumerOrderInfoLockManager() { + protected QueueLevelConsumerOrderInfoLockManager getConsumerOrderInfoLockManager() { return queueLevelConsumerOrderInfoLockManager; } @@ -577,6 +587,33 @@ public Long getLockFreeTimestamp() { return currentTime; } + @JSONField(serialize = false, deserialize = false) + public Long getMaxLockFreeTimestamp() { + if (offsetList == null || offsetList.isEmpty()) { + return null; + } + int num = offsetList.size(); + long maxTime = System.currentTimeMillis(); + for (int i = 0; i < num; i++) { + if (isNotAck(i)) { + if (invisibleTime == null || invisibleTime <= 0) { + return null; + } + long nextVisibleTime = popTime + invisibleTime; + if (offsetNextVisibleTime != null) { + Long time = offsetNextVisibleTime.get(this.getQueueOffset(i)); + if (time != null) { + nextVisibleTime = time; + } + } + if (maxTime < nextVisibleTime) { + maxTime = nextVisibleTime; + } + } + } + return maxTime; + } + @JSONField(serialize = false, deserialize = false) public void updateOffsetNextVisibleTime(long queueOffset, long nextVisibleTime) { if (this.offsetNextVisibleTime == null) { diff --git a/broker/src/main/java/org/apache/rocketmq/broker/pop/orderly/QueueLevelConsumerOrderInfoLockManager.java b/broker/src/main/java/org/apache/rocketmq/broker/pop/orderly/QueueLevelConsumerOrderInfoLockManager.java index d65b01d89c7..08569977e0f 100644 --- a/broker/src/main/java/org/apache/rocketmq/broker/pop/orderly/QueueLevelConsumerOrderInfoLockManager.java +++ b/broker/src/main/java/org/apache/rocketmq/broker/pop/orderly/QueueLevelConsumerOrderInfoLockManager.java @@ -29,6 +29,7 @@ import org.apache.rocketmq.broker.BrokerController; import org.apache.rocketmq.common.ThreadFactoryImpl; import org.apache.rocketmq.common.constant.LoggerName; +import org.apache.rocketmq.common.lite.LiteUtil; import org.apache.rocketmq.logging.org.slf4j.Logger; import org.apache.rocketmq.logging.org.slf4j.LoggerFactory; @@ -108,6 +109,10 @@ public void updateLockFreeTimestamp(String topic, String group, int queueId, Lon protected void notifyLockIsFree(Key key) { try { + if (LiteUtil.isLiteTopicQueue(key.topic)) { + this.brokerController.getLiteEventDispatcher().dispatch(key.group, key.topic, key.queueId, -1, -1); + return; + } this.brokerController.getPopMessageProcessor().notifyLongPollingRequestIfNeed(key.topic, key.group, key.queueId); } catch (Exception e) { POP_LOGGER.error("unexpect error when notifyLockIsFree. key:{}", key, e); diff --git a/broker/src/main/java/org/apache/rocketmq/broker/processor/AckMessageProcessor.java b/broker/src/main/java/org/apache/rocketmq/broker/processor/AckMessageProcessor.java index 13346933527..34a790efca7 100644 --- a/broker/src/main/java/org/apache/rocketmq/broker/processor/AckMessageProcessor.java +++ b/broker/src/main/java/org/apache/rocketmq/broker/processor/AckMessageProcessor.java @@ -21,7 +21,9 @@ import io.netty.channel.ChannelHandlerContext; import java.nio.charset.StandardCharsets; import java.util.BitSet; +import org.apache.commons.lang3.StringUtils; import org.apache.rocketmq.broker.BrokerController; +import org.apache.rocketmq.broker.lite.LiteMetadataUtil; import org.apache.rocketmq.broker.offset.ConsumerOffsetManager; import org.apache.rocketmq.broker.pop.PopConsumerLockService; import org.apache.rocketmq.broker.pop.orderly.ConsumerOrderInfoManager; @@ -30,6 +32,7 @@ import org.apache.rocketmq.common.TopicConfig; import org.apache.rocketmq.common.constant.LoggerName; import org.apache.rocketmq.common.help.FAQUrl; +import org.apache.rocketmq.common.lite.LiteUtil; import org.apache.rocketmq.common.message.MessageConst; import org.apache.rocketmq.common.message.MessageDecoder; import org.apache.rocketmq.common.message.MessageExtBrokerInner; @@ -144,6 +147,11 @@ private RemotingCommand processRequest(final Channel channel, RemotingCommand re return response; } + RemotingCommand ackLiteResponse = ackLite(requestHeader, null, response, channel); + if (ackLiteResponse != null) { + return ackLiteResponse; + } + long minOffset = this.brokerController.getMessageStore().getMinOffsetInQueue(requestHeader.getTopic(), requestHeader.getQueueId()); long maxOffset; try { @@ -456,7 +464,7 @@ protected void ackOrderlyNew(String topic, String consumeGroup, int qId, long ac long nextOffset = consumerOrderInfoManager.commitAndNext(topic, consumeGroup, qId, ackOffset, popTime); if (brokerController.getBrokerConfig().isPopConsumerKVServiceLog()) { POP_LOGGER.info("PopConsumerService ack orderly, time={}, topicId={}, groupId={}, queueId={}, " + - "offset={}, next={}", popTime, topic, consumeGroup, qId, ackOffset, nextOffset); + "offset={}, next={}", popTime, topic, consumeGroup, qId, ackOffset, nextOffset); } if (nextOffset > -1L) { @@ -481,4 +489,88 @@ protected void ackOrderlyNew(String topic, String consumeGroup, int qId, long ac consumerLockService.unlock(consumeGroup, topic); } } + + /** + * Currently, batch ack for lite messages is not supported, so we should ensure that all acknowledgements are individual. + */ + protected RemotingCommand ackLite(AckMessageRequestHeader requestHeader, BatchAckMessageRequestBody batchAckBody, + final RemotingCommand response, final Channel channel) { + if (batchAckBody != null) { + POP_LOGGER.warn("bad request, batch ack lite, {}", batchAckBody); + response.setCode(ResponseCode.ILLEGAL_OPERATION); + response.setRemark("batch ack lite is not supported."); + return response; + } + if (StringUtils.isBlank(requestHeader.getLiteTopic())) { + return null; + } + String group = requestHeader.getConsumerGroup(); + if (!requestHeader.getTopic().equals(LiteMetadataUtil.getLiteBindTopic(group, brokerController))) { + response.setCode(ResponseCode.INVALID_PARAMETER); + response.setRemark("group type or bind topic not match."); + return response; + } + + String lmqName = LiteUtil.toLmqName(requestHeader.getTopic(), requestHeader.getLiteTopic()); + long ackOffset = requestHeader.getOffset(); + long maxOffset = this.brokerController.getLiteLifecycleManager().getMaxOffsetInQueue(lmqName); + if (ackOffset > maxOffset) { + POP_LOGGER.warn("ack lite offset illegal, {}, {}, {}", lmqName, ackOffset, maxOffset); + response.setCode(ResponseCode.NO_MESSAGE); + response.setRemark("ack offset illegal."); + return response; + } + String[] extraInfo = ExtraInfoUtil.split(requestHeader.getExtraInfo()); + if (requestHeader.getQueueId() != 0 + || ExtraInfoUtil.getReviveQid(extraInfo) != KeyBuilder.POP_ORDER_REVIVE_QUEUE) { + response.setCode(ResponseCode.INVALID_PARAMETER); + response.setRemark("ack queue illegal."); + return response; + } + + long popTime = ExtraInfoUtil.getPopTime(extraInfo); + long invisibleTime = ExtraInfoUtil.getInvisibleTime(extraInfo); + + ConsumerOffsetManager consumerOffsetManager = this.brokerController.getConsumerOffsetManager(); + ConsumerOrderInfoManager consumerOrderInfoManager = + brokerController.getPopLiteMessageProcessor().getConsumerOrderInfoManager(); + PopConsumerLockService consumerLockService = this.brokerController.getPopLiteMessageProcessor().getLockService(); + + long oldOffset = consumerOffsetManager.queryOffset(group, lmqName, 0); + if (ackOffset < oldOffset) { + return response; + } + String lockKey = KeyBuilder.buildPopLiteLockKey(group, lmqName); + while (!consumerLockService.tryLock(lockKey)) { + } + + try { + oldOffset = consumerOffsetManager.queryOffset(group, lmqName, 0); + if (ackOffset < oldOffset) { + return response; + } + long nextOffset = consumerOrderInfoManager.commitAndNext(lmqName, group, 0, ackOffset, popTime); + if (nextOffset > -1L) { + if (!consumerOffsetManager.hasOffsetReset(lmqName, group, 0)) { + consumerOffsetManager.commitOffset("AckLiteHost", group, lmqName, 0, nextOffset); + } + if (!consumerOrderInfoManager.checkBlock(null, lmqName, group, 0, invisibleTime)) { + this.brokerController.getLiteEventDispatcher().dispatch(group, lmqName, 0, nextOffset, -1); + } + } + if (nextOffset == -1) { + POP_LOGGER.warn("ack lite, nextOffset illegal. lmq:{}, old:{}, commit:{}", lmqName, oldOffset, ackOffset); + response.setCode(ResponseCode.MESSAGE_ILLEGAL); + response.setRemark("ack offset illegal."); + return response; + } + } finally { + consumerLockService.unlock(lockKey); + } + + this.brokerController.getBrokerStatsManager().incBrokerAckNums(1); + this.brokerController.getBrokerStatsManager().incGroupAckNums(group, requestHeader.getTopic(), 1); + response.setCode(ResponseCode.SUCCESS); + return response; + } } 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 4361431bec0..4b8b3988758 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 @@ -67,7 +67,7 @@ import org.apache.rocketmq.broker.controller.ReplicasManager; import org.apache.rocketmq.broker.filter.ConsumerFilterData; import org.apache.rocketmq.broker.filter.ExpressionMessageFilter; - +import org.apache.rocketmq.broker.lite.LiteMetadataUtil; import org.apache.rocketmq.broker.metrics.InvocationStatus; import org.apache.rocketmq.broker.plugin.BrokerAttachedPlugin; import org.apache.rocketmq.broker.subscription.SubscriptionGroupManager; @@ -90,6 +90,7 @@ import org.apache.rocketmq.common.constant.FIleReadaheadMode; import org.apache.rocketmq.common.constant.LoggerName; import org.apache.rocketmq.common.constant.PermName; +import org.apache.rocketmq.common.lite.LiteUtil; import org.apache.rocketmq.common.message.MessageAccessor; import org.apache.rocketmq.common.message.MessageConst; import org.apache.rocketmq.common.message.MessageDecoder; @@ -800,6 +801,9 @@ private synchronized RemotingCommand deleteTopic(ChannelHandlerContext ctx, } try { + if (LiteMetadataUtil.isLiteMessageType(topic, brokerController)) { + brokerController.getLiteLifecycleManager().cleanByParentTopic(topic); + } for (String topicToClean : topicsToClean) { // delete topic deleteTopicInBroker(topicToClean); @@ -1181,9 +1185,27 @@ private RemotingCommand searchOffsetByTimestamp(ChannelHandlerContext ctx, return rewriteResult; } - long offset = this.brokerController.getMessageStore().getOffsetInQueueByTime(requestHeader.getTopic(), requestHeader.getQueueId(), - requestHeader.getTimestamp(), requestHeader.getBoundaryType()); + boolean queryOffset = true; + String topic = requestHeader.getTopic(); + int queueId = requestHeader.getQueueId(); + String liteTopic = requestHeader.getLiteTopic(); + if (StringUtils.isNotBlank(liteTopic)) { + topic = LiteUtil.toLmqName(topic, liteTopic); + long maxOffset = 0; + if (queueId == 0) { + maxOffset = this.brokerController.getLiteLifecycleManager().getMaxOffsetInQueue(topic); + } + // lite topic check max offset first + if (maxOffset <= 0) { + queryOffset = false; + } + } + long offset = 0L; + if (queryOffset) { + offset = this.brokerController.getMessageStore().getOffsetInQueueByTime(topic, queueId, + requestHeader.getTimestamp(), requestHeader.getBoundaryType()); + } responseHeader.setOffset(offset); response.setCode(ResponseCode.SUCCESS); @@ -1678,7 +1700,8 @@ private RemotingCommand deleteSubscriptionGroup(ChannelHandlerContext ctx, this.brokerController.getSubscriptionGroupManager().deleteSubscriptionGroupConfig(requestHeader.getGroupName()); - if (requestHeader.isCleanOffset()) { + if (requestHeader.isCleanOffset() + || LiteMetadataUtil.isLiteGroupType(requestHeader.getGroupName(), this.brokerController)) { this.brokerController.getConsumerOffsetManager().removeOffset(requestHeader.getGroupName()); this.brokerController.getPopInflightMessageCounter().clearInFlightMessageNumByGroupName(requestHeader.getGroupName()); } diff --git a/broker/src/main/java/org/apache/rocketmq/broker/processor/ChangeInvisibleTimeProcessor.java b/broker/src/main/java/org/apache/rocketmq/broker/processor/ChangeInvisibleTimeProcessor.java index 332930cd786..133e13ccb2c 100644 --- a/broker/src/main/java/org/apache/rocketmq/broker/processor/ChangeInvisibleTimeProcessor.java +++ b/broker/src/main/java/org/apache/rocketmq/broker/processor/ChangeInvisibleTimeProcessor.java @@ -22,6 +22,7 @@ import java.nio.charset.StandardCharsets; import java.util.concurrent.CompletableFuture; import java.util.concurrent.TimeUnit; +import org.apache.commons.lang3.StringUtils; import org.apache.rocketmq.broker.BrokerController; import org.apache.rocketmq.broker.offset.ConsumerOffsetManager; import org.apache.rocketmq.broker.pop.PopConsumerLockService; @@ -30,6 +31,7 @@ import org.apache.rocketmq.common.TopicConfig; import org.apache.rocketmq.common.constant.LoggerName; import org.apache.rocketmq.common.help.FAQUrl; +import org.apache.rocketmq.common.lite.LiteUtil; import org.apache.rocketmq.common.message.MessageConst; import org.apache.rocketmq.common.message.MessageDecoder; import org.apache.rocketmq.common.message.MessageExtBrokerInner; @@ -122,6 +124,12 @@ public CompletableFuture processRequestAsync(final Channel chan response.setRemark(errorInfo); return CompletableFuture.completedFuture(response); } + + CompletableFuture future = processChangeInvisibleTimeForLite(requestHeader, response, responseHeader); + if (future != null) { + return future; + } + long minOffset = this.brokerController.getMessageStore().getMinOffsetInQueue(requestHeader.getTopic(), requestHeader.getQueueId()); long maxOffset; try { @@ -354,6 +362,55 @@ private CompletableFuture appendCheckPointThenAckOrigin( }); } + protected CompletableFuture processChangeInvisibleTimeForLite( + ChangeInvisibleTimeRequestHeader requestHeader, + RemotingCommand response, ChangeInvisibleTimeResponseHeader responseHeader) { + if (StringUtils.isBlank(requestHeader.getLiteTopic())) { + return null; + } + String lmqName = LiteUtil.toLmqName(requestHeader.getTopic(), requestHeader.getLiteTopic()); + long maxOffset = this.brokerController.getLiteLifecycleManager().getMaxOffsetInQueue(lmqName); + if (requestHeader.getOffset() > maxOffset) { + POP_LOGGER.warn("process lite offset illegal, {}, {}, {}", lmqName, requestHeader.getOffset(), maxOffset); + response.setCode(ResponseCode.NO_MESSAGE); + return CompletableFuture.completedFuture(response); + } + + String group = requestHeader.getConsumerGroup(); + String[] extraInfo = ExtraInfoUtil.split(requestHeader.getExtraInfo()); + long popTime = ExtraInfoUtil.getPopTime(extraInfo); + + ConsumerOffsetManager consumerOffsetManager = this.brokerController.getConsumerOffsetManager(); + ConsumerOrderInfoManager consumerOrderInfoManager = + brokerController.getPopLiteMessageProcessor().getConsumerOrderInfoManager(); + PopConsumerLockService consumerLockService = this.brokerController.getPopLiteMessageProcessor().getLockService(); + + long oldOffset = consumerOffsetManager.queryOffset(group, lmqName, 0); + if (requestHeader.getOffset() < oldOffset) { + return CompletableFuture.completedFuture(response); + } + + while (!consumerLockService.tryLock(group, lmqName)) { + } + + try { + oldOffset = consumerOffsetManager.queryOffset(group, lmqName, 0); + if (requestHeader.getOffset() < oldOffset) { + return CompletableFuture.completedFuture(response); + } + long visibilityTimeout = System.currentTimeMillis() + requestHeader.getInvisibleTime(); + consumerOrderInfoManager.updateNextVisibleTime( + lmqName, group, 0, requestHeader.getOffset(), popTime, visibilityTimeout); + + responseHeader.setInvisibleTime(visibilityTimeout - popTime); + responseHeader.setPopTime(popTime); + responseHeader.setReviveQid(ExtraInfoUtil.getReviveQid(extraInfo)); + } finally { + consumerLockService.unlock(group, lmqName); + } + return CompletableFuture.completedFuture(response); + } + protected void doResponse(Channel channel, RemotingCommand request, final RemotingCommand response) { NettyRemotingAbstract.writeResponse(channel, request, response, null, brokerController.getBrokerMetricsManager().getRemotingMetricsManager()); diff --git a/broker/src/main/java/org/apache/rocketmq/broker/processor/LiteManagerProcessor.java b/broker/src/main/java/org/apache/rocketmq/broker/processor/LiteManagerProcessor.java new file mode 100644 index 00000000000..ac12983d61e --- /dev/null +++ b/broker/src/main/java/org/apache/rocketmq/broker/processor/LiteManagerProcessor.java @@ -0,0 +1,383 @@ +/* + * 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.processor; + +import com.google.common.annotations.VisibleForTesting; +import io.netty.channel.ChannelHandlerContext; +import java.util.List; + +import org.apache.commons.lang3.StringUtils; +import org.apache.rocketmq.broker.BrokerController; +import org.apache.rocketmq.broker.lite.AbstractLiteLifecycleManager; +import org.apache.rocketmq.broker.lite.LiteMetadataUtil; +import org.apache.rocketmq.broker.lite.LiteSharding; +import org.apache.rocketmq.common.Pair; +import org.apache.rocketmq.common.TopicConfig; +import org.apache.rocketmq.common.attribute.TopicMessageType; +import org.apache.rocketmq.common.constant.LoggerName; +import org.apache.rocketmq.common.lite.LiteLagInfo; +import org.apache.rocketmq.common.lite.LiteSubscription; +import org.apache.rocketmq.common.lite.LiteUtil; +import org.apache.rocketmq.logging.org.slf4j.Logger; +import org.apache.rocketmq.logging.org.slf4j.LoggerFactory; +import org.apache.rocketmq.remoting.common.RemotingHelper; +import org.apache.rocketmq.remoting.exception.RemotingCommandException; +import org.apache.rocketmq.remoting.netty.NettyRequestProcessor; +import org.apache.rocketmq.remoting.protocol.RemotingCommand; +import org.apache.rocketmq.remoting.protocol.RequestCode; +import org.apache.rocketmq.remoting.protocol.ResponseCode; +import org.apache.rocketmq.remoting.protocol.admin.OffsetWrapper; +import org.apache.rocketmq.remoting.protocol.admin.TopicOffset; +import org.apache.rocketmq.remoting.protocol.body.GetBrokerLiteInfoResponseBody; +import org.apache.rocketmq.remoting.protocol.body.GetLiteClientInfoResponseBody; +import org.apache.rocketmq.remoting.protocol.body.GetLiteGroupInfoResponseBody; +import org.apache.rocketmq.remoting.protocol.body.GetLiteTopicInfoResponseBody; +import org.apache.rocketmq.remoting.protocol.body.GetParentTopicInfoResponseBody; +import org.apache.rocketmq.remoting.protocol.header.GetLiteClientInfoRequestHeader; +import org.apache.rocketmq.remoting.protocol.header.GetLiteGroupInfoRequestHeader; +import org.apache.rocketmq.remoting.protocol.header.GetLiteTopicInfoRequestHeader; +import org.apache.rocketmq.remoting.protocol.header.GetParentTopicInfoRequestHeader; +import org.apache.rocketmq.remoting.protocol.header.TriggerLiteDispatchRequestHeader; +import org.apache.rocketmq.remoting.protocol.subscription.SubscriptionGroupConfig; + +import java.util.HashSet; +import java.util.Map; +import java.util.Set; + +public class LiteManagerProcessor implements NettyRequestProcessor { + private static final Logger LOGGER = LoggerFactory.getLogger(LoggerName.ROCKETMQ_POP_LITE_LOGGER_NAME); + + private static final int MAX_RETURN_COUNT = 10000; + private final BrokerController brokerController; + private final AbstractLiteLifecycleManager liteLifecycleManager; + private final LiteSharding liteSharding; + + public LiteManagerProcessor(BrokerController brokerController, + AbstractLiteLifecycleManager liteLifecycleManager, LiteSharding liteSharding) { + this.brokerController = brokerController; + this.liteLifecycleManager = liteLifecycleManager; + this.liteSharding = liteSharding; + } + + @Override + public RemotingCommand processRequest(ChannelHandlerContext ctx, RemotingCommand request) throws Exception { + switch (request.getCode()) { + case RequestCode.GET_BROKER_LITE_INFO: + return this.getBrokerLiteInfo(ctx, request); + case RequestCode.GET_PARENT_TOPIC_INFO: + return this.getParentTopicInfo(ctx, request); + case RequestCode.GET_LITE_TOPIC_INFO: + return this.getLiteTopicInfo(ctx, request); + case RequestCode.GET_LITE_CLIENT_INFO: + return this.getLiteClientInfo(ctx, request); + case RequestCode.GET_LITE_GROUP_INFO: + return this.getLiteGroupInfo(ctx, request); + case RequestCode.TRIGGER_LITE_DISPATCH: + return this.triggerLiteDispatch(ctx, request); + default: + break; + } + return null; + } + + @VisibleForTesting + protected RemotingCommand getBrokerLiteInfo(ChannelHandlerContext ctx, + RemotingCommand request) throws RemotingCommandException { + final RemotingCommand response = RemotingCommand.createResponseCommand(null); + + GetBrokerLiteInfoResponseBody body = new GetBrokerLiteInfoResponseBody(); + body.setStoreType(brokerController.getMessageStoreConfig().getStoreType()); + body.setMaxLmqNum(brokerController.getMessageStoreConfig().getMaxLmqConsumeQueueNum()); + body.setCurrentLmqNum(brokerController.getMessageStore().getQueueStore().getLmqNum()); + body.setLiteSubscriptionCount(brokerController.getLiteSubscriptionRegistry().getActiveSubscriptionNum()); + body.setOrderInfoCount(brokerController.getPopLiteMessageProcessor().getConsumerOrderInfoManager().getOrderInfoCount()); + body.setCqTableSize(brokerController.getMessageStore().getQueueStore().getConsumeQueueTable().size()); + body.setOffsetTableSize(brokerController.getConsumerOffsetManager().getOffsetTable().size()); + body.setEventMapSize(brokerController.getLiteEventDispatcher().getEventMapSize()); + body.setTopicMeta(LiteMetadataUtil.getTopicTtlMap(brokerController)); + body.setGroupMeta(LiteMetadataUtil.getSubscriberGroupMap(brokerController)); + + response.setBody(body.encode()); + response.setCode(ResponseCode.SUCCESS); + response.setRemark(null); + return response; + } + + @VisibleForTesting + protected RemotingCommand getParentTopicInfo(ChannelHandlerContext ctx, + RemotingCommand request) throws RemotingCommandException { + final GetParentTopicInfoRequestHeader requestHeader = + request.decodeCommandCustomHeader(GetParentTopicInfoRequestHeader.class); + final RemotingCommand response = RemotingCommand.createResponseCommand(null); + + String topic = requestHeader.getTopic(); + TopicConfig topicConfig = brokerController.getTopicConfigManager().selectTopicConfig(topic); + if (null == topicConfig) { + response.setCode(ResponseCode.TOPIC_NOT_EXIST); + response.setRemark(String.format("Topic [%s] not exist.", topic)); + return response; + } + if (!TopicMessageType.LITE.equals(topicConfig.getTopicMessageType())) { + response.setCode(ResponseCode.INVALID_PARAMETER); + response.setRemark(String.format("Topic [%s] type not match.", topic)); + return response; + } + + Map> subscriberGroupMap = LiteMetadataUtil.getSubscriberGroupMap(brokerController); + + GetParentTopicInfoResponseBody body = new GetParentTopicInfoResponseBody(); + body.setTopic(topic); + body.setTtl(topicConfig.getLiteTopicExpiration()); + body.setLmqNum(brokerController.getMessageStore().getQueueStore().getLmqNum()); + body.setLiteTopicCount(liteLifecycleManager.getLiteTopicCount(topic)); + body.setGroups(subscriberGroupMap != null ? subscriberGroupMap.get(topic) : null); + + response.setBody(body.encode()); + response.setCode(ResponseCode.SUCCESS); + response.setRemark(null); + return response; + } + + @VisibleForTesting + protected RemotingCommand getLiteTopicInfo(ChannelHandlerContext ctx, + RemotingCommand request) throws RemotingCommandException { + final GetLiteTopicInfoRequestHeader requestHeader = + request.decodeCommandCustomHeader(GetLiteTopicInfoRequestHeader.class); + final RemotingCommand response = RemotingCommand.createResponseCommand(null); + + String parentTopic = requestHeader.getParentTopic(); + String liteTopic = requestHeader.getLiteTopic(); + TopicConfig topicConfig = brokerController.getTopicConfigManager().selectTopicConfig(parentTopic); + if (null == topicConfig) { + response.setCode(ResponseCode.TOPIC_NOT_EXIST); + response.setRemark(String.format("Topic [%s] not exist.", parentTopic)); + return response; + } + if (!TopicMessageType.LITE.equals(topicConfig.getTopicMessageType())) { + response.setCode(ResponseCode.INVALID_PARAMETER); + response.setRemark(String.format("Topic [%s] type not match.", parentTopic)); + return response; + } + + String lmqName = LiteUtil.toLmqName(parentTopic, liteTopic); + TopicOffset topicOffset = new TopicOffset(); + long minOffset = 0; + long lastUpdateTimestamp = 0; + long maxOffset = liteLifecycleManager.getMaxOffsetInQueue(lmqName); + if (maxOffset > 0) { + minOffset = this.brokerController.getMessageStore().getMinOffsetInQueue(lmqName, 0); + lastUpdateTimestamp = brokerController.getMessageStore().getMessageStoreTimeStamp(lmqName, 0, maxOffset - 1); + } + topicOffset.setMinOffset(minOffset < 0 ? 0 : minOffset); + topicOffset.setMaxOffset(maxOffset < 0 ? 0 : maxOffset); + topicOffset.setLastUpdateTimestamp(lastUpdateTimestamp); + + GetLiteTopicInfoResponseBody body = new GetLiteTopicInfoResponseBody(); + body.setParentTopic(parentTopic); + body.setLiteTopic(liteTopic); + body.setSubscriber(brokerController.getLiteSubscriptionRegistry().getSubscriber(lmqName)); + body.setTopicOffset(topicOffset); + body.setShardingToBroker(brokerController.getBrokerConfig().getBrokerName().equals( + liteSharding.shardingByLmqName(parentTopic, lmqName))); + + response.setBody(body.encode()); + response.setCode(ResponseCode.SUCCESS); + response.setRemark(null); + return response; + } + + @VisibleForTesting + protected RemotingCommand getLiteClientInfo(ChannelHandlerContext ctx, + RemotingCommand request) throws RemotingCommandException { + final GetLiteClientInfoRequestHeader requestHeader = + request.decodeCommandCustomHeader(GetLiteClientInfoRequestHeader.class); + final RemotingCommand response = RemotingCommand.createResponseCommand(null); + + String parentTopic = requestHeader.getParentTopic(); + String group = requestHeader.getGroup(); + TopicConfig topicConfig = brokerController.getTopicConfigManager().selectTopicConfig(parentTopic); + if (null == topicConfig) { + response.setCode(ResponseCode.TOPIC_NOT_EXIST); + response.setRemark(String.format("Topic [%s] not exist.", parentTopic)); + return response; + } + if (!TopicMessageType.LITE.equals(topicConfig.getTopicMessageType())) { + response.setCode(ResponseCode.INVALID_PARAMETER); + response.setRemark(String.format("Topic [%s] type not match.", parentTopic)); + return response; + } + SubscriptionGroupConfig groupConfig = + brokerController.getSubscriptionGroupManager().findSubscriptionGroupConfig(group); + if (null == groupConfig) { + response.setCode(ResponseCode.SUBSCRIPTION_GROUP_NOT_EXIST); + response.setRemark(String.format("Group [%s] not exist.", group)); + return response; + } + if (!parentTopic.equals(groupConfig.getLiteBindTopic())) { + response.setCode(ResponseCode.INVALID_PARAMETER); + response.setRemark(String.format("Subscription [%s]-[%s] not match.", group, parentTopic)); + return response; + } + + String clientId = requestHeader.getClientId(); + int maxCount = Math.min(requestHeader.getMaxCount(), MAX_RETURN_COUNT); + Set returnSet = null; + int liteTopicCount = 0; + LiteSubscription liteSubscription = brokerController.getLiteSubscriptionRegistry().getLiteSubscription(clientId); + if (liteSubscription != null && liteSubscription.getLiteTopicSet() != null) { + Set liteTopicSet = liteSubscription.getLiteTopicSet(); + liteTopicCount = liteTopicSet.size(); + if (maxCount >= liteTopicCount) { + returnSet = liteTopicSet; + } else { + returnSet = new HashSet<>(maxCount); + int count = 0; + for (String topic : liteTopicSet) { + if (count >= maxCount) { + break; + } + returnSet.add(topic); + count++; + } + } + } else { + liteTopicCount = -1; + } + + GetLiteClientInfoResponseBody body = new GetLiteClientInfoResponseBody(); + body.setParentTopic(parentTopic); + body.setGroup(group); + body.setClientId(clientId); + body.setLiteTopicCount(liteTopicCount); + body.setLiteTopicSet(returnSet); + body.setLastAccessTime(brokerController.getLiteEventDispatcher().getClientLastAccessTime(clientId)); + + response.setBody(body.encode()); + response.setCode(ResponseCode.SUCCESS); + response.setRemark(null); + return response; + } + + @VisibleForTesting + protected RemotingCommand getLiteGroupInfo(ChannelHandlerContext ctx, RemotingCommand request) + throws RemotingCommandException { + final GetLiteGroupInfoRequestHeader requestHeader = + request.decodeCommandCustomHeader(GetLiteGroupInfoRequestHeader.class); + final String group = requestHeader.getGroup(); + final String liteTopic = requestHeader.getLiteTopic(); + final int topK = requestHeader.getTopK(); + LOGGER.info("Broker receive request to getLiteGroupInfo, group:{}, liteTopic:{}, caller:{}", + group, liteTopic, RemotingHelper.parseChannelRemoteAddr(ctx.channel())); + + SubscriptionGroupConfig groupConfig = + brokerController.getSubscriptionGroupManager().findSubscriptionGroupConfig(group); + if (null == groupConfig) { + return RemotingCommand.createResponseCommand(ResponseCode.SUBSCRIPTION_GROUP_NOT_EXIST, + String.format("Group [%s] not exist.", group)); + } + if (StringUtils.isEmpty(groupConfig.getLiteBindTopic())) { + return RemotingCommand.createResponseCommand(ResponseCode.INVALID_PARAMETER, + String.format("Group [%s] is not a LITE group.", group)); + } + String bindTopic = groupConfig.getLiteBindTopic(); + GetLiteGroupInfoResponseBody body = new GetLiteGroupInfoResponseBody(); + body.setGroup(group); + body.setParentTopic(bindTopic); + body.setLiteTopic(liteTopic); + + if (StringUtils.isEmpty(liteTopic)) { + Pair, Long> lagCountPair = brokerController.getBrokerMetricsManager() + .getLiteConsumerLagCalculator() + .getLagCountTopK(group, topK); + + Pair, Long> lagTimePair = brokerController.getBrokerMetricsManager() + .getLiteConsumerLagCalculator() + .getLagTimestampTopK(group, bindTopic, topK); + + body.setLagCountTopK(lagCountPair.getObject1()); + body.setTotalLagCount(lagCountPair.getObject2()); + body.setLagTimestampTopK(lagTimePair.getObject1()); + body.setEarliestUnconsumedTimestamp(lagTimePair.getObject2()); + } else { + String lmqName = LiteUtil.toLmqName(bindTopic, liteTopic); + long maxOffset = liteLifecycleManager.getMaxOffsetInQueue(lmqName); + if (maxOffset > 0) { + long commitOffset = brokerController.getConsumerOffsetManager().queryOffset(group, lmqName, 0); + if (commitOffset >= 0) { + // lag count and unconsumedTimestamp, reuse total field + body.setTotalLagCount(maxOffset - commitOffset); + body.setEarliestUnconsumedTimestamp(brokerController.getMessageStore().getMessageStoreTimeStamp( + lmqName, 0, commitOffset)); + + OffsetWrapper offsetWrapper = new OffsetWrapper(); + offsetWrapper.setBrokerOffset(maxOffset); + offsetWrapper.setConsumerOffset(commitOffset); + if (commitOffset - 1 >= 0) { + offsetWrapper.setLastTimestamp( + brokerController.getMessageStore().getMessageStoreTimeStamp(lmqName, 0, commitOffset - 1)); + } + body.setLiteTopicOffsetWrapper(offsetWrapper); + } + } else { + body.setTotalLagCount(-1); + body.setEarliestUnconsumedTimestamp(-1); + } + } + + final RemotingCommand response = RemotingCommand.createResponseCommand(null); + response.setBody(body.encode()); + response.setCode(ResponseCode.SUCCESS); + return response; + } + + @VisibleForTesting + protected RemotingCommand triggerLiteDispatch(ChannelHandlerContext ctx, RemotingCommand request) + throws RemotingCommandException { + final TriggerLiteDispatchRequestHeader requestHeader = + request.decodeCommandCustomHeader(TriggerLiteDispatchRequestHeader.class); + final String group = requestHeader.getGroup(); + final String clientId = requestHeader.getClientId(); + LOGGER.info("Broker receive request to triggerLiteDispatch, group:{}, clientId:{}, caller:{}", + group, clientId, RemotingHelper.parseChannelRemoteAddr(ctx.channel())); + SubscriptionGroupConfig groupConfig = + brokerController.getSubscriptionGroupManager().findSubscriptionGroupConfig(group); + if (null == groupConfig) { + return RemotingCommand.createResponseCommand(ResponseCode.SUBSCRIPTION_GROUP_NOT_EXIST, + String.format("Group [%s] not exist.", group)); + } + if (StringUtils.isEmpty(groupConfig.getLiteBindTopic())) { + return RemotingCommand.createResponseCommand(ResponseCode.INVALID_PARAMETER, + String.format("Group [%s] is not a LITE group.", group)); + } + + if (StringUtils.isNotEmpty(clientId)) { + brokerController.getLiteEventDispatcher().doFullDispatch(clientId, group); + } else { + brokerController.getLiteEventDispatcher().doFullDispatchByGroup(group); + } + + final RemotingCommand response = RemotingCommand.createResponseCommand(null); + response.setCode(ResponseCode.SUCCESS); + return response; + } + + @Override + public boolean rejectRequest() { + return false; + } +} diff --git a/broker/src/main/java/org/apache/rocketmq/broker/processor/LiteSubscriptionCtlProcessor.java b/broker/src/main/java/org/apache/rocketmq/broker/processor/LiteSubscriptionCtlProcessor.java new file mode 100644 index 00000000000..2b3ce6f9654 --- /dev/null +++ b/broker/src/main/java/org/apache/rocketmq/broker/processor/LiteSubscriptionCtlProcessor.java @@ -0,0 +1,136 @@ +/* + * 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.processor; + +import io.netty.channel.ChannelHandlerContext; +import java.util.Collections; +import java.util.Set; +import java.util.stream.Collectors; +import org.apache.commons.collections.CollectionUtils; +import org.apache.commons.lang3.StringUtils; +import org.apache.rocketmq.broker.BrokerController; +import org.apache.rocketmq.broker.lite.LiteSubscriptionRegistry; +import org.apache.rocketmq.broker.lite.LiteQuotaException; +import org.apache.rocketmq.broker.lite.LiteMetadataUtil; +import org.apache.rocketmq.common.constant.LoggerName; +import org.apache.rocketmq.common.lite.LiteSubscriptionDTO; +import org.apache.rocketmq.common.lite.LiteUtil; +import org.apache.rocketmq.remoting.netty.NettyRequestProcessor; +import org.apache.rocketmq.remoting.protocol.RemotingCommand; +import org.apache.rocketmq.remoting.protocol.ResponseCode; +import org.apache.rocketmq.remoting.protocol.body.LiteSubscriptionCtlRequestBody; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +public class LiteSubscriptionCtlProcessor implements NettyRequestProcessor { + protected final Logger log = LoggerFactory.getLogger(LoggerName.ROCKETMQ_POP_LITE_LOGGER_NAME); + + private final BrokerController brokerController; + private final LiteSubscriptionRegistry liteSubscriptionRegistry; + + public LiteSubscriptionCtlProcessor(BrokerController brokerController, LiteSubscriptionRegistry liteSubscriptionRegistry) { + this.brokerController = brokerController; + this.liteSubscriptionRegistry = liteSubscriptionRegistry; + } + + @Override + public RemotingCommand processRequest(ChannelHandlerContext ctx, RemotingCommand request) throws Exception { + if (request.getBody() == null) { + return RemotingCommand.createResponseCommand(ResponseCode.ILLEGAL_OPERATION, + "Request body is null."); + } + + final LiteSubscriptionCtlRequestBody requestBody = LiteSubscriptionCtlRequestBody + .decode(request.getBody(), LiteSubscriptionCtlRequestBody.class); + + Set entrySet = requestBody.getSubscriptionSet(); + if (CollectionUtils.isEmpty(entrySet)) { + return RemotingCommand.createResponseCommand(ResponseCode.ILLEGAL_OPERATION, + "LiteSubscriptionCtlRequestBody is empty."); + } + + try { + for (LiteSubscriptionDTO entry : entrySet) { + final String clientId = entry.getClientId(); + final String group = entry.getGroup(); + final String topic = entry.getTopic(); + if (StringUtils.isBlank(clientId)) { + log.warn("clientId is blank, {}", entry); + continue; + } + if (StringUtils.isBlank(group)) { + log.warn("group is blank, {}", entry); + continue; + } + if (StringUtils.isBlank(topic)) { + log.warn("topic is blank, {}", entry); + continue; + } + final Set lmqNameSet = toLmqNameSet(entry); + switch (entry.getAction()) { + case PARTIAL_ADD: + checkConsumeEnable(group); + this.liteSubscriptionRegistry.updateClientChannel(clientId, ctx.channel()); + this.liteSubscriptionRegistry.addPartialSubscription(clientId, group, topic, lmqNameSet, entry.getOffsetOption()); + break; + case PARTIAL_REMOVE: + this.liteSubscriptionRegistry.removePartialSubscription(clientId, group, topic, lmqNameSet); + break; + case COMPLETE_ADD: + checkConsumeEnable(group); + this.liteSubscriptionRegistry.updateClientChannel(clientId, ctx.channel()); + this.liteSubscriptionRegistry.addCompleteSubscription(clientId, group, topic, lmqNameSet, + entry.getVersion()); + break; + case COMPLETE_REMOVE: + this.liteSubscriptionRegistry.removeCompleteSubscription(clientId); + break; + } + } + return RemotingCommand.createResponseCommand(ResponseCode.SUCCESS, null); + } catch (LiteQuotaException e) { + return RemotingCommand.createResponseCommand(ResponseCode.LITE_SUBSCRIPTION_QUOTA_EXCEEDED, e.toString()); + } catch (IllegalStateException e) { + return RemotingCommand.createResponseCommand(ResponseCode.ILLEGAL_OPERATION, e.toString()); + } catch (Exception e) { + log.error("LiteSubscriptionCtlProcessor error", e); + return RemotingCommand.createResponseCommand(ResponseCode.SYSTEM_ERROR, e.toString()); + } + } + + private void checkConsumeEnable(String group) { + if (!LiteMetadataUtil.isConsumeEnable(group, brokerController)) { + throw new IllegalStateException("Consumer group is not allowed to consume."); + } + } + + private Set toLmqNameSet(LiteSubscriptionDTO liteSubscriptionDTO) { + if (CollectionUtils.isEmpty(liteSubscriptionDTO.getLiteTopicSet())) { + return Collections.emptySet(); + } + return liteSubscriptionDTO.getLiteTopicSet().stream() + .map(liteTopic -> LiteUtil.toLmqName(liteSubscriptionDTO.getTopic(), liteTopic)) + .collect(Collectors.toSet()); + } + + @Override + public boolean rejectRequest() { + return false; + } + +} diff --git a/broker/src/main/java/org/apache/rocketmq/broker/processor/PopLiteMessageProcessor.java b/broker/src/main/java/org/apache/rocketmq/broker/processor/PopLiteMessageProcessor.java new file mode 100644 index 00000000000..cb32b9757c9 --- /dev/null +++ b/broker/src/main/java/org/apache/rocketmq/broker/processor/PopLiteMessageProcessor.java @@ -0,0 +1,480 @@ +/* + * 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.processor; + +import com.google.common.annotations.VisibleForTesting; +import io.netty.channel.Channel; +import io.netty.channel.ChannelHandlerContext; +import io.opentelemetry.api.common.Attributes; +import org.apache.rocketmq.broker.BrokerController; +import org.apache.rocketmq.broker.lite.LiteEventDispatcher; +import org.apache.rocketmq.broker.longpolling.PollingResult; +import org.apache.rocketmq.broker.longpolling.PopLiteLongPollingService; +import org.apache.rocketmq.broker.metrics.LiteConsumerLagCalculator; +import org.apache.rocketmq.broker.offset.MemoryConsumerOrderInfoManager; +import org.apache.rocketmq.broker.pop.PopConsumerLockService; +import org.apache.rocketmq.broker.pop.orderly.ConsumerOrderInfoManager; +import org.apache.rocketmq.common.KeyBuilder; +import org.apache.rocketmq.common.MixAll; +import org.apache.rocketmq.common.Pair; +import org.apache.rocketmq.common.ServiceThread; +import org.apache.rocketmq.common.TopicConfig; +import org.apache.rocketmq.common.attribute.TopicMessageType; +import org.apache.rocketmq.common.constant.ConsumeInitMode; +import org.apache.rocketmq.common.constant.LoggerName; +import org.apache.rocketmq.common.constant.PermName; +import org.apache.rocketmq.common.help.FAQUrl; +import org.apache.rocketmq.common.message.MessageConst; +import org.apache.rocketmq.common.topic.TopicValidator; +import org.apache.rocketmq.logging.org.slf4j.Logger; +import org.apache.rocketmq.logging.org.slf4j.LoggerFactory; +import org.apache.rocketmq.remoting.exception.RemotingCommandException; +import org.apache.rocketmq.remoting.netty.NettyRemotingAbstract; +import org.apache.rocketmq.remoting.netty.NettyRequestProcessor; +import org.apache.rocketmq.remoting.protocol.RemotingCommand; +import org.apache.rocketmq.remoting.protocol.ResponseCode; +import org.apache.rocketmq.remoting.protocol.header.PopLiteMessageRequestHeader; +import org.apache.rocketmq.remoting.protocol.header.PopLiteMessageResponseHeader; +import org.apache.rocketmq.remoting.protocol.subscription.SubscriptionGroupConfig; +import org.apache.rocketmq.store.GetMessageResult; +import org.apache.rocketmq.store.GetMessageStatus; +import org.apache.rocketmq.store.SelectMappedBufferResult; +import org.apache.rocketmq.store.exception.ConsumeQueueException; + +import java.nio.ByteBuffer; +import java.util.Collections; +import java.util.HashSet; +import java.util.Iterator; +import java.util.List; +import java.util.Objects; +import java.util.Set; +import java.util.concurrent.TimeUnit; +import java.util.concurrent.atomic.AtomicLong; + +import static org.apache.rocketmq.broker.metrics.BrokerMetricsConstant.LABEL_CONSUMER_GROUP; +import static org.apache.rocketmq.broker.metrics.BrokerMetricsConstant.LABEL_IS_RETRY; +import static org.apache.rocketmq.broker.metrics.BrokerMetricsConstant.LABEL_IS_SYSTEM; +import static org.apache.rocketmq.broker.metrics.BrokerMetricsConstant.LABEL_TOPIC; + +/** + * Pop lite implementation, support FIFO consuming. + * This processor uses independent in-memory consumer order info and lock service, + * along with a specialized long polling service. + */ +public class PopLiteMessageProcessor implements NettyRequestProcessor { + private static final Logger LOGGER = LoggerFactory.getLogger(LoggerName.ROCKETMQ_POP_LITE_LOGGER_NAME); + private static final String BORN_TIME = "bornTime"; + + private final BrokerController brokerController; + private final PopLiteLongPollingService popLiteLongPollingService; + private final PopConsumerLockService lockService; + private final LiteEventDispatcher liteEventDispatcher; + private final ConsumerOrderInfoManager consumerOrderInfoManager; + private final PopLiteLockManager popLiteLockManager; + + public PopLiteMessageProcessor(final BrokerController brokerController, LiteEventDispatcher liteEventDispatcher) { + this.brokerController = brokerController; + this.popLiteLongPollingService = new PopLiteLongPollingService(brokerController, this, false); + this.lockService = new PopConsumerLockService(TimeUnit.MINUTES.toMillis(1)); + this.liteEventDispatcher = liteEventDispatcher; + this.consumerOrderInfoManager = new MemoryConsumerOrderInfoManager(brokerController); + this.popLiteLockManager = new PopLiteLockManager(); + } + + @Override + public boolean rejectRequest() { + return false; + } + + @Override + public RemotingCommand processRequest(final ChannelHandlerContext ctx, RemotingCommand request) + throws RemotingCommandException { + + final long beginTimeMills = brokerController.getMessageStore().now(); + Channel channel = ctx.channel(); + request.addExtFieldIfNotExist(BORN_TIME, String.valueOf(System.currentTimeMillis())); + if (Objects.equals(request.getExtFields().get(BORN_TIME), "0")) { + request.addExtField(BORN_TIME, String.valueOf(System.currentTimeMillis())); + } + RemotingCommand response = RemotingCommand.createResponseCommand(PopLiteMessageResponseHeader.class); + response.setOpaque(request.getOpaque()); + + final PopLiteMessageRequestHeader requestHeader = + request.decodeCommandCustomHeader(PopLiteMessageRequestHeader.class, true); + final PopLiteMessageResponseHeader responseHeader = (PopLiteMessageResponseHeader) response.readCustomHeader(); + RemotingCommand preCheckResponse = preCheck(ctx, requestHeader, response); + if (preCheckResponse != null) { + return preCheckResponse; + } + + String clientId = requestHeader.getClientId(); + String group = requestHeader.getConsumerGroup(); + String parentTopic = requestHeader.getTopic(); + int maxNum = requestHeader.getMaxMsgNum(); + long popTime = System.currentTimeMillis(); + long invisibleTime = requestHeader.getInvisibleTime(); + + Pair rst = popByClientId(channel.remoteAddress().toString(), parentTopic, + group, clientId, popTime, invisibleTime, maxNum, requestHeader.getAttemptId()); + + final GetMessageResult getMessageResult = rst.getObject2(); + if (getMessageResult != null && getMessageResult.getMessageCount() > 0) { + final byte[] r = readGetMessageResult(getMessageResult); + brokerController.getBrokerStatsManager().incGroupGetLatency(group, parentTopic, 0, + (int) (brokerController.getMessageStore().now() - beginTimeMills)); + brokerController.getBrokerStatsManager().incBrokerGetNums(parentTopic, getMessageResult.getMessageCount()); + brokerController.getBrokerStatsManager().incGroupGetNums(group, parentTopic, getMessageResult.getMessageCount()); + brokerController.getBrokerStatsManager().incGroupGetSize(group, parentTopic, getMessageResult.getBufferTotalSize()); + response.setCode(ResponseCode.SUCCESS); + response.setRemark(GetMessageStatus.FOUND.name()); + response.setBody(r); + } else { + response.setRemark(GetMessageStatus.NO_MESSAGE_IN_QUEUE.name()); + PollingResult pollingResult = popLiteLongPollingService.polling(ctx, request, requestHeader.getBornTime(), + requestHeader.getPollTime(), clientId, group); + if (PollingResult.POLLING_SUC.equals(pollingResult)) { + return null; + } else if (PollingResult.POLLING_FULL.equals(pollingResult)) { + response.setCode(ResponseCode.POLLING_FULL); + } else { + response.setCode(ResponseCode.POLLING_TIMEOUT); + } + } + + responseHeader.setPopTime(popTime); + responseHeader.setInvisibleTime(invisibleTime); + responseHeader.setReviveQid(KeyBuilder.POP_ORDER_REVIVE_QUEUE); + responseHeader.setOrderCountInfo(rst.getObject1().toString()); + // Since a single read operation potentially retrieving messages from multiple LMQs, + // we no longer utilize startOffset and msgOffset + NettyRemotingAbstract.writeResponse(channel, request, response, null, brokerController.getBrokerMetricsManager().getRemotingMetricsManager()); + return null; + } + + @VisibleForTesting + public RemotingCommand preCheck(ChannelHandlerContext ctx, + PopLiteMessageRequestHeader requestHeader, RemotingCommand response) { + if (requestHeader.isTimeoutTooMuch()) { + response.setCode(ResponseCode.POLLING_TIMEOUT); + response.setRemark(String.format("the broker[%s] pop message is timeout too much", + brokerController.getBrokerConfig().getBrokerIP1())); + return response; + } + + if (!PermName.isReadable(brokerController.getBrokerConfig().getBrokerPermission())) { + response.setCode(ResponseCode.NO_PERMISSION); + response.setRemark(String.format("the broker[%s] pop message is forbidden", + brokerController.getBrokerConfig().getBrokerIP1())); + return response; + } + + if (requestHeader.getMaxMsgNum() > 32) { + response.setCode(ResponseCode.INVALID_PARAMETER); + response.setRemark(String.format("the broker[%s] pop message's num is greater than 32", + brokerController.getBrokerConfig().getBrokerIP1())); + return response; + } + + TopicConfig topicConfig = brokerController.getTopicConfigManager().selectTopicConfig(requestHeader.getTopic()); + if (null == topicConfig) { + LOGGER.error("The parentTopic {} not exist, consumer: {} ", requestHeader.getTopic()); + 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))); + return response; + } + + if (!PermName.isReadable(topicConfig.getPerm())) { + response.setCode(ResponseCode.NO_PERMISSION); + response.setRemark(String.format("the topic [%s] peeking message is forbidden", requestHeader.getTopic())); + return response; + } + + if (!TopicMessageType.LITE.equals(topicConfig.getTopicMessageType())) { + response.setCode(ResponseCode.INVALID_PARAMETER); + response.setRemark(String.format("the topic [%s] message type not match", requestHeader.getTopic())); + return response; + } + + SubscriptionGroupConfig subscriptionGroupConfig = + brokerController.getSubscriptionGroupManager().findSubscriptionGroupConfig(requestHeader.getConsumerGroup()); + if (null == subscriptionGroupConfig) { + response.setCode(ResponseCode.SUBSCRIPTION_GROUP_NOT_EXIST); + response.setRemark(String.format("subscription group [%s] not exist, %s", + requestHeader.getConsumerGroup(), FAQUrl.suggestTodo(FAQUrl.SUBSCRIPTION_GROUP_NOT_EXIST))); + return response; + } + + if (!subscriptionGroupConfig.isConsumeEnable()) { + response.setCode(ResponseCode.NO_PERMISSION); + response.setRemark("subscription group no permission, " + requestHeader.getConsumerGroup()); + return response; + } + + if (!requestHeader.getTopic().equals(subscriptionGroupConfig.getLiteBindTopic())) { + response.setCode(ResponseCode.INVALID_PARAMETER); + response.setRemark("subscription bind topic not match, " + requestHeader.getConsumerGroup()); + return response; + } + + return null; + } + + private byte[] readGetMessageResult(GetMessageResult getMessageResult) { + final ByteBuffer byteBuffer = ByteBuffer.allocate(getMessageResult.getBufferTotalSize()); + try { + List messageBufferList = getMessageResult.getMessageBufferList(); + for (ByteBuffer bb : messageBufferList) { + byteBuffer.put(bb); + } + } finally { + getMessageResult.release(); + } + return byteBuffer.array(); + } + + public Pair popByClientId(String clientHost, String parentTopic, String group, + String clientId, long popTime, long invisibleTime, int maxNum, String attemptId) { + GetMessageResult getMessageResult = new GetMessageResult(); + StringBuilder orderCountInfoAll = new StringBuilder(); + AtomicLong total = new AtomicLong(0); + + Set processed = new HashSet<>(); // deduplication in one request + Iterator iterator = liteEventDispatcher.getEventIterator(clientId); + while (total.get() < maxNum && iterator.hasNext()) { + String lmqName = iterator.next(); // here event represents a lmq name + if (null == lmqName) { + break; + } + if (!processed.add(lmqName)) { + continue; // wait for next pop request or re-fetch in current process, here prefer the former approach + } + Pair pair = popLiteTopic(parentTopic, clientHost, group, lmqName, + maxNum - total.get(), popTime, invisibleTime, attemptId); + if (null == pair || pair.getObject2().getMessageCount() <= 0) { + continue; + } + GetMessageResult singleResult = pair.getObject2(); + total.addAndGet(singleResult.getMessageCount()); + for (SelectMappedBufferResult mappedBuffer : singleResult.getMessageMapedList()) { + getMessageResult.addMessage(mappedBuffer); + } + if (orderCountInfoAll.length() > 0) { + orderCountInfoAll.append(";"); + } + orderCountInfoAll.append(pair.getObject1()); + collectLiteConsumerLagMetrics(group, parentTopic, lmqName, singleResult, maxNum, total); + } + return new Pair<>(orderCountInfoAll, getMessageResult); + } + + @VisibleForTesting + public Pair popLiteTopic(String parentTopic, String clientHost, String group, + String lmqName, long maxNum, long popTime, long invisibleTime, String attemptId) { + if (!brokerController.getBrokerConfig().isEnableLiteEventMode() + && !brokerController.getLiteLifecycleManager().isLmqExist(lmqName)) { + return null; + } + String lockKey = KeyBuilder.buildPopLiteLockKey(group, lmqName); + if (!lockService.tryLock(lockKey)) { + return null; + } + try { + if (isFifoBlocked(attemptId, group, lmqName, invisibleTime)) { + return null; + } + final long consumeOffset = getPopOffset(group, lmqName); + GetMessageResult result = getMessage(clientHost, group, lmqName, consumeOffset, (int) maxNum); + return handleGetMessageResult(result, parentTopic, group, lmqName, popTime, invisibleTime, attemptId); + } catch (Throwable e) { + LOGGER.error("popLiteTopic error. {}, {}", group, lmqName, e); + } finally { + lockService.unlock(lockKey); + } + return null; + } + + public boolean isFifoBlocked(String attemptId, String group, String lmqName, long invisibleTime) { + return consumerOrderInfoManager.checkBlock(attemptId, lmqName, group, 0, invisibleTime); + } + + public long getPopOffset(String group, String lmqName) { + long offset = brokerController.getConsumerOffsetManager().queryOffset(group, lmqName, 0); + if (offset < 0L) { + try { + offset = brokerController.getPopMessageProcessor().getInitOffset(lmqName, group, 0, ConsumeInitMode.MAX, true); // reuse code, init as max + LOGGER.info("init offset, group:{}, topic:{}, offset:{}", group, lmqName, offset); + } catch (ConsumeQueueException e) { + throw new RuntimeException(e); + } + } + Long resetOffset = brokerController.getConsumerOffsetManager().queryThenEraseResetOffset(lmqName, group, 0); + if (resetOffset != null) { + consumerOrderInfoManager.clearBlock(lmqName, group, 0); + brokerController.getConsumerOffsetManager().commitOffset("ResetOffset", group, lmqName, 0, resetOffset); + LOGGER.info("find resetOffset, group:{}, topic:{}, resetOffset:{}", group, lmqName, resetOffset); + return resetOffset; + } + return offset; + } + + public Pair handleGetMessageResult(GetMessageResult result, String parentTopic, + String group, String lmqName, long popTime, long invisibleTime, String attemptId) { + if (null == result) { + return null; + } + + StringBuilder orderCountInfo = new StringBuilder(); + if (GetMessageStatus.FOUND.equals(result.getStatus()) && !result.getMessageQueueOffset().isEmpty()) { + consumerOrderInfoManager.update(attemptId, false, lmqName, group, 0, + popTime, invisibleTime, result.getMessageQueueOffset(), orderCountInfo, null); + recordPopLiteMetrics(result, parentTopic, group); + orderCountInfo = transformOrderCountInfo(orderCountInfo, result.getMessageCount()); + } + return new Pair<>(orderCountInfo, result); + } + + /** + * For order count information, we use a uniform format of one consume count per offset. + */ + @VisibleForTesting + public StringBuilder transformOrderCountInfo(StringBuilder orderCountInfo, int msgCount) { + if (null == orderCountInfo || orderCountInfo.length() <= 0) { + return new StringBuilder(String.join(";", Collections.nCopies(msgCount, "0"))); + } + String infoStr = orderCountInfo.toString(); + String[] infos = infoStr.split(";"); + if (infos.length > 1) { + // consume count of each offset + ";" + consume count of queueId + return new StringBuilder(infoStr.substring(0, infoStr.lastIndexOf(";"))); + } else { + // just consume count of queueId, like "0 0 N" + String[] split = orderCountInfo.toString().split(MessageConst.KEY_SEPARATOR); + if (split.length == 3) { + return new StringBuilder(String.join(";", Collections.nCopies(msgCount, split[2]))); + } else { + return new StringBuilder(String.join(";", Collections.nCopies(msgCount, "0"))); + } + } + } + + @VisibleForTesting + protected void recordPopLiteMetrics(GetMessageResult result, String parentTopic, String group) { + Attributes attributes = this.brokerController.getBrokerMetricsManager().newAttributesBuilder() + .put(LABEL_TOPIC, parentTopic) + .put(LABEL_CONSUMER_GROUP, group) + .put(LABEL_IS_SYSTEM, TopicValidator.isSystemTopic(parentTopic) || + MixAll.isSysConsumerGroup(group)) + .put(LABEL_IS_RETRY, false) + .build(); + this.brokerController.getBrokerMetricsManager().getMessagesOutTotal().add(result.getMessageCount(), attributes); + this.brokerController.getBrokerMetricsManager().getThroughputOutTotal().add(result.getBufferTotalSize(), attributes); + } + + private void collectLiteConsumerLagMetrics(String group, String topic, String liteTopic, + GetMessageResult getResult, long maxNum, AtomicLong total) { + if (!brokerController.getBrokerConfig().isLiteLagLatencyCollectEnable()) { + return; + } + + try { + final LiteConsumerLagCalculator lagCalculator = brokerController.getBrokerMetricsManager() + .getLiteConsumerLagCalculator(); + + if (total.get() < maxNum) { + // Batch not full, no consume lag + lagCalculator.removeLagInfo(group, topic, liteTopic); + return; + } + + // Batch full, check for potential consume lag + long storeTimestamp = brokerController.getMessageStore() + .getMessageStoreTimeStamp(liteTopic, 0, getResult.getNextBeginOffset()); + if (storeTimestamp > 0) { + lagCalculator.updateLagInfo(group, topic, liteTopic, storeTimestamp); + } else { + // no next msg, no consume lag + lagCalculator.removeLagInfo(group, topic, liteTopic); + } + } catch (Exception e) { + LOGGER.warn("Failed to collect lite consumer lag metrics for group={}, topic={}, liteTopic={}", + group, topic, liteTopic, e); + } + } + + // tiered store ensures reading lmq from local storage + public GetMessageResult getMessage(String clientHost, String group, String lmqName, long offset, int batchSize) { + GetMessageResult result = brokerController.getMessageStore().getMessage(group, lmqName, 0, offset, batchSize, null); + if (null == result) { + return null; + } + if (GetMessageStatus.OFFSET_TOO_SMALL.equals(result.getStatus()) + || GetMessageStatus.OFFSET_OVERFLOW_BADLY.equals(result.getStatus()) + || GetMessageStatus.OFFSET_FOUND_NULL.equals(result.getStatus()) + || GetMessageStatus.NO_MATCHED_MESSAGE.equals(result.getStatus()) + || GetMessageStatus.MESSAGE_WAS_REMOVING.equals(result.getStatus()) + || GetMessageStatus.NO_MATCHED_LOGIC_QUEUE.equals(result.getStatus())) { + + long correctOffset = result.getNextBeginOffset(); // >=0 + brokerController.getConsumerOffsetManager().commitOffset("CorrectOffset", group, lmqName, 0, correctOffset); + LOGGER.warn("correct offset, {}, {}, from {} to {}", group, lmqName, offset, correctOffset); + return brokerController.getMessageStore().getMessage(group, lmqName, 0, correctOffset, batchSize, null); + } + return result; + } + + public class PopLiteLockManager extends ServiceThread { + @Override + public String getServiceName() { + if (brokerController.getBrokerConfig().isInBrokerContainer()) { + return brokerController.getBrokerIdentity().getIdentifier() + PopLiteLockManager.class.getSimpleName(); + } + return PopLiteLockManager.class.getSimpleName(); + } + + @Override + public void run() { + while (!isStopped()) { + try { + waitForRunning(60000); + lockService.removeTimeout(); + } catch (Exception ignored) { + } + } + } + } + + public PopLiteLongPollingService getPopLiteLongPollingService() { + return popLiteLongPollingService; + } + + public PopConsumerLockService getLockService() { + return lockService; + } + + public ConsumerOrderInfoManager getConsumerOrderInfoManager() { + return consumerOrderInfoManager; + } + + public void startPopLiteLockManager() { + popLiteLockManager.start(); + } + + public void stopPopLiteLockManager() { + popLiteLockManager.shutdown(); + } +} 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 76be358c486..d8e026a16b0 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 @@ -20,6 +20,7 @@ import io.netty.channel.ChannelHandlerContext; import java.util.List; import java.util.Objects; +import org.apache.commons.lang3.StringUtils; import org.apache.rocketmq.broker.BrokerController; import org.apache.rocketmq.broker.client.ClientChannelInfo; import org.apache.rocketmq.broker.client.ConsumerGroupInfo; @@ -39,6 +40,7 @@ import org.apache.rocketmq.common.constant.PermName; import org.apache.rocketmq.common.filter.ExpressionType; import org.apache.rocketmq.common.help.FAQUrl; +import org.apache.rocketmq.common.lite.LiteUtil; import org.apache.rocketmq.common.sysflag.PullSysFlag; import org.apache.rocketmq.logging.org.slf4j.Logger; import org.apache.rocketmq.logging.org.slf4j.LoggerFactory; @@ -531,6 +533,7 @@ private RemotingCommand processRequest(final Channel channel, RemotingCommand re final boolean useResetOffsetFeature = brokerController.getBrokerConfig().isUseServerSideResetOffset(); String topic = requestHeader.getTopic(); + String liteTopic = requestHeader.getLiteTopic(); String group = requestHeader.getConsumerGroup(); int queueId = requestHeader.getQueueId(); Long resetOffset = brokerController.getConsumerOffsetManager().queryThenEraseResetOffset(topic, group, queueId); @@ -556,7 +559,11 @@ private RemotingCommand processRequest(final Channel channel, RemotingCommand re } else { SubscriptionData finalSubscriptionData = subscriptionData; RemotingCommand finalResponse = response; - messageStore.getMessageAsync(group, topic, queueId, requestHeader.getQueueOffset(), + String storeTopic = topic; + if (StringUtils.isNotBlank(liteTopic)) { + storeTopic = LiteUtil.toLmqName(topic, liteTopic); + } + messageStore.getMessageAsync(group, storeTopic, queueId, requestHeader.getQueueOffset(), requestHeader.getMaxMsgNums(), messageFilter) .thenApply(result -> { if (null == result) { 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 c8e7e4c1287..5f5671fb7a0 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 @@ -33,6 +33,7 @@ import org.apache.rocketmq.common.attribute.TopicMessageType; import org.apache.rocketmq.common.constant.PermName; import org.apache.rocketmq.common.help.FAQUrl; +import org.apache.rocketmq.common.lite.LiteUtil; import org.apache.rocketmq.common.message.MessageAccessor; import org.apache.rocketmq.common.message.MessageClientIDSetter; import org.apache.rocketmq.common.message.MessageConst; @@ -280,6 +281,13 @@ public RemotingCommand sendMessage(final ChannelHandlerContext ctx, oriProps.put(MessageConst.PROPERTY_UNIQ_CLIENT_MESSAGE_ID_KEYIDX, uniqKey); } + // liteTopic multi dispatch + String liteTopic = oriProps.get(MessageConst.PROPERTY_LITE_TOPIC); + if (StringUtils.isNotEmpty(liteTopic)) { + String lmqName = LiteUtil.toLmqName(requestHeader.getTopic(), liteTopic); + oriProps.put(MessageConst.PROPERTY_INNER_MULTI_DISPATCH, lmqName); + } + MessageAccessor.setProperties(msgInner, oriProps); // check properties to ensure exclusive, don't check topic meta config to keep the behavior consistent int msgPriority = msgInner.getPriority(); @@ -445,8 +453,8 @@ private RemotingCommand handlePutMessageResult(PutMessageResult putMessageResult response.setRemark("[PC_SYNCHRONIZED]broker busy, start flow control for a while"); break; case LMQ_CONSUME_QUEUE_NUM_EXCEEDED: - response.setCode(ResponseCode.SYSTEM_ERROR); - response.setRemark("[LMQ_CONSUME_QUEUE_NUM_EXCEEDED]broker config enableLmq and enableMultiDispatch, lmq consumeQueue num exceed maxLmqConsumeQueueNum config num, default limit 2w."); + response.setCode(ResponseCode.LMQ_QUOTA_EXCEEDED); + response.setRemark("[LMQ_CONSUME_QUEUE_NUM_EXCEEDED]lmq consume queue num exceeded."); break; case UNKNOWN_ERROR: response.setCode(ResponseCode.SYSTEM_ERROR); diff --git a/broker/src/main/java/org/apache/rocketmq/broker/util/HookUtils.java b/broker/src/main/java/org/apache/rocketmq/broker/util/HookUtils.java index dec42351d9f..94be46ea405 100644 --- a/broker/src/main/java/org/apache/rocketmq/broker/util/HookUtils.java +++ b/broker/src/main/java/org/apache/rocketmq/broker/util/HookUtils.java @@ -38,6 +38,7 @@ import org.apache.rocketmq.store.PutMessageResult; import org.apache.rocketmq.store.PutMessageStatus; import org.apache.rocketmq.store.config.BrokerRole; +import org.apache.rocketmq.store.queue.ConsumeQueueStoreInterface; import org.apache.rocketmq.store.timer.TimerMessageStore; public class HookUtils { @@ -151,6 +152,30 @@ public static PutMessageResult handleScheduleMessage(BrokerController brokerCont return null; } + public static PutMessageResult handleLmqQuota(BrokerController brokerController, final MessageExtBrokerInner msg) { + if (!brokerController.getMessageStoreConfig().isEnableLmqQuota() + || !brokerController.getMessageStoreConfig().isEnableLmq() + || !brokerController.getMessageStoreConfig().isEnableMultiDispatch() + || !msg.needDispatchLMQ()) { + return null; + } + + ConsumeQueueStoreInterface cqStore = brokerController.getMessageStore().getQueueStore(); + String[] queueNames = + msg.getProperty(MessageConst.PROPERTY_INNER_MULTI_DISPATCH).split(MixAll.LMQ_DISPATCH_SEPARATOR); + for (String queueName : queueNames) { + if (!MixAll.isLmq(queueName)) { + continue; + } + if (cqStore.getLmqNum() >= brokerController.getMessageStoreConfig().getMaxLmqConsumeQueueNum()) { + if (!cqStore.isLmqExist(queueName)) { + return new PutMessageResult(PutMessageStatus.LMQ_CONSUME_QUEUE_NUM_EXCEEDED, null); + } + } + } + return null; + } + private static boolean isRolledTimerMessage(MessageExtBrokerInner msg) { return TimerMessageStore.TIMER_TOPIC.equals(msg.getTopic()); } diff --git a/broker/src/main/resources/rmq.broker.logback.xml b/broker/src/main/resources/rmq.broker.logback.xml index fd63ef174da..837112837b7 100644 --- a/broker/src/main/resources/rmq.broker.logback.xml +++ b/broker/src/main/resources/rmq.broker.logback.xml @@ -531,6 +531,37 @@ + + + brokerContainerLogDir + ${file.separator} + + + + ${user.home}${file.separator}logs${file.separator}rocketmqlogs${file.separator}lite.log + true + + ${user.home}${file.separator}logs${file.separator}rocketmqlogs${file.separator}otherdays${file.separator}lite.%i.log + + 1 + 10 + + + 128MB + + + %d{yyy-MM-dd HH:mm:ss,GMT+8} %p %t - %m%n + UTF-8 + + + + + + + + brokerContainerLogDir @@ -696,6 +727,10 @@ + + + + diff --git a/broker/src/test/java/org/apache/rocketmq/broker/lite/AbstractLiteLifecycleManagerTest.java b/broker/src/test/java/org/apache/rocketmq/broker/lite/AbstractLiteLifecycleManagerTest.java new file mode 100644 index 00000000000..d5742ea3eea --- /dev/null +++ b/broker/src/test/java/org/apache/rocketmq/broker/lite/AbstractLiteLifecycleManagerTest.java @@ -0,0 +1,282 @@ +/* + * 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.lite; + +import java.util.Collections; +import java.util.List; +import java.util.concurrent.ConcurrentHashMap; +import java.util.concurrent.ConcurrentMap; + +import org.apache.rocketmq.broker.BrokerController; +import org.apache.rocketmq.broker.config.v1.RocksDBConsumerOffsetManager; +import org.apache.rocketmq.broker.pop.orderly.ConsumerOrderInfoManager; +import org.apache.rocketmq.broker.processor.PopLiteMessageProcessor; +import org.apache.rocketmq.broker.subscription.SubscriptionGroupManager; +import org.apache.rocketmq.broker.topic.TopicConfigManager; +import org.apache.rocketmq.common.BrokerConfig; +import org.apache.rocketmq.common.Pair; +import org.apache.rocketmq.common.TopicAttributes; +import org.apache.rocketmq.common.TopicConfig; +import org.apache.rocketmq.common.attribute.TopicMessageType; +import org.apache.rocketmq.common.lite.LiteUtil; +import org.apache.rocketmq.remoting.protocol.subscription.SubscriptionGroupConfig; +import org.apache.rocketmq.store.MessageStore; +import org.junit.After; +import org.junit.Assert; +import org.junit.Before; +import org.junit.Test; +import org.junit.runner.RunWith; +import org.mockito.Mock; +import org.mockito.Mockito; +import org.mockito.junit.MockitoJUnitRunner; + +import static org.apache.rocketmq.broker.offset.ConsumerOffsetManager.TOPIC_GROUP_SEPARATOR; +import static org.mockito.ArgumentMatchers.anyLong; +import static org.mockito.ArgumentMatchers.anyString; +import static org.mockito.Mockito.when; +import static org.mockito.Mockito.anyInt; +import static org.mockito.Mockito.verify; +import static org.mockito.Mockito.times; +import static org.mockito.Mockito.never; +import static org.mockito.Mockito.atLeastOnce; + +@RunWith(MockitoJUnitRunner.class) +public class AbstractLiteLifecycleManagerTest { + private static final String PARENT_TOPIC = "parentTopic"; + private static final String EXIST_LMQ_NAME = LiteUtil.toLmqName(PARENT_TOPIC, "HW"); + private static final String GROUP = "group"; + + @Mock + private BrokerController brokerController; + @Mock + private LiteSharding liteSharding; + @Mock + private MessageStore messageStore; + @Mock + private TopicConfigManager topicConfigManager; + @Mock + private SubscriptionGroupManager subscriptionGroupManager; + @Mock + private RocksDBConsumerOffsetManager consumerOffsetManager; + @Mock + private PopLiteMessageProcessor popLiteMessageProcessor; + @Mock + private ConsumerOrderInfoManager consumerOrderInfoManager; + @Mock + private LiteSubscriptionRegistry liteSubscriptionRegistry; + + private TestLiteLifecycleManager lifecycleManager; + private BrokerConfig brokerConfig; + + private final TopicConfig topicConfig = new TopicConfig(PARENT_TOPIC, 1, 1); + private final SubscriptionGroupConfig groupConfig = new SubscriptionGroupConfig(); + private final ConcurrentMap> offsetTable = new ConcurrentHashMap<>(); + + @Before + public void setUp() { + brokerConfig = new BrokerConfig(); + when(brokerController.getBrokerConfig()).thenReturn(brokerConfig); + when(brokerController.getMessageStore()).thenReturn(messageStore); + when(brokerController.getTopicConfigManager()).thenReturn(topicConfigManager); + when(brokerController.getSubscriptionGroupManager()).thenReturn(subscriptionGroupManager); + when(brokerController.getConsumerOffsetManager()).thenReturn(consumerOffsetManager); + when(brokerController.getPopLiteMessageProcessor()).thenReturn(popLiteMessageProcessor); + when(popLiteMessageProcessor.getConsumerOrderInfoManager()).thenReturn(consumerOrderInfoManager); + when(brokerController.getLiteSubscriptionRegistry()).thenReturn(liteSubscriptionRegistry); + + topicConfig.getAttributes().put( + TopicAttributes.TOPIC_MESSAGE_TYPE_ATTRIBUTE.getName(), TopicMessageType.LITE.getValue()); + ConcurrentMap topicConfigTable = new ConcurrentHashMap<>(); + topicConfigTable.put(PARENT_TOPIC, topicConfig); + when(topicConfigManager.getTopicConfigTable()).thenReturn(topicConfigTable); + when(topicConfigManager.selectTopicConfig(PARENT_TOPIC)).thenReturn(topicConfig); + + groupConfig.setGroupName(GROUP); + groupConfig.setLiteBindTopic(PARENT_TOPIC); + ConcurrentMap groupTable = new ConcurrentHashMap<>(); + groupTable.put(GROUP, groupConfig); + when(subscriptionGroupManager.getSubscriptionGroupTable()).thenReturn(groupTable); + + when(consumerOffsetManager.getOffsetTable()).thenReturn(offsetTable); + when(consumerOffsetManager.getPullOffsetTable()).thenReturn(offsetTable); + + TestLiteLifecycleManager testObject = new TestLiteLifecycleManager(brokerController, liteSharding); + lifecycleManager = Mockito.spy(testObject); + lifecycleManager.init(); + } + + @After + public void reset() { + topicConfig.getAttributes().clear(); + groupConfig.getAttributes().clear(); + offsetTable.clear(); + } + + @Test + public void testIsSubscriptionActive() { + when(liteSharding.shardingByLmqName(PARENT_TOPIC, EXIST_LMQ_NAME)).thenReturn(brokerConfig.getBrokerName()); + Assert.assertTrue(lifecycleManager.isSubscriptionActive(PARENT_TOPIC, EXIST_LMQ_NAME)); + Assert.assertFalse(lifecycleManager.isSubscriptionActive("whatever", "whatever")); + + when(liteSharding.shardingByLmqName(anyString(), anyString())).thenReturn(brokerConfig.getBrokerName()); + Assert.assertTrue(lifecycleManager.isSubscriptionActive(PARENT_TOPIC, EXIST_LMQ_NAME)); + Assert.assertTrue(lifecycleManager.isSubscriptionActive("whatever", "whatever")); + + when(liteSharding.shardingByLmqName(anyString(), anyString())).thenReturn("otherBrokerName"); + Assert.assertTrue(lifecycleManager.isSubscriptionActive(PARENT_TOPIC, EXIST_LMQ_NAME)); + Assert.assertFalse(lifecycleManager.isSubscriptionActive("whatever", "whatever")); + } + + @Test + public void testIsLmqExist() { + Assert.assertTrue(lifecycleManager.isLmqExist(EXIST_LMQ_NAME)); + Assert.assertFalse(lifecycleManager.isLmqExist("whatever")); + } + + @Test + public void testGetLiteTopicCount() { + Assert.assertEquals(1, lifecycleManager.getLiteTopicCount(PARENT_TOPIC)); + verify(lifecycleManager).collectByParentTopic(PARENT_TOPIC); + + Assert.assertEquals(0, lifecycleManager.getLiteTopicCount("whatever")); + verify(lifecycleManager, never()).collectByParentTopic("whatever"); + } + + @Test + public void testIsLiteTopicExpired() { + // not lite topic queue + Assert.assertFalse(lifecycleManager.isLiteTopicExpired(PARENT_TOPIC, "whatever", 10L)); + + // maxOffset invalid + Assert.assertFalse(lifecycleManager.isLiteTopicExpired(PARENT_TOPIC, EXIST_LMQ_NAME, 0L)); + + // less than minLiteTTl + long mockStoreTime = System.currentTimeMillis(); + when(messageStore.getMessageStoreTimeStamp(anyString(), anyInt(), anyLong())).thenReturn(mockStoreTime); + Assert.assertFalse(lifecycleManager.isLiteTopicExpired(PARENT_TOPIC, EXIST_LMQ_NAME, 100L)); + + // topic ttl not found + mockStoreTime = System.currentTimeMillis() - brokerConfig.getMinLiteTTl() - 2000; + when(messageStore.getMessageStoreTimeStamp(anyString(), anyInt(), anyLong())).thenReturn(mockStoreTime); + Assert.assertFalse(lifecycleManager.isLiteTopicExpired(PARENT_TOPIC, EXIST_LMQ_NAME, 100L)); + + // topic ttl no expiration + topicConfig.getAttributes().put(TopicAttributes.LITE_EXPIRATION_ATTRIBUTE.getName(), "-1"); + lifecycleManager.updateMetadata(); + mockStoreTime = System.currentTimeMillis() - brokerConfig.getMinLiteTTl() - 2000; + when(messageStore.getMessageStoreTimeStamp(anyString(), anyInt(), anyLong())).thenReturn(mockStoreTime); + Assert.assertFalse(lifecycleManager.isLiteTopicExpired(PARENT_TOPIC, EXIST_LMQ_NAME, 100L)); + + // topic ttl expired + topicConfig.getAttributes().put( + TopicAttributes.LITE_EXPIRATION_ATTRIBUTE.getName(), "" + brokerConfig.getMinLiteTTl() / 1000 / 60); + lifecycleManager.updateMetadata(); + mockStoreTime = System.currentTimeMillis() - brokerConfig.getMinLiteTTl() - 2000; + when(messageStore.getMessageStoreTimeStamp(anyString(), anyInt(), anyLong())).thenReturn(mockStoreTime); + Assert.assertTrue(lifecycleManager.isLiteTopicExpired(PARENT_TOPIC, EXIST_LMQ_NAME, 100L)); + } + + @Test + public void testDeleteLmq() { + lifecycleManager.updateMetadata(); + String otherKey = "otherTopic@otherGroup"; + String removeKey = EXIST_LMQ_NAME + TOPIC_GROUP_SEPARATOR + GROUP; + offsetTable.put(otherKey, new ConcurrentHashMap<>()); + offsetTable.put(removeKey, new ConcurrentHashMap<>()); + + // sharding to this broker + when(liteSharding.shardingByLmqName(PARENT_TOPIC, EXIST_LMQ_NAME)).thenReturn(brokerConfig.getBrokerName()); + lifecycleManager.deleteLmq(PARENT_TOPIC, EXIST_LMQ_NAME); + + Assert.assertTrue(offsetTable.containsKey(otherKey)); + Assert.assertFalse(offsetTable.containsKey(removeKey)); + verify(consumerOffsetManager).removeConsumerOffset(removeKey); + verify(messageStore).deleteTopics(Collections.singleton(EXIST_LMQ_NAME)); + verify(liteSubscriptionRegistry).cleanSubscription(EXIST_LMQ_NAME, false); + verify(consumerOrderInfoManager, times(1)).remove(EXIST_LMQ_NAME, GROUP); + + // not sharding to this broker + when(liteSharding.shardingByLmqName(PARENT_TOPIC, EXIST_LMQ_NAME)).thenReturn("otherBrokerName"); + lifecycleManager.deleteLmq(PARENT_TOPIC, EXIST_LMQ_NAME); + + Assert.assertTrue(offsetTable.containsKey(otherKey)); + Assert.assertFalse(offsetTable.containsKey(removeKey)); + verify(consumerOffsetManager, times(2)).removeConsumerOffset(removeKey); + verify(messageStore, times(2)).deleteTopics(Collections.singleton(EXIST_LMQ_NAME)); + verify(liteSubscriptionRegistry, times(2)).cleanSubscription(EXIST_LMQ_NAME, false); + } + + @Test + public void testCleanExpiredLiteTopic() { + String removeKey = EXIST_LMQ_NAME + TOPIC_GROUP_SEPARATOR + GROUP; + when(liteSharding.shardingByLmqName(PARENT_TOPIC, EXIST_LMQ_NAME)).thenReturn(brokerConfig.getBrokerName()); + + lifecycleManager.cleanExpiredLiteTopic(); + verify(consumerOffsetManager).removeConsumerOffset(removeKey); + verify(messageStore).deleteTopics(Collections.singleton(EXIST_LMQ_NAME)); + verify(liteSubscriptionRegistry).cleanSubscription(EXIST_LMQ_NAME, false); + } + + @Test + public void testCleanByParentTopic() { + String removeKey = EXIST_LMQ_NAME + TOPIC_GROUP_SEPARATOR + GROUP; + when(liteSharding.shardingByLmqName(PARENT_TOPIC, EXIST_LMQ_NAME)).thenReturn(brokerConfig.getBrokerName()); + + lifecycleManager.cleanByParentTopic(PARENT_TOPIC); + verify(consumerOffsetManager).removeConsumerOffset(removeKey); + verify(messageStore).deleteTopics(Collections.singleton(EXIST_LMQ_NAME)); + verify(liteSubscriptionRegistry).cleanSubscription(EXIST_LMQ_NAME, false); + + lifecycleManager.cleanByParentTopic("whatever"); + verify(lifecycleManager, never()).collectByParentTopic("whatever"); + } + + @Test + public void testRun() throws InterruptedException { + brokerConfig.setLiteTtlCheckInterval(100L); + when(liteSharding.shardingByLmqName(PARENT_TOPIC, EXIST_LMQ_NAME)).thenReturn(brokerConfig.getBrokerName()); + lifecycleManager.start(); + Thread.sleep(300); + lifecycleManager.shutdown(); + + verify(consumerOffsetManager, atLeastOnce()).removeConsumerOffset(anyString()); + verify(messageStore, atLeastOnce()).deleteTopics(Collections.singleton(EXIST_LMQ_NAME)); + verify(liteSubscriptionRegistry, atLeastOnce()).cleanSubscription(EXIST_LMQ_NAME, false); + } + + private static class TestLiteLifecycleManager extends AbstractLiteLifecycleManager { + public TestLiteLifecycleManager(BrokerController brokerController, LiteSharding liteSharding) { + super(brokerController, liteSharding); + } + + @Override + public long getMaxOffsetInQueue(String lmqName) { + return EXIST_LMQ_NAME.equals(lmqName) ? 100 : -1; + } + + @Override + public List> collectExpiredLiteTopic() { + return Collections.singletonList(new Pair<>(PARENT_TOPIC, EXIST_LMQ_NAME)); + } + + @Override + public List collectByParentTopic(String parentTopic) { + return PARENT_TOPIC.equals(parentTopic) ? Collections.singletonList(EXIST_LMQ_NAME) : Collections.emptyList(); + } + } +} diff --git a/broker/src/test/java/org/apache/rocketmq/broker/lite/LiteEventDispatcherTest.java b/broker/src/test/java/org/apache/rocketmq/broker/lite/LiteEventDispatcherTest.java new file mode 100644 index 00000000000..1360ec56764 --- /dev/null +++ b/broker/src/test/java/org/apache/rocketmq/broker/lite/LiteEventDispatcherTest.java @@ -0,0 +1,581 @@ +/* + * 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.lite; + +import com.google.common.cache.Cache; +import org.apache.commons.lang3.reflect.FieldUtils; +import org.apache.rocketmq.broker.BrokerController; +import org.apache.rocketmq.broker.longpolling.PopLiteLongPollingService; +import org.apache.rocketmq.broker.offset.ConsumerOffsetManager; +import org.apache.rocketmq.broker.pop.orderly.ConsumerOrderInfoManager; +import org.apache.rocketmq.broker.processor.PopLiteMessageProcessor; +import org.apache.rocketmq.broker.subscription.SubscriptionGroupManager; +import org.apache.rocketmq.common.BrokerConfig; +import org.apache.rocketmq.common.entity.ClientGroup; +import org.apache.rocketmq.common.lite.LiteSubscription; +import org.apache.rocketmq.common.lite.LiteUtil; +import org.junit.After; +import org.junit.Assert; +import org.junit.Before; +import org.junit.Test; +import org.junit.runner.RunWith; +import org.mockito.Mock; +import org.mockito.Mockito; +import org.mockito.junit.MockitoJUnitRunner; + +import java.util.Arrays; +import java.util.Collections; +import java.util.HashSet; +import java.util.Iterator; +import java.util.List; +import java.util.Map; +import java.util.Set; +import java.util.UUID; +import java.util.concurrent.ConcurrentMap; +import java.util.concurrent.ConcurrentSkipListSet; + +import static org.apache.rocketmq.broker.lite.LiteEventDispatcher.COMPARATOR; +import static org.mockito.ArgumentMatchers.anyString; +import static org.mockito.Mockito.never; +import static org.mockito.Mockito.times; +import static org.mockito.Mockito.verify; +import static org.mockito.Mockito.when; + + +@RunWith(MockitoJUnitRunner.class) +public class LiteEventDispatcherTest { + + @Mock + private BrokerController brokerController; + @Mock + private LiteSubscriptionRegistry liteSubscriptionRegistry; + @Mock + private AbstractLiteLifecycleManager liteLifecycleManager; + @Mock + private ConsumerOffsetManager consumerOffsetManager; + @Mock + private PopLiteMessageProcessor popLiteMessageProcessor; + @Mock + private PopLiteLongPollingService popLiteLongPollingService; + @Mock + private ConsumerOrderInfoManager consumerOrderInfoManager; + @Mock + private SubscriptionGroupManager subscriptionGroupManager; + + private BrokerConfig brokerConfig; + private LiteEventDispatcher liteEventDispatcher; + private ConcurrentMap clientEventMap; + private Cache blacklist; + + @SuppressWarnings("unchecked") + @Before + public void setUp() throws IllegalAccessException { + brokerConfig = new BrokerConfig(); + when(brokerController.getBrokerConfig()).thenReturn(brokerConfig); + when(brokerController.getConsumerOffsetManager()).thenReturn(consumerOffsetManager); + when(brokerController.getPopLiteMessageProcessor()).thenReturn(popLiteMessageProcessor); + when(brokerController.getSubscriptionGroupManager()).thenReturn(subscriptionGroupManager); + when(popLiteMessageProcessor.getPopLiteLongPollingService()).thenReturn(popLiteLongPollingService); + when(popLiteMessageProcessor.getConsumerOrderInfoManager()).thenReturn(consumerOrderInfoManager); + + LiteEventDispatcher testObject = new LiteEventDispatcher(brokerController, liteSubscriptionRegistry, liteLifecycleManager); + liteEventDispatcher = Mockito.spy(testObject); + liteEventDispatcher.init(); + + clientEventMap = (ConcurrentMap) + FieldUtils.readDeclaredField(testObject, "clientEventMap", true); + blacklist = (Cache) FieldUtils.readDeclaredField(testObject, "blacklist", true); + } + + @After + public void reset() { + brokerConfig = new BrokerConfig(); + clientEventMap.clear(); + blacklist.invalidateAll(); + } + + @Test + public void testFullDispatchRequestComparator() { + LiteEventDispatcher.FullDispatchRequest request1 = + new LiteEventDispatcher.FullDispatchRequest("client1", "whatever", 1000); + LiteEventDispatcher.FullDispatchRequest request2 = + new LiteEventDispatcher.FullDispatchRequest("client2", "whatever", 2000); + LiteEventDispatcher.FullDispatchRequest request3 = + new LiteEventDispatcher.FullDispatchRequest("client1", "whatever", 1000); + + Assert.assertTrue(COMPARATOR.compare(request1, request2) < 0); + Assert.assertTrue(COMPARATOR.compare(request2, request1) > 0); + Assert.assertEquals(0, COMPARATOR.compare(request1, request3)); + } + + @Test + public void testFullDispatchSet() { + ConcurrentSkipListSet set = + new ConcurrentSkipListSet<>(COMPARATOR); + + LiteEventDispatcher.FullDispatchRequest request1 = + new LiteEventDispatcher.FullDispatchRequest("client1", "whatever", 1000); + LiteEventDispatcher.FullDispatchRequest request2 = + new LiteEventDispatcher.FullDispatchRequest("client2", "whatever", 2000); + LiteEventDispatcher.FullDispatchRequest request3 = + new LiteEventDispatcher.FullDispatchRequest("client1", "whatever", 1000); + LiteEventDispatcher.FullDispatchRequest request4 = + new LiteEventDispatcher.FullDispatchRequest("client3", "whatever", 500); + LiteEventDispatcher.FullDispatchRequest request5 = + new LiteEventDispatcher.FullDispatchRequest("client4", "whatever", 1000); + LiteEventDispatcher.FullDispatchRequest request6 = + new LiteEventDispatcher.FullDispatchRequest(null, "whatever", 1000); + + set.add(request1); + set.add(request3); + set.add(request6); + Assert.assertEquals(1, set.size()); + Assert.assertEquals(request1, set.pollFirst()); + + set.clear(); + set.add(request1); + set.add(request2); + set.add(request3); + set.add(request4); + set.add(request5); + Assert.assertEquals(4, set.size()); + Assert.assertEquals(request4, set.pollFirst()); + Assert.assertEquals(request1, set.pollFirst()); + Assert.assertEquals(request5, set.pollFirst()); + Assert.assertEquals(request2, set.pollFirst()); + } + + @Test + public void testEventSetIterator() { + LiteEventDispatcher.ClientEventSet clientEventSet = liteEventDispatcher.new ClientEventSet("group"); + clientEventSet.offer("event1"); + clientEventSet.offer("event2"); + + LiteEventDispatcher.EventSetIterator iterator = new LiteEventDispatcher.EventSetIterator(clientEventSet); + + Assert.assertTrue(iterator.hasNext()); + Assert.assertEquals("event1", iterator.next()); + Assert.assertTrue(iterator.hasNext()); + Assert.assertEquals("event2", iterator.next()); + Assert.assertFalse(iterator.hasNext()); + } + + @Test + public void testLiteSubscriptionIterator() { + Iterator topicIterator = Arrays.asList("event1", "event2").iterator(); + + LiteEventDispatcher.LiteSubscriptionIterator iterator = + new LiteEventDispatcher.LiteSubscriptionIterator("parentTopic", topicIterator); + + Assert.assertTrue(iterator.hasNext()); + Assert.assertEquals("event1", iterator.next()); + Assert.assertTrue(iterator.hasNext()); + Assert.assertEquals("event2", iterator.next()); + Assert.assertFalse(iterator.hasNext()); + } + + @Test + public void testClientEventSet_offerAndPoll() { + brokerConfig.setMaxClientEventCount(3); + LiteEventDispatcher.ClientEventSet clientEventSet = liteEventDispatcher.new ClientEventSet("group"); + + Assert.assertTrue(clientEventSet.offer("event1")); + Assert.assertTrue(clientEventSet.offer("event2")); + Assert.assertTrue(clientEventSet.offer("event1")); + Assert.assertTrue(clientEventSet.offer("event3")); + Assert.assertFalse(clientEventSet.offer("event4")); + + Assert.assertEquals(3, clientEventSet.size()); + Assert.assertEquals("event1", clientEventSet.poll()); + Assert.assertEquals("event2", clientEventSet.poll()); + Assert.assertEquals("event3", clientEventSet.poll()); + Assert.assertEquals(0, clientEventSet.size()); + Assert.assertNull(clientEventSet.poll()); + } + + @Test + public void testClientEventSet_isLowWaterMark() { + brokerConfig.setMaxClientEventCount(10); + LiteEventDispatcher.ClientEventSet clientEventSet = liteEventDispatcher.new ClientEventSet("group"); + Assert.assertTrue(clientEventSet.isLowWaterMark()); + + for (int i = 0; i < 4; i++) { + clientEventSet.offer("event" + i); + } + Assert.assertFalse(clientEventSet.isLowWaterMark()); + } + + @Test + public void testClientEventSetMaybeBlock() throws Exception { + LiteEventDispatcher.ClientEventSet clientEventSet = liteEventDispatcher.new ClientEventSet("group"); + Assert.assertFalse(clientEventSet.maybeBlock()); + + clientEventSet.offer("event"); + FieldUtils.writeDeclaredField(clientEventSet, "lastAccessTime", 0L, true); + Assert.assertTrue(clientEventSet.maybeBlock()); + clientEventSet.poll(); + Assert.assertFalse(clientEventSet.maybeBlock()); + } + + @Test + public void testGetAllSubscriber_noSubscribers() { + when(liteSubscriptionRegistry.getSubscriber("event")).thenReturn(null); + Object result = liteEventDispatcher.getAllSubscriber("group", "event"); + Assert.assertNull(result); + } + + @Test + @SuppressWarnings("unchecked") + public void testGetAllSubscriber_singleSubscriber() { + Set subscribers = new HashSet<>(); + subscribers.add(new ClientGroup("clientId", "group")); + when(liteSubscriptionRegistry.getSubscriber("event")).thenReturn(subscribers); + + Object result = liteEventDispatcher.getAllSubscriber("group", "event"); // specified + Assert.assertTrue(result instanceof List); + Assert.assertEquals(1, ((List) result).size()); + Assert.assertEquals("clientId", ((List) result).get(0).clientId); + + result = liteEventDispatcher.getAllSubscriber(null, "event"); // not specified + Assert.assertTrue(result instanceof List); + Assert.assertEquals(1, ((List) result).size()); + Assert.assertEquals("clientId", ((List) result).get(0).clientId); + + result = liteEventDispatcher.getAllSubscriber("otherGroup", "event"); // specified but not match + Assert.assertNull(result); + } + + @Test + @SuppressWarnings("unchecked") + public void testGetAllSubscriber_multipleSubscribers() { + Set subscribers = new HashSet<>(); + subscribers.add(new ClientGroup("clientId1", "group1")); + subscribers.add(new ClientGroup("clientId2", "group1")); + subscribers.add(new ClientGroup("clientId3", "group2")); + when(liteSubscriptionRegistry.getSubscriber("event")).thenReturn(subscribers); + + Object result = liteEventDispatcher.getAllSubscriber("group1", "event"); // specified + Assert.assertTrue(result instanceof List); + Assert.assertEquals(2, ((List) result).size()); + Assert.assertEquals("clientId1", ((List) result).get(0).clientId); + + result = liteEventDispatcher.getAllSubscriber("group2", "event"); // specified + Assert.assertTrue(result instanceof List); + Assert.assertEquals(1, ((List) result).size()); + Assert.assertEquals("clientId3", ((List) result).get(0).clientId); + + result = liteEventDispatcher.getAllSubscriber("otherGroup", "event"); // specified but not match + Assert.assertNull(result); + + result = liteEventDispatcher.getAllSubscriber(null, "event"); // not specified + Assert.assertTrue(result instanceof Map); + Assert.assertEquals(2, ((Map) result).size()); + Assert.assertEquals(2, ((Map>) result).get("group1").size()); + Assert.assertEquals(1, ((Map>) result).get("group2").size()); + } + + @Test + public void testTryDispatchToClient() { + brokerConfig.setMaxClientEventCount(1); + String clientId = "clientId"; + + boolean result = liteEventDispatcher.tryDispatchToClient("event1", clientId, "group"); + Assert.assertTrue(result); + + // not in blacklist + result = liteEventDispatcher.tryDispatchToClient("event2", clientId, "group"); + Assert.assertFalse(result); + verify(liteEventDispatcher).scheduleFullDispatch(clientId, "group", false); + + // in blacklist + blacklist.put(clientId, Boolean.TRUE); + result = liteEventDispatcher.tryDispatchToClient("event3", clientId, "group"); + Assert.assertFalse(result); + verify(liteEventDispatcher).scheduleFullDispatch(clientId, "group", true); + + blacklist.invalidate(clientId); + result = liteEventDispatcher.tryDispatchToClient("event3", clientId, "group"); + Assert.assertFalse(result); + verify(liteEventDispatcher, times(2)).scheduleFullDispatch(clientId, "group", false); + } + + @Test + public void testSelectAndDispatch_empty_or_singleClient() { + List clients = Collections.singletonList(new ClientGroup("client", "group")); + // disable event mode + brokerConfig.setEnableLiteEventMode(false); + liteEventDispatcher.selectAndDispatch("event", clients, null); + verify(liteEventDispatcher, never()).tryDispatchToClient(anyString(), anyString(), anyString()); + + // empty list + liteEventDispatcher.selectAndDispatch("event", Collections.emptyList(), null); + verify(liteEventDispatcher, never()).tryDispatchToClient(anyString(), anyString(), anyString()); + + // event mode + brokerConfig.setMaxClientEventCount(2); + brokerConfig.setEnableLiteEventMode(true); + + liteEventDispatcher.selectAndDispatch("event1", clients, null); + liteEventDispatcher.selectAndDispatch("event2", clients, "client"); // exclude + liteEventDispatcher.selectAndDispatch("event3", clients, null); + verify(popLiteLongPollingService, times(2)).notifyMessageArriving("client", true, 0, "group"); + } + + @Test + public void testSelectAndDispatch_multipleClients() { + brokerConfig.setMaxClientEventCount(2); + String client1 = UUID.randomUUID().toString(); + String client2 = UUID.randomUUID().toString(); + List clients = Arrays.asList( + new ClientGroup(client1, "group"), + new ClientGroup(client2, "group")); + + // no fallback + liteEventDispatcher.selectAndDispatch("event1", clients, client1); + verify(popLiteLongPollingService).notifyMessageArriving(client2, true, 0, "group"); + + // no fallback + liteEventDispatcher.selectAndDispatch("event2", clients, client2); + verify(popLiteLongPollingService).notifyMessageArriving(client1, true, 0, "group"); + + // fallback + blacklist.put(client1, Boolean.TRUE); + liteEventDispatcher.selectAndDispatch("event3", clients, null); + verify(popLiteLongPollingService, times(2)).notifyMessageArriving(client2, true, 0, "group"); + + // fallback + blacklist.invalidate(client1); + blacklist.put(client2, Boolean.TRUE); + liteEventDispatcher.selectAndDispatch("event4", clients, null); + verify(popLiteLongPollingService, times(2)).notifyMessageArriving(client1, true, 0, "group"); + + // queue all full + liteEventDispatcher.selectAndDispatch("event5", clients, null); + verify(popLiteLongPollingService, times(2)).notifyMessageArriving(client1, true, 0, "group"); + verify(popLiteLongPollingService, times(2)).notifyMessageArriving(client2, true, 0, "group"); + } + + @Test + public void testDispatch() { + // disable event mode + brokerConfig.setEnableLiteEventMode(false); + liteEventDispatcher.dispatch("group", "event", 0, 0, System.currentTimeMillis()); + verify(liteEventDispatcher, never()).getAllSubscriber(anyString(), anyString()); + + // event mode + brokerConfig.setEnableLiteEventMode(true); + liteEventDispatcher.dispatch("group", "event", 1, 0, System.currentTimeMillis()); // queue id not match + liteEventDispatcher.dispatch("group", "event", 0, 0, System.currentTimeMillis()); // queue name not match + verify(liteEventDispatcher, never()).getAllSubscriber(anyString(), anyString()); + + // do dispatch + liteEventDispatcher.dispatch("group", LiteUtil.toLmqName("p", "l"), 0, 0, System.currentTimeMillis()); + verify(liteEventDispatcher).getAllSubscriber(anyString(), anyString()); + } + + @Test + public void testDoFullDispatch_disable_or_emptySubscription() { + String clientId = "clientId"; + String group = "group"; + + // disable event mode + brokerConfig.setEnableLiteEventMode(false); + liteEventDispatcher.doFullDispatch(clientId, group); + verify(liteSubscriptionRegistry, never()).getLiteSubscription(clientId); + + // empty subscription + brokerConfig.setEnableLiteEventMode(true); + when(liteSubscriptionRegistry.getLiteSubscription("clientId")).thenReturn(null); + liteEventDispatcher.doFullDispatch(clientId, group); + verify(liteLifecycleManager, never()).getMaxOffsetInQueue(anyString()); + } + + @Test + public void testDoFullDispatch_maybeBlock() throws Exception { + int num = 10; + String clientId = "clientId"; + String group = "group"; + LiteSubscription subscription = new LiteSubscription(); + subscription.setTopic("parentTopic"); + for (int i = 0; i < num; i++) { + subscription.addLiteTopic(LiteUtil.toLmqName(subscription.getTopic(), "l" + i)); + } + when(liteSubscriptionRegistry.getLiteSubscription(clientId)).thenReturn(subscription); + + // maybe block + liteEventDispatcher.tryDispatchToClient("event", clientId, group); + Assert.assertNotNull(clientEventMap.get(clientId)); + FieldUtils.writeDeclaredField(clientEventMap.get(clientId), "lastAccessTime", 0L, true); + liteEventDispatcher.doFullDispatch(clientId, group); + verify(liteEventDispatcher).scheduleFullDispatch(clientId, group, true); + verify(liteLifecycleManager, never()).getMaxOffsetInQueue(anyString()); + } + + @Test + public void testDoFullDispatch_highWaterMark() throws Exception { + int num = 10; + String clientId = "clientId"; + String group = "group"; + LiteSubscription subscription = new LiteSubscription(); + subscription.setTopic("parentTopic"); + for (int i = 0; i < num; i++) { + subscription.addLiteTopic(LiteUtil.toLmqName(subscription.getTopic(), "l" + i)); + } + when(liteSubscriptionRegistry.getLiteSubscription(clientId)).thenReturn(subscription); + + brokerConfig.setMaxClientEventCount(1); + + // active consuming + liteEventDispatcher.tryDispatchToClient("event", clientId, group); + liteEventDispatcher.doFullDispatch(clientId, group); + + verify(liteEventDispatcher).scheduleFullDispatch(clientId, group, false); + verify(liteLifecycleManager, never()).getMaxOffsetInQueue(anyString()); + + // not active consuming + clientEventMap.clear(); + liteEventDispatcher.tryDispatchToClient("event", clientId, group); + FieldUtils.writeDeclaredField(clientEventMap.get(clientId), "lastAccessTime", System.currentTimeMillis() - 6000L, true); + liteEventDispatcher.doFullDispatch(clientId, group); + + verify(liteEventDispatcher).scheduleFullDispatch(clientId, group, true); + verify(liteLifecycleManager, never()).getMaxOffsetInQueue(anyString()); + } + + @Test + public void testDoFullDispatch_multipleTopics() { + String clientId = "clientId"; + String group = "group"; + + String lmqName1 = "lmqName1"; + String lmqName2 = "lmqName2"; + String lmqName3 = "lmqName2"; + LiteSubscription subscription = new LiteSubscription(); + subscription.setTopic("parentTopic"); + subscription.addLiteTopic(lmqName1); + subscription.addLiteTopic(lmqName2); + subscription.addLiteTopic(lmqName3); + when(liteSubscriptionRegistry.getLiteSubscription(clientId)).thenReturn(subscription); + + + when(liteLifecycleManager.getMaxOffsetInQueue(lmqName1)).thenReturn(0L); + + when(liteLifecycleManager.getMaxOffsetInQueue(lmqName2)).thenReturn(10L); + when(consumerOffsetManager.queryOffset(group, lmqName2, 0)).thenReturn(10L); + + when(liteLifecycleManager.getMaxOffsetInQueue(lmqName3)).thenReturn(10L); + when(consumerOffsetManager.queryOffset(group, lmqName3, 0)).thenReturn(5L); + + liteEventDispatcher.doFullDispatch(clientId, group); + + verify(liteLifecycleManager).getMaxOffsetInQueue(lmqName1); + verify(liteLifecycleManager).getMaxOffsetInQueue(lmqName2); + verify(liteLifecycleManager).getMaxOffsetInQueue(lmqName3); + verify(consumerOffsetManager, never()).queryOffset(group, lmqName1, 0); + verify(consumerOffsetManager).queryOffset(group, lmqName2, 0); + verify(consumerOffsetManager).queryOffset(group, lmqName3, 0); + + verify(liteEventDispatcher, never()).scheduleFullDispatch(clientId, group, true); + verify(popLiteLongPollingService, times(2)).notifyMessageArriving(clientId, true, 0, group); + } + + @Test + public void testDoFullDispatch_eventQueueFull() throws IllegalAccessException { + brokerConfig.setMaxClientEventCount(2); + String clientId = "clientId"; + String group = "group"; + + String lmqName1 = "lmqName1"; + String lmqName2 = "lmqName2"; + String lmqName3 = "lmqName3"; + LiteSubscription subscription = new LiteSubscription(); + subscription.setTopic("parentTopic"); + subscription.addLiteTopic(lmqName1); + subscription.addLiteTopic(lmqName2); + subscription.addLiteTopic(lmqName3); + when(liteSubscriptionRegistry.getLiteSubscription(clientId)).thenReturn(subscription); + + when(liteLifecycleManager.getMaxOffsetInQueue(lmqName1)).thenReturn(10L); + when(consumerOffsetManager.queryOffset(group, lmqName1, 0)).thenReturn(5L); + + when(liteLifecycleManager.getMaxOffsetInQueue(lmqName2)).thenReturn(10L); + when(consumerOffsetManager.queryOffset(group, lmqName2, 0)).thenReturn(5L); + + when(liteLifecycleManager.getMaxOffsetInQueue(lmqName3)).thenReturn(10L); + when(consumerOffsetManager.queryOffset(group, lmqName3, 0)).thenReturn(5L); + + // active consuming + liteEventDispatcher.doFullDispatch(clientId, group); + verify(liteEventDispatcher).scheduleFullDispatch(clientId, group, false); + verify(popLiteLongPollingService, times(2)).notifyMessageArriving(clientId, true, 0, group); + Assert.assertNotNull(clientEventMap.get(clientId).poll()); + Assert.assertNotNull(clientEventMap.get(clientId).poll()); + + // not active consuming + FieldUtils.writeDeclaredField(clientEventMap.get(clientId), "lastAccessTime", System.currentTimeMillis() - 6000L, true); + liteEventDispatcher.doFullDispatch(clientId, group); + verify(liteEventDispatcher).scheduleFullDispatch(clientId, group, true); + verify(popLiteLongPollingService, times(4)).notifyMessageArriving(clientId, true, 0, group); + } + + @Test + public void testDoFullDispatchByGroup() { + String group = "group"; + String clientId1 = "client1"; + String clientId2 = "client2"; + List clientIds = Arrays.asList(clientId1, clientId2); + Mockito.when(liteSubscriptionRegistry.getAllClientIdByGroup(group)).thenReturn(clientIds); + + liteEventDispatcher.doFullDispatchByGroup(group); + + verify(liteSubscriptionRegistry, times(1)).getAllClientIdByGroup(group); + verify(liteEventDispatcher, times(1)).doFullDispatch(clientId1, group); + verify(liteEventDispatcher, times(1)).doFullDispatch(clientId2, group); + } + + @Test + public void testScan() throws Exception { + String clientId = "clientId"; + String group = "group"; + String event = "event"; + liteEventDispatcher.tryDispatchToClient(event, clientId, group); + + Assert.assertNotNull(clientEventMap.get(clientId)); + FieldUtils.writeDeclaredField(clientEventMap.get(clientId), "lastAccessTime", 0L, true); + liteEventDispatcher.scan(); + verify(liteEventDispatcher).getAllSubscriber(group, event); + } + + @Test + public void testFullDispatchDeduplication() throws InterruptedException { + String clientId1 = "clientId1"; + String clientId2 = "clientId2"; + String group = "group"; + brokerConfig.setLiteEventFullDispatchDelayTime(10L); + liteEventDispatcher.scheduleFullDispatch(clientId1, group, false); + liteEventDispatcher.scheduleFullDispatch(clientId1, group, false); + liteEventDispatcher.scheduleFullDispatch(clientId1, group, false); + liteEventDispatcher.scheduleFullDispatch(clientId1, group, false); + liteEventDispatcher.scheduleFullDispatch(clientId2, group, false); + + Thread.sleep(20L); + liteEventDispatcher.scan(); + verify(liteEventDispatcher, times(1)).doFullDispatch(clientId1, group); + verify(liteEventDispatcher, times(1)).doFullDispatch(clientId2, group); + } +} diff --git a/broker/src/test/java/org/apache/rocketmq/broker/lite/LiteLifecycleManagerTest.java b/broker/src/test/java/org/apache/rocketmq/broker/lite/LiteLifecycleManagerTest.java new file mode 100644 index 00000000000..d2f40b4b75b --- /dev/null +++ b/broker/src/test/java/org/apache/rocketmq/broker/lite/LiteLifecycleManagerTest.java @@ -0,0 +1,206 @@ +/* + * 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.lite; + +import org.apache.rocketmq.broker.BrokerController; +import org.apache.rocketmq.broker.subscription.SubscriptionGroupManager; +import org.apache.rocketmq.broker.topic.TopicConfigManager; +import org.apache.rocketmq.common.BrokerConfig; +import org.apache.rocketmq.common.Pair; +import org.apache.rocketmq.common.TopicAttributes; +import org.apache.rocketmq.common.TopicConfig; +import org.apache.rocketmq.common.UtilAll; +import org.apache.rocketmq.common.attribute.TopicMessageType; +import org.apache.rocketmq.common.lite.LiteUtil; +import org.apache.rocketmq.store.MessageStore; +import org.junit.AfterClass; +import org.junit.Assert; +import org.junit.BeforeClass; +import org.junit.Ignore; +import org.junit.Test; +import org.junit.runner.RunWith; +import org.mockito.Mockito; +import org.mockito.junit.MockitoJUnitRunner; + +import java.io.File; +import java.util.List; +import java.util.UUID; +import java.util.concurrent.ConcurrentHashMap; +import java.util.concurrent.ConcurrentMap; +import java.util.stream.Collectors; +import java.util.stream.IntStream; + +import static java.util.concurrent.TimeUnit.MILLISECONDS; +import static java.util.concurrent.TimeUnit.SECONDS; +import static org.awaitility.Awaitility.await; +import static org.mockito.ArgumentMatchers.anyLong; +import static org.mockito.ArgumentMatchers.anyString; +import static org.mockito.ArgumentMatchers.eq; +import static org.mockito.Mockito.when; + +@RunWith(MockitoJUnitRunner.class) +public class LiteLifecycleManagerTest { + + private final static BrokerConfig BROKER_CONFIG = new BrokerConfig(); + private final static ConcurrentMap TOPIC_CONFIG_TABLE = new ConcurrentHashMap<>(); + private static String storePathRootDir; + private static MessageStore messageStore; + private static LiteLifecycleManager liteLifecycleManager; + private static TopicConfig mockTopicConfig = new TopicConfig(); + + @BeforeClass + public static void setUp() throws Exception { + storePathRootDir = System.getProperty("java.io.tmpdir") + File.separator + "store-lifecycleTest"; + UtilAll.deleteFile(new File(storePathRootDir)); + + messageStore = LiteTestUtil.buildMessageStore(storePathRootDir, BROKER_CONFIG, TOPIC_CONFIG_TABLE, false); + messageStore.load(); + messageStore.start(); + + BrokerController brokerController = Mockito.mock(BrokerController.class); + LiteSharding liteSharding = Mockito.mock(LiteSharding.class); + TopicConfigManager topicConfigManager = Mockito.mock(TopicConfigManager.class); + SubscriptionGroupManager subscriptionGroupManager = Mockito.mock(SubscriptionGroupManager.class); + + when(brokerController.getBrokerConfig()).thenReturn(BROKER_CONFIG); + when(brokerController.getMessageStore()).thenReturn(messageStore); + when(brokerController.getTopicConfigManager()).thenReturn(topicConfigManager); + when(brokerController.getSubscriptionGroupManager()).thenReturn(subscriptionGroupManager); + when(topicConfigManager.getTopicConfigTable()).thenReturn(TOPIC_CONFIG_TABLE); + when(topicConfigManager.selectTopicConfig(anyString())).thenReturn(mockTopicConfig); + when(subscriptionGroupManager.getSubscriptionGroupTable()).thenReturn(new ConcurrentHashMap<>()); + + LiteLifecycleManager testObject = new LiteLifecycleManager(brokerController, liteSharding); + liteLifecycleManager = Mockito.spy(testObject); + liteLifecycleManager.init(); + } + + @AfterClass + public static void reset() { + messageStore.shutdown(); + messageStore.destroy(); + UtilAll.deleteFile(new File(storePathRootDir)); + } + + @Test + public void testGetMaxOffsetInQueue() { + int num = 3; + String topic = UUID.randomUUID().toString(); + for (int i = 0; i < num; i++) { + messageStore.putMessage(LiteTestUtil.buildMessage(topic, null)); + } + await().atMost(5, SECONDS).pollInterval(200, MILLISECONDS).until(() -> messageStore.dispatchBehindBytes() <= 0); + Assert.assertEquals(num, liteLifecycleManager.getMaxOffsetInQueue(topic)); + Assert.assertEquals(0, liteLifecycleManager.getMaxOffsetInQueue(UUID.randomUUID().toString())); + } + + @Test + public void testCollectByParentTopic() { + int num = 3; + String parentTopic = UUID.randomUUID().toString(); + for (int i = 0; i < num; i++) { + messageStore.putMessage(LiteTestUtil.buildMessage(parentTopic, UUID.randomUUID().toString())); + messageStore.putMessage(LiteTestUtil.buildMessage(UUID.randomUUID().toString(), UUID.randomUUID().toString())); + } + await().atMost(5, SECONDS).pollInterval(200, MILLISECONDS).until(() -> messageStore.dispatchBehindBytes() <= 0); + List result = liteLifecycleManager.collectByParentTopic(parentTopic); + Assert.assertEquals(num, result.size()); + for (String lmqName : result) { + Assert.assertTrue(LiteUtil.belongsTo(lmqName, parentTopic)); + } + + result = liteLifecycleManager.collectByParentTopic(UUID.randomUUID().toString()); + Assert.assertEquals(0, result.size()); + } + + @Test + public void testCollectExpiredLiteTopic() { + int num = 3; + String parentTopic = UUID.randomUUID().toString(); + for (int i = 0; i < num; i++) { + messageStore.putMessage(LiteTestUtil.buildMessage(parentTopic, UUID.randomUUID().toString())); + messageStore.putMessage(LiteTestUtil.buildMessage(UUID.randomUUID().toString(), null)); + } + await().atMost(5, SECONDS).pollInterval(200, MILLISECONDS).until(() -> messageStore.dispatchBehindBytes() <= 0); + + when(liteLifecycleManager.isLiteTopicExpired(anyString(), anyString(), anyLong())).thenReturn(false); + List> result = liteLifecycleManager.collectExpiredLiteTopic(); + Assert.assertEquals(0, result.size()); + + when(liteLifecycleManager.isLiteTopicExpired(eq(parentTopic), anyString(), anyLong())).thenReturn(true); + result = liteLifecycleManager.collectExpiredLiteTopic(); + Assert.assertEquals(num, result.size()); + for (Pair pair : result) { + Assert.assertEquals(parentTopic, pair.getObject1()); + Assert.assertTrue(LiteUtil.belongsTo(pair.getObject2(), parentTopic)); + } + } + + @Ignore + @Test + public void testCleanExpiredLiteTopic() { + int num = 3; + String parentTopic = UUID.randomUUID().toString(); + List liteTopics = + IntStream.range(0, 3).mapToObj(i -> UUID.randomUUID().toString()).collect(Collectors.toList()); + for (int i = 0; i < num; i++) { + messageStore.putMessage(LiteTestUtil.buildMessage(parentTopic, liteTopics.get(i))); + } + await().atMost(5, SECONDS).pollInterval(200, MILLISECONDS).until(() -> messageStore.dispatchBehindBytes() <= 0); + + for (int i = 0; i < num; i++) { + String lmqName = LiteUtil.toLmqName(parentTopic, liteTopics.get(i)); + Assert.assertTrue(messageStore.getQueueStore().getConsumeQueueTable().containsKey(lmqName)); + } + + when(liteLifecycleManager.isLiteTopicExpired(eq(parentTopic), anyString(), anyLong())).thenReturn(true); + liteLifecycleManager.cleanExpiredLiteTopic(); + + for (int i = 0; i < num; i++) { + String lmqName = LiteUtil.toLmqName(parentTopic, liteTopics.get(i)); + Assert.assertFalse(messageStore.getQueueStore().getConsumeQueueTable().containsKey(lmqName)); + } + } + + @Test + public void testCleanByParentTopic() { + int num = 3; + String parentTopic = UUID.randomUUID().toString(); + mockTopicConfig.getAttributes().put( + TopicAttributes.TOPIC_MESSAGE_TYPE_ATTRIBUTE.getName(), TopicMessageType.LITE.getValue()); + + List liteTopics = + IntStream.range(0, 3).mapToObj(i -> UUID.randomUUID().toString()).collect(Collectors.toList()); + for (int i = 0; i < num; i++) { + messageStore.putMessage(LiteTestUtil.buildMessage(parentTopic, liteTopics.get(i))); + } + await().atMost(5, SECONDS).pollInterval(200, MILLISECONDS).until(() -> messageStore.dispatchBehindBytes() <= 0); + + for (int i = 0; i < num; i++) { + String lmqName = LiteUtil.toLmqName(parentTopic, liteTopics.get(i)); + Assert.assertTrue(messageStore.getQueueStore().getConsumeQueueTable().containsKey(lmqName)); + } + + liteLifecycleManager.cleanByParentTopic(parentTopic); + + for (int i = 0; i < num; i++) { + String lmqName = LiteUtil.toLmqName(parentTopic, liteTopics.get(i)); + Assert.assertFalse(messageStore.getQueueStore().getConsumeQueueTable().containsKey(lmqName)); + } + } +} diff --git a/broker/src/test/java/org/apache/rocketmq/broker/lite/LiteShardingImplTest.java b/broker/src/test/java/org/apache/rocketmq/broker/lite/LiteShardingImplTest.java new file mode 100644 index 00000000000..72fa83c8b1d --- /dev/null +++ b/broker/src/test/java/org/apache/rocketmq/broker/lite/LiteShardingImplTest.java @@ -0,0 +1,133 @@ +/* + * 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.lite; + +import com.google.common.hash.Hashing; + +import java.util.ArrayList; +import java.util.List; + +import org.apache.rocketmq.broker.BrokerController; +import org.apache.rocketmq.broker.topic.TopicRouteInfoManager; +import org.apache.rocketmq.client.impl.producer.TopicPublishInfo; +import org.apache.rocketmq.common.BrokerConfig; +import org.apache.rocketmq.common.lite.LiteUtil; +import org.apache.rocketmq.common.message.MessageQueue; +import org.junit.Before; +import org.junit.Test; +import org.junit.runner.RunWith; +import org.mockito.Mock; +import org.mockito.junit.MockitoJUnitRunner; + +import static org.junit.jupiter.api.Assertions.assertEquals; +import static org.mockito.Mockito.mock; +import static org.mockito.Mockito.when; + +@RunWith(MockitoJUnitRunner.class) +public class LiteShardingImplTest { + + @Mock + private BrokerController brokerController; + + @Mock + private TopicRouteInfoManager topicRouteInfoManager; + + private LiteShardingImpl liteSharding; + + @Before + public void setUp() { + liteSharding = new LiteShardingImpl(brokerController, topicRouteInfoManager); + } + + /** + * Test normal case: multiple MessageQueues, verify consistent hash selects correct brokerName + */ + @Test + public void testShardingByLmqName_NormalCase() { + // Prepare data + String parentTopic = "TestTopic"; + String liteTopic = "lite_topic"; + String lmqName = LiteUtil.toLmqName(parentTopic, liteTopic); + String brokerName1 = "BrokerA"; + String brokerName2 = "BrokerB"; + + TopicPublishInfo topicPublishInfo = mock(TopicPublishInfo.class); + List messageQueues = new ArrayList<>(); + MessageQueue mq1 = mock(MessageQueue.class); + MessageQueue mq2 = mock(MessageQueue.class); + when(mq1.getBrokerName()).thenReturn(brokerName1); +// when(mq2.getBrokerName()).thenReturn(brokerName2); + messageQueues.add(mq1); + messageQueues.add(mq2); + + when(topicPublishInfo.getMessageQueueList()).thenReturn(messageQueues); + when(topicRouteInfoManager.tryToFindTopicPublishInfo(parentTopic)).thenReturn(topicPublishInfo); + + // Execute method + String brokerName = liteSharding.shardingByLmqName(parentTopic, lmqName); + + // Verify consistent hash selected bucket + int bucket = Hashing.consistentHash(liteTopic.hashCode(), messageQueues.size()); + MessageQueue expectedMq = messageQueues.get(bucket); + String expectedBrokerName = expectedMq.getBrokerName(); + + assertEquals(expectedBrokerName, brokerName); + } + + /** + * Test edge case: empty MessageQueue list should return current broker name + */ + @Test + public void testShardingByLmqName_EmptyQueueList() { + String parentTopic = "TestTopic"; + String lmqName = "LmqName2"; + String currentBrokerName = "CurrentBroker"; + + BrokerConfig brokerConfig = mock(BrokerConfig.class); + when(brokerController.getBrokerConfig()).thenReturn(brokerConfig); + when(brokerConfig.getBrokerName()).thenReturn(currentBrokerName); + + TopicPublishInfo topicPublishInfo = mock(TopicPublishInfo.class); + when(topicPublishInfo.getMessageQueueList()).thenReturn(new ArrayList<>()); + when(topicRouteInfoManager.tryToFindTopicPublishInfo(parentTopic)).thenReturn(topicPublishInfo); + + String brokerName = liteSharding.shardingByLmqName(parentTopic, lmqName); + + assertEquals(currentBrokerName, brokerName); + } + + /** + * Test exception case: tryToFindTopicPublishInfo returns null, should return current broker name + */ + @Test + public void testShardingByLmqName_NullTopicPublishInfo() { + String parentTopic = "TestTopic"; + String lmqName = "LmqName3"; + String currentBrokerName = "CurrentBroker"; + + BrokerConfig brokerConfig = mock(BrokerConfig.class); + when(brokerController.getBrokerConfig()).thenReturn(brokerConfig); + when(brokerConfig.getBrokerName()).thenReturn(currentBrokerName); + + when(topicRouteInfoManager.tryToFindTopicPublishInfo(parentTopic)).thenReturn(null); + + String brokerName = liteSharding.shardingByLmqName(parentTopic, lmqName); + + assertEquals(currentBrokerName, brokerName); + } +} diff --git a/broker/src/test/java/org/apache/rocketmq/broker/lite/LiteSubscriptionRegistryImplTest.java b/broker/src/test/java/org/apache/rocketmq/broker/lite/LiteSubscriptionRegistryImplTest.java new file mode 100644 index 00000000000..bf300ef4d95 --- /dev/null +++ b/broker/src/test/java/org/apache/rocketmq/broker/lite/LiteSubscriptionRegistryImplTest.java @@ -0,0 +1,874 @@ +/* + * 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.lite; + +import io.netty.channel.Channel; +import java.util.HashSet; +import java.util.List; +import java.util.Set; +import java.util.concurrent.ConcurrentHashMap; +import org.apache.rocketmq.broker.BrokerController; +import org.apache.rocketmq.broker.offset.ConsumerOffsetManager; +import org.apache.rocketmq.broker.pop.orderly.QueueLevelConsumerManager; +import org.apache.rocketmq.broker.processor.PopLiteMessageProcessor; +import org.apache.rocketmq.broker.subscription.SubscriptionGroupManager; +import org.apache.rocketmq.common.BrokerConfig; +import org.apache.rocketmq.common.attribute.LiteSubModel; +import org.apache.rocketmq.common.entity.ClientGroup; +import org.apache.rocketmq.common.lite.LiteSubscription; +import org.apache.rocketmq.common.lite.OffsetOption; +import org.apache.rocketmq.remoting.protocol.subscription.SubscriptionGroupConfig; +import org.junit.Before; +import org.junit.Test; + +import static org.apache.rocketmq.common.SubscriptionGroupAttributes.LITE_SUB_MODEL_ATTRIBUTE; +import static org.junit.jupiter.api.Assertions.assertEquals; +import static org.junit.jupiter.api.Assertions.assertFalse; +import static org.junit.jupiter.api.Assertions.assertNotNull; +import static org.junit.jupiter.api.Assertions.assertNull; +import static org.junit.jupiter.api.Assertions.assertThrows; +import static org.junit.jupiter.api.Assertions.assertTrue; +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.clearInvocations; +import static org.mockito.Mockito.mock; +import static org.mockito.Mockito.never; +import static org.mockito.Mockito.times; +import static org.mockito.Mockito.verify; +import static org.mockito.Mockito.when; + +public class LiteSubscriptionRegistryImplTest { + + private LiteSubscriptionRegistryImpl registry; + private LiteCtlListener mockListener; + private AbstractLiteLifecycleManager mockLifecycleManager; + private BrokerConfig mockBrokerConfig; + private SubscriptionGroupManager mockSubscriptionGroupManager; + private ConsumerOffsetManager mockConsumerOffsetManager; + + @Before + public void setUp() { + BrokerController mockBrokerController = mock(BrokerController.class); + mockLifecycleManager = mock(AbstractLiteLifecycleManager.class); + mockBrokerConfig = mock(BrokerConfig.class); + mockSubscriptionGroupManager = mock(SubscriptionGroupManager.class); + mockConsumerOffsetManager = mock(ConsumerOffsetManager.class); + PopLiteMessageProcessor mockPopLiteMessageProcessor = mock(PopLiteMessageProcessor.class); + QueueLevelConsumerManager mockConsumerOrderInfoManager = mock(QueueLevelConsumerManager.class); + + when(mockBrokerController.getBrokerConfig()).thenReturn(mockBrokerConfig); + when(mockBrokerController.getSubscriptionGroupManager()).thenReturn(mockSubscriptionGroupManager); + when(mockBrokerController.getConsumerOffsetManager()).thenReturn(mockConsumerOffsetManager); + when(mockBrokerController.getPopLiteMessageProcessor()).thenReturn(mockPopLiteMessageProcessor); + when(mockPopLiteMessageProcessor.getConsumerOrderInfoManager()).thenReturn(mockConsumerOrderInfoManager); + when(mockConsumerOrderInfoManager.getTable()).thenReturn(new ConcurrentHashMap<>()); + when(mockBrokerConfig.getMaxLiteSubscriptionCount()).thenReturn(1000L); + when(mockBrokerConfig.getLiteSubscriptionCheckTimeoutMills()).thenReturn(60000L); + when(mockBrokerConfig.getLiteSubscriptionCheckInterval()).thenReturn(10000L); + + registry = new LiteSubscriptionRegistryImpl(mockBrokerController, mockLifecycleManager); + mockListener = mock(LiteCtlListener.class); + registry.addListener(mockListener); + } + + // Test addIncremental method + @Test + public void testAddPartialSubscription_BasicFunctionality() { + String clientId = "client1"; + String group = "group1"; + String topic = "topic1"; + Set liteTopicSet = new HashSet<>(); + liteTopicSet.add("lmq1"); + liteTopicSet.add("lmq2"); + + when(mockLifecycleManager.isSubscriptionActive(anyString(), anyString())).thenReturn(true); + + registry.addPartialSubscription(clientId, group, topic, liteTopicSet, null); + + LiteSubscription subscription = registry.getLiteSubscription(clientId); + assertNotNull(subscription); + assertEquals(group, subscription.getGroup()); + assertEquals(topic, subscription.getTopic()); + assertTrue(subscription.getLiteTopicSet().containsAll(liteTopicSet)); + + assertEquals(liteTopicSet.size(), registry.liteTopic2Group.size()); + Set topicGroupSet = registry.liteTopic2Group.get("lmq1"); + assertEquals(1, topicGroupSet.size()); + ClientGroup registeredGroup = topicGroupSet.iterator().next(); + assertEquals(clientId, registeredGroup.clientId); + assertEquals(group, registeredGroup.group); + + verify(mockListener, times(2)).onRegister(eq(clientId), eq(group), anyString()); + } + + @Test + public void testAddPartialSubscription_ExclusiveMode() { + String existingClientId = "existingClient"; + String newClientId = "newClient"; + String group = "group"; + String topic = "topic"; + String liteTopic = "lmq1"; + + Set liteTopicSet = new HashSet<>(); + liteTopicSet.add(liteTopic); + + when(mockLifecycleManager.isSubscriptionActive(anyString(), anyString())).thenReturn(true); + + // Mock subscription group config for reset offset behavior + SubscriptionGroupConfig subscriptionGroupConfig = new SubscriptionGroupConfig(); + subscriptionGroupConfig.setGroupName(group); + subscriptionGroupConfig.getAttributes().put(LITE_SUB_MODEL_ATTRIBUTE.getName(), LiteSubModel.Exclusive.name()); + when(mockSubscriptionGroupManager.findSubscriptionGroupConfig(group)).thenReturn(subscriptionGroupConfig); + + // Add existing client + registry.addPartialSubscription(existingClientId, group, topic, liteTopicSet, null); + + // Verify that the existing client is correctly registered + LiteSubscription existingSubscription = registry.getLiteSubscription(existingClientId); + assertNotNull(existingSubscription); + assertTrue(existingSubscription.getLiteTopicSet().contains(liteTopic)); + + // Execute exclusive mode addition + Set newLiteTopicSet = new HashSet<>(); + newLiteTopicSet.add(liteTopic); + registry.addPartialSubscription(newClientId, group, topic, newLiteTopicSet, null); + + // Verify that new client subscription has been added. + LiteSubscription newSubscription = registry.getLiteSubscription(newClientId); + assertNotNull(newSubscription); + assertTrue(newSubscription.getLiteTopicSet().contains(liteTopic)); + + assertEquals(liteTopicSet.size(), registry.liteTopic2Group.size()); + Set topicGroupSet = registry.liteTopic2Group.get(liteTopic); + assertEquals(1, topicGroupSet.size()); + ClientGroup registeredGroup = topicGroupSet.iterator().next(); + assertEquals(newClientId, registeredGroup.clientId); + assertEquals(group, registeredGroup.group); + + verify(mockListener).onRegister(existingClientId, group, liteTopic); + verify(mockListener).onRegister(newClientId, group, liteTopic); + verify(mockListener).onUnregister(existingClientId, group, liteTopic); + } + + @Test + public void testAddPartialSubscription_NonExclusiveMode() { + // Add an existing client subscription first + String existingClientId = "existingClient"; + String newClientId = "newClient"; + String group = "group1"; + String topic = "topic1"; + String liteTopic = "lmq1"; + + Set existingLiteTopicSet = new HashSet<>(); + existingLiteTopicSet.add(liteTopic); + + when(mockLifecycleManager.isSubscriptionActive(anyString(), anyString())).thenReturn(true); + + // Mock subscription group config + SubscriptionGroupConfig subscriptionGroupConfig = new SubscriptionGroupConfig(); + subscriptionGroupConfig.setGroupName(group); + when(mockSubscriptionGroupManager.findSubscriptionGroupConfig(group)).thenReturn(subscriptionGroupConfig); + + // Add existing client + registry.addPartialSubscription(existingClientId, group, topic, existingLiteTopicSet, null); + + // Add new client in non-exclusive mode + Set newLiteTopicSet = new HashSet<>(); + newLiteTopicSet.add(liteTopic); + registry.addPartialSubscription(newClientId, group, topic, newLiteTopicSet, null); + + // Verify both client subscriptions exist + LiteSubscription existingSubscription = registry.getLiteSubscription(existingClientId); + LiteSubscription newSubscription = registry.getLiteSubscription(newClientId); + assertNotNull(existingSubscription); + assertNotNull(newSubscription); + assertTrue(existingSubscription.getLiteTopicSet().contains(liteTopic)); + assertTrue(newSubscription.getLiteTopicSet().contains(liteTopic)); + + // Verify listener was only called for registration, not unregistration + verify(mockListener, times(2)).onRegister(anyString(), eq(group), eq(liteTopic)); + verify(mockListener, never()).onUnregister(anyString(), anyString(), anyString()); + } + + @Test + public void testAddPartialSubscription_WithEmptyLiteTopicSet() { + String clientId = "client1"; + String group = "group1"; + String topic = "topic1"; + Set liteTopicSet = new HashSet<>(); + + registry.addPartialSubscription(clientId, group, topic, liteTopicSet, null); + + LiteSubscription subscription = registry.getLiteSubscription(clientId); + assertNotNull(subscription); + assertEquals(group, subscription.getGroup()); + assertEquals(topic, subscription.getTopic()); + assertTrue(subscription.getLiteTopicSet().isEmpty()); + + // Verify listener was not called + verify(mockListener, never()).onRegister(anyString(), anyString(), anyString()); + } + + @Test + public void testAddPartialSubscription_InactiveSubscription() { + String clientId = "client1"; + String group = "group1"; + String topic = "topic1"; + String inactiveLiteTopic = "inactive_lmq1"; + + Set liteTopicSet = new HashSet<>(); + liteTopicSet.add(inactiveLiteTopic); + + // Mock inactive subscription + when(mockLifecycleManager.isSubscriptionActive(topic, inactiveLiteTopic)).thenReturn(false); + + // Should not add inactive subscriptions + registry.addPartialSubscription(clientId, group, topic, liteTopicSet, null); + + LiteSubscription subscription = registry.getLiteSubscription(clientId); + assertNotNull(subscription); + assertFalse(subscription.getLiteTopicSet().contains(inactiveLiteTopic)); + assertEquals(0, registry.getActiveSubscriptionNum()); + } + + @Test + public void testAddPartialSubscription_ExclusiveModeDifferentGroups() { + // Add two clients from different groups + String client1 = "client1"; + String group1 = "group1"; + String client2 = "client2"; + String group2 = "group2"; + String topic = "topic1"; + String liteTopic = "lmq1"; + + Set liteTopicSet = new HashSet<>(); + liteTopicSet.add(liteTopic); + + when(mockLifecycleManager.isSubscriptionActive(anyString(), anyString())).thenReturn(true); + + // Mock subscription group configs + SubscriptionGroupConfig subscriptionGroupConfig1 = new SubscriptionGroupConfig(); + subscriptionGroupConfig1.setGroupName(group1); + subscriptionGroupConfig1.getAttributes().put(LITE_SUB_MODEL_ATTRIBUTE.getName(), LiteSubModel.Exclusive.name()); + when(mockSubscriptionGroupManager.findSubscriptionGroupConfig(group1)).thenReturn(subscriptionGroupConfig1); + + SubscriptionGroupConfig subscriptionGroupConfig2 = new SubscriptionGroupConfig(); + subscriptionGroupConfig2.setGroupName(group2); + subscriptionGroupConfig2.getAttributes().put(LITE_SUB_MODEL_ATTRIBUTE.getName(), LiteSubModel.Exclusive.name()); + when(mockSubscriptionGroupManager.findSubscriptionGroupConfig(group2)).thenReturn(subscriptionGroupConfig2); + + // Add first client + registry.addPartialSubscription(client1, group1, topic, liteTopicSet, null); + + // Add second client + registry.addPartialSubscription(client2, group2, topic, liteTopicSet, null); + + // Verify both clients are registered for the same topic + Set observers = registry.getSubscriber(liteTopic); + assertEquals(2, observers.size()); + + // Add new client in exclusive mode from the same group as client1 + String client3 = "client3"; + registry.addPartialSubscription(client3, group1, topic, liteTopicSet, null); + + // Verify only client1 was removed (same group), client2 remains (different group) + observers = registry.getSubscriber(liteTopic); + assertEquals(2, observers.size()); // client2(group2) and client3(group1) + + boolean hasClient2 = false; + boolean hasClient3 = false; + for (ClientGroup cg : observers) { + if (cg.clientId.equals(client2) && cg.group.equals(group2)) { + hasClient2 = true; + } + if (cg.clientId.equals(client3) && cg.group.equals(group1)) { + hasClient3 = true; + } + } + + assertTrue(hasClient2, "Client2 (group2) should still be registered"); + assertTrue(hasClient3, "Client3 (group1) should be registered"); + + // Verify listener calls + verify(mockListener).onUnregister(client1, group1, liteTopic); // Same group client1 removed + verify(mockListener, never()).onUnregister(client2, group2, liteTopic); // Different group client2 retained + } + + @Test + public void testAddPartialSubscription_QuotaLimit() { + // Set quota to 1 + when(mockBrokerConfig.getMaxLiteSubscriptionCount()).thenReturn(1L); + + when(mockLifecycleManager.isSubscriptionActive(anyString(), anyString())).thenReturn(true); + + // Add first subscription + String clientId1 = "client1"; + String group1 = "group1"; + String topic1 = "topic1"; + Set liteTopicSet1 = new HashSet<>(); + liteTopicSet1.add("lmq1"); + + registry.addPartialSubscription(clientId1, group1, topic1, liteTopicSet1, null); + + // Try to add second subscription, should throw exception + String clientId2 = "client2"; + String group2 = "group2"; + String topic2 = "topic2"; + Set liteTopicSet2 = new HashSet<>(); + liteTopicSet2.add("lmq2"); + + assertThrows(LiteQuotaException.class, () -> { + registry.addPartialSubscription(clientId2, group2, topic2, liteTopicSet2, null); + }); + } + + // Test removeIncremental method + @Test + public void testRemovePartialSubscription() { + String clientId = "client1"; + String group = "group1"; + String topic = "topic1"; + String liteTopic1 = "lmq1"; + String liteTopic2 = "lmq2"; + + Set liteTopicSet = new HashSet<>(); + liteTopicSet.add(liteTopic1); + liteTopicSet.add(liteTopic2); + + when(mockLifecycleManager.isSubscriptionActive(anyString(), anyString())).thenReturn(true); + + // Add subscriptions first + registry.addPartialSubscription(clientId, group, topic, liteTopicSet, null); + + // Verify subscriptions were added + LiteSubscription subscription = registry.getLiteSubscription(clientId); + assertTrue(subscription.getLiteTopicSet().contains(liteTopic1)); + assertTrue(subscription.getLiteTopicSet().contains(liteTopic2)); + + // Remove some subscriptions + Set toRemove = new HashSet<>(); + toRemove.add(liteTopic1); + registry.removePartialSubscription(clientId, group, topic, toRemove); + + // Verify removal was successful + subscription = registry.getLiteSubscription(clientId); + assertFalse(subscription.getLiteTopicSet().contains(liteTopic1)); + assertTrue(subscription.getLiteTopicSet().contains(liteTopic2)); + + verify(mockListener).onUnregister(clientId, group, liteTopic1); + verify(mockListener, never()).onUnregister(clientId, group, liteTopic2); + } + + // Test addAll method + @Test + public void testAddCompleteSubscription() { + String clientId = "client1"; + String group = "group1"; + String topic = "topic1"; + String liteTopic1 = "lmq1"; + String liteTopic2 = "lmq2"; + String liteTopic3 = "lmq3"; + + // Initial subscriptions + Set initialSet = new HashSet<>(); + initialSet.add(liteTopic1); + initialSet.add(liteTopic2); + + // New full subscription set + Set newFullSet = new HashSet<>(); + newFullSet.add(liteTopic2); + newFullSet.add(liteTopic3); + + when(mockLifecycleManager.isSubscriptionActive(anyString(), anyString())).thenReturn(true); + + // Add initial subscriptions + registry.addPartialSubscription(clientId, group, topic, initialSet, null); + + // Reset mock to ignore previous interactions + clearInvocations(mockListener); + + // Update with addAll + registry.addCompleteSubscription(clientId, group, topic, newFullSet, 1L); + + // Verify update results + LiteSubscription subscription = registry.getLiteSubscription(clientId); + assertFalse(subscription.getLiteTopicSet().contains(liteTopic1)); // Should be removed + assertTrue(subscription.getLiteTopicSet().contains(liteTopic2)); // Should be retained + assertTrue(subscription.getLiteTopicSet().contains(liteTopic3)); // Should be added + + // Verify that liteTopic1 was unregistered (no longer in new set) + verify(mockListener).onUnregister(clientId, group, liteTopic1); + + // Verify that liteTopic3 was registered (new in the set) + verify(mockListener).onRegister(clientId, group, liteTopic3); + + // Verify that liteTopic2 was neither unregistered nor registered again + // (it was already registered and remains in the new set) + verify(mockListener, never()).onUnregister(clientId, group, liteTopic2); + } + + // Test removeAll method + @Test + public void testRemoveCompleteSubscription() { + String clientId = "client1"; + String group = "group1"; + String topic = "topic1"; + String liteTopic1 = "lmq1"; + String liteTopic2 = "lmq2"; + + Set liteTopicSet = new HashSet<>(); + liteTopicSet.add(liteTopic1); + liteTopicSet.add(liteTopic2); + + when(mockLifecycleManager.isSubscriptionActive(anyString(), anyString())).thenReturn(true); + + // Add subscriptions + registry.addPartialSubscription(clientId, group, topic, liteTopicSet, null); + + // Verify subscriptions were added + assertNotNull(registry.getLiteSubscription(clientId)); + assertEquals(2, registry.getActiveSubscriptionNum()); + + // Remove all subscriptions + registry.removeCompleteSubscription(clientId); + + // Verify all subscriptions were removed + assertNull(registry.getLiteSubscription(clientId)); + assertEquals(0, registry.getActiveSubscriptionNum()); + + verify(mockListener).onRemoveAll(clientId, group); + } + + @Test + public void testRemoveCompleteSubscription_NonExistentClient() { + String nonExistentClientId = "nonexistent"; + + // Should not throw exception + registry.removeCompleteSubscription(nonExistentClientId); + + // Verify no changes to registry state + assertEquals(0, registry.getActiveSubscriptionNum()); + assertNull(registry.getLiteSubscription(nonExistentClientId)); + } + + // Test cleanSubscription method + @Test + public void testCleanSubscription() { + String clientId = "client1"; + String group = "group1"; + String topic = "topic1"; + String liteTopic1 = "lmq1"; + String liteTopic2 = "lmq2"; + + Set liteTopicSet = new HashSet<>(); + liteTopicSet.add(liteTopic1); + liteTopicSet.add(liteTopic2); + + when(mockLifecycleManager.isSubscriptionActive(anyString(), anyString())).thenReturn(true); + + // Add subscription + registry.addPartialSubscription(clientId, group, topic, liteTopicSet, null); + assertEquals(2, registry.getActiveSubscriptionNum()); + + // Verify subscription was added + LiteSubscription subscription = registry.getLiteSubscription(clientId); + assertTrue(subscription.getLiteTopicSet().contains(liteTopic1)); + assertTrue(subscription.getLiteTopicSet().contains(liteTopic2)); + + // Clean subscription + registry.cleanSubscription(liteTopic1, true); + registry.cleanSubscription(liteTopic2, false); + + // Verify subscription was cleaned + subscription = registry.getLiteSubscription(clientId); + assertFalse(subscription.getLiteTopicSet().contains(liteTopic1)); + assertFalse(subscription.getLiteTopicSet().contains(liteTopic2)); + assertNull(registry.getSubscriber(liteTopic1)); + assertNull(registry.getSubscriber(liteTopic2)); + assertEquals(0, registry.getActiveSubscriptionNum()); + } + + // Test getSubscriber method + @Test + public void testGetSubscriber() { + String clientId = "client1"; + String group = "group1"; + String topic = "topic1"; + String liteTopic = "lmq1"; + + Set liteTopicSet = new HashSet<>(); + liteTopicSet.add(liteTopic); + + when(mockLifecycleManager.isSubscriptionActive(anyString(), anyString())).thenReturn(true); + + registry.addPartialSubscription(clientId, group, topic, liteTopicSet, null); + + Set observers = registry.getSubscriber(liteTopic); + assertNotNull(observers); + assertEquals(1, observers.size()); + ClientGroup clientGroup = observers.iterator().next(); + assertEquals(clientId, clientGroup.clientId); + assertEquals(group, clientGroup.group); + } + + @Test + public void testGetSubscriber_NonExistentTopic() { + String nonExistentTopic = "nonexistent_lmq"; + + Set result = registry.getSubscriber(nonExistentTopic); + + // Should return null for non-existent topic + assertNull(result); + } + + // Test updateClientChannel method + @Test + public void testUpdateClientChannel() { + String clientId = "client1"; + Channel mockChannel = mock(Channel.class); + + registry.updateClientChannel(clientId, mockChannel); + + // Verify channel was updated + assertEquals(mockChannel, registry.clientChannels.get(clientId)); + } + + // Test getActiveSubscriptionNum method + @Test + public void testGetActiveSubscriptionNum() { + String clientId1 = "client1"; + String clientId2 = "client2"; + String group = "group1"; + String topic = "topic1"; + String liteTopic1 = "lmq1"; + String liteTopic2 = "lmq2"; + + Set liteTopicSet1 = new HashSet<>(); + liteTopicSet1.add(liteTopic1); + + Set liteTopicSet2 = new HashSet<>(); + liteTopicSet2.add(liteTopic1); // Same topic + liteTopicSet2.add(liteTopic2); // New topic + + when(mockLifecycleManager.isSubscriptionActive(anyString(), anyString())).thenReturn(true); + + // Initial state + assertEquals(0, registry.getActiveSubscriptionNum()); + + // Add first client + registry.addPartialSubscription(clientId1, group, topic, liteTopicSet1, null); + assertEquals(1, registry.getActiveSubscriptionNum()); + + // Add second client + registry.addPartialSubscription(clientId2, group, topic, liteTopicSet2, null); + assertEquals(3, registry.getActiveSubscriptionNum()); // 3 references: client1->topic1, client2->topic1, client2->topic2 + } + + // Test cleanupExpiredSubscriptions method + @Test + public void testCleanupExpiredSubscriptions_NoExpiredClients() { + String clientId = "client1"; + String group = "group1"; + String topic = "topic1"; + Set liteTopics = new HashSet<>(); + liteTopics.add("lmq1"); + liteTopics.add("lmq2"); + + LiteSubscription subscription = new LiteSubscription(); + subscription.setGroup(group); + subscription.setTopic(topic); + subscription.addLiteTopic(liteTopics); + subscription.setUpdateTime(System.currentTimeMillis()); // Not expired + + Channel channel = mock(Channel.class); + + registry.client2Subscription.put(clientId, subscription); + registry.clientChannels.put(clientId, channel); + + // Initialize liteTopic2Group + for (String lmq : liteTopics) { + registry.liteTopic2Group.computeIfAbsent(lmq, k -> ConcurrentHashMap.newKeySet()) + .add(new ClientGroup(clientId, group)); + } + + registry.activeNum.set(liteTopics.size()); + + // Perform cleanup with a timeout of 10 seconds + registry.cleanupExpiredSubscriptions(10000); + + // Verify that the client has not been cleaned up + assertNotNull(registry.client2Subscription.get(clientId)); + assertNotNull(registry.clientChannels.get(clientId)); + assertEquals(liteTopics.size(), registry.activeNum.get()); + } + + @Test + public void testCleanupExpiredSubscriptions_WithExpiredClients() { + String clientId = "client1"; + String group = "group1"; + String topic = "topic1"; + Set liteTopics = new HashSet<>(); + liteTopics.add("lmq1"); + liteTopics.add("lmq2"); + + LiteSubscription subscription = new LiteSubscription(); + subscription.setGroup(group); + subscription.setTopic(topic); + subscription.addLiteTopic(liteTopics); + subscription.setUpdateTime(System.currentTimeMillis() - 20000); + + Channel channel = mock(Channel.class); + + registry.client2Subscription.put(clientId, subscription); + registry.clientChannels.put(clientId, channel); + + // Initialize liteTopic2Group + for (String lmq : liteTopics) { + registry.liteTopic2Group.computeIfAbsent(lmq, k -> ConcurrentHashMap.newKeySet()) + .add(new ClientGroup(clientId, group)); + } + + registry.activeNum.set(liteTopics.size()); + + LiteCtlListener mockListener = mock(LiteCtlListener.class); + registry.addListener(mockListener); + + // Perform cleanup with a timeout of 10 seconds + registry.cleanupExpiredSubscriptions(10000); + + // Verify that the client has been cleaned up + assertNull(registry.client2Subscription.get(clientId)); + assertNull(registry.clientChannels.get(clientId)); + assertEquals(0, registry.activeNum.get()); + + // Verify that the listener was called + verify(mockListener, times(1)).onUnregister(eq(clientId), eq(group), eq("lmq1")); + verify(mockListener, times(1)).onUnregister(eq(clientId), eq(group), eq("lmq2")); + verify(mockListener, times(1)).onRemoveAll(eq(clientId), eq(group)); + + // Verify that topics in liteTopic2Group have been removed + assertNull(registry.liteTopic2Group.get("lmq1")); + assertNull(registry.liteTopic2Group.get("lmq2")); + } + + @Test + public void testCleanupExpiredSubscriptions_ExpiredClientWithNoSubscriptions() { + String clientId = "client1"; + String group = "group1"; + String topic = "topic1"; + Set liteTopics = new HashSet<>(); + + LiteSubscription subscription = new LiteSubscription(); + subscription.setGroup(group); + subscription.setTopic(topic); + subscription.addLiteTopic(liteTopics); + subscription.setUpdateTime(System.currentTimeMillis() - 20000); // Expired + + Channel channel = mock(Channel.class); + + registry.client2Subscription.put(clientId, subscription); + registry.clientChannels.put(clientId, channel); + + registry.activeNum.set(0); + + LiteCtlListener mockListener = mock(LiteCtlListener.class); + registry.addListener(mockListener); + + // Perform cleanup with 10 second timeout + registry.cleanupExpiredSubscriptions(10000); + + // Verify that the client has been cleaned up + assertNull(registry.client2Subscription.get(clientId)); + assertNull(registry.clientChannels.get(clientId)); + assertEquals(0, registry.activeNum.get()); + + // Verify that the listener was not called + verify(mockListener, never()).onUnregister(anyString(), anyString(), anyString()); + } + + // Test removeTopicGroup method + @Test + public void testRemoveTopicGroup_EmptyTopicGroupSet() { + String clientId = "client1"; + String group = "group1"; + String liteTopic = "lmq1"; + + ClientGroup clientGroup = new ClientGroup(clientId, group); + + // Initialize with a single client + Set topicGroupSet = ConcurrentHashMap.newKeySet(); + topicGroupSet.add(clientGroup); + registry.liteTopic2Group.put(liteTopic, topicGroupSet); + registry.activeNum.set(1); + + // Remove the only client + registry.removeTopicGroup(clientGroup, liteTopic, false); + + // Verify that the topic is completely removed from liteTopic2Group + assertNull(registry.liteTopic2Group.get(liteTopic)); + assertEquals(0, registry.getActiveSubscriptionNum()); + } + + // Test excludeClientByLmqName method + @Test + public void testExcludeClientByLmqName_EmptyClientSet() { + String newClientId = "newClient"; + String group = "group1"; + String lmqName = "lmq1"; + + // Ensure the liteTopic2Group map exists but is empty + registry.liteTopic2Group.put(lmqName, ConcurrentHashMap.newKeySet()); + + // Should not throw any exception + registry.excludeClientByLmqName(newClientId, group, lmqName); + + // Verify no changes + assertTrue(registry.liteTopic2Group.get(lmqName).isEmpty()); + } + + @Test + public void testGetAllClientIdByGroup() { + String group1 = "group1"; + String group2 = "group2"; + String clientId1 = "client1"; + String clientId2 = "client2"; + String clientId3 = "client3"; + String topic = "parentTopic"; + + LiteSubscription sub1 = new LiteSubscription(); + sub1.setGroup(group1); + sub1.setTopic(topic); + + LiteSubscription sub2 = new LiteSubscription(); + sub2.setGroup(group1); + sub2.setTopic(topic); + + LiteSubscription sub3 = new LiteSubscription(); + sub3.setGroup(group2); + sub3.setTopic(topic); + + registry.client2Subscription.put(clientId1, sub1); + registry.client2Subscription.put(clientId2, sub2); + registry.client2Subscription.put(clientId3, sub3); + + List result; + + // group1 + result = registry.getAllClientIdByGroup(group1); + assertEquals(2, result.size()); + assertTrue(result.contains(clientId1)); + assertTrue(result.contains(clientId2)); + + // group2 + result = registry.getAllClientIdByGroup(group2); + assertEquals(1, result.size()); + assertTrue(result.contains(clientId3)); + + // not exist + result = registry.getAllClientIdByGroup("notExistGroup"); + assertTrue(result.isEmpty()); + + // null + result = registry.getAllClientIdByGroup(null); + assertTrue(result.isEmpty()); + } + + @Test + public void testResetOffset_minOffset() { + String lmqName = "lmq1"; + String group = "group1"; + String clientId = "client1"; + + when(mockConsumerOffsetManager.queryOffset(group, lmqName, 0)).thenReturn(100L); + + OffsetOption offsetOption = new OffsetOption(OffsetOption.Type.POLICY, OffsetOption.POLICY_MIN_VALUE); + registry.resetOffset(lmqName, group, clientId, offsetOption); + + verify(mockConsumerOffsetManager).assignResetOffset(lmqName, group, 0, 0L); + } + + @Test + public void testResetOffset_maxOffset() { + String lmqName = "lmq1"; + String group = "group1"; + String clientId = "client1"; + long maxOffset = 500L; + + when(mockConsumerOffsetManager.queryOffset(group, lmqName, 0)).thenReturn(100L); + when(mockLifecycleManager.getMaxOffsetInQueue(lmqName)).thenReturn(maxOffset); + + OffsetOption offsetOption = new OffsetOption(OffsetOption.Type.POLICY, OffsetOption.POLICY_MAX_VALUE); + registry.resetOffset(lmqName, group, clientId, offsetOption); + + verify(mockConsumerOffsetManager).assignResetOffset(lmqName, group, 0, maxOffset); + } + + @Test + public void testResetOffset_absolute() { + String lmqName = "lmq1"; + String group = "group1"; + String clientId = "client1"; + long specifiedOffset = 250L; + + when(mockConsumerOffsetManager.queryOffset(group, lmqName, 0)).thenReturn(100L); + + OffsetOption offsetOption = new OffsetOption(OffsetOption.Type.OFFSET, specifiedOffset); + registry.resetOffset(lmqName, group, clientId, offsetOption); + + verify(mockConsumerOffsetManager).assignResetOffset(lmqName, group, 0, specifiedOffset); + } + + @Test + public void testResetOffset_LastN() { + String lmqName = "lmq1"; + String group1 = "group1"; + String group2 = "group2"; + String clientId = "client1"; + long currentOffset = 100L; + long lastN = 20L; + long expectedTargetOffset = 80L; + + when(mockConsumerOffsetManager.queryOffset(group1, lmqName, 0)).thenReturn(currentOffset); + when(mockConsumerOffsetManager.queryOffset(group2, lmqName, 0)).thenReturn(-1L); + + OffsetOption offsetOption = new OffsetOption(OffsetOption.Type.TAIL_N, lastN); + + registry.resetOffset(lmqName, group1, clientId, offsetOption); + registry.resetOffset(lmqName, group2, clientId, offsetOption); + + verify(mockConsumerOffsetManager).assignResetOffset(lmqName, group1, 0, expectedTargetOffset); + verify(mockConsumerOffsetManager, never()).assignResetOffset(lmqName, group2, 0, expectedTargetOffset); + } + + @Test + public void testResetOffset_timestamp_not_supported() { + String lmqName = "lmq1"; + String group = "group1"; + String clientId = "client1"; + long timestamp = System.currentTimeMillis(); + + when(mockConsumerOffsetManager.queryOffset(group, lmqName, 0)).thenReturn(100L); + + OffsetOption offsetOption = new OffsetOption(OffsetOption.Type.TIMESTAMP, timestamp); + registry.resetOffset(lmqName, group, clientId, offsetOption); + + verify(mockConsumerOffsetManager, never()).assignResetOffset(anyString(), anyString(), anyInt(), anyLong()); + } +} \ No newline at end of file diff --git a/broker/src/test/java/org/apache/rocketmq/broker/lite/LiteTestUtil.java b/broker/src/test/java/org/apache/rocketmq/broker/lite/LiteTestUtil.java new file mode 100644 index 00000000000..eabc5ea3f0d --- /dev/null +++ b/broker/src/test/java/org/apache/rocketmq/broker/lite/LiteTestUtil.java @@ -0,0 +1,83 @@ +/* + * 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.lite; + +import org.apache.commons.lang3.StringUtils; +import org.apache.rocketmq.common.BrokerConfig; +import org.apache.rocketmq.common.TopicConfig; +import org.apache.rocketmq.common.lite.LiteUtil; +import org.apache.rocketmq.common.message.MessageAccessor; +import org.apache.rocketmq.common.message.MessageConst; +import org.apache.rocketmq.common.message.MessageDecoder; +import org.apache.rocketmq.common.message.MessageExtBrokerInner; +import org.apache.rocketmq.store.DefaultMessageStore; +import org.apache.rocketmq.store.MessageStore; +import org.apache.rocketmq.store.RocksDBMessageStore; +import org.apache.rocketmq.store.config.FlushDiskType; +import org.apache.rocketmq.store.config.MessageStoreConfig; +import org.apache.rocketmq.store.stats.BrokerStatsManager; + +import java.net.InetSocketAddress; +import java.util.concurrent.ConcurrentMap; + +public class LiteTestUtil { + + public static MessageStore buildMessageStore(String storePathRootDir, final BrokerConfig brokerConfig, + final ConcurrentMap topicConfigTable, boolean isRocksDBStore) throws Exception { + MessageStoreConfig storeConfig = new MessageStoreConfig(); + storeConfig.setMappedFileSizeCommitLog(1024 * 1024 * 10); + storeConfig.setMappedFileSizeConsumeQueue(1024 * 1024 * 10); + storeConfig.setMaxHashSlotNum(10000); + storeConfig.setMaxIndexNum(100 * 100); + storeConfig.setFlushDiskType(FlushDiskType.SYNC_FLUSH); + storeConfig.setFlushIntervalConsumeQueue(1); + storeConfig.setHaListenPort(0); + storeConfig.setEnableLmq(true); + storeConfig.setEnableMultiDispatch(true); + storeConfig.setStorePathRootDir(storePathRootDir); + + BrokerStatsManager brokerStatsManager = new BrokerStatsManager(brokerConfig); + MessageStore messageStore; + if (isRocksDBStore) { + messageStore = new RocksDBMessageStore(storeConfig, brokerStatsManager, null, brokerConfig, topicConfigTable); + } else { + messageStore = new DefaultMessageStore(storeConfig, brokerStatsManager, null, brokerConfig, topicConfigTable); + } + return messageStore; + } + + public static MessageExtBrokerInner buildMessage(String parentTopic, String liteTopic) { + MessageExtBrokerInner msg = new MessageExtBrokerInner(); + msg.setTopic(parentTopic); + msg.setTags("TAG1"); + msg.setKeys("Hello"); + msg.setBody("HW".getBytes()); + msg.setQueueId(0); + msg.setSysFlag(0); + msg.setBornTimestamp(System.currentTimeMillis()); + msg.setStoreHost(new InetSocketAddress("localhost", 10911)); + msg.setBornHost(new InetSocketAddress("localhost", 0)); + + if (StringUtils.isNotEmpty(liteTopic)) { + String lmqName = LiteUtil.toLmqName(parentTopic, liteTopic); + MessageAccessor.putProperty(msg, MessageConst.PROPERTY_INNER_MULTI_DISPATCH, lmqName); + } + msg.setPropertiesString(MessageDecoder.messageProperties2String(msg.getProperties())); + return msg; + } +} diff --git a/broker/src/test/java/org/apache/rocketmq/broker/lite/RocksDBLiteLifecycleManagerTest.java b/broker/src/test/java/org/apache/rocketmq/broker/lite/RocksDBLiteLifecycleManagerTest.java new file mode 100644 index 00000000000..90b4e47f6a3 --- /dev/null +++ b/broker/src/test/java/org/apache/rocketmq/broker/lite/RocksDBLiteLifecycleManagerTest.java @@ -0,0 +1,242 @@ +/* + * 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.lite; + +import org.apache.rocketmq.broker.BrokerController; +import org.apache.rocketmq.broker.subscription.SubscriptionGroupManager; +import org.apache.rocketmq.broker.topic.TopicConfigManager; +import org.apache.rocketmq.common.BrokerConfig; +import org.apache.rocketmq.common.Pair; +import org.apache.rocketmq.common.TopicAttributes; +import org.apache.rocketmq.common.TopicConfig; +import org.apache.rocketmq.common.UtilAll; +import org.apache.rocketmq.common.attribute.TopicMessageType; +import org.apache.rocketmq.common.lite.LiteUtil; +import org.apache.rocketmq.store.MessageStore; +import org.apache.rocketmq.store.plugin.AbstractPluginMessageStore; +import org.apache.rocketmq.store.plugin.MessageStorePluginContext; +import org.apache.rocketmq.tieredstore.TieredMessageStore; +import org.junit.AfterClass; +import org.junit.Assert; +import org.junit.BeforeClass; +import org.junit.Ignore; +import org.junit.Test; +import org.junit.runner.RunWith; +import org.mockito.Mockito; +import org.mockito.junit.MockitoJUnitRunner; + +import java.io.File; +import java.util.List; +import java.util.UUID; +import java.util.concurrent.ConcurrentHashMap; +import java.util.concurrent.ConcurrentMap; +import java.util.stream.Collectors; +import java.util.stream.IntStream; + +import static java.util.concurrent.TimeUnit.MILLISECONDS; +import static java.util.concurrent.TimeUnit.SECONDS; +import static org.awaitility.Awaitility.await; +import static org.mockito.ArgumentMatchers.anyLong; +import static org.mockito.ArgumentMatchers.anyString; +import static org.mockito.ArgumentMatchers.eq; +import static org.mockito.Mockito.when; + +@RunWith(MockitoJUnitRunner.class) +public class RocksDBLiteLifecycleManagerTest { + + private final static BrokerConfig BROKER_CONFIG = new BrokerConfig(); + private final static ConcurrentMap TOPIC_CONFIG_TABLE = new ConcurrentHashMap<>(); + private static String storePathRootDir; + private static MessageStore messageStore; + private static RocksDBLiteLifecycleManager liteLifecycleManager; + private static TopicConfig mockTopicConfig = new TopicConfig(); + + @BeforeClass + public static void setUp() throws Exception { + storePathRootDir = System.getProperty("java.io.tmpdir") + File.separator + "store-rocksDBLifecycleTest"; + UtilAll.deleteFile(new File(storePathRootDir)); + + messageStore = LiteTestUtil.buildMessageStore(storePathRootDir, BROKER_CONFIG, TOPIC_CONFIG_TABLE, true); + messageStore.load(); + messageStore.start(); + + BrokerController brokerController = Mockito.mock(BrokerController.class); + LiteSharding liteSharding = Mockito.mock(LiteSharding.class); + TopicConfigManager topicConfigManager = Mockito.mock(TopicConfigManager.class); + SubscriptionGroupManager subscriptionGroupManager = Mockito.mock(SubscriptionGroupManager.class); + + when(brokerController.getBrokerConfig()).thenReturn(BROKER_CONFIG); + when(brokerController.getMessageStore()).thenReturn(messageStore); + when(brokerController.getTopicConfigManager()).thenReturn(topicConfigManager); + when(brokerController.getSubscriptionGroupManager()).thenReturn(subscriptionGroupManager); + when(topicConfigManager.getTopicConfigTable()).thenReturn(TOPIC_CONFIG_TABLE); + when(topicConfigManager.selectTopicConfig(anyString())).thenReturn(mockTopicConfig); + when(subscriptionGroupManager.getSubscriptionGroupTable()).thenReturn(new ConcurrentHashMap<>()); + + RocksDBLiteLifecycleManager testObject = new RocksDBLiteLifecycleManager(brokerController, liteSharding); + liteLifecycleManager = Mockito.spy(testObject); + liteLifecycleManager.init(); + } + + @AfterClass + public static void reset() { + messageStore.shutdown(); + messageStore.destroy(); + UtilAll.deleteFile(new File(storePathRootDir)); + mockTopicConfig = new TopicConfig(); + } + + @Ignore + @Test + public void testInit_tieredStore() { + BrokerController brokerController = Mockito.mock(BrokerController.class); + LiteSharding liteSharding = Mockito.mock(LiteSharding.class); + MessageStorePluginContext context = Mockito.mock(MessageStorePluginContext.class); + + TieredMessageStore tieredMessageStore = new TieredMessageStore(context, messageStore); + when(brokerController.getBrokerConfig()).thenReturn(BROKER_CONFIG); + when(brokerController.getMessageStore()).thenReturn(tieredMessageStore); + + RocksDBLiteLifecycleManager manager = new RocksDBLiteLifecycleManager(brokerController, liteSharding); + manager.init(); + Assert.assertEquals(0, manager.getMaxOffsetInQueue(UUID.randomUUID().toString())); + } + + @Test + public void testInit_otherStore() { + BrokerController brokerController = Mockito.mock(BrokerController.class); + LiteSharding liteSharding = Mockito.mock(LiteSharding.class); + AbstractPluginMessageStore pluginMessageStore = Mockito.mock(AbstractPluginMessageStore.class); + + when(brokerController.getBrokerConfig()).thenReturn(BROKER_CONFIG); + when(brokerController.getMessageStore()).thenReturn(pluginMessageStore); + + RocksDBLiteLifecycleManager manager = new RocksDBLiteLifecycleManager(brokerController, liteSharding); + manager.init(); + Assert.assertThrows(NullPointerException.class, () -> manager.getMaxOffsetInQueue("HW")); + } + + @Test + public void testGetMaxOffsetInQueue() { + int num = 3; + String topic = UUID.randomUUID().toString(); + for (int i = 0; i < num; i++) { + messageStore.putMessage(LiteTestUtil.buildMessage(topic, null)); + } + await().atMost(5, SECONDS).pollInterval(200, MILLISECONDS).until(() -> messageStore.dispatchBehindBytes() <= 0); + Assert.assertEquals(num, liteLifecycleManager.getMaxOffsetInQueue(topic)); + Assert.assertEquals(0, liteLifecycleManager.getMaxOffsetInQueue(UUID.randomUUID().toString())); + } + + @Test + public void testCollectByParentTopic() { + int num = 3; + String parentTopic = UUID.randomUUID().toString(); + for (int i = 0; i < num; i++) { + messageStore.putMessage(LiteTestUtil.buildMessage(parentTopic, UUID.randomUUID().toString())); + messageStore.putMessage(LiteTestUtil.buildMessage(UUID.randomUUID().toString(), UUID.randomUUID().toString())); + } + await().atMost(5, SECONDS).pollInterval(200, MILLISECONDS).until(() -> messageStore.dispatchBehindBytes() <= 0); + List result = liteLifecycleManager.collectByParentTopic(parentTopic); + Assert.assertEquals(num, result.size()); + for (String lmqName : result) { + Assert.assertTrue(LiteUtil.belongsTo(lmqName, parentTopic)); + } + + result = liteLifecycleManager.collectByParentTopic(UUID.randomUUID().toString()); + Assert.assertEquals(0, result.size()); + } + + @Test + public void testCollectExpiredLiteTopic() { + int num = 3; + String parentTopic = UUID.randomUUID().toString(); + for (int i = 0; i < num; i++) { + messageStore.putMessage(LiteTestUtil.buildMessage(parentTopic, UUID.randomUUID().toString())); + messageStore.putMessage(LiteTestUtil.buildMessage(UUID.randomUUID().toString(), null)); + } + await().atMost(5, SECONDS).pollInterval(200, MILLISECONDS).until(() -> messageStore.dispatchBehindBytes() <= 0); + + when(liteLifecycleManager.isLiteTopicExpired(anyString(), anyString(), anyLong())).thenReturn(false); + List> result = liteLifecycleManager.collectExpiredLiteTopic(); + Assert.assertEquals(0, result.size()); + + when(liteLifecycleManager.isLiteTopicExpired(eq(parentTopic), anyString(), anyLong())).thenReturn(true); + result = liteLifecycleManager.collectExpiredLiteTopic(); + Assert.assertEquals(num, result.size()); + for (Pair pair : result) { + Assert.assertEquals(parentTopic, pair.getObject1()); + Assert.assertTrue(LiteUtil.belongsTo(pair.getObject2(), parentTopic)); + } + } + + @Test + public void testCleanExpiredLiteTopic() throws Exception { + int num = 3; + String parentTopic = UUID.randomUUID().toString(); + List liteTopics = + IntStream.range(0, 3).mapToObj(i -> UUID.randomUUID().toString()).collect(Collectors.toList()); + for (int i = 0; i < num; i++) { + messageStore.putMessage(LiteTestUtil.buildMessage(parentTopic, liteTopics.get(i))); + } + await().atMost(5, SECONDS).pollInterval(200, MILLISECONDS).until(() -> messageStore.dispatchBehindBytes() <= 0); + + for (int i = 0; i < num; i++) { + String lmqName = LiteUtil.toLmqName(parentTopic, liteTopics.get(i)); + Assert.assertEquals(1, (long) messageStore.getQueueStore().getMaxOffset(lmqName, 0)); + Assert.assertEquals(1, liteLifecycleManager.getMaxOffsetInQueue(lmqName)); + } + + when(liteLifecycleManager.isLiteTopicExpired(eq(parentTopic), anyString(), anyLong())).thenReturn(true); + liteLifecycleManager.cleanExpiredLiteTopic(); + + for (int i = 0; i < num; i++) { + String lmqName = LiteUtil.toLmqName(parentTopic, liteTopics.get(i)); + Assert.assertEquals(0, (long) messageStore.getQueueStore().getMaxOffset(lmqName, 0)); + Assert.assertEquals(0, liteLifecycleManager.getMaxOffsetInQueue(lmqName)); + } + } + + @Test + public void testCleanByParentTopic() throws Exception { + int num = 3; + String parentTopic = UUID.randomUUID().toString(); + mockTopicConfig.getAttributes().put( + TopicAttributes.TOPIC_MESSAGE_TYPE_ATTRIBUTE.getName(), TopicMessageType.LITE.getValue()); + List liteTopics = + IntStream.range(0, 3).mapToObj(i -> UUID.randomUUID().toString()).collect(Collectors.toList()); + for (int i = 0; i < num; i++) { + messageStore.putMessage(LiteTestUtil.buildMessage(parentTopic, liteTopics.get(i))); + } + await().atMost(5, SECONDS).pollInterval(200, MILLISECONDS).until(() -> messageStore.dispatchBehindBytes() <= 0); + + for (int i = 0; i < num; i++) { + String lmqName = LiteUtil.toLmqName(parentTopic, liteTopics.get(i)); + Assert.assertEquals(1, (long) messageStore.getQueueStore().getMaxOffset(lmqName, 0)); + Assert.assertEquals(1, liteLifecycleManager.getMaxOffsetInQueue(lmqName)); + } + + liteLifecycleManager.cleanByParentTopic(parentTopic); + + for (int i = 0; i < num; i++) { + String lmqName = LiteUtil.toLmqName(parentTopic, liteTopics.get(i)); + Assert.assertEquals(0, (long) messageStore.getQueueStore().getMaxOffset(lmqName, 0)); + Assert.assertEquals(0, liteLifecycleManager.getMaxOffsetInQueue(lmqName)); + } + } +} diff --git a/broker/src/test/java/org/apache/rocketmq/broker/longpolling/PopLiteLongPollingServiceTest.java b/broker/src/test/java/org/apache/rocketmq/broker/longpolling/PopLiteLongPollingServiceTest.java new file mode 100644 index 00000000000..f0fb2b05040 --- /dev/null +++ b/broker/src/test/java/org/apache/rocketmq/broker/longpolling/PopLiteLongPollingServiceTest.java @@ -0,0 +1,219 @@ +/* + * 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.longpolling; + +import com.googlecode.concurrentlinkedhashmap.ConcurrentLinkedHashMap; +import io.netty.channel.Channel; +import io.netty.channel.ChannelHandlerContext; +import org.apache.commons.lang3.reflect.FieldUtils; +import org.apache.rocketmq.broker.BrokerController; +import org.apache.rocketmq.common.BrokerConfig; +import org.apache.rocketmq.remoting.netty.NettyRequestProcessor; +import org.apache.rocketmq.remoting.protocol.RemotingCommand; +import org.junit.Before; +import org.junit.Test; +import org.junit.runner.RunWith; +import org.mockito.Mock; +import org.mockito.junit.MockitoJUnitRunner; + +import java.util.concurrent.ConcurrentSkipListSet; +import java.util.concurrent.ExecutorService; +import java.util.concurrent.atomic.AtomicLong; + +import static org.junit.Assert.assertEquals; +import static org.junit.Assert.assertFalse; +import static org.junit.Assert.assertTrue; +import static org.mockito.ArgumentMatchers.any; +import static org.mockito.Mockito.mock; +import static org.mockito.Mockito.never; +import static org.mockito.Mockito.verify; +import static org.mockito.Mockito.when; + +@RunWith(MockitoJUnitRunner.class) +public class PopLiteLongPollingServiceTest { + + @Mock + private BrokerController brokerController; + @Mock + private NettyRequestProcessor processor; + @Mock + private ChannelHandlerContext ctx; + @Mock + private ExecutorService pullMessageExecutor; + + private BrokerConfig brokerConfig; + private PopLiteLongPollingService popLiteLongPollingService; + private ConcurrentLinkedHashMap> pollingMap; + private AtomicLong totalPollingNum; + + @SuppressWarnings("unchecked") + @Before + public void init() throws IllegalAccessException { + brokerConfig = new BrokerConfig(); + when(brokerController.getBrokerConfig()).thenReturn(brokerConfig); + when(brokerController.getPullMessageExecutor()).thenReturn(pullMessageExecutor); + popLiteLongPollingService = new PopLiteLongPollingService(brokerController, processor, true); + pollingMap = (ConcurrentLinkedHashMap>) + FieldUtils.readDeclaredField(popLiteLongPollingService, "pollingMap", true); + totalPollingNum = (AtomicLong) FieldUtils.readDeclaredField(popLiteLongPollingService, "totalPollingNum", true); + } + + @Test + public void testNotifyMessageArriving_noRequest() { + assertFalse(popLiteLongPollingService.notifyMessageArriving("clientId", true, 0, "group")); + } + + @Test + public void testNotifyMessageArriving_inactiveChannel() throws Exception { + String clientId = "clientId"; + String group = "group"; + + ChannelHandlerContext ctx = mock(ChannelHandlerContext.class); + RemotingCommand remotingCommand = mock(RemotingCommand.class); + Channel channel = mock(Channel.class); + when(channel.isActive()).thenReturn(false); + when(ctx.channel()).thenReturn(channel); + + PollingResult result = popLiteLongPollingService.polling( + ctx, remotingCommand, System.currentTimeMillis(), 10000, clientId, group); + assertEquals(PollingResult.POLLING_SUC, result); + assertEquals(1, totalPollingNum.get()); + + assertFalse(popLiteLongPollingService.notifyMessageArriving(clientId, true, 0, group)); + assertEquals(0, totalPollingNum.get()); + } + + @Test + public void testNotifyMessageArriving_success() throws Exception { + String clientId = "clientId"; + String group = "group"; + + ChannelHandlerContext ctx = mock(ChannelHandlerContext.class); + RemotingCommand remotingCommand1 = mock(RemotingCommand.class); + RemotingCommand remotingCommand2 = mock(RemotingCommand.class); + Channel channel = mock(Channel.class); + when(channel.isActive()).thenReturn(true); + when(ctx.channel()).thenReturn(channel); + + PollingResult result1 = popLiteLongPollingService.polling( + ctx, remotingCommand1, System.currentTimeMillis(), 10000, clientId, group); + PollingResult result2 = popLiteLongPollingService.polling( + ctx, remotingCommand2, System.currentTimeMillis(), 15000, clientId, group); + + assertEquals(PollingResult.POLLING_SUC, result1); + assertEquals(PollingResult.POLLING_SUC, result2); + assertEquals(2, totalPollingNum.get()); + + assertTrue(popLiteLongPollingService.notifyMessageArriving(clientId, true, 0, group)); + assertEquals(1, totalPollingNum.get()); + assertEquals(remotingCommand1, pollingMap.get(clientId).pollFirst().getRemotingCommand()); // notify last + } + + @Test + public void testWakeUp_nullRequest() { + assertFalse(popLiteLongPollingService.wakeUp(null)); + } + + @Test + public void testWakeUp_completeRequest() { + PopRequest request = mock(PopRequest.class); + when(request.complete()).thenReturn(false); + + assertFalse(popLiteLongPollingService.wakeUp(request)); + } + + @Test + public void testWakeUp_inactiveChannel() { + PopRequest request = mock(PopRequest.class); + when(request.complete()).thenReturn(true); + when(request.getCtx()).thenReturn(ctx); + Channel channel = mock(Channel.class); + when(ctx.channel()).thenReturn(channel); + when(channel.isActive()).thenReturn(false); + + assertFalse(popLiteLongPollingService.wakeUp(request)); + verify(pullMessageExecutor, never()).submit(any(Runnable.class)); + } + + @Test + public void testWakeUp_success() { + PopRequest request = mock(PopRequest.class); + when(request.complete()).thenReturn(true); + when(request.getCtx()).thenReturn(ctx); + Channel channel = mock(Channel.class); + when(ctx.channel()).thenReturn(channel); + when(channel.isActive()).thenReturn(true); + + assertTrue(popLiteLongPollingService.wakeUp(request)); + verify(pullMessageExecutor).submit(any(Runnable.class)); + } + + @Test + public void testPolling_notPolling() { + ChannelHandlerContext ctx = mock(ChannelHandlerContext.class); + RemotingCommand remotingCommand = mock(RemotingCommand.class); + + PollingResult result = popLiteLongPollingService.polling(ctx, remotingCommand, 0, 0, "clientId", "group"); + assertEquals(PollingResult.NOT_POLLING, result); + } + + @Test + public void testPolling_timeout() { + ChannelHandlerContext ctx = mock(ChannelHandlerContext.class); + RemotingCommand remotingCommand = mock(RemotingCommand.class); + + PollingResult result = + popLiteLongPollingService.polling(ctx, remotingCommand, System.currentTimeMillis(), 40, "clientId", "group"); + assertEquals(PollingResult.POLLING_TIMEOUT, result); + } + + @Test + public void testPolling_success() { + ChannelHandlerContext ctx = mock(ChannelHandlerContext.class); + RemotingCommand remotingCommand = mock(RemotingCommand.class); + + PollingResult result = popLiteLongPollingService.polling( + ctx, remotingCommand, System.currentTimeMillis(), 10000, "clientId", "group"); + assertEquals(PollingResult.POLLING_SUC, result); + } + + @Test + public void testPolling_totalPollingFull() { + ChannelHandlerContext ctx = mock(ChannelHandlerContext.class); + RemotingCommand remotingCommand = mock(RemotingCommand.class); + totalPollingNum.set(brokerConfig.getMaxPopPollingSize() + 1); + + PollingResult result = popLiteLongPollingService.polling( + ctx, remotingCommand, System.currentTimeMillis(), 10000, "clientId", "group"); + assertEquals(PollingResult.POLLING_FULL, result); + } + + @Test + public void testPolling_singlePollingFull() { + ChannelHandlerContext ctx = mock(ChannelHandlerContext.class); + RemotingCommand remotingCommand = mock(RemotingCommand.class); + brokerConfig.setPopPollingSize(-1); + + PollingResult result = popLiteLongPollingService.polling( + ctx, remotingCommand, System.currentTimeMillis(), 10000, "clientId", "group"); + assertEquals(PollingResult.POLLING_SUC, result); + + result = popLiteLongPollingService.polling( + ctx, remotingCommand, System.currentTimeMillis(), 10000, "clientId", "group"); + assertEquals(PollingResult.POLLING_FULL, result); + } +} diff --git a/broker/src/test/java/org/apache/rocketmq/broker/metrics/LiteConsumerLagCalculatorTest.java b/broker/src/test/java/org/apache/rocketmq/broker/metrics/LiteConsumerLagCalculatorTest.java new file mode 100644 index 00000000000..732ca7dfbd4 --- /dev/null +++ b/broker/src/test/java/org/apache/rocketmq/broker/metrics/LiteConsumerLagCalculatorTest.java @@ -0,0 +1,405 @@ +/* + * 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.metrics; + +import java.util.ArrayList; +import java.util.Collections; +import java.util.List; +import java.util.concurrent.ConcurrentHashMap; +import java.util.concurrent.ConcurrentMap; +import java.util.concurrent.PriorityBlockingQueue; +import org.apache.rocketmq.broker.BrokerController; +import org.apache.rocketmq.broker.offset.ConsumerOffsetManager; +import org.apache.rocketmq.common.BrokerConfig; +import org.apache.rocketmq.common.Pair; +import org.apache.rocketmq.common.entity.TopicGroup; +import org.apache.rocketmq.common.lite.LiteLagInfo; +import org.apache.rocketmq.common.lite.LiteUtil; +import org.junit.Before; +import org.junit.Test; +import org.junit.runner.RunWith; +import org.mockito.Mock; +import org.mockito.junit.MockitoJUnitRunner; + +import static org.junit.Assert.assertEquals; +import static org.junit.Assert.assertFalse; +import static org.junit.Assert.assertNotNull; +import static org.junit.Assert.assertTrue; +import static org.mockito.Mockito.doReturn; +import static org.mockito.Mockito.spy; +import static org.mockito.Mockito.when; + +@RunWith(MockitoJUnitRunner.class) +public class LiteConsumerLagCalculatorTest { + + private LiteConsumerLagCalculator liteConsumerLagCalculator; + + @Mock + private BrokerController brokerController; + + @Mock + private ConsumerOffsetManager consumerOffsetManager; + + private final BrokerConfig brokerConfig = new BrokerConfig(); + + @Before + public void setUp() { + when(brokerController.getBrokerConfig()).thenReturn(brokerConfig); + when(brokerController.getConsumerOffsetManager()).thenReturn(consumerOffsetManager); + + liteConsumerLagCalculator = new LiteConsumerLagCalculator(brokerController); + } + + @Test + public void testUpdateLagInfo() { + String group = "testGroup"; + String topic = "testTopic"; + String lmqName = LiteUtil.toLmqName(topic, "lmq1"); + long storeTimestamp = System.currentTimeMillis(); + + liteConsumerLagCalculator.updateLagInfo(group, topic, lmqName, storeTimestamp); + + TopicGroup topicGroup = new TopicGroup(topic, group); + PriorityBlockingQueue lagHeap = + liteConsumerLagCalculator.topicGroupLagTimeMap.get(topicGroup); + assertNotNull(lagHeap); + assertEquals(1, lagHeap.size()); + LiteConsumerLagCalculator.LagTimeInfo lagInfo = lagHeap.peek(); + assertNotNull(lagInfo); + assertEquals(lmqName, lagInfo.getLmqName()); + assertEquals(storeTimestamp, lagInfo.getLagTimestamp()); + } + + @Test + public void testUpdateLagInfo_KeepSmallestWhenExceedsCapacity() { + String group = "testGroup"; + String topic = "testTopic"; + + // Set topK to 3, so the heap will retain at most 3 elements + brokerConfig.setLiteLagLatencyTopK(3); + + // Add 5 elements with timestamps 1000, 2000, 3000, 4000, 5000 + // Expected result is to retain the smallest 3: 1000, 2000, 3000 + liteConsumerLagCalculator.updateLagInfo(group, topic, + LiteUtil.toLmqName(topic, "lmq1"), 3000L); + liteConsumerLagCalculator.updateLagInfo(group, topic, + LiteUtil.toLmqName(topic, "lmq2"), 1000L); + liteConsumerLagCalculator.updateLagInfo(group, topic, + LiteUtil.toLmqName(topic, "lmq3"), 5000L); + liteConsumerLagCalculator.updateLagInfo(group, topic, + LiteUtil.toLmqName(topic, "lmq4"), 2000L); + liteConsumerLagCalculator.updateLagInfo(group, topic, + LiteUtil.toLmqName(topic, "lmq5"), 4000L); + + // Verify that the heap contains only 3 elements + TopicGroup topicGroup = new TopicGroup(topic, group); + PriorityBlockingQueue lagHeap = + liteConsumerLagCalculator.topicGroupLagTimeMap.get(topicGroup); + assertNotNull(lagHeap); + assertEquals(3, lagHeap.size()); + + // Verify that the retained elements have the smallest timestamps: 1000, 2000, 3000 + List timestamps = new ArrayList<>(); + for (LiteConsumerLagCalculator.LagTimeInfo info : lagHeap) { + timestamps.add(info.getLagTimestamp()); + } + Collections.sort(timestamps); + assertEquals(3, timestamps.size()); + assertEquals(1000L, timestamps.get(0).longValue()); + assertEquals(2000L, timestamps.get(1).longValue()); + assertEquals(3000L, timestamps.get(2).longValue()); + } + + @Test + public void testRemoveLagInfo() { + String group = "testGroup"; + String topic = "testTopic"; + String lmqName = LiteUtil.toLmqName(topic, "lmq1"); + long storeTimestamp = System.currentTimeMillis(); + + liteConsumerLagCalculator.updateLagInfo(group, topic, lmqName, storeTimestamp); + liteConsumerLagCalculator.removeLagInfo(group, topic, lmqName); + + TopicGroup topicGroup = new TopicGroup(topic, group); + PriorityBlockingQueue lagHeap = + liteConsumerLagCalculator.topicGroupLagTimeMap.get(topicGroup); + assertTrue(lagHeap.isEmpty()); + } + + @Test + public void testOffsetTableForEachByGroup() { + String testTopic = "testTopic"; + String liteTopic = "lmq1"; + String testGroup = "testGroup"; + String otherGroup = "otherGroup"; + String lmqName = LiteUtil.toLmqName(testTopic, liteTopic); + String key = lmqName + "@" + testGroup; + + // Prepare test data without thread-safe classes + ConcurrentMap> offsetTable = new ConcurrentHashMap<>(); + ConcurrentMap queueOffsetMap = new ConcurrentHashMap<>(); + queueOffsetMap.put(0, 100L); + offsetTable.put(key, queueOffsetMap); + + when(consumerOffsetManager.getOffsetTable()).thenReturn(offsetTable); + + // Test processing all groups + final boolean[] processed = {false}; + liteConsumerLagCalculator.offsetTableForEachByGroup(null, (topicGroup, offset) -> { + processed[0] = true; + assertEquals(lmqName, topicGroup.topic); + assertEquals(testGroup, topicGroup.group); + assertEquals(Long.valueOf(100L), offset); + }); + assertTrue(processed[0]); + + // Test processing specific group + processed[0] = false; + liteConsumerLagCalculator.offsetTableForEachByGroup(testGroup, (topicGroup, offset) -> { + processed[0] = true; + assertEquals(lmqName, topicGroup.topic); + assertEquals(testGroup, topicGroup.group); + assertEquals(Long.valueOf(100L), offset); + }); + assertTrue(processed[0]); + + // Test processing non-matching group + processed[0] = false; + liteConsumerLagCalculator.offsetTableForEachByGroup(otherGroup, + (topicGroup, offset) -> processed[0] = true); + assertFalse(processed[0]); + } + + @Test + public void testGetLagTimestampTopK_NormalCase() { + // Prepare test data + String group = "testGroup"; + String parentTopic = "testParentTopic"; + String lmq1 = LiteUtil.toLmqName(parentTopic, "lmq1"); + String lmq2 = LiteUtil.toLmqName(parentTopic, "lmq2"); + String lmq3 = LiteUtil.toLmqName(parentTopic, "lmq3"); + + long timestamp1 = 1000L; + long timestamp2 = 2000L; + long timestamp3 = 1500L; + + // Consumer offsets + long consumerOffset1 = 50L; +// long consumerOffset2 = 30L; + long consumerOffset3 = 40L; + + // Max offsets + long maxOffset1 = 100L; +// long maxOffset2 = 80L; + long maxOffset3 = 90L; + + // Create a spy of the calculator to allow partial mocking + LiteConsumerLagCalculator spyCalculator = spy(liteConsumerLagCalculator); + + // Add lag info to the spy calculator + spyCalculator.updateLagInfo(group, parentTopic, lmq1, timestamp1); + spyCalculator.updateLagInfo(group, parentTopic, lmq2, timestamp2); + spyCalculator.updateLagInfo(group, parentTopic, lmq3, timestamp3); + + // Mock getOffset and getMaxOffset methods on the spy + doReturn(consumerOffset1).when(spyCalculator).getOffset(group, lmq1); +// doReturn(consumerOffset2).when(spyCalculator).getOffset(group, lmq2); + doReturn(consumerOffset3).when(spyCalculator).getOffset(group, lmq3); + + doReturn(maxOffset1).when(spyCalculator).getMaxOffset(lmq1); +// doReturn(maxOffset2).when(spyCalculator).getMaxOffset(lmq2); + doReturn(maxOffset3).when(spyCalculator).getMaxOffset(lmq3); + + // Test with topK = 2 + Pair, Long> result = spyCalculator.getLagTimestampTopK(group, parentTopic, 2); + + // Verify results + assertNotNull(result); + assertEquals(2, result.getObject1().size()); + + // Should be sorted by timestamp in ascending order + assertEquals(timestamp1, result.getObject1().get(0).getEarliestUnconsumedTimestamp()); + assertEquals(timestamp3, result.getObject1().get(1).getEarliestUnconsumedTimestamp()); + + // Verify lag counts (maxOffset - consumerOffset) + assertEquals(maxOffset1 - consumerOffset1, result.getObject1().get(0).getLagCount()); + assertEquals(maxOffset3 - consumerOffset3, result.getObject1().get(1).getLagCount()); + + // Verify lite topics + assertEquals("lmq1", result.getObject1().get(0).getLiteTopic()); + assertEquals("lmq3", result.getObject1().get(1).getLiteTopic()); + + // Verify earliest timestamp + assertEquals(timestamp1, result.getObject2().longValue()); + } + + @Test + public void testGetLagCountTopK_NormalCase() { + String group = "testGroup"; + String topic = "testTopic"; + String lmqName1 = LiteUtil.toLmqName(topic, "lmq1"); + String lmqName2 = LiteUtil.toLmqName(topic, "lmq2"); + String lmqName3 = LiteUtil.toLmqName(topic, "lmq3"); + + // Prepare offset table data + ConcurrentMap> offsetTable = new ConcurrentHashMap<>(); + ConcurrentMap queueOffsetMap1 = new ConcurrentHashMap<>(); + ConcurrentMap queueOffsetMap2 = new ConcurrentHashMap<>(); + ConcurrentMap queueOffsetMap3 = new ConcurrentHashMap<>(); + + long consumerOffset1 = 50L; + long consumerOffset2 = 30L; + long consumerOffset3 = 70L; + + queueOffsetMap1.put(0, consumerOffset1); + queueOffsetMap2.put(0, consumerOffset2); + queueOffsetMap3.put(0, consumerOffset3); + + offsetTable.put(lmqName1 + "@" + group, queueOffsetMap1); + offsetTable.put(lmqName2 + "@" + group, queueOffsetMap2); + offsetTable.put(lmqName3 + "@" + group, queueOffsetMap3); + + when(consumerOffsetManager.getOffsetTable()).thenReturn(offsetTable); + + // Mock store timestamps + long timestamp1 = 1000L; + long timestamp2 = 2000L; + long timestamp3 = 1500L; + + // Create a spy of the calculator to allow partial mocking + LiteConsumerLagCalculator spyCalculator = spy(liteConsumerLagCalculator); + + // Mock getStoreTimestamp method on the spy + doReturn(timestamp1).when(spyCalculator).getStoreTimestamp(lmqName1, consumerOffset1); + doReturn(timestamp2).when(spyCalculator).getStoreTimestamp(lmqName2, consumerOffset2); + doReturn(timestamp3).when(spyCalculator).getStoreTimestamp(lmqName3, consumerOffset3); + + // Mock getMaxOffset method on the spy + doReturn(100L).when(spyCalculator).getMaxOffset(lmqName1); + doReturn(80L).when(spyCalculator).getMaxOffset(lmqName2); + doReturn(90L).when(spyCalculator).getMaxOffset(lmqName3); + + // Test with topK = 2 + Pair, Long> result = spyCalculator.getLagCountTopK(group, 2); + + // Verify results + assertNotNull(result); + assertNotNull(result.getObject1()); + assertEquals(2, result.getObject1().size()); + + // Should be sorted by lag count in descending order + // lmq1: 100-50=50, lmq2: 80-30=50, lmq3: 90-70=20 + // So order should be lmq1(50), lmq2(50) or lmq2(50), lmq1(50) (both have same lag count) + LiteLagInfo first = result.getObject1().get(0); + LiteLagInfo second = result.getObject1().get(1); + + // Verify lag counts + assertEquals(50L, first.getLagCount()); + assertEquals(50L, second.getLagCount()); + + // Verify lite topics + assertTrue(first.getLiteTopic().equals("lmq1") || first.getLiteTopic().equals("lmq2")); + assertTrue(second.getLiteTopic().equals("lmq1") || second.getLiteTopic().equals("lmq2")); + + // Verify timestamps + assertTrue(first.getEarliestUnconsumedTimestamp() == timestamp1 || first.getEarliestUnconsumedTimestamp() == timestamp2); + assertTrue(second.getEarliestUnconsumedTimestamp() == timestamp1 || second.getEarliestUnconsumedTimestamp() == timestamp2); + + // Verify total lag count + assertEquals(120L, result.getObject2().longValue()); // 50 + 50 + 20 + } + + @Test + public void testCalculateLiteLagCount() { + brokerConfig.setLiteLagCountMetricsEnable(true); + + String group = "testGroup"; + String parentTopic = "testParentTopic"; + String lmqName = LiteUtil.toLmqName(parentTopic, "lmq1"); + + ConcurrentMap> offsetTable = new ConcurrentHashMap<>(); + ConcurrentMap queueOffsetMap = new ConcurrentHashMap<>(); + queueOffsetMap.put(0, 50L); + offsetTable.put(lmqName + "@" + group, queueOffsetMap); + + when(consumerOffsetManager.getOffsetTable()).thenReturn(offsetTable); + + LiteConsumerLagCalculator spyCalculator = spy(liteConsumerLagCalculator); + doReturn(100L).when(spyCalculator).getMaxOffset(lmqName); + + final ConsumerLagCalculator.CalculateLagResult[] result = {null}; + spyCalculator.calculateLiteLagCount(lagResult -> result[0] = lagResult); + + assertNotNull(result[0]); + assertEquals(group, result[0].group); + // The metrics of liteTopic are aggregated under its parent topic + assertEquals(parentTopic, result[0].topic); + assertEquals(50L, result[0].lag); + } + + @Test + public void testCalculateLiteLagLatency() { + brokerConfig.setLiteLagLatencyMetricsEnable(true); + + String group = "testGroup"; + String parentTopic = "testParentTopic"; + String lmqName = LiteUtil.toLmqName(parentTopic, "lmq1"); + long storeTimestamp = System.currentTimeMillis(); + + liteConsumerLagCalculator.updateLagInfo(group, parentTopic, lmqName, storeTimestamp); + + final ConsumerLagCalculator.CalculateLagResult[] result = {null}; + liteConsumerLagCalculator.calculateLiteLagLatency(lagResult -> result[0] = lagResult); + + assertNotNull(result[0]); + assertEquals(group, result[0].group); + // The metrics of liteTopic are aggregated under its parent topic + assertEquals(parentTopic, result[0].topic); + assertEquals(storeTimestamp, result[0].earliestUnconsumedTimestamp); + } + + @Test + public void testUpdateLagInfoWithDuplicateElements() { + String group = "testGroup"; + String parentTopic = "testParentTopic"; + String lmqName1 = "lmq1"; + String lmqName2 = "lmq2"; + String lmqName3 = "lmq3"; + long storeTimestamp1 = 1000L; + long storeTimestamp2 = 2000L; + long storeTimestamp3 = 3000L; + + // Add three LMQs with different timestamps, each added three times + for (int i = 0; i < 3; i++) { + liteConsumerLagCalculator.updateLagInfo(group, parentTopic, lmqName1, storeTimestamp1 + i * 100); + liteConsumerLagCalculator.updateLagInfo(group, parentTopic, lmqName2, storeTimestamp2 + i * 100); + liteConsumerLagCalculator.updateLagInfo(group, parentTopic, lmqName3, storeTimestamp3 + i * 100); + } + + // Verify that the heap contains exactly 3 elements + PriorityBlockingQueue lagHeap = liteConsumerLagCalculator.topicGroupLagTimeMap + .get(new TopicGroup(parentTopic, group)); + assertNotNull(lagHeap); + assertEquals(3, lagHeap.size()); + + // Verify that each LMQ is present with its latest timestamp + assertTrue(lagHeap.contains(new LiteConsumerLagCalculator.LagTimeInfo(lmqName1, storeTimestamp1 + 200))); + assertTrue(lagHeap.contains(new LiteConsumerLagCalculator.LagTimeInfo(lmqName2, storeTimestamp2 + 200))); + assertTrue(lagHeap.contains(new LiteConsumerLagCalculator.LagTimeInfo(lmqName3, storeTimestamp3 + 200))); + } +} diff --git a/broker/src/test/java/org/apache/rocketmq/broker/offset/RocksDBConsumerOffsetManagerTest.java b/broker/src/test/java/org/apache/rocketmq/broker/offset/RocksDBConsumerOffsetManagerTest.java index 1227d339bd1..28476c7e1b0 100644 --- a/broker/src/test/java/org/apache/rocketmq/broker/offset/RocksDBConsumerOffsetManagerTest.java +++ b/broker/src/test/java/org/apache/rocketmq/broker/offset/RocksDBConsumerOffsetManagerTest.java @@ -17,84 +17,83 @@ package org.apache.rocketmq.broker.offset; +import java.io.File; +import java.util.UUID; import java.util.concurrent.ConcurrentHashMap; import java.util.concurrent.ConcurrentMap; import org.apache.rocketmq.broker.BrokerController; import org.apache.rocketmq.broker.config.v1.RocksDBConsumerOffsetManager; import org.apache.rocketmq.common.BrokerConfig; +import org.apache.rocketmq.common.MixAll; +import org.apache.rocketmq.common.UtilAll; import org.apache.rocketmq.store.config.MessageStoreConfig; import org.junit.After; import org.junit.Assert; +import org.junit.Assume; import org.junit.Before; import org.junit.Test; import org.mockito.Mockito; +import static org.apache.rocketmq.broker.offset.ConsumerOffsetManager.TOPIC_GROUP_SEPARATOR; import static org.assertj.core.api.Assertions.assertThat; public class RocksDBConsumerOffsetManagerTest { + private static final String SKIP_MAC_KEY = "skipMac"; + private static final String KEY = "FooBar@FooBarGroup"; private BrokerController brokerController; private ConsumerOffsetManager consumerOffsetManager; + private BrokerConfig brokerConfig; + @Before - @SuppressWarnings("DoubleBraceInitialization") public void init() { - if (notToBeExecuted()) { - return; - } +// System.setProperty(SKIP_MAC_KEY, "false"); + skipMacIfNecessary(); brokerController = Mockito.mock(BrokerController.class); + brokerConfig = new BrokerConfig(); MessageStoreConfig messageStoreConfig = new MessageStoreConfig(); Mockito.when(brokerController.getMessageStoreConfig()).thenReturn(messageStoreConfig); - Mockito.when(brokerController.getBrokerConfig()).thenReturn(new BrokerConfig()); + Mockito.when(brokerController.getBrokerConfig()).thenReturn(brokerConfig); consumerOffsetManager = new RocksDBConsumerOffsetManager(brokerController); consumerOffsetManager.load(); ConcurrentHashMap> offsetTable = new ConcurrentHashMap<>(512); - offsetTable.put(KEY,new ConcurrentHashMap() {{ - put(1,2L); - put(2,3L); - }}); + ConcurrentHashMap innerMap = new ConcurrentHashMap<>(); + innerMap.put(1, 2L); + innerMap.put(2, 3L); + offsetTable.put(KEY, innerMap); consumerOffsetManager.setOffsetTable(offsetTable); } @After public void destroy() { - if (notToBeExecuted()) { - return; - } if (consumerOffsetManager != null) { consumerOffsetManager.stop(); + File file = new File(((RocksDBConsumerOffsetManager) consumerOffsetManager).rocksdbConfigFilePath(null, false)); + UtilAll.deleteFile(file); } } @Test public void cleanOffsetByTopic_NotExist() { - if (notToBeExecuted()) { - return; - } consumerOffsetManager.cleanOffsetByTopic("InvalidTopic"); assertThat(consumerOffsetManager.getOffsetTable().containsKey(KEY)).isTrue(); } @Test public void cleanOffsetByTopic_Exist() { - if (notToBeExecuted()) { - return; - } consumerOffsetManager.cleanOffsetByTopic("FooBar"); assertThat(!consumerOffsetManager.getOffsetTable().containsKey(KEY)).isTrue(); } @Test public void testOffsetPersistInMemory() { - if (notToBeExecuted()) { - return; - } ConcurrentMap> offsetTable = consumerOffsetManager.getOffsetTable(); ConcurrentMap table = new ConcurrentHashMap<>(); table.put(0, 1L); @@ -110,7 +109,176 @@ public void testOffsetPersistInMemory() { Assert.assertEquals(table, offsetTableLoaded); } - private boolean notToBeExecuted() { - return false; + @Test + public void testCommitOffset_persist_periodically() { + brokerConfig.setPersistConsumerOffsetIncrementally(false); + String group = UUID.randomUUID().toString(); + String topic = UUID.randomUUID().toString(); + String key = topic + TOPIC_GROUP_SEPARATOR + group; + + // 1. commit but not persist + Assert.assertFalse(consumerOffsetManager.getOffsetTable().containsKey(key)); + consumerOffsetManager.commitOffset("ClientID", group, topic, 0, 1); + Assert.assertTrue(consumerOffsetManager.getOffsetTable().containsKey(key)); + + consumerOffsetManager.stop(); + consumerOffsetManager.getOffsetTable().clear(); + consumerOffsetManager.load(); + Assert.assertFalse(consumerOffsetManager.getOffsetTable().containsKey(key)); // not in kv + + // 2. commit and persist + consumerOffsetManager.commitOffset("ClientID", group, topic, 0, 1); + Assert.assertTrue(consumerOffsetManager.getOffsetTable().containsKey(key)); + consumerOffsetManager.persist(); + consumerOffsetManager.stop(); + consumerOffsetManager.getOffsetTable().clear(); + consumerOffsetManager.load(); + Assert.assertTrue(consumerOffsetManager.getOffsetTable().containsKey(key)); // load from kv + } + + @Test + public void testCommitOffset_persist_incrementally() { + brokerConfig.setPersistConsumerOffsetIncrementally(true); + String group = UUID.randomUUID().toString(); + String topic = UUID.randomUUID().toString(); + String key = topic + TOPIC_GROUP_SEPARATOR + group; + + // commit but not persist + Assert.assertFalse(consumerOffsetManager.getOffsetTable().containsKey(key)); + consumerOffsetManager.commitOffset("ClientID", group, topic, 0, 1); + Assert.assertTrue(consumerOffsetManager.getOffsetTable().containsKey(key)); + + consumerOffsetManager.stop(); + consumerOffsetManager.getOffsetTable().clear(); + consumerOffsetManager.load(); + Assert.assertTrue(consumerOffsetManager.getOffsetTable().containsKey(key)); // reload from kv + } + + @Test + public void testRemoveConsumerOffset() { + String group = UUID.randomUUID().toString(); + String topic = UUID.randomUUID().toString(); + String key = topic + TOPIC_GROUP_SEPARATOR + group; + + // commit and persist + Assert.assertFalse(consumerOffsetManager.getOffsetTable().containsKey(key)); + consumerOffsetManager.commitOffset("ClientID", group, topic, 0, 1); + Assert.assertTrue(consumerOffsetManager.getOffsetTable().containsKey(key)); + consumerOffsetManager.persist(); + + consumerOffsetManager.removeConsumerOffset(key); + consumerOffsetManager.stop(); + consumerOffsetManager.getOffsetTable().clear(); + consumerOffsetManager.load(); + Assert.assertFalse(consumerOffsetManager.getOffsetTable().containsKey(key)); // removed from kv + } + + @Test + public void testRemoveOffset() { + String group = UUID.randomUUID().toString(); + String topic1 = UUID.randomUUID().toString(); + String topic2 = UUID.randomUUID().toString(); + String key1 = topic1 + TOPIC_GROUP_SEPARATOR + group; + String key2 = topic2 + TOPIC_GROUP_SEPARATOR + group; + + // commit and persist + Assert.assertFalse(consumerOffsetManager.getOffsetTable().containsKey(key1)); + Assert.assertFalse(consumerOffsetManager.getOffsetTable().containsKey(key2)); + consumerOffsetManager.commitOffset("ClientID", group, topic1, 0, 1); + consumerOffsetManager.commitOffset("ClientID", group, topic2, 0, 1); + Assert.assertTrue(consumerOffsetManager.getOffsetTable().containsKey(key1)); + Assert.assertTrue(consumerOffsetManager.getOffsetTable().containsKey(key2)); + consumerOffsetManager.persist(); + + // remove all offsets by group + consumerOffsetManager.removeOffset(group); + Assert.assertFalse(consumerOffsetManager.getOffsetTable().containsKey(key1)); + Assert.assertFalse(consumerOffsetManager.getOffsetTable().containsKey(key2)); + consumerOffsetManager.stop(); + consumerOffsetManager.getOffsetTable().clear(); + consumerOffsetManager.load(); + Assert.assertFalse(consumerOffsetManager.getOffsetTable().containsKey(key1)); // removed from kv + Assert.assertFalse(consumerOffsetManager.getOffsetTable().containsKey(key2)); // removed from kv + } + + @Test + // similar to testRemoveOffset() + public void testCleanOffset() { + String group = UUID.randomUUID().toString(); + String topic1 = UUID.randomUUID().toString(); + String topic2 = UUID.randomUUID().toString(); + String key1 = topic1 + TOPIC_GROUP_SEPARATOR + group; + String key2 = topic2 + TOPIC_GROUP_SEPARATOR + group; + + // commit and persist + Assert.assertFalse(consumerOffsetManager.getOffsetTable().containsKey(key1)); + Assert.assertFalse(consumerOffsetManager.getOffsetTable().containsKey(key2)); + consumerOffsetManager.commitOffset("ClientID", group, topic1, 0, 1); + consumerOffsetManager.commitOffset("ClientID", group, topic2, 0, 1); + Assert.assertTrue(consumerOffsetManager.getOffsetTable().containsKey(key1)); + Assert.assertTrue(consumerOffsetManager.getOffsetTable().containsKey(key2)); + consumerOffsetManager.persist(); + + // remove all offsets by group + consumerOffsetManager.cleanOffset(group); + Assert.assertFalse(consumerOffsetManager.getOffsetTable().containsKey(key1)); + Assert.assertFalse(consumerOffsetManager.getOffsetTable().containsKey(key2)); + consumerOffsetManager.stop(); + consumerOffsetManager.getOffsetTable().clear(); + consumerOffsetManager.load(); + Assert.assertFalse(consumerOffsetManager.getOffsetTable().containsKey(key1)); // removed from kv + Assert.assertFalse(consumerOffsetManager.getOffsetTable().containsKey(key2)); // removed from kv + } + + @Test + public void testCleanOffsetByTopic() { + String group1 = UUID.randomUUID().toString(); + String group2 = UUID.randomUUID().toString(); + String topic = UUID.randomUUID().toString(); + String key1 = topic + TOPIC_GROUP_SEPARATOR + group1; + String key2 = topic + TOPIC_GROUP_SEPARATOR + group2; + + // commit and persist + Assert.assertFalse(consumerOffsetManager.getOffsetTable().containsKey(key1)); + Assert.assertFalse(consumerOffsetManager.getOffsetTable().containsKey(key2)); + consumerOffsetManager.commitOffset("ClientID", group1, topic, 0, 1); + consumerOffsetManager.commitOffset("ClientID", group2, topic, 0, 1); + Assert.assertTrue(consumerOffsetManager.getOffsetTable().containsKey(key1)); + Assert.assertTrue(consumerOffsetManager.getOffsetTable().containsKey(key2)); + consumerOffsetManager.persist(); + + // remove all offsets by group + consumerOffsetManager.cleanOffsetByTopic(topic); + Assert.assertFalse(consumerOffsetManager.getOffsetTable().containsKey(key1)); + Assert.assertFalse(consumerOffsetManager.getOffsetTable().containsKey(key2)); + consumerOffsetManager.stop(); + consumerOffsetManager.getOffsetTable().clear(); + consumerOffsetManager.load(); + Assert.assertFalse(consumerOffsetManager.getOffsetTable().containsKey(key1)); // removed from kv + Assert.assertFalse(consumerOffsetManager.getOffsetTable().containsKey(key2)); // removed from kv + } + + @Test + public void testUpdateDataVersion() { + Assert.assertEquals(0, consumerOffsetManager.getDataVersion().getCounter().get()); + for (int i = 0; i < 10; i++) { + ((RocksDBConsumerOffsetManager) consumerOffsetManager).updateDataVersion(); + } + Assert.assertEquals(10, consumerOffsetManager.getDataVersion().getCounter().get()); + } + + @Test + public void testLoadDataVersion() { + for (int i = 0; i < 10; i++) { + ((RocksDBConsumerOffsetManager) consumerOffsetManager).updateDataVersion(); + } + consumerOffsetManager.stop(); + consumerOffsetManager.load(); + Assert.assertEquals(10, consumerOffsetManager.getDataVersion().getCounter().get()); + } + + private static void skipMacIfNecessary() { + boolean skipMac = Boolean.parseBoolean(System.getProperty(SKIP_MAC_KEY, "true")); + Assume.assumeFalse(MixAll.isMac() && skipMac); } -} +} \ No newline at end of file diff --git a/broker/src/test/java/org/apache/rocketmq/broker/offset/RocksDBOffsetSerializeWrapperTest.java b/broker/src/test/java/org/apache/rocketmq/broker/offset/RocksDBOffsetSerializeWrapperTest.java index c01e63f31f7..5ef6cf00f85 100644 --- a/broker/src/test/java/org/apache/rocketmq/broker/offset/RocksDBOffsetSerializeWrapperTest.java +++ b/broker/src/test/java/org/apache/rocketmq/broker/offset/RocksDBOffsetSerializeWrapperTest.java @@ -26,7 +26,6 @@ import org.junit.Test; import static org.junit.Assert.assertEquals; -import static org.junit.Assert.assertNotNull; public class RocksDBOffsetSerializeWrapperTest { @@ -37,12 +36,6 @@ public void setUp() { wrapper = new RocksDBOffsetSerializeWrapper(); } - @Test - public void testGetOffsetTable_ShouldReturnConcurrentHashMap() { - ConcurrentMap offsetTable = wrapper.getOffsetTable(); - assertNotNull("The offsetTable should not be null", offsetTable); - } - @Test public void testSetOffsetTable_ShouldSetTheOffsetTableCorrectly() { ConcurrentMap newOffsetTable = new ConcurrentHashMap<>(); diff --git a/broker/src/test/java/org/apache/rocketmq/broker/pop/orderly/ConsumerOrderInfoManagerTest.java b/broker/src/test/java/org/apache/rocketmq/broker/pop/orderly/ConsumerOrderInfoManagerTest.java index 7ab3c921edd..a5a5dfc2357 100644 --- a/broker/src/test/java/org/apache/rocketmq/broker/pop/orderly/ConsumerOrderInfoManagerTest.java +++ b/broker/src/test/java/org/apache/rocketmq/broker/pop/orderly/ConsumerOrderInfoManagerTest.java @@ -18,6 +18,10 @@ package org.apache.rocketmq.broker.pop.orderly; import java.time.Duration; +import java.util.ArrayList; +import java.util.Arrays; +import java.util.HashMap; +import java.util.List; import java.util.Map; import java.util.UUID; import java.util.concurrent.ConcurrentHashMap; @@ -530,4 +534,56 @@ public void testReentrant() { assertTrue(consumerOrderInfoManager.checkBlock(null, TOPIC, GROUP, QUEUE_ID_0, 3000)); assertFalse(consumerOrderInfoManager.checkBlock(attemptId, TOPIC, GROUP, QUEUE_ID_0, 3000)); } + + @Test + public void testGetMaxLockFreeTimestamp() { + QueueLevelConsumerManager.OrderInfo orderInfo = new QueueLevelConsumerManager.OrderInfo(); + orderInfo.setOffsetList(new ArrayList<>()); + assertNull(orderInfo.getMaxLockFreeTimestamp()); + + QueueLevelConsumerManager.OrderInfo nullOrderInfo = new QueueLevelConsumerManager.OrderInfo(); + nullOrderInfo.setOffsetList(null); + assertNull(nullOrderInfo.getMaxLockFreeTimestamp()); + + List offsetList = Arrays.asList(100L, 1L, 2L); + + QueueLevelConsumerManager.OrderInfo allAckOrderInfo = new QueueLevelConsumerManager.OrderInfo(); + allAckOrderInfo.setOffsetList(offsetList); + allAckOrderInfo.setCommitOffsetBit(7); + allAckOrderInfo.setPopTime(System.currentTimeMillis()); + allAckOrderInfo.setInvisibleTime(30000L); + assertEquals(System.currentTimeMillis(), allAckOrderInfo.getMaxLockFreeTimestamp(), 1000L); + + QueueLevelConsumerManager.OrderInfo unackOrderInfo = new QueueLevelConsumerManager.OrderInfo(); + unackOrderInfo.setOffsetList(offsetList); + unackOrderInfo.setCommitOffsetBit(0); + long popTime = System.currentTimeMillis(); + unackOrderInfo.setPopTime(popTime); + unackOrderInfo.setInvisibleTime(30000L); + Long expectedTime = popTime + 30000L; + assertEquals(expectedTime, unackOrderInfo.getMaxLockFreeTimestamp()); + + QueueLevelConsumerManager.OrderInfo hasVisibleButAckedOrderInfo = new QueueLevelConsumerManager.OrderInfo(); + hasVisibleButAckedOrderInfo.setOffsetList(offsetList); + hasVisibleButAckedOrderInfo.setCommitOffsetBit(1); + hasVisibleButAckedOrderInfo.setPopTime(popTime); + hasVisibleButAckedOrderInfo.setInvisibleTime(30000L); + Map offsetNextVisibleTime = new HashMap<>(); + offsetNextVisibleTime.put(100L, popTime + 60000L); + hasVisibleButAckedOrderInfo.setOffsetNextVisibleTime(offsetNextVisibleTime); + assertEquals(Long.valueOf(popTime + 30000L), hasVisibleButAckedOrderInfo.getMaxLockFreeTimestamp()); + + QueueLevelConsumerManager.OrderInfo multiUnackOrderInfo = new QueueLevelConsumerManager.OrderInfo(); + multiUnackOrderInfo.setOffsetList(offsetList); + multiUnackOrderInfo.setCommitOffsetBit(0); + multiUnackOrderInfo.setPopTime(popTime); + multiUnackOrderInfo.setInvisibleTime(30000L); + Map multiOffsetNextVisibleTime = new HashMap<>(); + multiOffsetNextVisibleTime.put(100L, popTime + 20000L); + multiOffsetNextVisibleTime.put(101L, popTime + 40000L); + multiOffsetNextVisibleTime.put(102L, popTime + 60000L); + multiUnackOrderInfo.setOffsetNextVisibleTime(multiOffsetNextVisibleTime); + assertEquals(Long.valueOf(popTime + 60000L), multiUnackOrderInfo.getMaxLockFreeTimestamp()); + } + } 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 572be63e3f6..656c783e1f4 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 @@ -41,6 +41,7 @@ import org.apache.rocketmq.broker.config.v1.RocksDBSubscriptionGroupManager; import org.apache.rocketmq.broker.config.v1.RocksDBTopicConfigManager; import org.apache.rocketmq.broker.metrics.BrokerMetricsManager; +import org.apache.rocketmq.broker.lite.LiteLifecycleManager; import org.apache.rocketmq.broker.offset.ConsumerOffsetManager; import org.apache.rocketmq.broker.schedule.ScheduleMessageService; import org.apache.rocketmq.broker.topic.TopicConfigManager; @@ -57,6 +58,7 @@ import org.apache.rocketmq.common.constant.FIleReadaheadMode; import org.apache.rocketmq.common.constant.PermName; import org.apache.rocketmq.common.consumer.ConsumeFromWhere; +import org.apache.rocketmq.common.lite.LiteUtil; import org.apache.rocketmq.common.message.MessageAccessor; import org.apache.rocketmq.common.message.MessageConst; import org.apache.rocketmq.common.message.MessageExt; @@ -114,6 +116,7 @@ import org.apache.rocketmq.remoting.protocol.header.ResetOffsetRequestHeader; import org.apache.rocketmq.remoting.protocol.header.ResumeCheckHalfMessageRequestHeader; import org.apache.rocketmq.remoting.protocol.header.SearchOffsetRequestHeader; +import org.apache.rocketmq.remoting.protocol.header.SearchOffsetResponseHeader; import org.apache.rocketmq.remoting.protocol.header.UpdateAclRequestHeader; import org.apache.rocketmq.remoting.protocol.header.UpdateUserRequestHeader; import org.apache.rocketmq.remoting.protocol.heartbeat.ConsumeType; @@ -712,11 +715,53 @@ public void testSearchOffsetByTimestamp() throws Exception { searchOffsetRequestHeader.setQueueId(0); searchOffsetRequestHeader.setTimestamp(System.currentTimeMillis()); RemotingCommand request = RemotingCommand.createRequestCommand(RequestCode.SEARCH_OFFSET_BY_TIMESTAMP, searchOffsetRequestHeader); - request.addExtField("topic", "topic"); - request.addExtField("queueId", "0"); - request.addExtField("timestamp", System.currentTimeMillis() + ""); + request.makeCustomHeaderToNet(); + RemotingCommand response = adminBrokerProcessor.processRequest(handlerContext, request); + assertThat(response.getCode()).isEqualTo(ResponseCode.SUCCESS); + } + + @Test + public void testSearchOffsetByTimestampWithLiteTopic() throws Exception { + // Prepare test data + String topic = "testTopic"; + String liteTopic = "liteTestTopic"; + long timestamp = System.currentTimeMillis(); + long mockOffset = 100L; + long mockMaxOffset = 500L; + + MessageStore messageStore = mock(MessageStore.class); + LiteLifecycleManager liteLifecycleManager = mock(LiteLifecycleManager.class); + when(brokerController.getMessageStore()).thenReturn(messageStore); + when(brokerController.getLiteLifecycleManager()).thenReturn(liteLifecycleManager); + + when(liteLifecycleManager.getMaxOffsetInQueue(anyString())).thenReturn(mockMaxOffset); + when(messageStore.getOffsetInQueueByTime(anyString(), anyInt(), anyLong(), any(BoundaryType.class))) + .thenReturn(mockOffset); + + SearchOffsetRequestHeader requestHeader = new SearchOffsetRequestHeader(); + requestHeader.setTopic(topic); + requestHeader.setQueueId(0); + requestHeader.setTimestamp(timestamp); + requestHeader.setLiteTopic(liteTopic); + requestHeader.setBoundaryType(BoundaryType.LOWER); + RemotingCommand request = RemotingCommand.createRequestCommand(RequestCode.SEARCH_OFFSET_BY_TIMESTAMP, requestHeader); + request.makeCustomHeaderToNet(); + RemotingCommand response = adminBrokerProcessor.processRequest(handlerContext, request); + assertThat(response.getCode()).isEqualTo(ResponseCode.SUCCESS); + assertThat(response.readCustomHeader()).isInstanceOf(SearchOffsetResponseHeader.class); + + SearchOffsetResponseHeader responseHeader = (SearchOffsetResponseHeader) response.readCustomHeader(); + assertThat(responseHeader.getOffset()).isEqualTo(mockOffset); + + // Verify that the LMQ conversion logic is correctly invoked + // When maxOffset > 0, the offset query operation should be executed + String expectedLmqTopic = LiteUtil.toLmqName(topic, liteTopic); + verify(liteLifecycleManager).getMaxOffsetInQueue(expectedLmqTopic); + verify(messageStore).getOffsetInQueueByTime(eq(expectedLmqTopic), eq(0), anyLong(), any(BoundaryType.class)); + // Verify that queueId is correctly set to 0 (LMQ characteristic) + verify(messageStore).getOffsetInQueueByTime(anyString(), eq(0), anyLong(), any(BoundaryType.class)); } @Test diff --git a/broker/src/test/java/org/apache/rocketmq/broker/processor/LiteManagerProcessorTest.java b/broker/src/test/java/org/apache/rocketmq/broker/processor/LiteManagerProcessorTest.java new file mode 100644 index 00000000000..c6cf7312818 --- /dev/null +++ b/broker/src/test/java/org/apache/rocketmq/broker/processor/LiteManagerProcessorTest.java @@ -0,0 +1,741 @@ +/* + * 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.processor; + +import io.netty.channel.ChannelHandlerContext; +import java.util.ArrayList; +import java.util.HashSet; +import java.util.List; +import java.util.Set; +import java.util.concurrent.ConcurrentHashMap; +import java.util.concurrent.ConcurrentMap; +import org.apache.commons.lang3.StringUtils; +import org.apache.rocketmq.broker.BrokerController; +import org.apache.rocketmq.broker.lite.AbstractLiteLifecycleManager; +import org.apache.rocketmq.broker.lite.LiteEventDispatcher; +import org.apache.rocketmq.broker.lite.LiteSharding; +import org.apache.rocketmq.broker.lite.LiteSubscriptionRegistry; +import org.apache.rocketmq.broker.metrics.BrokerMetricsManager; +import org.apache.rocketmq.broker.metrics.LiteConsumerLagCalculator; +import org.apache.rocketmq.broker.offset.ConsumerOffsetManager; +import org.apache.rocketmq.broker.offset.MemoryConsumerOrderInfoManager; +import org.apache.rocketmq.broker.pop.orderly.ConsumerOrderInfoManager; +import org.apache.rocketmq.broker.subscription.SubscriptionGroupManager; +import org.apache.rocketmq.broker.topic.TopicConfigManager; +import org.apache.rocketmq.common.BrokerConfig; +import org.apache.rocketmq.common.Pair; +import org.apache.rocketmq.common.TopicConfig; +import org.apache.rocketmq.common.attribute.TopicMessageType; +import org.apache.rocketmq.common.entity.ClientGroup; +import org.apache.rocketmq.common.lite.LiteLagInfo; +import org.apache.rocketmq.common.lite.LiteSubscription; +import org.apache.rocketmq.common.lite.LiteUtil; +import org.apache.rocketmq.remoting.exception.RemotingCommandException; +import org.apache.rocketmq.remoting.protocol.RemotingCommand; +import org.apache.rocketmq.remoting.protocol.RequestCode; +import org.apache.rocketmq.remoting.protocol.ResponseCode; +import org.apache.rocketmq.remoting.protocol.admin.TopicOffset; +import org.apache.rocketmq.remoting.protocol.body.GetBrokerLiteInfoResponseBody; +import org.apache.rocketmq.remoting.protocol.body.GetLiteClientInfoResponseBody; +import org.apache.rocketmq.remoting.protocol.body.GetLiteGroupInfoResponseBody; +import org.apache.rocketmq.remoting.protocol.body.GetLiteTopicInfoResponseBody; +import org.apache.rocketmq.remoting.protocol.body.GetParentTopicInfoResponseBody; +import org.apache.rocketmq.remoting.protocol.header.GetLiteClientInfoRequestHeader; +import org.apache.rocketmq.remoting.protocol.header.GetLiteGroupInfoRequestHeader; +import org.apache.rocketmq.remoting.protocol.header.GetLiteTopicInfoRequestHeader; +import org.apache.rocketmq.remoting.protocol.header.GetParentTopicInfoRequestHeader; +import org.apache.rocketmq.remoting.protocol.header.TriggerLiteDispatchRequestHeader; +import org.apache.rocketmq.remoting.protocol.subscription.SubscriptionGroupConfig; +import org.apache.rocketmq.store.MessageStore; +import org.apache.rocketmq.store.config.MessageStoreConfig; +import org.apache.rocketmq.store.queue.ConsumeQueueStoreInterface; +import org.junit.Before; +import org.junit.Test; +import org.junit.runner.RunWith; +import org.mockito.Mock; +import org.mockito.junit.MockitoJUnitRunner; + +import static org.junit.Assert.assertEquals; +import static org.junit.Assert.assertNotNull; +import static org.junit.Assert.assertNull; +import static org.junit.Assert.assertTrue; +import static org.mockito.Mockito.mock; +import static org.mockito.Mockito.never; +import static org.mockito.Mockito.times; +import static org.mockito.Mockito.verify; +import static org.mockito.Mockito.when; + +@RunWith(MockitoJUnitRunner.class) +public class LiteManagerProcessorTest { + + @Mock + private BrokerController brokerController; + + @Mock + private AbstractLiteLifecycleManager liteLifecycleManager; + + @Mock + private LiteSharding liteSharding; + + @Mock + private ChannelHandlerContext ctx; + + @Mock + private MessageStoreConfig messageStoreConfig; + + @Mock + private MessageStore messageStore; + + @Mock + private ConsumeQueueStoreInterface consumeQueueStore; + + @Mock + private TopicConfigManager topicConfigManager; + + @Mock + private SubscriptionGroupManager subscriptionGroupManager; + + @Mock + private LiteSubscriptionRegistry liteSubscriptionRegistry; + + @Mock + private ConsumerOffsetManager consumerOffsetManager; + + @Mock + private BrokerMetricsManager brokerMetricsManager; + + @Mock + private LiteConsumerLagCalculator liteConsumerLagCalculator; + + @Mock + private LiteEventDispatcher liteEventDispatcher; + + @Mock + private PopLiteMessageProcessor popLiteMessageProcessor; + + private LiteManagerProcessor processor; + + @Before + public void setUp() { + processor = new LiteManagerProcessor(brokerController, liteLifecycleManager, liteSharding); + + when(brokerController.getMessageStoreConfig()).thenReturn(messageStoreConfig); + when(brokerController.getMessageStore()).thenReturn(messageStore); + when(brokerController.getTopicConfigManager()).thenReturn(topicConfigManager); + when(brokerController.getSubscriptionGroupManager()).thenReturn(subscriptionGroupManager); + when(brokerController.getLiteSubscriptionRegistry()).thenReturn(liteSubscriptionRegistry); + when(brokerController.getBrokerMetricsManager()).thenReturn(brokerMetricsManager); + when(brokerController.getLiteEventDispatcher()).thenReturn(liteEventDispatcher); + when(brokerController.getPopLiteMessageProcessor()).thenReturn(popLiteMessageProcessor); + when(brokerController.getConsumerOffsetManager()).thenReturn(consumerOffsetManager); + + ConsumerOrderInfoManager consumerOrderInfoManager = new MemoryConsumerOrderInfoManager(brokerController); + when(popLiteMessageProcessor.getConsumerOrderInfoManager()).thenReturn(consumerOrderInfoManager); + + when(messageStore.getQueueStore()).thenReturn(consumeQueueStore); + when(consumeQueueStore.getConsumeQueueTable()).thenReturn(new ConcurrentHashMap<>()); + when(brokerMetricsManager.getLiteConsumerLagCalculator()).thenReturn(liteConsumerLagCalculator); + + when(consumerOffsetManager.getOffsetTable()).thenReturn(new ConcurrentHashMap<>()); + } + + @Test + public void testProcessRequest_GetBrokerLiteInfo() throws Exception { + RemotingCommand request = mock(RemotingCommand.class); + when(request.getCode()).thenReturn(RequestCode.GET_BROKER_LITE_INFO); + + ConcurrentMap topicConfigTable = new ConcurrentHashMap<>(); + when(topicConfigManager.getTopicConfigTable()).thenReturn(topicConfigTable); + + ConcurrentMap subscriptionGroupTable = new ConcurrentHashMap<>(); + when(subscriptionGroupManager.getSubscriptionGroupTable()).thenReturn(subscriptionGroupTable); + + RemotingCommand response = processor.processRequest(ctx, request); + + assertNotNull(response); + assertEquals(ResponseCode.SUCCESS, response.getCode()); + } + + @Test + public void testProcessRequest_UnsupportedRequestCode() throws Exception { + RemotingCommand request = mock(RemotingCommand.class); + when(request.getCode()).thenReturn(99999); + + assertNull(processor.processRequest(ctx, request)); + } + + @Test + public void testGetBrokerLiteInfo() throws RemotingCommandException { + when(messageStoreConfig.getStoreType()).thenReturn("RocksDB"); + when(messageStoreConfig.getMaxLmqConsumeQueueNum()).thenReturn(10000); + when(consumeQueueStore.getLmqNum()).thenReturn(100); + when(liteSubscriptionRegistry.getActiveSubscriptionNum()).thenReturn(50); + + ConcurrentHashMap topicConfigMap = new ConcurrentHashMap<>(); + topicConfigMap.put("SYSTEM_TOPIC", new TopicConfig("SYSTEM_TOPIC")); + when(topicConfigManager.getTopicConfigTable()).thenReturn(topicConfigMap); + + ConcurrentHashMap subscriptionGroupMap = new ConcurrentHashMap<>(); + SubscriptionGroupConfig config = new SubscriptionGroupConfig(); + config.setGroupName("test_group"); + config.setLiteBindTopic("test_topic"); + subscriptionGroupMap.put("test_group", config); + when(subscriptionGroupManager.getSubscriptionGroupTable()).thenReturn(subscriptionGroupMap); + + RemotingCommand request = RemotingCommand.createRequestCommand(RequestCode.GET_BROKER_LITE_INFO, null); + + RemotingCommand response = processor.getBrokerLiteInfo(ctx, request); + + assertEquals(ResponseCode.SUCCESS, response.getCode()); + assertNotNull(response.getBody()); + + GetBrokerLiteInfoResponseBody body = GetBrokerLiteInfoResponseBody.decode(response.getBody(), GetBrokerLiteInfoResponseBody.class); + assertEquals("RocksDB", body.getStoreType()); + assertEquals(10000, body.getMaxLmqNum()); + assertEquals(100, body.getCurrentLmqNum()); + assertEquals(50, body.getLiteSubscriptionCount()); + assertNotNull(body.getTopicMeta()); + assertNotNull(body.getGroupMeta()); + } + + @Test + public void testGetParentTopicInfo_TopicNotExist() throws RemotingCommandException { + GetParentTopicInfoRequestHeader requestHeader = new GetParentTopicInfoRequestHeader(); + requestHeader.setTopic("nonexistent_topic"); + + RemotingCommand request = RemotingCommand.createRequestCommand(RequestCode.GET_PARENT_TOPIC_INFO, requestHeader); + request.makeCustomHeaderToNet(); + + when(topicConfigManager.selectTopicConfig("nonexistent_topic")).thenReturn(null); + + RemotingCommand response = processor.getParentTopicInfo(ctx, request); + + assertEquals(ResponseCode.TOPIC_NOT_EXIST, response.getCode()); + assertTrue(response.getRemark().contains("nonexistent_topic")); + } + + @Test + public void testGetParentTopicInfo_InvalidTopicType() throws RemotingCommandException { + GetParentTopicInfoRequestHeader requestHeader = new GetParentTopicInfoRequestHeader(); + requestHeader.setTopic("invalid_topic"); + + RemotingCommand request = RemotingCommand.createRequestCommand(RequestCode.GET_PARENT_TOPIC_INFO, requestHeader); + request.makeCustomHeaderToNet(); + + TopicConfig topicConfig = new TopicConfig(); + topicConfig.setTopicName("invalid_topic"); + topicConfig.setTopicMessageType(TopicMessageType.NORMAL); + + when(topicConfigManager.selectTopicConfig("invalid_topic")).thenReturn(topicConfig); + + RemotingCommand response = processor.getParentTopicInfo(ctx, request); + + assertEquals(ResponseCode.INVALID_PARAMETER, response.getCode()); + assertTrue(response.getRemark().contains("invalid_topic")); + } + + @Test + public void testGetParentTopicInfo_Success() throws RemotingCommandException { + GetParentTopicInfoRequestHeader requestHeader = new GetParentTopicInfoRequestHeader(); + requestHeader.setTopic("parent_topic"); + + RemotingCommand request = RemotingCommand.createRequestCommand(RequestCode.GET_PARENT_TOPIC_INFO, requestHeader); + request.makeCustomHeaderToNet(); + + TopicConfig topicConfig = new TopicConfig(); + topicConfig.setTopicName("parent_topic"); + topicConfig.setTopicMessageType(TopicMessageType.LITE); + topicConfig.setLiteTopicExpiration(3600); + + when(topicConfigManager.selectTopicConfig("parent_topic")).thenReturn(topicConfig); + when(consumeQueueStore.getLmqNum()).thenReturn(200); + when(liteLifecycleManager.getLiteTopicCount("parent_topic")).thenReturn(10); + + ConcurrentHashMap subscriptionGroupMap = new ConcurrentHashMap<>(); + SubscriptionGroupConfig config = new SubscriptionGroupConfig(); + config.setGroupName("test_group"); + config.setLiteBindTopic("parent_topic"); + subscriptionGroupMap.put("test_group", config); + when(subscriptionGroupManager.getSubscriptionGroupTable()).thenReturn(subscriptionGroupMap); + + RemotingCommand response = processor.getParentTopicInfo(ctx, request); + + assertEquals(ResponseCode.SUCCESS, response.getCode()); + assertNotNull(response.getBody()); + + GetParentTopicInfoResponseBody body = GetParentTopicInfoResponseBody.decode(response.getBody(), GetParentTopicInfoResponseBody.class); + assertEquals("parent_topic", body.getTopic()); + assertEquals(3600, body.getTtl()); + assertEquals(200, body.getLmqNum()); + assertEquals(10, body.getLiteTopicCount()); + assertTrue(body.getGroups().contains("test_group")); + } + + @Test + public void testGetLiteTopicInfo_ParentTopicNotExist() throws RemotingCommandException { + GetLiteTopicInfoRequestHeader requestHeader = new GetLiteTopicInfoRequestHeader(); + requestHeader.setParentTopic("nonexistent_parent"); + requestHeader.setLiteTopic("lite_topic"); + + RemotingCommand request = RemotingCommand.createRequestCommand(RequestCode.GET_LITE_TOPIC_INFO, requestHeader); + request.makeCustomHeaderToNet(); + + when(topicConfigManager.selectTopicConfig("nonexistent_parent")).thenReturn(null); + + RemotingCommand response = processor.getLiteTopicInfo(ctx, request); + + assertEquals(ResponseCode.TOPIC_NOT_EXIST, response.getCode()); + assertTrue(response.getRemark().contains("nonexistent_parent")); + } + + @Test + public void testGetLiteTopicInfo_InvalidParentTopicType() throws RemotingCommandException { + GetLiteTopicInfoRequestHeader requestHeader = new GetLiteTopicInfoRequestHeader(); + requestHeader.setParentTopic("invalid_parent"); + requestHeader.setLiteTopic("lite_topic"); + + RemotingCommand request = RemotingCommand.createRequestCommand(RequestCode.GET_LITE_TOPIC_INFO, requestHeader); + request.makeCustomHeaderToNet(); + + TopicConfig topicConfig = new TopicConfig(); + topicConfig.setTopicName("invalid_parent"); + topicConfig.setTopicMessageType(TopicMessageType.NORMAL); + + when(topicConfigManager.selectTopicConfig("invalid_parent")).thenReturn(topicConfig); + + RemotingCommand response = processor.getLiteTopicInfo(ctx, request); + + assertEquals(ResponseCode.INVALID_PARAMETER, response.getCode()); + assertTrue(response.getRemark().contains("invalid_parent")); + } + + @Test + public void testGetLiteTopicInfo_Success() throws RemotingCommandException { + GetLiteTopicInfoRequestHeader requestHeader = new GetLiteTopicInfoRequestHeader(); + requestHeader.setParentTopic("parent_topic"); + requestHeader.setLiteTopic("lite_topic"); + + RemotingCommand request = RemotingCommand.createRequestCommand(RequestCode.GET_LITE_TOPIC_INFO, requestHeader); + request.makeCustomHeaderToNet(); + + TopicConfig topicConfig = new TopicConfig(); + topicConfig.setTopicName("parent_topic"); + topicConfig.setTopicMessageType(TopicMessageType.LITE); + + String lmqName = LiteUtil.toLmqName("parent_topic", "lite_topic"); + long maxOffset = 100L; + long minOffset = 10L; + long lastUpdateTimestamp = System.currentTimeMillis(); + + when(topicConfigManager.selectTopicConfig("parent_topic")).thenReturn(topicConfig); + when(liteLifecycleManager.getMaxOffsetInQueue(lmqName)).thenReturn(maxOffset); + when(messageStore.getMinOffsetInQueue(lmqName, 0)).thenReturn(minOffset); + when(messageStore.getMessageStoreTimeStamp(lmqName, 0, maxOffset - 1)).thenReturn(lastUpdateTimestamp); + Set subscribers = new HashSet<>(); + subscribers.add(new ClientGroup("clientId1", "group1")); + when(liteSubscriptionRegistry.getSubscriber(lmqName)).thenReturn(subscribers); + when(brokerController.getBrokerConfig()).thenReturn(mock(BrokerConfig.class)); + when(brokerController.getBrokerConfig().getBrokerName()).thenReturn("broker1"); + when(liteSharding.shardingByLmqName("parent_topic", lmqName)).thenReturn("broker1"); + + RemotingCommand response = processor.getLiteTopicInfo(ctx, request); + + assertEquals(ResponseCode.SUCCESS, response.getCode()); + assertNotNull(response.getBody()); + + GetLiteTopicInfoResponseBody body = GetLiteTopicInfoResponseBody.decode(response.getBody(), GetLiteTopicInfoResponseBody.class); + assertEquals("parent_topic", body.getParentTopic()); + assertEquals("lite_topic", body.getLiteTopic()); + assertEquals(subscribers, body.getSubscriber()); + + TopicOffset topicOffset = body.getTopicOffset(); + assertEquals(minOffset, topicOffset.getMinOffset()); + assertEquals(maxOffset, topicOffset.getMaxOffset()); + assertEquals(lastUpdateTimestamp, topicOffset.getLastUpdateTimestamp()); + assertTrue(body.isShardingToBroker()); + } + + @Test + public void testGetLiteClientInfo_ParentTopicNotExist() throws RemotingCommandException { + GetLiteClientInfoRequestHeader requestHeader = new GetLiteClientInfoRequestHeader(); + requestHeader.setParentTopic("nonexistent_parent"); + requestHeader.setGroup("group1"); + requestHeader.setClientId("client1"); + requestHeader.setMaxCount(100); + + RemotingCommand request = RemotingCommand.createRequestCommand(RequestCode.GET_LITE_CLIENT_INFO, requestHeader); + request.makeCustomHeaderToNet(); + + when(topicConfigManager.selectTopicConfig("nonexistent_parent")).thenReturn(null); + + RemotingCommand response = processor.getLiteClientInfo(ctx, request); + + assertEquals(ResponseCode.TOPIC_NOT_EXIST, response.getCode()); + assertTrue(response.getRemark().contains("nonexistent_parent")); + } + + @Test + public void testGetLiteClientInfo_GroupNotExist() throws RemotingCommandException { + GetLiteClientInfoRequestHeader requestHeader = new GetLiteClientInfoRequestHeader(); + requestHeader.setParentTopic("parent_topic"); + requestHeader.setGroup("nonexistent_group"); + requestHeader.setClientId("client1"); + requestHeader.setMaxCount(100); + + RemotingCommand request = RemotingCommand.createRequestCommand(RequestCode.GET_LITE_CLIENT_INFO, requestHeader); + request.makeCustomHeaderToNet(); + + TopicConfig topicConfig = new TopicConfig(); + topicConfig.setTopicName("parent_topic"); + topicConfig.setTopicMessageType(TopicMessageType.LITE); + + when(topicConfigManager.selectTopicConfig("parent_topic")).thenReturn(topicConfig); + when(subscriptionGroupManager.findSubscriptionGroupConfig("nonexistent_group")).thenReturn(null); + + RemotingCommand response = processor.getLiteClientInfo(ctx, request); + + assertEquals(ResponseCode.SUBSCRIPTION_GROUP_NOT_EXIST, response.getCode()); + assertTrue(response.getRemark().contains("nonexistent_group")); + } + + @Test + public void testGetLiteClientInfo_NoSubscription() throws RemotingCommandException { + GetLiteClientInfoRequestHeader requestHeader = new GetLiteClientInfoRequestHeader(); + requestHeader.setParentTopic("parent_topic"); + requestHeader.setGroup("group1"); + requestHeader.setClientId("client1"); + requestHeader.setMaxCount(100); + + RemotingCommand request = RemotingCommand.createRequestCommand(RequestCode.GET_LITE_CLIENT_INFO, requestHeader); + request.makeCustomHeaderToNet(); + + TopicConfig topicConfig = new TopicConfig(); + topicConfig.setTopicName("parent_topic"); + topicConfig.setTopicMessageType(TopicMessageType.LITE); + + SubscriptionGroupConfig groupConfig = new SubscriptionGroupConfig(); + groupConfig.setGroupName("group1"); + groupConfig.setLiteBindTopic("parent_topic"); + + when(topicConfigManager.selectTopicConfig("parent_topic")).thenReturn(topicConfig); + when(subscriptionGroupManager.findSubscriptionGroupConfig("group1")).thenReturn(groupConfig); + when(liteSubscriptionRegistry.getLiteSubscription("client1")).thenReturn(null); + + RemotingCommand response = processor.getLiteClientInfo(ctx, request); + + assertEquals(ResponseCode.SUCCESS, response.getCode()); + assertNotNull(response.getBody()); + + GetLiteClientInfoResponseBody body = GetLiteClientInfoResponseBody.decode(response.getBody(), GetLiteClientInfoResponseBody.class); + assertEquals("parent_topic", body.getParentTopic()); + assertEquals("group1", body.getGroup()); + assertEquals("client1", body.getClientId()); + assertEquals(-1, body.getLiteTopicCount()); + assertNull(body.getLiteTopicSet()); + } + + @Test + public void testGetLiteClientInfo_WithSubscription() throws RemotingCommandException { + GetLiteClientInfoRequestHeader requestHeader = new GetLiteClientInfoRequestHeader(); + requestHeader.setParentTopic("parent_topic"); + requestHeader.setGroup("group1"); + requestHeader.setClientId("client1"); + requestHeader.setMaxCount(100); + + RemotingCommand request = RemotingCommand.createRequestCommand(RequestCode.GET_LITE_CLIENT_INFO, requestHeader); + request.makeCustomHeaderToNet(); + + TopicConfig topicConfig = new TopicConfig(); + topicConfig.setTopicName("parent_topic"); + topicConfig.setTopicMessageType(TopicMessageType.LITE); + + SubscriptionGroupConfig groupConfig = new SubscriptionGroupConfig(); + groupConfig.setGroupName("group1"); + groupConfig.setLiteBindTopic("parent_topic"); + + Set liteTopicSet = new HashSet<>(); + liteTopicSet.add("lite_topic1"); + liteTopicSet.add("lite_topic2"); + + LiteSubscription liteSubscription = new LiteSubscription(); + liteSubscription.setLiteTopicSet(liteTopicSet); + + when(topicConfigManager.selectTopicConfig("parent_topic")).thenReturn(topicConfig); + when(subscriptionGroupManager.findSubscriptionGroupConfig("group1")).thenReturn(groupConfig); + when(liteSubscriptionRegistry.getLiteSubscription("client1")).thenReturn(liteSubscription); + + RemotingCommand response = processor.getLiteClientInfo(ctx, request); + + assertEquals(ResponseCode.SUCCESS, response.getCode()); + assertNotNull(response.getBody()); + + GetLiteClientInfoResponseBody body = GetLiteClientInfoResponseBody.decode(response.getBody(), GetLiteClientInfoResponseBody.class); + assertEquals("parent_topic", body.getParentTopic()); + assertEquals("group1", body.getGroup()); + assertEquals("client1", body.getClientId()); + assertEquals(2, body.getLiteTopicCount()); + assertEquals(liteTopicSet, body.getLiteTopicSet()); + } + + @Test + public void testGetLiteGroupInfo_GroupNotExist() throws RemotingCommandException { + GetLiteGroupInfoRequestHeader requestHeader = new GetLiteGroupInfoRequestHeader(); + requestHeader.setGroup("nonexistent_group"); + requestHeader.setLiteTopic(""); + requestHeader.setTopK(10); + + RemotingCommand request = RemotingCommand.createRequestCommand(RequestCode.GET_LITE_GROUP_INFO, requestHeader); + request.makeCustomHeaderToNet(); + + when(subscriptionGroupManager.findSubscriptionGroupConfig("nonexistent_group")).thenReturn(null); + + RemotingCommand response = processor.getLiteGroupInfo(ctx, request); + + assertEquals(ResponseCode.SUBSCRIPTION_GROUP_NOT_EXIST, response.getCode()); + assertTrue(response.getRemark().contains("nonexistent_group")); + } + + @Test + public void testGetLiteGroupInfo_NotLiteGroup() throws RemotingCommandException { + GetLiteGroupInfoRequestHeader requestHeader = new GetLiteGroupInfoRequestHeader(); + requestHeader.setGroup("normal_group"); + requestHeader.setLiteTopic(""); + requestHeader.setTopK(10); + + RemotingCommand request = RemotingCommand.createRequestCommand(RequestCode.GET_LITE_GROUP_INFO, requestHeader); + request.makeCustomHeaderToNet(); + + SubscriptionGroupConfig groupConfig = new SubscriptionGroupConfig(); + groupConfig.setGroupName("normal_group"); + groupConfig.setLiteBindTopic(""); + + when(subscriptionGroupManager.findSubscriptionGroupConfig("normal_group")).thenReturn(groupConfig); + + RemotingCommand response = processor.getLiteGroupInfo(ctx, request); + + assertEquals(ResponseCode.INVALID_PARAMETER, response.getCode()); + assertTrue(response.getRemark().contains("normal_group")); + assertTrue(response.getRemark().contains("not a LITE group")); + } + + @Test + public void testGetLiteGroupInfo_GetTopKInfo() throws RemotingCommandException { + GetLiteGroupInfoRequestHeader requestHeader = new GetLiteGroupInfoRequestHeader(); + requestHeader.setGroup("lite_group"); + requestHeader.setLiteTopic(""); + requestHeader.setTopK(10); + + RemotingCommand request = RemotingCommand.createRequestCommand(RequestCode.GET_LITE_GROUP_INFO, requestHeader); + request.makeCustomHeaderToNet(); + + SubscriptionGroupConfig groupConfig = new SubscriptionGroupConfig(); + groupConfig.setGroupName("lite_group"); + groupConfig.setLiteBindTopic("parent_topic"); + + List lagCountList = new ArrayList<>(); + LiteLagInfo lagCountInfo = new LiteLagInfo(); + lagCountInfo.setLiteTopic("topic1"); + lagCountInfo.setLagCount(100L); + lagCountList.add(lagCountInfo); + Pair, Long> lagCountPair = new Pair<>(lagCountList, 100L); + + List lagTimeList = new ArrayList<>(); + LiteLagInfo lagTimeInfo = new LiteLagInfo(); + lagTimeInfo.setLiteTopic("topic1"); + lagTimeInfo.setEarliestUnconsumedTimestamp(System.currentTimeMillis()); + lagTimeList.add(lagTimeInfo); + Pair, Long> lagTimePair = new Pair<>(lagTimeList, System.currentTimeMillis()); + + when(subscriptionGroupManager.findSubscriptionGroupConfig("lite_group")).thenReturn(groupConfig); + when(liteConsumerLagCalculator.getLagCountTopK("lite_group", 10)).thenReturn(lagCountPair); + when(liteConsumerLagCalculator.getLagTimestampTopK("lite_group", "parent_topic", 10)).thenReturn(lagTimePair); + + RemotingCommand response = processor.getLiteGroupInfo(ctx, request); + + assertEquals(ResponseCode.SUCCESS, response.getCode()); + assertNotNull(response.getBody()); + + GetLiteGroupInfoResponseBody body = GetLiteGroupInfoResponseBody.decode(response.getBody(), GetLiteGroupInfoResponseBody.class); + assertEquals("lite_group", body.getGroup()); + assertEquals("parent_topic", body.getParentTopic()); + assertTrue(StringUtils.isEmpty(body.getLiteTopic())); + List actualLagCountList = body.getLagCountTopK(); + assertEquals(lagCountList.size(), actualLagCountList.size()); + for (int i = 0; i < lagCountList.size(); i++) { + LiteLagInfo expected = lagCountList.get(i); + LiteLagInfo actual = actualLagCountList.get(i); + assertEquals(expected.getLiteTopic(), actual.getLiteTopic()); + assertEquals(expected.getLagCount(), actual.getLagCount()); + } + assertEquals(Long.valueOf(100L), Long.valueOf(body.getTotalLagCount())); + List actualLagTimeList = body.getLagTimestampTopK(); + assertEquals(lagTimeList.size(), actualLagTimeList.size()); + for (int i = 0; i < lagTimeList.size(); i++) { + LiteLagInfo expected = lagTimeList.get(i); + LiteLagInfo actual = actualLagTimeList.get(i); + assertEquals(expected.getLiteTopic(), actual.getLiteTopic()); + assertEquals(expected.getEarliestUnconsumedTimestamp(), actual.getEarliestUnconsumedTimestamp()); + } + } + + @Test + public void testGetLiteGroupInfo_SpecificLiteTopic_WithMessages() throws RemotingCommandException { + GetLiteGroupInfoRequestHeader requestHeader = new GetLiteGroupInfoRequestHeader(); + requestHeader.setGroup("lite_group"); + requestHeader.setLiteTopic("specific_lite_topic"); + requestHeader.setTopK(10); + + RemotingCommand request = RemotingCommand.createRequestCommand(RequestCode.GET_LITE_GROUP_INFO, requestHeader); + request.makeCustomHeaderToNet(); + + SubscriptionGroupConfig groupConfig = new SubscriptionGroupConfig(); + groupConfig.setGroupName("lite_group"); + groupConfig.setLiteBindTopic("parent_topic"); + + String lmqName = LiteUtil.toLmqName("parent_topic", "specific_lite_topic"); + long maxOffset = 100L; + long commitOffset = 50L; + long messageTimestamp = System.currentTimeMillis() - 10000; + + when(subscriptionGroupManager.findSubscriptionGroupConfig("lite_group")).thenReturn(groupConfig); + when(liteLifecycleManager.getMaxOffsetInQueue(lmqName)).thenReturn(maxOffset); + when(consumerOffsetManager.queryOffset("lite_group", lmqName, 0)).thenReturn(commitOffset); + when(messageStore.getMessageStoreTimeStamp(lmqName, 0, commitOffset)).thenReturn(messageTimestamp); + when(brokerController.getConsumerOffsetManager()).thenReturn(consumerOffsetManager); + + RemotingCommand response = processor.getLiteGroupInfo(ctx, request); + + assertEquals(ResponseCode.SUCCESS, response.getCode()); + assertNotNull(response.getBody()); + + GetLiteGroupInfoResponseBody body = GetLiteGroupInfoResponseBody.decode(response.getBody(), GetLiteGroupInfoResponseBody.class); + assertEquals("lite_group", body.getGroup()); + assertEquals("parent_topic", body.getParentTopic()); + assertEquals("specific_lite_topic", body.getLiteTopic()); + assertEquals(maxOffset - commitOffset, body.getTotalLagCount()); + assertEquals(messageTimestamp, body.getEarliestUnconsumedTimestamp()); + } + + @Test + public void testGetLiteGroupInfo_SpecificLiteTopic_WithoutMessages() throws RemotingCommandException { + GetLiteGroupInfoRequestHeader requestHeader = new GetLiteGroupInfoRequestHeader(); + requestHeader.setGroup("lite_group"); + requestHeader.setLiteTopic("specific_lite_topic"); + requestHeader.setTopK(10); + + RemotingCommand request = RemotingCommand.createRequestCommand(RequestCode.GET_LITE_GROUP_INFO, requestHeader); + request.makeCustomHeaderToNet(); + + SubscriptionGroupConfig groupConfig = new SubscriptionGroupConfig(); + groupConfig.setGroupName("lite_group"); + groupConfig.setLiteBindTopic("parent_topic"); + + String lmqName = LiteUtil.toLmqName("parent_topic", "specific_lite_topic"); + long maxOffset = 0L; + + when(subscriptionGroupManager.findSubscriptionGroupConfig("lite_group")).thenReturn(groupConfig); + when(liteLifecycleManager.getMaxOffsetInQueue(lmqName)).thenReturn(maxOffset); + + RemotingCommand response = processor.getLiteGroupInfo(ctx, request); + + assertEquals(ResponseCode.SUCCESS, response.getCode()); + assertNotNull(response.getBody()); + + GetLiteGroupInfoResponseBody body = GetLiteGroupInfoResponseBody.decode(response.getBody(), GetLiteGroupInfoResponseBody.class); + assertEquals("lite_group", body.getGroup()); + assertEquals("parent_topic", body.getParentTopic()); + assertEquals("specific_lite_topic", body.getLiteTopic()); + assertEquals(-1, body.getTotalLagCount()); + assertEquals(-1L, body.getEarliestUnconsumedTimestamp()); + } + + @Test + public void testGetLiteGroupInfo_SpecificLiteTopic_ZeroCommitOffset() throws RemotingCommandException { + GetLiteGroupInfoRequestHeader requestHeader = new GetLiteGroupInfoRequestHeader(); + requestHeader.setGroup("lite_group"); + requestHeader.setLiteTopic("specific_lite_topic"); + requestHeader.setTopK(10); + + RemotingCommand request = RemotingCommand.createRequestCommand(RequestCode.GET_LITE_GROUP_INFO, requestHeader); + request.makeCustomHeaderToNet(); + + SubscriptionGroupConfig groupConfig = new SubscriptionGroupConfig(); + groupConfig.setGroupName("lite_group"); + groupConfig.setLiteBindTopic("parent_topic"); + + String lmqName = LiteUtil.toLmqName("parent_topic", "specific_lite_topic"); + long maxOffset = 100L; + long commitOffset = 0L; + + when(subscriptionGroupManager.findSubscriptionGroupConfig("lite_group")).thenReturn(groupConfig); + when(liteLifecycleManager.getMaxOffsetInQueue(lmqName)).thenReturn(maxOffset); + when(consumerOffsetManager.queryOffset("lite_group", lmqName, 0)).thenReturn(commitOffset); + when(brokerController.getConsumerOffsetManager()).thenReturn(consumerOffsetManager); + + RemotingCommand response = processor.getLiteGroupInfo(ctx, request); + + assertEquals(ResponseCode.SUCCESS, response.getCode()); + assertNotNull(response.getBody()); + + GetLiteGroupInfoResponseBody body = GetLiteGroupInfoResponseBody.decode(response.getBody(), GetLiteGroupInfoResponseBody.class); + assertEquals("lite_group", body.getGroup()); + assertEquals("parent_topic", body.getParentTopic()); + assertEquals("specific_lite_topic", body.getLiteTopic()); + assertEquals(maxOffset - commitOffset, body.getTotalLagCount()); + assertEquals(0, body.getEarliestUnconsumedTimestamp()); + } + + @Test + public void testTriggerLiteDispatch() throws Exception { + String group = "group"; + String clientId = "clientId"; + TriggerLiteDispatchRequestHeader requestHeader; + + // with clientId + requestHeader = new TriggerLiteDispatchRequestHeader(); + requestHeader.setGroup(group); + requestHeader.setClientId(clientId); + RemotingCommand request = RemotingCommand.createRequestCommand(RequestCode.TRIGGER_LITE_DISPATCH, requestHeader); + request.makeCustomHeaderToNet(); + + SubscriptionGroupConfig groupConfig = new SubscriptionGroupConfig(); + groupConfig.setGroupName(group); + groupConfig.setLiteBindTopic("parent_topic"); + when(subscriptionGroupManager.findSubscriptionGroupConfig(group)).thenReturn(groupConfig); + + RemotingCommand response = processor.triggerLiteDispatch(ctx, request); + + assertNotNull(response); + assertEquals(ResponseCode.SUCCESS, response.getCode()); + verify(liteEventDispatcher, times(1)).doFullDispatch(clientId, group); + verify(liteEventDispatcher, never()).doFullDispatchByGroup(group); + + // without clientId + requestHeader = new TriggerLiteDispatchRequestHeader(); + requestHeader.setGroup(group); + request = RemotingCommand.createRequestCommand(RequestCode.TRIGGER_LITE_DISPATCH, requestHeader); + request.makeCustomHeaderToNet(); + + response = processor.triggerLiteDispatch(ctx, request); + + assertNotNull(response); + assertEquals(ResponseCode.SUCCESS, response.getCode()); + verify(liteEventDispatcher, times(1)).doFullDispatch(clientId, group); + verify(liteEventDispatcher, times(1)).doFullDispatchByGroup(group); + } +} diff --git a/broker/src/test/java/org/apache/rocketmq/broker/processor/LiteSubscriptionCtlProcessorTest.java b/broker/src/test/java/org/apache/rocketmq/broker/processor/LiteSubscriptionCtlProcessorTest.java new file mode 100644 index 00000000000..cc4692955fa --- /dev/null +++ b/broker/src/test/java/org/apache/rocketmq/broker/processor/LiteSubscriptionCtlProcessorTest.java @@ -0,0 +1,253 @@ +/* + * 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.processor; + +import io.netty.channel.Channel; +import io.netty.channel.ChannelHandlerContext; +import java.util.Collections; +import java.util.HashSet; +import java.util.Set; +import org.apache.rocketmq.broker.BrokerController; +import org.apache.rocketmq.broker.lite.LiteSubscriptionRegistry; +import org.apache.rocketmq.broker.subscription.SubscriptionGroupManager; +import org.apache.rocketmq.common.lite.LiteSubscriptionAction; +import org.apache.rocketmq.common.lite.LiteSubscriptionDTO; +import org.apache.rocketmq.remoting.protocol.RemotingCommand; +import org.apache.rocketmq.remoting.protocol.ResponseCode; +import org.apache.rocketmq.remoting.protocol.body.LiteSubscriptionCtlRequestBody; +import org.apache.rocketmq.remoting.protocol.subscription.SubscriptionGroupConfig; +import org.junit.Test; +import org.junit.runner.RunWith; +import org.mockito.InjectMocks; +import org.mockito.Mock; +import org.mockito.junit.MockitoJUnitRunner; + +import static org.junit.jupiter.api.Assertions.assertEquals; +import static org.junit.jupiter.api.Assertions.assertTrue; +import static org.mockito.Mockito.any; +import static org.mockito.Mockito.anySet; +import static org.mockito.Mockito.eq; +import static org.mockito.Mockito.verify; +import static org.mockito.Mockito.when; + +@RunWith(MockitoJUnitRunner.class) +public class LiteSubscriptionCtlProcessorTest { + + @Mock + private BrokerController brokerController; + + @Mock + private SubscriptionGroupManager subscriptionGroupManager; + + @Mock + private LiteSubscriptionRegistry liteSubscriptionRegistry; + + @Mock + private ChannelHandlerContext ctx; + + @Mock + private Channel channel; + + @InjectMocks + private LiteSubscriptionCtlProcessor processor; + + @Test + public void testProcessRequest_BodyIsNull() throws Exception { + RemotingCommand request = RemotingCommand.createRequestCommand(0, null); + RemotingCommand response = processor.processRequest(ctx, request); + assertEquals(ResponseCode.ILLEGAL_OPERATION, response.getCode()); + } + + @Test + public void testProcessRequest_SubscriptionSetIsEmpty() throws Exception { + LiteSubscriptionCtlRequestBody requestBody = new LiteSubscriptionCtlRequestBody(); + requestBody.setSubscriptionSet(Collections.emptySet()); + RemotingCommand request = RemotingCommand.createRequestCommand(0, null); + request.setBody(requestBody.encode()); + RemotingCommand response = processor.processRequest(ctx, request); + assertEquals(ResponseCode.ILLEGAL_OPERATION, response.getCode()); + } + + @Test + public void testProcessRequest_ActionIsIncrementalAdd() throws Exception { + String clientId = "clientId"; + String group = "group"; + String topic = "topic"; + String liteTopic = "liteTopic"; + Set liteTopicSet = new HashSet<>(); + liteTopicSet.add(liteTopic); + + LiteSubscriptionDTO dto = new LiteSubscriptionDTO(); + dto.setClientId(clientId); + dto.setGroup(group); + dto.setTopic(topic); + dto.setLiteTopicSet(liteTopicSet); + dto.setAction(LiteSubscriptionAction.PARTIAL_ADD); + dto.setVersion(1L); + + Set subscriptionSet = new HashSet<>(); + subscriptionSet.add(dto); + + LiteSubscriptionCtlRequestBody requestBody = new LiteSubscriptionCtlRequestBody(); + requestBody.setSubscriptionSet(subscriptionSet); + RemotingCommand request = RemotingCommand.createRequestCommand(0, null); + request.setBody(requestBody.encode()); + + when(ctx.channel()).thenReturn(channel); + when(brokerController.getSubscriptionGroupManager()).thenReturn(subscriptionGroupManager); + SubscriptionGroupConfig groupConfig = new SubscriptionGroupConfig(); + groupConfig.setConsumeEnable(true); + when(subscriptionGroupManager.findSubscriptionGroupConfig(group)).thenReturn(groupConfig); + + RemotingCommand response = processor.processRequest(ctx, request); + + assertEquals(ResponseCode.SUCCESS, response.getCode()); + verify(liteSubscriptionRegistry).updateClientChannel(eq(clientId), eq(channel)); + verify(liteSubscriptionRegistry).addPartialSubscription(eq(clientId), eq(group), eq(topic), anySet(), any()); + } + + @Test + public void testProcessRequest_ActionIsAllAdd() throws Exception { + String clientId = "clientId"; + String group = "group"; + String topic = "topic"; + String liteTopic = "liteTopic"; + Set liteTopicSet = new HashSet<>(); + liteTopicSet.add(liteTopic); + + LiteSubscriptionDTO dto = new LiteSubscriptionDTO(); + dto.setClientId(clientId); + dto.setGroup(group); + dto.setTopic(topic); + dto.setLiteTopicSet(liteTopicSet); + dto.setAction(LiteSubscriptionAction.COMPLETE_ADD); + dto.setVersion(1L); + + Set subscriptionSet = new HashSet<>(); + subscriptionSet.add(dto); + + LiteSubscriptionCtlRequestBody requestBody = new LiteSubscriptionCtlRequestBody(); + requestBody.setSubscriptionSet(subscriptionSet); + RemotingCommand request = RemotingCommand.createRequestCommand(0, null); + request.setBody(requestBody.encode()); + + when(ctx.channel()).thenReturn(channel); + when(brokerController.getSubscriptionGroupManager()).thenReturn(subscriptionGroupManager); + SubscriptionGroupConfig groupConfig = new SubscriptionGroupConfig(); + groupConfig.setConsumeEnable(true); + when(subscriptionGroupManager.findSubscriptionGroupConfig(group)).thenReturn(groupConfig); + + RemotingCommand response = processor.processRequest(ctx, request); + + assertEquals(ResponseCode.SUCCESS, response.getCode()); + verify(liteSubscriptionRegistry).updateClientChannel(eq(clientId), eq(channel)); + verify(liteSubscriptionRegistry).addCompleteSubscription(eq(clientId), eq(group), eq(topic), anySet(), eq(1L)); + } + + @Test + public void testProcessRequest_ActionIsIncrementalRemove() throws Exception { + String clientId = "clientId"; + String group = "group"; + String topic = "topic"; + String liteTopic = "liteTopic"; + Set liteTopicSet = new HashSet<>(); + liteTopicSet.add(liteTopic); + + LiteSubscriptionDTO dto = new LiteSubscriptionDTO(); + dto.setClientId(clientId); + dto.setGroup(group); + dto.setTopic(topic); + dto.setLiteTopicSet(liteTopicSet); + dto.setAction(LiteSubscriptionAction.PARTIAL_REMOVE); + + Set subscriptionSet = new HashSet<>(); + subscriptionSet.add(dto); + + LiteSubscriptionCtlRequestBody requestBody = new LiteSubscriptionCtlRequestBody(); + requestBody.setSubscriptionSet(subscriptionSet); + RemotingCommand request = RemotingCommand.createRequestCommand(0, null); + request.setBody(requestBody.encode()); + + RemotingCommand response = processor.processRequest(ctx, request); + + assertEquals(ResponseCode.SUCCESS, response.getCode()); + verify(liteSubscriptionRegistry).removePartialSubscription(eq(clientId), eq(group), eq(topic), anySet()); + } + + @Test + public void testProcessRequest_ActionIsAllRemove() throws Exception { + String clientId = "clientId"; + + LiteSubscriptionDTO dto = new LiteSubscriptionDTO(); + String group = "group"; + String topic = "topic"; + dto.setClientId(clientId); + dto.setTopic(topic); + dto.setGroup(group); + dto.setAction(LiteSubscriptionAction.COMPLETE_REMOVE); + + Set subscriptionSet = new HashSet<>(); + subscriptionSet.add(dto); + + LiteSubscriptionCtlRequestBody requestBody = new LiteSubscriptionCtlRequestBody(); + requestBody.setSubscriptionSet(subscriptionSet); + RemotingCommand request = RemotingCommand.createRequestCommand(0, null); + request.setBody(requestBody.encode()); + + RemotingCommand response = processor.processRequest(ctx, request); + + assertEquals(ResponseCode.SUCCESS, response.getCode()); + verify(liteSubscriptionRegistry).removeCompleteSubscription(eq(clientId)); + } + + @Test + public void testProcessRequest_CheckConsumeEnableThrowsException() throws Exception { + String clientId = "clientId"; + String group = "group"; + String topic = "topic"; + String liteTopic = "liteTopic"; + Set liteTopicSet = new HashSet<>(); + liteTopicSet.add(liteTopic); + + LiteSubscriptionDTO dto = new LiteSubscriptionDTO(); + dto.setClientId(clientId); + dto.setGroup(group); + dto.setTopic(topic); + dto.setLiteTopicSet(liteTopicSet); + dto.setAction(LiteSubscriptionAction.PARTIAL_ADD); + + Set subscriptionSet = new HashSet<>(); + subscriptionSet.add(dto); + + LiteSubscriptionCtlRequestBody requestBody = new LiteSubscriptionCtlRequestBody(); + requestBody.setSubscriptionSet(subscriptionSet); + RemotingCommand request = RemotingCommand.createRequestCommand(0, null); + request.setBody(requestBody.encode()); + + when(brokerController.getSubscriptionGroupManager()).thenReturn(subscriptionGroupManager); + SubscriptionGroupConfig groupConfig = new SubscriptionGroupConfig(); + groupConfig.setConsumeEnable(false); + when(subscriptionGroupManager.findSubscriptionGroupConfig(group)).thenReturn(groupConfig); + + RemotingCommand response = processor.processRequest(ctx, request); + + assertEquals(ResponseCode.ILLEGAL_OPERATION, response.getCode()); + assertTrue(response.getRemark().contains("Consumer group is not allowed to consume.")); + } + +} diff --git a/broker/src/test/java/org/apache/rocketmq/broker/processor/PopLiteMessageProcessorTest.java b/broker/src/test/java/org/apache/rocketmq/broker/processor/PopLiteMessageProcessorTest.java new file mode 100644 index 00000000000..453cb8fd14e --- /dev/null +++ b/broker/src/test/java/org/apache/rocketmq/broker/processor/PopLiteMessageProcessorTest.java @@ -0,0 +1,490 @@ +/* + * 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.processor; + +import io.netty.channel.ChannelHandlerContext; +import org.apache.commons.lang3.reflect.FieldUtils; +import org.apache.rocketmq.broker.BrokerController; +import org.apache.rocketmq.broker.lite.AbstractLiteLifecycleManager; +import org.apache.rocketmq.broker.lite.LiteEventDispatcher; +import org.apache.rocketmq.broker.longpolling.PopLiteLongPollingService; +import org.apache.rocketmq.broker.offset.ConsumerOffsetManager; +import org.apache.rocketmq.broker.pop.PopConsumerLockService; +import org.apache.rocketmq.broker.pop.orderly.ConsumerOrderInfoManager; +import org.apache.rocketmq.broker.subscription.SubscriptionGroupManager; +import org.apache.rocketmq.broker.topic.TopicConfigManager; +import org.apache.rocketmq.common.BrokerConfig; +import org.apache.rocketmq.common.Pair; +import org.apache.rocketmq.common.TopicConfig; +import org.apache.rocketmq.common.attribute.TopicMessageType; +import org.apache.rocketmq.common.constant.PermName; +import org.apache.rocketmq.common.message.MessageConst; +import org.apache.rocketmq.remoting.protocol.RemotingCommand; +import org.apache.rocketmq.remoting.protocol.ResponseCode; +import org.apache.rocketmq.remoting.protocol.header.PopLiteMessageRequestHeader; +import org.apache.rocketmq.remoting.protocol.header.PopLiteMessageResponseHeader; +import org.apache.rocketmq.remoting.protocol.subscription.SubscriptionGroupConfig; +import org.apache.rocketmq.store.GetMessageResult; +import org.apache.rocketmq.store.GetMessageStatus; +import org.apache.rocketmq.store.MessageStore; +import org.apache.rocketmq.store.SelectMappedBufferResult; +import org.apache.rocketmq.store.exception.ConsumeQueueException; +import org.junit.Before; +import org.junit.Test; +import org.junit.runner.RunWith; +import org.mockito.Mock; +import org.mockito.Mockito; +import org.mockito.junit.MockitoJUnitRunner; + +import java.util.Iterator; + +import static org.assertj.core.api.Assertions.assertThat; +import static org.junit.Assert.assertEquals; +import static org.junit.Assert.assertFalse; +import static org.junit.Assert.assertNotNull; +import static org.junit.Assert.assertNull; +import static org.junit.Assert.assertTrue; +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.Mockito.doNothing; +import static org.mockito.Mockito.when; +import static org.mockito.Mockito.verify; +import static org.mockito.Mockito.times; +import static org.mockito.Mockito.mock; +import static org.mockito.Mockito.never; +import static org.mockito.Mockito.doReturn; + +@RunWith(MockitoJUnitRunner.class) +public class PopLiteMessageProcessorTest { + + @Mock + private BrokerController brokerController; + @Mock + private MessageStore messageStore; + @Mock + private LiteEventDispatcher liteEventDispatcher; + @Mock + private PopLiteLongPollingService popLiteLongPollingService; + @Mock + private PopConsumerLockService lockService; + @Mock + private ConsumerOrderInfoManager consumerOrderInfoManager; + @Mock + private ConsumerOffsetManager consumerOffsetManager; + @Mock + private PopMessageProcessor popMessageProcessor; + @Mock + private TopicConfigManager topicConfigManager; + @Mock + private SubscriptionGroupManager subscriptionGroupManager; + @Mock + private AbstractLiteLifecycleManager liteLifecycleManager; + + private BrokerConfig brokerConfig; + private PopLiteMessageProcessor popLiteMessageProcessor; + + @Before + public void setUp() throws Exception { + brokerConfig = new BrokerConfig(); + when(brokerController.getConsumerOffsetManager()).thenReturn(consumerOffsetManager); + when(brokerController.getPopMessageProcessor()).thenReturn(popMessageProcessor); + when(brokerController.getBrokerConfig()).thenReturn(brokerConfig); + when(brokerController.getMessageStore()).thenReturn(messageStore); + when(brokerController.getTopicConfigManager()).thenReturn(topicConfigManager); + when(brokerController.getSubscriptionGroupManager()).thenReturn(subscriptionGroupManager); + when(brokerController.getLiteLifecycleManager()).thenReturn(liteLifecycleManager); + + PopLiteMessageProcessor testObject = new PopLiteMessageProcessor(brokerController, liteEventDispatcher); + FieldUtils.writeDeclaredField(testObject, "popLiteLongPollingService", popLiteLongPollingService, true); + FieldUtils.writeDeclaredField(testObject, "lockService", lockService, true); + FieldUtils.writeDeclaredField(testObject, "consumerOrderInfoManager", consumerOrderInfoManager, true); + popLiteMessageProcessor = Mockito.spy(testObject); + } + + @Test + public void testRejectRequest() { + assertFalse(popLiteMessageProcessor.rejectRequest()); + } + + @Test + public void testTransformOrderCountInfo_empty() { + StringBuilder result = popLiteMessageProcessor.transformOrderCountInfo(new StringBuilder(), 3); + assertEquals("0;0;0", result.toString()); + } + + @Test + public void testTransformOrderCountInfo_onlyQueueIdInfo() { + StringBuilder input = new StringBuilder("0" + MessageConst.KEY_SEPARATOR + "0" + MessageConst.KEY_SEPARATOR + "2"); + StringBuilder result = popLiteMessageProcessor.transformOrderCountInfo(input, 3); + assertEquals("2;2;2", result.toString()); + } + + @Test + public void testTransformOrderCountInfo_consumeCountAndQueueIdInfo() { + StringBuilder input = new StringBuilder("0 qo0%0 0;0 qo0%1 1;0 0 1"); + StringBuilder result = popLiteMessageProcessor.transformOrderCountInfo(input, 2); + assertEquals("0 qo0%0 0;0 qo0%1 1", result.toString()); + } + + @Test + public void testIsFifoBlocked() { + when(consumerOrderInfoManager.checkBlock(anyString(), anyString(), anyString(), anyInt(), anyLong())) + .thenReturn(true); + assertTrue(popLiteMessageProcessor.isFifoBlocked("attemptId", "group", "lmqName", 1000L)); + verify(consumerOrderInfoManager).checkBlock("attemptId", "lmqName", "group", 0, 1000L); + } + + @Test + public void testGetPopOffset_normal() throws ConsumeQueueException { + String group = "group"; + String lmqName = "lmqName"; + long consumerOffset = 100L; + + // exist + when(consumerOffsetManager.queryOffset(group, lmqName, 0)).thenReturn(consumerOffset); + when(consumerOffsetManager.queryThenEraseResetOffset(lmqName, group, 0)).thenReturn(null); + assertEquals(consumerOffset, popLiteMessageProcessor.getPopOffset(group, lmqName)); + + // not exist, init mode + long initOffset = 10L; + when(consumerOffsetManager.queryOffset(group, lmqName, 0)).thenReturn(-1L); + when(popMessageProcessor.getInitOffset(lmqName, group, 0, 1, true)).thenReturn(initOffset); + + assertEquals(initOffset, popLiteMessageProcessor.getPopOffset(group, lmqName)); + + verify(consumerOffsetManager, times(2)).queryThenEraseResetOffset(lmqName, group, 0); + verify(consumerOrderInfoManager, never()).clearBlock(anyString(), anyString(), anyInt()); + verify(consumerOffsetManager, never()).commitOffset(anyString(), anyString(), anyString(), anyInt(), anyLong()); + } + + + @Test + public void testGetPopOffset_resetOffset() { + String group = "group"; + String lmqName = "lmq"; + long consumerOffset = 100L; + long resetOffset = 50L; + + when(consumerOffsetManager.queryOffset(group, lmqName, 0)).thenReturn(consumerOffset); + when(consumerOffsetManager.queryThenEraseResetOffset(lmqName, group, 0)).thenReturn(resetOffset); + + assertEquals(resetOffset, popLiteMessageProcessor.getPopOffset(group, lmqName)); + + verify(consumerOffsetManager).queryOffset(group, lmqName, 0); + verify(consumerOffsetManager).queryThenEraseResetOffset(lmqName, group, 0); + verify(consumerOrderInfoManager).clearBlock(lmqName, group, 0); + verify(consumerOffsetManager).commitOffset("ResetOffset", group, lmqName, 0, resetOffset); + } + + @SuppressWarnings("unchecked") + @Test + public void testPopByClientId_noEvent() { + Iterator mockIterator = mock(Iterator.class); + when(mockIterator.hasNext()).thenReturn(false); + when(liteEventDispatcher.getEventIterator("clientId")).thenReturn(mockIterator); + + Pair result = popLiteMessageProcessor.popByClientId( + "clientHost", "parentTopic", "group", "clientId", System.currentTimeMillis(), 6000L, 32, "attemptId"); + + assertEquals(0, result.getObject1().length()); + assertEquals(0, result.getObject2().getMessageCount()); + verify(liteEventDispatcher).getEventIterator("clientId"); + } + + @SuppressWarnings("unchecked") + @Test + public void testPopByClientId_oneEvent() { + String event = "lmqName"; + int msgCount = 1; + GetMessageResult mockResult = mockGetMessageResult(GetMessageStatus.FOUND, msgCount, 100L); + long pollTime = System.currentTimeMillis(); + + Iterator mockIterator = mock(Iterator.class); + when(mockIterator.hasNext()).thenReturn(true, false); + when(mockIterator.next()).thenReturn(event); + when(liteEventDispatcher.getEventIterator("clientId")).thenReturn(mockIterator); + doReturn(new Pair<>(new StringBuilder("0"), mockResult)) + .when(popLiteMessageProcessor) + .popLiteTopic(anyString(), anyString(), anyString(), anyString(), anyLong(), anyLong(), anyLong(), anyString()); + + Pair result = popLiteMessageProcessor.popByClientId( + "clientHost", "parentTopic", "group", "clientId", pollTime, 6000L, 32, "attemptId"); + + assertEquals(msgCount, result.getObject2().getMessageCount()); + verify(mockIterator, times(2)).hasNext(); + verify(popLiteMessageProcessor).popLiteTopic("parentTopic" ,"clientHost", "group", event, 32L, pollTime, 6000L, "attemptId"); + } + + @SuppressWarnings("unchecked") + @Test + public void testPopByClientId_resultFull() { + String event1 = "lmqName1"; + String event2 = "lmqName2"; + int msgCount = 1; + GetMessageResult mockResult = mockGetMessageResult(GetMessageStatus.FOUND, msgCount, 100L); + long pollTime = System.currentTimeMillis(); + + Iterator mockIterator = mock(Iterator.class); + when(mockIterator.hasNext()).thenReturn(true, true, true, true, false); + when(mockIterator.next()).thenReturn(event1, event2, "event3", "event4"); + when(liteEventDispatcher.getEventIterator("clientId")).thenReturn(mockIterator); + doReturn(new Pair<>(new StringBuilder("0"), mockResult)) + .when(popLiteMessageProcessor) + .popLiteTopic(anyString(), anyString(), anyString(), anyString(), anyLong(), anyLong(), anyLong(), anyString()); + + Pair result = popLiteMessageProcessor.popByClientId( + "clientHost", "parentTopic", "group", "clientId", pollTime, 6000L, 2, "attemptId"); + + assertEquals(2, result.getObject2().getMessageCount()); + assertEquals("0;0", result.getObject1().toString()); + verify(mockIterator, times(2)).hasNext(); + verify(popLiteMessageProcessor).popLiteTopic("parentTopic", "clientHost", "group", event1, 2L, pollTime, 6000L, "attemptId"); + verify(popLiteMessageProcessor).popLiteTopic("parentTopic", "clientHost", "group", event2, 1L, pollTime, 6000L, "attemptId"); + } + + @SuppressWarnings("unchecked") + @Test + public void testPopByClientId_duplicateEvent() { + String event1 = "lmqName1"; + String event2 = "lmqName2"; + String event3 = "lmqName1"; + int msgCount = 1; + GetMessageResult mockResult = mockGetMessageResult(GetMessageStatus.FOUND, msgCount, 100L); + long pollTime = System.currentTimeMillis(); + + Iterator mockIterator = mock(Iterator.class); + when(mockIterator.hasNext()).thenReturn(true, true, true, false); + when(mockIterator.next()).thenReturn(event1, event2, event3); + when(liteEventDispatcher.getEventIterator("clientId")).thenReturn(mockIterator); + doReturn(new Pair<>(new StringBuilder("0"), mockResult)) + .when(popLiteMessageProcessor) + .popLiteTopic(anyString(), anyString(), anyString(), anyString(), anyLong(), anyLong(), anyLong(), anyString()); + + Pair result = popLiteMessageProcessor.popByClientId( + "clientHost", "parentTopic", "group", "clientId", pollTime, 6000L, 32, "attemptId"); + + assertEquals(2, result.getObject2().getMessageCount()); + assertEquals("0;0", result.getObject1().toString()); + verify(mockIterator, times(4)).hasNext(); + verify(popLiteMessageProcessor).popLiteTopic("parentTopic", "clientHost", "group", event1, 32L, pollTime, 6000L, "attemptId"); + verify(popLiteMessageProcessor).popLiteTopic("parentTopic", "clientHost", "group", event2, 31L, pollTime, 6000L, "attemptId"); + } + + @Test + public void testGetMessage_found() { + String group = "group"; + String lmqName = "lmqName"; + String clientHost = "clientHost"; + long offset = 50L; + int batchSize = 16; + GetMessageResult mockResult = mockGetMessageResult(GetMessageStatus.FOUND, 1, 100L); + when(messageStore.getMessage(group, lmqName, 0, offset, batchSize, null)).thenReturn(mockResult); + + GetMessageResult getMessageResult = + popLiteMessageProcessor.getMessage(clientHost, group, lmqName, offset, batchSize); + assertEquals(mockResult, getMessageResult); + verify(consumerOffsetManager, never()).commitOffset(clientHost, group, lmqName, 0, 100L); + } + + @Test + public void testGetMessage_notFound() { + String group = "group"; + String lmqName = "lmqName"; + String clientHost = "clientHost"; + long offset = 50L; + long nextBeginOffset = 100L; + int batchSize = 16; + + GetMessageResult firstResult = mockGetMessageResult(GetMessageStatus.MESSAGE_WAS_REMOVING, 0, nextBeginOffset); + when(messageStore.getMessage(group, lmqName, 0, offset, batchSize, null)).thenReturn(firstResult); + GetMessageResult secondResult = mockGetMessageResult(GetMessageStatus.FOUND, batchSize, nextBeginOffset + batchSize); + when(messageStore.getMessage(group, lmqName, 0, nextBeginOffset, batchSize, null)).thenReturn(secondResult); + + GetMessageResult getMessageResult = + popLiteMessageProcessor.getMessage(clientHost, group, lmqName, offset, batchSize); + assertEquals(secondResult, getMessageResult); + assertEquals(116, secondResult.getNextBeginOffset()); + verify(consumerOffsetManager).commitOffset("CorrectOffset", group, lmqName, 0, nextBeginOffset); + } + + @Test + public void testHandleGetMessageResult_nullResult() { + Pair result = popLiteMessageProcessor.handleGetMessageResult( + null, "parentTopic", "group", "lmqName", System.currentTimeMillis(), 6000L, "attemptId"); + assertNull(result); + } + + @Test + public void testHandleGetMessageResult_found() { + int msgCount = 2; + GetMessageResult getResult = mockGetMessageResult(GetMessageStatus.FOUND, msgCount, 100L); + getResult.getMessageQueueOffset().add(0L); + getResult.getMessageQueueOffset().add(1L); + + doNothing().when(popLiteMessageProcessor).recordPopLiteMetrics(any(), anyString(), anyString()); + + Pair result = popLiteMessageProcessor.handleGetMessageResult( + getResult, "parentTopic", "group", "lmqName", System.currentTimeMillis(), 6000L, "attemptId"); + + assertNotNull(result); + assertEquals(getResult, result.getObject2()); + assertEquals("0;0", result.getObject1().toString()); + } + + @Test + public void testPopLiteTopic_lockFailed() { + when(lockService.tryLock(anyString())).thenReturn(false); + + Pair result = popLiteMessageProcessor.popLiteTopic("parentTopic", + "clientHost", "group", "lmqName", 32L, System.currentTimeMillis(), 6000L, "attemptId"); + + assertNull(result); + verify(lockService).tryLock(anyString()); + verify(lockService, never()).unlock(anyString()); + } + + @Test + public void testPopLiteTopic_fifoBlocked() { + when(lockService.tryLock(anyString())).thenReturn(true); + when(consumerOrderInfoManager.checkBlock(anyString(), anyString(), anyString(), anyInt(), anyLong())) + .thenReturn(true); + + Pair result = popLiteMessageProcessor.popLiteTopic("parentTopic", + "clientHost", "group", "lmqName", 32L, System.currentTimeMillis(), 6000L, "attemptId"); + + assertThat(result).isNull(); + verify(lockService).tryLock(anyString()); + verify(lockService).unlock(anyString()); + } + + @Test + public void testPopLiteTopic_lmqNotExist() { + when(liteLifecycleManager.isLmqExist("lmqName")).thenReturn(false); + brokerConfig.setEnableLiteEventMode(false); + + Pair result = popLiteMessageProcessor.popLiteTopic("parentTopic", + "clientHost", "group", "lmqName", 32L, System.currentTimeMillis(), 6000L, "attemptId"); + + assertThat(result).isNull(); + verify(lockService, never()).tryLock(anyString()); + } + + @Test + public void testPopLiteTopic_found() { + when(lockService.tryLock(anyString())).thenReturn(true); + when(consumerOrderInfoManager.checkBlock(anyString(), anyString(), anyString(), anyInt(), anyLong())) + .thenReturn(false); + GetMessageResult mockResult = mockGetMessageResult(GetMessageStatus.FOUND, 1, 100L); + when(messageStore.getMessage("group", "lmqName", 0, 0, 32, null)).thenReturn(mockResult); + + Pair result = popLiteMessageProcessor.popLiteTopic("parentTopic", + "clientHost", "group", "lmqName", 32L, System.currentTimeMillis(), 6000L, "attemptId"); + + assertEquals(mockResult, result.getObject2()); + verify(lockService).tryLock(anyString()); + verify(lockService).unlock(anyString()); + } + + @Test + public void testPreCheck() { + final String parentTopic = "parentTopic"; + final String group = "group"; + final TopicConfig topicConfig = new TopicConfig(); + final SubscriptionGroupConfig groupConfig = new SubscriptionGroupConfig(); + final ChannelHandlerContext ctx = mock(ChannelHandlerContext.class); + + RemotingCommand response = RemotingCommand.createResponseCommand(PopLiteMessageResponseHeader.class); + PopLiteMessageRequestHeader requestHeader = new PopLiteMessageRequestHeader(); + when(topicConfigManager.selectTopicConfig(parentTopic)).thenReturn(topicConfig); + when(subscriptionGroupManager.findSubscriptionGroupConfig(group)).thenReturn(groupConfig); + + // timeout too much + requestHeader.setBornTime(System.currentTimeMillis() - 60000); + requestHeader.setPollTime(30000); + + RemotingCommand result = popLiteMessageProcessor.preCheck(ctx, requestHeader, response); + assertEquals(ResponseCode.POLLING_TIMEOUT, result.getCode()); + + // not readable + brokerConfig.setBrokerPermission(PermName.PERM_WRITE); + requestHeader.setBornTime(System.currentTimeMillis()); + requestHeader.setPollTime(30000); + + result = popLiteMessageProcessor.preCheck(ctx, requestHeader, response); + assertEquals(ResponseCode.NO_PERMISSION, result.getCode()); + brokerConfig.setBrokerPermission(PermName.PERM_READ | PermName.PERM_WRITE); + + // topic not exist + requestHeader.setTopic("whatever"); + + result = popLiteMessageProcessor.preCheck(ctx, requestHeader, response); + assertEquals(ResponseCode.TOPIC_NOT_EXIST, result.getCode()); + + // not lite topic type + requestHeader.setTopic(parentTopic); + + result = popLiteMessageProcessor.preCheck(ctx, requestHeader, response); + assertEquals(ResponseCode.INVALID_PARAMETER, result.getCode()); + + // group not exist + topicConfig.setPerm(PermName.PERM_READ | PermName.PERM_WRITE); + topicConfig.setTopicMessageType(TopicMessageType.LITE); + requestHeader.setConsumerGroup("whatever"); + + result = popLiteMessageProcessor.preCheck(ctx, requestHeader, response); + assertEquals(ResponseCode.SUBSCRIPTION_GROUP_NOT_EXIST, result.getCode()); + + // group disable + groupConfig.setConsumeEnable(false); + requestHeader.setConsumerGroup(group); + + result = popLiteMessageProcessor.preCheck(ctx, requestHeader, response); + assertEquals(ResponseCode.NO_PERMISSION, result.getCode()); + groupConfig.setConsumeEnable(true); + + // bind topic not match + groupConfig.setLiteBindTopic("otherTopic"); + requestHeader.setMaxMsgNum(32); + + result = popLiteMessageProcessor.preCheck(ctx, requestHeader, response); + assertEquals(ResponseCode.INVALID_PARAMETER, result.getCode()); + + // normal + groupConfig.setLiteBindTopic(parentTopic); + result = popLiteMessageProcessor.preCheck(ctx, requestHeader, response); + assertNull(result); + } + + + private GetMessageResult mockGetMessageResult(GetMessageStatus status, int messageCount, long nextBeginOffset) { + GetMessageResult getMessageResult = new GetMessageResult(); + getMessageResult.setStatus(status); + getMessageResult.setMinOffset(0); + getMessageResult.setMaxOffset(1024); + getMessageResult.setNextBeginOffset(nextBeginOffset); + + if (GetMessageStatus.FOUND.equals(status)) { + for (int i = 0; i < messageCount; i++) { + getMessageResult.addMessage(Mockito.mock(SelectMappedBufferResult.class)); + } + } + return getMessageResult; + } +} diff --git a/client/src/main/java/org/apache/rocketmq/client/impl/MQClientAPIImpl.java b/client/src/main/java/org/apache/rocketmq/client/impl/MQClientAPIImpl.java index f8e662331f4..8294ffd422f 100644 --- a/client/src/main/java/org/apache/rocketmq/client/impl/MQClientAPIImpl.java +++ b/client/src/main/java/org/apache/rocketmq/client/impl/MQClientAPIImpl.java @@ -17,6 +17,24 @@ package org.apache.rocketmq.client.impl; import com.alibaba.fastjson2.JSON; +import java.io.UnsupportedEncodingException; +import java.nio.ByteBuffer; +import java.util.ArrayList; +import java.util.Arrays; +import java.util.BitSet; +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.Properties; +import java.util.Set; +import java.util.concurrent.ConcurrentHashMap; +import java.util.concurrent.ConcurrentMap; +import java.util.concurrent.TimeUnit; +import java.util.concurrent.atomic.AtomicInteger; import org.apache.commons.lang3.ArrayUtils; import org.apache.commons.lang3.StringUtils; import org.apache.rocketmq.client.ClientConfig; @@ -108,7 +126,12 @@ import org.apache.rocketmq.remoting.protocol.body.ConsumerRunningInfo; import org.apache.rocketmq.remoting.protocol.body.CreateTopicListRequestBody; import org.apache.rocketmq.remoting.protocol.body.EpochEntryCache; +import org.apache.rocketmq.remoting.protocol.body.GetBrokerLiteInfoResponseBody; import org.apache.rocketmq.remoting.protocol.body.GetConsumerStatusBody; +import org.apache.rocketmq.remoting.protocol.body.GetLiteClientInfoResponseBody; +import org.apache.rocketmq.remoting.protocol.body.GetLiteGroupInfoResponseBody; +import org.apache.rocketmq.remoting.protocol.body.GetLiteTopicInfoResponseBody; +import org.apache.rocketmq.remoting.protocol.body.GetParentTopicInfoResponseBody; import org.apache.rocketmq.remoting.protocol.body.GroupList; import org.apache.rocketmq.remoting.protocol.body.HARuntimeInfo; import org.apache.rocketmq.remoting.protocol.body.KVTable; @@ -165,10 +188,14 @@ import org.apache.rocketmq.remoting.protocol.header.GetConsumerStatusRequestHeader; import org.apache.rocketmq.remoting.protocol.header.GetEarliestMsgStoretimeRequestHeader; import org.apache.rocketmq.remoting.protocol.header.GetEarliestMsgStoretimeResponseHeader; +import org.apache.rocketmq.remoting.protocol.header.GetLiteClientInfoRequestHeader; +import org.apache.rocketmq.remoting.protocol.header.GetLiteGroupInfoRequestHeader; +import org.apache.rocketmq.remoting.protocol.header.GetLiteTopicInfoRequestHeader; import org.apache.rocketmq.remoting.protocol.header.GetMaxOffsetRequestHeader; import org.apache.rocketmq.remoting.protocol.header.GetMaxOffsetResponseHeader; import org.apache.rocketmq.remoting.protocol.header.GetMinOffsetRequestHeader; import org.apache.rocketmq.remoting.protocol.header.GetMinOffsetResponseHeader; +import org.apache.rocketmq.remoting.protocol.header.GetParentTopicInfoRequestHeader; import org.apache.rocketmq.remoting.protocol.header.GetProducerConnectionListRequestHeader; import org.apache.rocketmq.remoting.protocol.header.GetSubscriptionGroupConfigRequestHeader; import org.apache.rocketmq.remoting.protocol.header.GetTopicConfigRequestHeader; @@ -179,6 +206,8 @@ import org.apache.rocketmq.remoting.protocol.header.ListAclsRequestHeader; import org.apache.rocketmq.remoting.protocol.header.ListUsersRequestHeader; import org.apache.rocketmq.remoting.protocol.header.LockBatchMqRequestHeader; +import org.apache.rocketmq.remoting.protocol.header.PopLiteMessageRequestHeader; +import org.apache.rocketmq.remoting.protocol.header.PopLiteMessageResponseHeader; import org.apache.rocketmq.remoting.protocol.header.PopMessageRequestHeader; import org.apache.rocketmq.remoting.protocol.header.PopMessageResponseHeader; import org.apache.rocketmq.remoting.protocol.header.PullMessageRequestHeader; @@ -203,6 +232,7 @@ import org.apache.rocketmq.remoting.protocol.header.SendMessageRequestHeader; import org.apache.rocketmq.remoting.protocol.header.SendMessageRequestHeaderV2; import org.apache.rocketmq.remoting.protocol.header.SendMessageResponseHeader; +import org.apache.rocketmq.remoting.protocol.header.TriggerLiteDispatchRequestHeader; import org.apache.rocketmq.remoting.protocol.header.UnlockBatchMqRequestHeader; import org.apache.rocketmq.remoting.protocol.header.UnregisterClientRequestHeader; import org.apache.rocketmq.remoting.protocol.header.UpdateAclRequestHeader; @@ -236,26 +266,8 @@ import org.apache.rocketmq.remoting.protocol.subscription.SubscriptionGroupConfig; import org.apache.rocketmq.remoting.rpchook.DynamicalExtFieldRPCHook; import org.apache.rocketmq.remoting.rpchook.StreamTypeRPCHook; -import static org.apache.rocketmq.common.message.MessageConst.TIMER_ENGINE_TYPE; -import java.io.UnsupportedEncodingException; -import java.nio.ByteBuffer; -import java.util.ArrayList; -import java.util.Arrays; -import java.util.BitSet; -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.Properties; -import java.util.Set; -import java.util.concurrent.ConcurrentHashMap; -import java.util.concurrent.ConcurrentMap; -import java.util.concurrent.TimeUnit; -import java.util.concurrent.atomic.AtomicInteger; +import static org.apache.rocketmq.common.message.MessageConst.TIMER_ENGINE_TYPE; import static org.apache.rocketmq.remoting.protocol.RemotingSysResponseCode.SUCCESS; public class MQClientAPIImpl implements NameServerUpdateCallback, StartAndShutdown { @@ -321,6 +333,8 @@ public MQClientAPIImpl(final NettyClientConfig nettyClientConfig, this.remotingClient.registerRPCHook(new DynamicalExtFieldRPCHook()); this.remotingClient.registerProcessor(RequestCode.CHECK_TRANSACTION_STATE, this.clientRemotingProcessor, null); + this.remotingClient.registerProcessor(RequestCode.NOTIFY_UNSUBSCRIBE_LITE, this.clientRemotingProcessor, null); + this.remotingClient.registerProcessor(RequestCode.NOTIFY_CONSUMER_IDS_CHANGED, this.clientRemotingProcessor, null); this.remotingClient.registerProcessor(RequestCode.RESET_CONSUMER_CLIENT_OFFSET, this.clientRemotingProcessor, null); @@ -848,6 +862,34 @@ public void operationFail(Throwable throwable) { }); } + public void popLiteMessageAsync( + final String brokerName, final String addr, final PopLiteMessageRequestHeader requestHeader, + final long timeoutMillis, final PopCallback popCallback + ) throws RemotingException, InterruptedException { + final String bindTopic = requestHeader.getTopic(); + final RemotingCommand request = RemotingCommand.createRequestCommand(RequestCode.POP_LITE_MESSAGE, requestHeader); + this.remotingClient.invokeAsync(addr, request, timeoutMillis, new InvokeCallback() { + @Override + public void operationComplete(ResponseFuture responseFuture) { + } + + @Override + public void operationSucceed(RemotingCommand response) { + try { + PopResult popResult = MQClientAPIImpl.this.processPopLiteResponse(brokerName, response, bindTopic, requestHeader); + popCallback.onSuccess(popResult); + } catch (Exception e) { + popCallback.onException(e); + } + } + + @Override + public void operationFail(Throwable throwable) { + popCallback.onException(throwable); + } + }); + } + public void ackMessageAsync( final String addr, final long timeOut, @@ -857,6 +899,15 @@ public void ackMessageAsync( ackMessageAsync(addr, timeOut, ackCallback, requestHeader, null); } + public void ackLiteMessageAsync( + final String addr, + final long timeout, + final AckCallback ackCallback, + final AckMessageRequestHeader requestHeader + ) throws RemotingException, MQBrokerException, InterruptedException { + ackMessageAsync(addr, timeout, ackCallback, requestHeader, null); + } + public void batchAckMessageAsync( final String addr, final long timeOut, @@ -1176,6 +1227,62 @@ private PopResult processPopResponse(final String brokerName, final RemotingComm return popResult; } + private PopResult processPopLiteResponse(final String brokerName, final RemotingCommand response, String topic, + CommandCustomHeader requestHeader) throws MQBrokerException, RemotingCommandException { + PopStatus popStatus; + List msgFoundList = null; + switch (response.getCode()) { + case ResponseCode.SUCCESS: + popStatus = PopStatus.FOUND; + ByteBuffer byteBuffer = ByteBuffer.wrap(response.getBody()); + msgFoundList = MessageDecoder.decodesBatch( + byteBuffer, + clientConfig.isDecodeReadBody(), + clientConfig.isDecodeDecompressBody(), + true); + break; + case ResponseCode.POLLING_FULL: + popStatus = PopStatus.POLLING_FULL; + break; + case ResponseCode.POLLING_TIMEOUT: + popStatus = PopStatus.POLLING_NOT_FOUND; + break; + case ResponseCode.PULL_NOT_FOUND: + popStatus = PopStatus.POLLING_NOT_FOUND; + break; + default: + throw new MQBrokerException(response.getCode(), response.getRemark()); + } + + PopResult popResult = new PopResult(popStatus, msgFoundList); + PopLiteMessageResponseHeader responseHeader = response.decodeCommandCustomHeader(PopLiteMessageResponseHeader.class); + if (popStatus != PopStatus.FOUND) { + return popResult; + } + + List orderCountList = ExtraInfoUtil.parseLiteOrderCountInfo(responseHeader.getOrderCountInfo(), msgFoundList.size()); + for (int i = 0; i < msgFoundList.size(); i++) { + MessageExt messageExt = msgFoundList.get(i); + String[] queues = StringUtils.split( + messageExt.getProperty(MessageConst.PROPERTY_INNER_MULTI_DISPATCH), MixAll.LMQ_DISPATCH_SEPARATOR); + String[] queueOffsets = StringUtils.split( + messageExt.getProperty(MessageConst.PROPERTY_INNER_MULTI_QUEUE_OFFSET), MixAll.LMQ_DISPATCH_SEPARATOR); + + if (null == queues || null == queueOffsets || queues.length != 1 || queues.length != queueOffsets.length) { + continue; + } + messageExt.getProperties().put(MessageConst.PROPERTY_POP_CK, + ExtraInfoUtil.buildExtraInfo(0, responseHeader.getPopTime(), responseHeader.getInvisibleTime(), + responseHeader.getReviveQid(), topic, brokerName, 0, Long.parseLong(queueOffsets[0]))); + messageExt.getProperties().computeIfAbsent( + MessageConst.PROPERTY_FIRST_POP_TIME, k -> String.valueOf(responseHeader.getPopTime())); + messageExt.setBrokerName(brokerName); + messageExt.setReconsumeTimes(orderCountList != null ? orderCountList.get(i) : 0); + messageExt.setQueueOffset(Long.parseLong(queueOffsets[0])); + } + return popResult; + } + /** * Build queue offset sorted map * @@ -3662,4 +3769,84 @@ public void switchTimerEngine(String brokerAddr, String engineType, long timeout throw new MQBrokerException(response.getCode(), response.getRemark()); } + + public GetBrokerLiteInfoResponseBody getBrokerLiteInfo(String addr, long timeoutMillis) + throws RemotingException, MQBrokerException, InterruptedException { + return invokeBrokerMethod(addr, RequestCode.GET_BROKER_LITE_INFO, null, + GetBrokerLiteInfoResponseBody.class, timeoutMillis); + } + + public GetParentTopicInfoResponseBody getParentTopicInfo(String addr, String topic, long timeoutMillis) + throws RemotingException, MQBrokerException, InterruptedException { + GetParentTopicInfoRequestHeader requestHeader = new GetParentTopicInfoRequestHeader(); + requestHeader.setTopic(topic); + return invokeBrokerMethod(addr, RequestCode.GET_PARENT_TOPIC_INFO, requestHeader, + GetParentTopicInfoResponseBody.class, timeoutMillis); + } + + public GetLiteTopicInfoResponseBody getLiteTopicInfo(String addr, String parentTopic, String liteTopic, + long timeoutMillis) + throws RemotingException, MQBrokerException, InterruptedException { + GetLiteTopicInfoRequestHeader requestHeader = new GetLiteTopicInfoRequestHeader(); + requestHeader.setParentTopic(parentTopic); + requestHeader.setLiteTopic(liteTopic); + return invokeBrokerMethod(addr, RequestCode.GET_LITE_TOPIC_INFO, requestHeader, + GetLiteTopicInfoResponseBody.class, timeoutMillis); + } + + public GetLiteClientInfoResponseBody getLiteClientInfo(String addr, String parentTopic, String group, + String clientId, long timeoutMillis) + throws RemotingException, MQBrokerException, InterruptedException { + GetLiteClientInfoRequestHeader requestHeader = new GetLiteClientInfoRequestHeader(); + requestHeader.setParentTopic(parentTopic); + requestHeader.setGroup(group); + requestHeader.setClientId(clientId); + return invokeBrokerMethod(addr, RequestCode.GET_LITE_CLIENT_INFO, requestHeader, + GetLiteClientInfoResponseBody.class, timeoutMillis); + } + + public GetLiteGroupInfoResponseBody getLiteGroupInfo(String addr, String group, + String liteTopic, int topK, long timeoutMillis) + throws RemotingException, MQBrokerException, InterruptedException { + GetLiteGroupInfoRequestHeader requestHeader = new GetLiteGroupInfoRequestHeader(); + requestHeader.setGroup(group); + requestHeader.setTopK(topK); + requestHeader.setLiteTopic(liteTopic); + return invokeBrokerMethod(addr, RequestCode.GET_LITE_GROUP_INFO, requestHeader, + GetLiteGroupInfoResponseBody.class, timeoutMillis); + } + + public void triggerLiteDispatch(String addr, String group, String clientId, long timeoutMillis) + throws RemotingException, MQBrokerException, InterruptedException { + TriggerLiteDispatchRequestHeader requestHeader = new TriggerLiteDispatchRequestHeader(); + requestHeader.setGroup(group); + requestHeader.setClientId(clientId); + invokeBrokerMethod(addr, RequestCode.TRIGGER_LITE_DISPATCH, requestHeader, null, timeoutMillis); + } + + private R invokeBrokerMethod( + final String addr, + final int requestCode, + final T requestHeader, + final Class responseClass, + final long timeoutMillis + ) throws RemotingException, MQBrokerException, InterruptedException { + RemotingCommand request = RemotingCommand.createRequestCommand(requestCode, requestHeader); + + RemotingCommand response = this.remotingClient.invokeSync( + MixAll.brokerVIPChannel(this.clientConfig.isVipChannelEnabled(), addr), + request, + timeoutMillis + ); + + if (response.getCode() == SUCCESS) { + if (response.getBody() != null) { + return RemotingSerializable.decode(response.getBody(), responseClass); + } + return null; + } + + throw new MQBrokerException(response.getCode(), response.getRemark(), addr); + } + } diff --git a/client/src/main/java/org/apache/rocketmq/client/impl/mqclient/MQClientAPIExt.java b/client/src/main/java/org/apache/rocketmq/client/impl/mqclient/MQClientAPIExt.java index 90895034070..75bc5438f39 100644 --- a/client/src/main/java/org/apache/rocketmq/client/impl/mqclient/MQClientAPIExt.java +++ b/client/src/main/java/org/apache/rocketmq/client/impl/mqclient/MQClientAPIExt.java @@ -21,6 +21,7 @@ import java.util.List; import java.util.Set; import java.util.concurrent.CompletableFuture; +import java.util.concurrent.CompletionException; import org.apache.rocketmq.client.ClientConfig; import org.apache.rocketmq.client.consumer.AckCallback; import org.apache.rocketmq.client.consumer.AckResult; @@ -40,6 +41,7 @@ import org.apache.rocketmq.client.producer.SendResult; import org.apache.rocketmq.common.ObjectCreator; import org.apache.rocketmq.common.constant.LoggerName; +import org.apache.rocketmq.common.lite.LiteSubscriptionDTO; import org.apache.rocketmq.common.message.Message; import org.apache.rocketmq.common.message.MessageBatch; import org.apache.rocketmq.common.message.MessageClientIDSetter; @@ -57,6 +59,8 @@ import org.apache.rocketmq.remoting.protocol.RemotingCommand; import org.apache.rocketmq.remoting.protocol.RequestCode; import org.apache.rocketmq.remoting.protocol.ResponseCode; +import org.apache.rocketmq.remoting.protocol.body.GetLiteTopicInfoResponseBody; +import org.apache.rocketmq.remoting.protocol.body.LiteSubscriptionCtlRequestBody; import org.apache.rocketmq.remoting.protocol.body.LockBatchRequestBody; import org.apache.rocketmq.remoting.protocol.body.LockBatchResponseBody; import org.apache.rocketmq.remoting.protocol.body.UnlockBatchRequestBody; @@ -65,14 +69,17 @@ import org.apache.rocketmq.remoting.protocol.header.ConsumerSendMsgBackRequestHeader; import org.apache.rocketmq.remoting.protocol.header.GetConsumerListByGroupRequestHeader; import org.apache.rocketmq.remoting.protocol.header.GetConsumerListByGroupResponseBody; +import org.apache.rocketmq.remoting.protocol.header.GetLiteTopicInfoRequestHeader; import org.apache.rocketmq.remoting.protocol.header.GetMaxOffsetRequestHeader; import org.apache.rocketmq.remoting.protocol.header.GetMaxOffsetResponseHeader; import org.apache.rocketmq.remoting.protocol.header.GetMinOffsetRequestHeader; import org.apache.rocketmq.remoting.protocol.header.GetMinOffsetResponseHeader; import org.apache.rocketmq.remoting.protocol.header.HeartbeatRequestHeader; +import org.apache.rocketmq.remoting.protocol.header.LiteSubscriptionCtlRequestHeader; import org.apache.rocketmq.remoting.protocol.header.LockBatchMqRequestHeader; import org.apache.rocketmq.remoting.protocol.header.NotificationRequestHeader; import org.apache.rocketmq.remoting.protocol.header.NotificationResponseHeader; +import org.apache.rocketmq.remoting.protocol.header.PopLiteMessageRequestHeader; import org.apache.rocketmq.remoting.protocol.header.PopMessageRequestHeader; import org.apache.rocketmq.remoting.protocol.header.PullMessageRequestHeader; import org.apache.rocketmq.remoting.protocol.header.QueryConsumerOffsetRequestHeader; @@ -252,6 +259,31 @@ public void onException(Throwable t) { return future; } + public CompletableFuture popLiteMessageAsync( + String brokerAddr, + String brokerName, + PopLiteMessageRequestHeader requestHeader, + long timeoutMillis + ) { + CompletableFuture future = new CompletableFuture<>(); + try { + this.popLiteMessageAsync(brokerName, brokerAddr, requestHeader, timeoutMillis, new PopCallback() { + @Override + public void onSuccess(PopResult popResult) { + future.complete(popResult); + } + + @Override + public void onException(Throwable t) { + future.completeExceptionally(t); + } + }); + } catch (Throwable t) { + future.completeExceptionally(t); + } + return future; + } + public CompletableFuture ackMessageAsync( String brokerAddr, AckMessageRequestHeader requestHeader, @@ -668,6 +700,58 @@ public CompletableFuture recallMessageAsync(String brokerAddr, }); } + public CompletableFuture syncLiteSubscriptionAsync( + String brokerAddr, + LiteSubscriptionDTO liteSubscriptionDTO, + long timeoutMillis + ) { + LiteSubscriptionCtlRequestBody requestBody = new LiteSubscriptionCtlRequestBody(); + requestBody.setSubscriptionSet(Collections.singleton(liteSubscriptionDTO)); + RemotingCommand request = RemotingCommand + .createRequestCommand(RequestCode.LITE_SUBSCRIPTION_CTL, new LiteSubscriptionCtlRequestHeader()); + request.setBody(requestBody.encode()); + + return getRemotingClient() + .invoke(brokerAddr, request, timeoutMillis) + .thenCompose(response -> { + if (ResponseCode.SUCCESS == response.getCode()) { + return CompletableFuture.completedFuture(null); + } else { + CompletableFuture future = new CompletableFuture<>(); + future.completeExceptionally( + new MQBrokerException(response.getCode(), response.getRemark(), brokerAddr) + ); + return future; + } + }); + } + + public CompletableFuture getLiteTopicInfoAsync( + String addr, + String parentTopic, + String liteTopic, + long timeoutMillis + ) { + GetLiteTopicInfoRequestHeader requestHeader = new GetLiteTopicInfoRequestHeader(); + requestHeader.setParentTopic(parentTopic); + requestHeader.setLiteTopic(liteTopic); + RemotingCommand request = RemotingCommand.createRequestCommand(RequestCode.GET_LITE_TOPIC_INFO, requestHeader); + + return this.getRemotingClient() + .invoke(addr, request, timeoutMillis) + .thenApply(response -> { + if (ResponseCode.SUCCESS == response.getCode()) { + try { + return GetLiteTopicInfoResponseBody.decode(response.getBody(), GetLiteTopicInfoResponseBody.class); + } catch (Exception e) { + throw new CompletionException(e); + } + } else { + throw new CompletionException(new MQBrokerException(response.getCode(), response.getRemark())); + } + }); + } + public CompletableFuture invoke(String brokerAddr, RemotingCommand request, long timeoutMillis) { return getRemotingClient().invoke(brokerAddr, request, timeoutMillis); } diff --git a/client/src/test/java/org/apache/rocketmq/client/impl/MQClientAPIImplTest.java b/client/src/test/java/org/apache/rocketmq/client/impl/MQClientAPIImplTest.java index c8d23274bd9..27b3d685715 100644 --- a/client/src/test/java/org/apache/rocketmq/client/impl/MQClientAPIImplTest.java +++ b/client/src/test/java/org/apache/rocketmq/client/impl/MQClientAPIImplTest.java @@ -541,7 +541,7 @@ public void testPopLmqMessage_async() throws Exception { message.setBody("body".getBytes()); message.setTopic(topic); message.putUserProperty("key", "value"); - message.putUserProperty(MessageConst.PROPERTY_INNER_MULTI_DISPATCH, lmqTopic); + MessageAccessor.putProperty(message, MessageConst.PROPERTY_INNER_MULTI_DISPATCH, lmqTopic); message.getProperties().put(MessageConst.PROPERTY_INNER_MULTI_QUEUE_OFFSET, String.valueOf(0)); response.setBody(MessageDecoder.encode(message, false)); responseFuture.setResponseCommand(response); diff --git a/common/src/main/java/org/apache/rocketmq/common/BrokerConfig.java b/common/src/main/java/org/apache/rocketmq/common/BrokerConfig.java index 8b5a43fd358..99539b74342 100644 --- a/common/src/main/java/org/apache/rocketmq/common/BrokerConfig.java +++ b/common/src/main/java/org/apache/rocketmq/common/BrokerConfig.java @@ -491,6 +491,39 @@ public class BrokerConfig extends BrokerIdentity { private boolean enableCreateSysGroup = true; + private boolean enableLiteEventMode = true; + + private long liteEventCheckInterval = 10 * 1000; + + private long liteTtlCheckInterval = 120 * 1000; + + private long minLiteTTl = 15 * 60 * 1000; + + private long liteSubscriptionCheckInterval = TimeUnit.MINUTES.toMillis(2); + + private long liteSubscriptionCheckTimeoutMills = TimeUnit.MINUTES.toMillis(3); + + // make sense for rocksdb store + private boolean persistConsumerOffsetIncrementally = false; + + private long maxLiteSubscriptionCount = 100000; + + private boolean enableLitePopLog = false; + + private int maxClientEventCount = 100; + + private long liteEventFullDispatchDelayTime = 10 * 1000; + + // lite metrics + // whether to collect storeTime in popLiteProcessor + private boolean liteLagLatencyCollectEnable = false; + + private boolean liteLagLatencyMetricsEnable = false; + + private boolean liteLagCountMetricsEnable = false; + + private int liteLagLatencyTopK = 50; + public String getConfigBlackList() { return configBlackList; } @@ -2215,4 +2248,124 @@ public boolean isUseSeparateRetryQueue() { public void setUseSeparateRetryQueue(boolean useSeparateRetryQueue) { this.useSeparateRetryQueue = useSeparateRetryQueue; } + + public boolean isEnableLiteEventMode() { + return enableLiteEventMode; + } + + public void setEnableLiteEventMode(boolean enableLiteEventMode) { + this.enableLiteEventMode = enableLiteEventMode; + } + + public long getLiteEventCheckInterval() { + return liteEventCheckInterval; + } + + public void setLiteEventCheckInterval(long liteEventCheckInterval) { + this.liteEventCheckInterval = liteEventCheckInterval; + } + + public long getLiteTtlCheckInterval() { + return liteTtlCheckInterval; + } + + public void setLiteTtlCheckInterval(long liteTtlCheckInterval) { + this.liteTtlCheckInterval = liteTtlCheckInterval; + } + + public long getMinLiteTTl() { + return minLiteTTl; + } + + public void setMinLiteTTl(long minLiteTTl) { + this.minLiteTTl = minLiteTTl; + } + + public long getLiteSubscriptionCheckInterval() { + return liteSubscriptionCheckInterval; + } + + public void setLiteSubscriptionCheckInterval(long liteSubscriptionCheckInterval) { + this.liteSubscriptionCheckInterval = liteSubscriptionCheckInterval; + } + + public long getLiteSubscriptionCheckTimeoutMills() { + return liteSubscriptionCheckTimeoutMills; + } + + public void setLiteSubscriptionCheckTimeoutMills(long liteSubscriptionCheckTimeoutMills) { + this.liteSubscriptionCheckTimeoutMills = liteSubscriptionCheckTimeoutMills; + } + + public boolean isPersistConsumerOffsetIncrementally() { + return persistConsumerOffsetIncrementally; + } + + public void setPersistConsumerOffsetIncrementally(boolean persistConsumerOffsetIncrementally) { + this.persistConsumerOffsetIncrementally = persistConsumerOffsetIncrementally; + } + + public long getMaxLiteSubscriptionCount() { + return maxLiteSubscriptionCount; + } + + public void setMaxLiteSubscriptionCount(long maxLiteSubscriptionCount) { + this.maxLiteSubscriptionCount = maxLiteSubscriptionCount; + } + + public boolean isEnableLitePopLog() { + return enableLitePopLog; + } + + public void setEnableLitePopLog(boolean enableLitePopLog) { + this.enableLitePopLog = enableLitePopLog; + } + + public int getMaxClientEventCount() { + return maxClientEventCount; + } + + public void setMaxClientEventCount(int maxClientEventCount) { + this.maxClientEventCount = maxClientEventCount; + } + + public long getLiteEventFullDispatchDelayTime() { + return liteEventFullDispatchDelayTime; + } + + public void setLiteEventFullDispatchDelayTime(long liteEventFullDispatchDelayTime) { + this.liteEventFullDispatchDelayTime = liteEventFullDispatchDelayTime; + } + + public boolean isLiteLagLatencyCollectEnable() { + return liteLagLatencyCollectEnable; + } + + public void setLiteLagLatencyCollectEnable(boolean liteLagLatencyCollectEnable) { + this.liteLagLatencyCollectEnable = liteLagLatencyCollectEnable; + } + + public boolean isLiteLagLatencyMetricsEnable() { + return liteLagLatencyMetricsEnable; + } + + public void setLiteLagLatencyMetricsEnable(boolean liteLagLatencyMetricsEnable) { + this.liteLagLatencyMetricsEnable = liteLagLatencyMetricsEnable; + } + + public boolean isLiteLagCountMetricsEnable() { + return liteLagCountMetricsEnable; + } + + public void setLiteLagCountMetricsEnable(boolean liteLagCountMetricsEnable) { + this.liteLagCountMetricsEnable = liteLagCountMetricsEnable; + } + + public int getLiteLagLatencyTopK() { + return liteLagLatencyTopK; + } + + public void setLiteLagLatencyTopK(int liteLagLatencyTopK) { + this.liteLagLatencyTopK = liteLagLatencyTopK; + } } diff --git a/common/src/main/java/org/apache/rocketmq/common/KeyBuilder.java b/common/src/main/java/org/apache/rocketmq/common/KeyBuilder.java index 910a73b7137..19fe9ec5286 100644 --- a/common/src/main/java/org/apache/rocketmq/common/KeyBuilder.java +++ b/common/src/main/java/org/apache/rocketmq/common/KeyBuilder.java @@ -79,4 +79,8 @@ public static String buildPollingKey(String topic, String cid, int queueId) { public static boolean isPopRetryTopicV2(String retryTopic) { return retryTopic.startsWith(MixAll.RETRY_GROUP_TOPIC_PREFIX) && retryTopic.contains(String.valueOf(POP_RETRY_SEPARATOR_V2)); } + + public static String buildPopLiteLockKey(String group, String lmqName) { + return group + PopAckConstants.SPLIT + lmqName; + } } diff --git a/common/src/main/java/org/apache/rocketmq/common/MixAll.java b/common/src/main/java/org/apache/rocketmq/common/MixAll.java index e0255032ed3..efde29a891b 100644 --- a/common/src/main/java/org/apache/rocketmq/common/MixAll.java +++ b/common/src/main/java/org/apache/rocketmq/common/MixAll.java @@ -574,6 +574,7 @@ public static boolean isSysConsumerGroupPullMessage(String consumerGroup) { public static boolean topicAllowsLMQ(String topic) { return !topic.startsWith(MixAll.RETRY_GROUP_TOPIC_PREFIX) + && !topic.startsWith(MixAll.DLQ_GROUP_TOPIC_PREFIX) && !topic.startsWith(TopicValidator.SYSTEM_TOPIC_PREFIX) && !topic.equals(TopicValidator.RMQ_SYS_SCHEDULE_TOPIC); } diff --git a/common/src/main/java/org/apache/rocketmq/common/ServiceThread.java b/common/src/main/java/org/apache/rocketmq/common/ServiceThread.java index 96195d53090..cec00bab02b 100644 --- a/common/src/main/java/org/apache/rocketmq/common/ServiceThread.java +++ b/common/src/main/java/org/apache/rocketmq/common/ServiceThread.java @@ -41,7 +41,9 @@ public ServiceThread() { } - public abstract String getServiceName(); + public String getServiceName() { + return this.getClass().getSimpleName(); + } public void start() { log.info("Try to start service thread:{} started:{} lastThread:{}", getServiceName(), started.get(), thread); diff --git a/common/src/main/java/org/apache/rocketmq/common/SubscriptionGroupAttributes.java b/common/src/main/java/org/apache/rocketmq/common/SubscriptionGroupAttributes.java index 845f407939b..12f5dbf67e0 100644 --- a/common/src/main/java/org/apache/rocketmq/common/SubscriptionGroupAttributes.java +++ b/common/src/main/java/org/apache/rocketmq/common/SubscriptionGroupAttributes.java @@ -16,12 +16,19 @@ */ package org.apache.rocketmq.common; +import static com.google.common.collect.Sets.newHashSet; + import java.util.HashMap; import java.util.Map; import org.apache.rocketmq.common.attribute.Attribute; +import org.apache.rocketmq.common.attribute.BooleanAttribute; +import org.apache.rocketmq.common.attribute.EnumAttribute; import org.apache.rocketmq.common.attribute.LongRangeAttribute; +import org.apache.rocketmq.common.attribute.StringAttribute; +import org.apache.rocketmq.common.attribute.LiteSubModel; public class SubscriptionGroupAttributes { + public static final Map ALL; public static final LongRangeAttribute PRIORITY_FACTOR_ATTRIBUTE = new LongRangeAttribute( "priority.factor", @@ -31,8 +38,57 @@ public class SubscriptionGroupAttributes { 100 ); + public static final StringAttribute LITE_BIND_TOPIC_ATTRIBUTE = new StringAttribute( + "lite.bind.topic", + true + ); + + public static final EnumAttribute LITE_SUB_MODEL_ATTRIBUTE = new EnumAttribute( + "lite.sub.model", + true, + newHashSet(LiteSubModel.Shared.name(), LiteSubModel.Exclusive.name()), + LiteSubModel.Shared.name() + ); + + public static final BooleanAttribute LITE_SUB_RESET_OFFSET_EXCLUSIVE_ATTRIBUTE = new BooleanAttribute( + "lite.sub.reset.offset.exclusive", + true, + false + ); + + public static final BooleanAttribute LITE_SUB_RESET_OFFSET_UNSUBSCRIBE_ATTRIBUTE = new BooleanAttribute( + "lite.sub.reset.offset.unsubscribe", + true, + false + ); + + /** + * client-side lite subscription quota limit + */ + public static final LongRangeAttribute LITE_SUB_CLIENT_QUOTA_ATTRIBUTE = new LongRangeAttribute( + "lite.sub.client.quota", + true, + -1, + Long.MAX_VALUE, + 2000 + ); + + public static final LongRangeAttribute LITE_SUB_CLIENT_MAX_EVENT_COUNT = new LongRangeAttribute( + "lite.sub.client.max.event.cnt", + true, + 10, + Long.MAX_VALUE, + 400 + ); + static { ALL = new HashMap<>(); ALL.put(PRIORITY_FACTOR_ATTRIBUTE.getName(), PRIORITY_FACTOR_ATTRIBUTE); + ALL.put(LITE_BIND_TOPIC_ATTRIBUTE.getName(), LITE_BIND_TOPIC_ATTRIBUTE); + ALL.put(LITE_SUB_CLIENT_QUOTA_ATTRIBUTE.getName(), LITE_SUB_CLIENT_QUOTA_ATTRIBUTE); + ALL.put(LITE_SUB_MODEL_ATTRIBUTE.getName(), LITE_SUB_MODEL_ATTRIBUTE); + ALL.put(LITE_SUB_RESET_OFFSET_EXCLUSIVE_ATTRIBUTE.getName(), LITE_SUB_RESET_OFFSET_EXCLUSIVE_ATTRIBUTE); + ALL.put(LITE_SUB_RESET_OFFSET_UNSUBSCRIBE_ATTRIBUTE.getName(), LITE_SUB_RESET_OFFSET_UNSUBSCRIBE_ATTRIBUTE); + ALL.put(LITE_SUB_CLIENT_MAX_EVENT_COUNT.getName(), LITE_SUB_CLIENT_MAX_EVENT_COUNT); } } diff --git a/common/src/main/java/org/apache/rocketmq/common/TopicAttributes.java b/common/src/main/java/org/apache/rocketmq/common/TopicAttributes.java index c507748c677..6a70088e0d4 100644 --- a/common/src/main/java/org/apache/rocketmq/common/TopicAttributes.java +++ b/common/src/main/java/org/apache/rocketmq/common/TopicAttributes.java @@ -18,6 +18,7 @@ import java.util.HashMap; import java.util.Map; +import java.util.concurrent.TimeUnit; import org.apache.rocketmq.common.attribute.Attribute; import org.apache.rocketmq.common.attribute.EnumAttribute; import org.apache.rocketmq.common.attribute.LongRangeAttribute; @@ -52,6 +53,14 @@ public class TopicAttributes { -1 ); + public static final LongRangeAttribute LITE_EXPIRATION_ATTRIBUTE = new LongRangeAttribute( + "lite.topic.expiration", + true, + -1, + TimeUnit.DAYS.toMinutes(30), + -1 + ); + public static final Map ALL; static { @@ -60,5 +69,6 @@ public class TopicAttributes { ALL.put(CLEANUP_POLICY_ATTRIBUTE.getName(), CLEANUP_POLICY_ATTRIBUTE); ALL.put(TOPIC_MESSAGE_TYPE_ATTRIBUTE.getName(), TOPIC_MESSAGE_TYPE_ATTRIBUTE); ALL.put(TOPIC_RESERVE_TIME_ATTRIBUTE.getName(), TOPIC_RESERVE_TIME_ATTRIBUTE); + ALL.put(LITE_EXPIRATION_ATTRIBUTE.getName(), LITE_EXPIRATION_ATTRIBUTE); } } diff --git a/common/src/main/java/org/apache/rocketmq/common/TopicConfig.java b/common/src/main/java/org/apache/rocketmq/common/TopicConfig.java index ebb009c5674..18389b58191 100644 --- a/common/src/main/java/org/apache/rocketmq/common/TopicConfig.java +++ b/common/src/main/java/org/apache/rocketmq/common/TopicConfig.java @@ -19,13 +19,14 @@ import com.alibaba.fastjson2.JSON; import com.alibaba.fastjson2.TypeReference; import com.alibaba.fastjson2.annotation.JSONField; -import org.apache.rocketmq.common.attribute.TopicMessageType; -import org.apache.rocketmq.common.constant.PermName; - import java.util.HashMap; import java.util.Map; import java.util.Objects; +import org.apache.commons.lang3.math.NumberUtils; +import org.apache.rocketmq.common.attribute.TopicMessageType; +import org.apache.rocketmq.common.constant.PermName; +import static org.apache.rocketmq.common.TopicAttributes.LITE_EXPIRATION_ATTRIBUTE; import static org.apache.rocketmq.common.TopicAttributes.TOPIC_MESSAGE_TYPE_ATTRIBUTE; public class TopicConfig { @@ -216,6 +217,26 @@ public void setTopicMessageType(TopicMessageType topicMessageType) { attributes.put(TOPIC_MESSAGE_TYPE_ATTRIBUTE.getName(), topicMessageType.getValue()); } + @JSONField(serialize = false, deserialize = false) + public void setLiteTopicExpiration(int liteTopicExpiration) { + if (!TopicMessageType.LITE.equals(getTopicMessageType())) { + return; + } + attributes.put(LITE_EXPIRATION_ATTRIBUTE.getName(), String.valueOf(liteTopicExpiration)); + } + + @JSONField(serialize = false, deserialize = false) + public int getLiteTopicExpiration() { + if (!TopicMessageType.LITE.equals(getTopicMessageType())) { + return -1; + } + String content = attributes.get(LITE_EXPIRATION_ATTRIBUTE.getName()); + if (content == null) { + return -1; + } + return NumberUtils.toInt(content, -1); + } + @Override public boolean equals(Object o) { if (this == o) { diff --git a/common/src/main/java/org/apache/rocketmq/common/attribute/LiteSubModel.java b/common/src/main/java/org/apache/rocketmq/common/attribute/LiteSubModel.java new file mode 100644 index 00000000000..5e326d54afc --- /dev/null +++ b/common/src/main/java/org/apache/rocketmq/common/attribute/LiteSubModel.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.common.attribute; + +public enum LiteSubModel { + Shared, + Exclusive +} diff --git a/common/src/main/java/org/apache/rocketmq/common/attribute/StringAttribute.java b/common/src/main/java/org/apache/rocketmq/common/attribute/StringAttribute.java new file mode 100644 index 00000000000..e66d688c789 --- /dev/null +++ b/common/src/main/java/org/apache/rocketmq/common/attribute/StringAttribute.java @@ -0,0 +1,32 @@ +/* + * 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.common.attribute; + +import static com.google.common.base.Preconditions.checkNotNull; + +public class StringAttribute extends Attribute { + + public StringAttribute(String name, boolean changeable) { + super(name, changeable); + } + + @Override + public void verify(String value) { + checkNotNull(value); + } +} diff --git a/common/src/main/java/org/apache/rocketmq/common/attribute/TopicMessageType.java b/common/src/main/java/org/apache/rocketmq/common/attribute/TopicMessageType.java index 9a89d30e8f8..9d3cb7608e5 100644 --- a/common/src/main/java/org/apache/rocketmq/common/attribute/TopicMessageType.java +++ b/common/src/main/java/org/apache/rocketmq/common/attribute/TopicMessageType.java @@ -29,16 +29,18 @@ public enum TopicMessageType { DELAY("DELAY"), TRANSACTION("TRANSACTION"), PRIORITY("PRIORITY"), + LITE("LITE"), MIXED("MIXED"); private final String value; + TopicMessageType(String value) { this.value = value; } public static Set topicMessageTypeSet() { return Sets.newHashSet(UNSPECIFIED.value, NORMAL.value, FIFO.value, DELAY.value, TRANSACTION.value, - PRIORITY.value, MIXED.value); + PRIORITY.value, LITE.value, MIXED.value); } public String getValue() { @@ -58,6 +60,8 @@ public static TopicMessageType parseFromMessageProperty(Map mess return TopicMessageType.FIFO; } else if (messageProperty.get(MessageConst.PROPERTY_PRIORITY) != null) { return TopicMessageType.PRIORITY; + } else if (messageProperty.get(MessageConst.PROPERTY_LITE_TOPIC) != null) { + return TopicMessageType.LITE; } return TopicMessageType.NORMAL; } diff --git a/common/src/main/java/org/apache/rocketmq/common/constant/LoggerName.java b/common/src/main/java/org/apache/rocketmq/common/constant/LoggerName.java index 4a8d307987b..e92b4cdf9c2 100644 --- a/common/src/main/java/org/apache/rocketmq/common/constant/LoggerName.java +++ b/common/src/main/java/org/apache/rocketmq/common/constant/LoggerName.java @@ -47,6 +47,7 @@ public class LoggerName { public static final String WATER_MARK_LOGGER_NAME = "RocketmqWaterMark"; public static final String FILTER_LOGGER_NAME = "RocketmqFilter"; public static final String ROCKETMQ_POP_LOGGER_NAME = "RocketmqPop"; + public static final String ROCKETMQ_POP_LITE_LOGGER_NAME = "RocketmqPopLite"; public static final String FAILOVER_LOGGER_NAME = "RocketmqFailover"; public static final String STDOUT_LOGGER_NAME = "STDOUT"; public static final String PROXY_LOGGER_NAME = "RocketmqProxy"; diff --git a/common/src/main/java/org/apache/rocketmq/common/entity/ClientGroup.java b/common/src/main/java/org/apache/rocketmq/common/entity/ClientGroup.java new file mode 100644 index 00000000000..44d467a6040 --- /dev/null +++ b/common/src/main/java/org/apache/rocketmq/common/entity/ClientGroup.java @@ -0,0 +1,61 @@ +/* + * 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.common.entity; + +import java.util.Objects; + +public class ClientGroup { + + public final String clientId; + public final String group; + /** + * Cache the hash code for the object + */ + private int hash; // Default to 0 + + public ClientGroup(String clientId, String group) { + this.clientId = clientId; + this.group = group; + } + + @Override + public boolean equals(Object o) { + if (o == null || getClass() != o.getClass()) { + return false; + } + ClientGroup that = (ClientGroup) o; + return Objects.equals(clientId, that.clientId) + && Objects.equals(group, that.group); + } + + @Override + public int hashCode() { + if (hash == 0) { + hash = Objects.hash(clientId, group); + } + return hash; + } + + @Override + public String toString() { + return "ClientGroup{" + + "clientId='" + clientId + '\'' + + ", group='" + group + '\'' + + '}'; + } +} diff --git a/common/src/main/java/org/apache/rocketmq/common/entity/TopicGroup.java b/common/src/main/java/org/apache/rocketmq/common/entity/TopicGroup.java new file mode 100644 index 00000000000..7ee2d6d32f9 --- /dev/null +++ b/common/src/main/java/org/apache/rocketmq/common/entity/TopicGroup.java @@ -0,0 +1,60 @@ +/* + * 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.common.entity; + +import java.util.Objects; + +public class TopicGroup { + + public final String topic; + public final String group; + /** + * Cache the hash code for the object + */ + private int hash; // Default to 0 + + public TopicGroup(String topic, String group) { + this.topic = topic; + this.group = group; + } + + @Override + public boolean equals(Object o) { + if (o == null || getClass() != o.getClass()) { + return false; + } + TopicGroup that = (TopicGroup) o; + return Objects.equals(topic, that.topic) && Objects.equals(group, that.group); + } + + @Override + public int hashCode() { + if (hash == 0) { + hash = Objects.hash(topic, group); + } + return hash; + } + + @Override + public String toString() { + return "TopicGroup{" + + "topic='" + topic + '\'' + + ", group='" + group + '\'' + + '}'; + } +} diff --git a/common/src/main/java/org/apache/rocketmq/common/lite/LiteLagInfo.java b/common/src/main/java/org/apache/rocketmq/common/lite/LiteLagInfo.java new file mode 100644 index 00000000000..5a3caf0371d --- /dev/null +++ b/common/src/main/java/org/apache/rocketmq/common/lite/LiteLagInfo.java @@ -0,0 +1,48 @@ +/* + * 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.common.lite; + +public class LiteLagInfo { + private String liteTopic; + private long lagCount; + // earliest unconsumed timestamp + private long earliestUnconsumedTimestamp = -1; + + public String getLiteTopic() { + return liteTopic; + } + + public void setLiteTopic(String liteTopic) { + this.liteTopic = liteTopic; + } + + public long getLagCount() { + return lagCount; + } + + public void setLagCount(long lagCount) { + this.lagCount = lagCount; + } + + public long getEarliestUnconsumedTimestamp() { + return earliestUnconsumedTimestamp; + } + + public void setEarliestUnconsumedTimestamp(long earliestUnconsumedTimestamp) { + this.earliestUnconsumedTimestamp = earliestUnconsumedTimestamp; + } +} diff --git a/common/src/main/java/org/apache/rocketmq/common/lite/LiteSubscription.java b/common/src/main/java/org/apache/rocketmq/common/lite/LiteSubscription.java new file mode 100644 index 00000000000..abf7c9ee3af --- /dev/null +++ b/common/src/main/java/org/apache/rocketmq/common/lite/LiteSubscription.java @@ -0,0 +1,98 @@ +/* + * 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.common.lite; + +import java.util.Collection; +import java.util.Set; +import java.util.concurrent.ConcurrentHashMap; + +public class LiteSubscription { + private String group; + private String topic; + private final Set liteTopicSet = ConcurrentHashMap.newKeySet(); + private volatile long updateTime = System.currentTimeMillis(); + + public boolean addLiteTopic(String liteTopic) { + updateTime(); + return this.liteTopicSet.add(liteTopic); + } + + public void addLiteTopic(Collection set) { + updateTime(); + this.liteTopicSet.addAll(set); + } + + public boolean removeLiteTopic(String liteTopic) { + updateTime(); + return this.liteTopicSet.remove(liteTopic); + } + + public void removeLiteTopic(Collection set) { + updateTime(); + this.liteTopicSet.removeAll(set); + } + + public String getGroup() { + return group; + } + + public LiteSubscription setGroup(String group) { + this.group = group; + return this; + } + + public String getTopic() { + return topic; + } + + public LiteSubscription setTopic(String topic) { + this.topic = topic; + return this; + } + + public Set getLiteTopicSet() { + return liteTopicSet; + } + + public LiteSubscription setLiteTopicSet(Set liteTopicSet) { + this.liteTopicSet.addAll(liteTopicSet); + return this; + } + + public long getUpdateTime() { + return updateTime; + } + + public void setUpdateTime(long updateTime) { + this.updateTime = updateTime; + } + + private void updateTime() { + this.updateTime = System.currentTimeMillis(); + } + + @Override + public String toString() { + return "LiteSubscription{" + + "group='" + group + '\'' + + ", topic='" + topic + '\'' + + ", liteTopicSet=" + liteTopicSet + + ", updateTime=" + updateTime + + '}'; + } +} diff --git a/common/src/main/java/org/apache/rocketmq/common/lite/LiteSubscriptionAction.java b/common/src/main/java/org/apache/rocketmq/common/lite/LiteSubscriptionAction.java new file mode 100644 index 00000000000..dbd7686a058 --- /dev/null +++ b/common/src/main/java/org/apache/rocketmq/common/lite/LiteSubscriptionAction.java @@ -0,0 +1,25 @@ +/* + * 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.common.lite; + +public enum LiteSubscriptionAction { + PARTIAL_ADD, + PARTIAL_REMOVE, + COMPLETE_ADD, + COMPLETE_REMOVE +} diff --git a/common/src/main/java/org/apache/rocketmq/common/lite/LiteSubscriptionDTO.java b/common/src/main/java/org/apache/rocketmq/common/lite/LiteSubscriptionDTO.java new file mode 100644 index 00000000000..967fbfc8a10 --- /dev/null +++ b/common/src/main/java/org/apache/rocketmq/common/lite/LiteSubscriptionDTO.java @@ -0,0 +1,104 @@ +/* + * 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.common.lite; + +import java.util.Set; + +public class LiteSubscriptionDTO { + private LiteSubscriptionAction action; + private String clientId; + private String group; + private String topic; + private Set liteTopicSet; + private OffsetOption offsetOption; + private long version; + + public LiteSubscriptionAction getAction() { + return action; + } + + public LiteSubscriptionDTO setAction(LiteSubscriptionAction action) { + this.action = action; + return this; + } + + public String getClientId() { + return clientId; + } + + public LiteSubscriptionDTO setClientId(String clientId) { + this.clientId = clientId; + return this; + } + + public String getGroup() { + return group; + } + + public LiteSubscriptionDTO setGroup(String group) { + this.group = group; + return this; + } + + public String getTopic() { + return topic; + } + + public LiteSubscriptionDTO setTopic(String topic) { + this.topic = topic; + return this; + } + + public Set getLiteTopicSet() { + return liteTopicSet; + } + + public LiteSubscriptionDTO setLiteTopicSet(Set liteTopicSet) { + this.liteTopicSet = liteTopicSet; + return this; + } + + public OffsetOption getOffsetOption() { + return offsetOption; + } + + public void setOffsetOption(OffsetOption offsetOption) { + this.offsetOption = offsetOption; + } + + public long getVersion() { + return version; + } + + public LiteSubscriptionDTO setVersion(long version) { + this.version = version; + return this; + } + + @Override + public String toString() { + return "LiteSubscriptionDTO{" + "action=" + action + + ", clientId='" + clientId + '\'' + + ", group='" + group + '\'' + + ", topic='" + topic + '\'' + + ", liteTopicSet=" + liteTopicSet + + ", offsetOption=" + offsetOption + + ", version=" + version + + '}'; + } +} diff --git a/common/src/main/java/org/apache/rocketmq/common/lite/LiteUtil.java b/common/src/main/java/org/apache/rocketmq/common/lite/LiteUtil.java new file mode 100644 index 00000000000..0f1e0205e96 --- /dev/null +++ b/common/src/main/java/org/apache/rocketmq/common/lite/LiteUtil.java @@ -0,0 +1,111 @@ +/* + * 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.common.lite; + +import org.apache.commons.lang3.StringUtils; +import org.apache.rocketmq.common.MixAll; +import org.apache.rocketmq.common.Pair; + +public class LiteUtil { + + public static final char SEPARATOR = '$'; + public static final String LITE_TOPIC_PREFIX = MixAll.LMQ_PREFIX + SEPARATOR; + + /** + * Lite Topic: A specific type of message topic implemented based on LMQ, which has no retry topic. + * A lite topic's underlying storage is a lmq (Light Message Queue), + * but the reverse is not true: lmq is not necessarily a lite topic, + * we use "$" as a separator to achieve the distinction and assume "$" is not allowed for topic name. + * pattern like: %LMQ%$parentTopic$liteTopic + * + * @param parentTopic act as namespace + * @param liteTopic here means child topic string + * @return lmqName + */ + public static String toLmqName(String parentTopic, String liteTopic) { + if (StringUtils.isEmpty(parentTopic) || StringUtils.isEmpty(liteTopic)) { + return null; + } + return LITE_TOPIC_PREFIX + parentTopic + SEPARATOR + liteTopic; + } + + /** + * whether lmqName is queue of a lite topic, here we only check the prefix. + * @param lmqName + * @return + */ + public static boolean isLiteTopicQueue(String lmqName) { + return lmqName != null && lmqName.startsWith(LITE_TOPIC_PREFIX); + } + + public static String getParentTopic(String lmqName) { + if (!isLiteTopicQueue(lmqName)) { + return null; + } + int index = lmqName.indexOf(SEPARATOR, LITE_TOPIC_PREFIX.length()); + if (index == -1 || index == lmqName.length() - 1 || index == LITE_TOPIC_PREFIX.length()) { + return null; + } + if (lmqName.indexOf(SEPARATOR, index + 1) != -1) { + return null; + } + return lmqName.substring(LITE_TOPIC_PREFIX.length(), index); + } + + public static String getLiteTopic(String lmqName) { + if (!isLiteTopicQueue(lmqName)) { + return null; + } + int index = lmqName.indexOf(SEPARATOR, LITE_TOPIC_PREFIX.length()); + if (index == -1 || index == lmqName.length() - 1 || index == LITE_TOPIC_PREFIX.length()) { + return null; + } + if (lmqName.indexOf(SEPARATOR, index + 1) != -1) { + return null; + } + return lmqName.substring(index + 1); + } + + /** + * %LMQ%${parentTopic}${liteTopic} + * parse parent topic and child topic from lmqName + * @param lmqName + * @return + */ + public static Pair getParentAndLiteTopic(String lmqName) { + if (null == lmqName || !lmqName.startsWith(LITE_TOPIC_PREFIX)) { + return null; + } + String[] array = StringUtils.split(lmqName, SEPARATOR); + if (array.length != 3) { + return null; + } + return new Pair<>(array[1], array[2]); + } + + /** + * whether lmqName is queue of a lite topic and belongs to the specified parent, + * here we only check the prefix. + * @param lmqName + * @param parentTopic + * @return + */ + public static boolean belongsTo(String lmqName, String parentTopic) { + return lmqName != null && lmqName.startsWith(LITE_TOPIC_PREFIX + parentTopic + SEPARATOR); + } +} diff --git a/common/src/main/java/org/apache/rocketmq/common/lite/OffsetOption.java b/common/src/main/java/org/apache/rocketmq/common/lite/OffsetOption.java new file mode 100644 index 00000000000..a72414c3b29 --- /dev/null +++ b/common/src/main/java/org/apache/rocketmq/common/lite/OffsetOption.java @@ -0,0 +1,86 @@ +/* + * 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.common.lite; + +import java.util.Objects; + +public class OffsetOption { + + public static final long POLICY_LAST_VALUE = 0L; + public static final long POLICY_MIN_VALUE = 1L; + public static final long POLICY_MAX_VALUE = 2L; + + private Type type; + private long value; + + public OffsetOption() { + } + + public OffsetOption(Type type, long value) { + this.type = type; + this.value = value; + } + + public Type getType() { + return type; + } + + public void setType(Type type) { + this.type = type; + } + + public long getValue() { + return value; + } + + public void setValue(long value) { + this.value = value; + } + + @Override + public boolean equals(Object o) { + if (o == null || getClass() != o.getClass()) { + return false; + } + + OffsetOption option = (OffsetOption) o; + return value == option.value && type == option.type; + } + + @Override + public int hashCode() { + int result = Objects.hashCode(type); + result = 31 * result + Long.hashCode(value); + return result; + } + + @Override + public String toString() { + return "OffsetOption{" + "type=" + type + + ", value=" + value + + '}'; + } + + public enum Type { + POLICY, + OFFSET, + TAIL_N, + TIMESTAMP + } + +} \ No newline at end of file diff --git a/common/src/main/java/org/apache/rocketmq/common/message/MessageAccessor.java b/common/src/main/java/org/apache/rocketmq/common/message/MessageAccessor.java index 62e3bbd7e6e..1e17e1e19df 100644 --- a/common/src/main/java/org/apache/rocketmq/common/message/MessageAccessor.java +++ b/common/src/main/java/org/apache/rocketmq/common/message/MessageAccessor.java @@ -90,6 +90,10 @@ public static String getConsumeStartTimeStamp(final Message msg) { return msg.getProperty(MessageConst.PROPERTY_CONSUME_START_TIMESTAMP); } + public static void setLiteTopic(final Message msg, String liteTopic) { + MessageAccessor.putProperty(msg, MessageConst.PROPERTY_LITE_TOPIC, liteTopic); + } + public static Message cloneMessage(final Message msg) { Message newMsg = new Message(msg.getTopic(), msg.getBody()); newMsg.setFlag(msg.getFlag()); diff --git a/common/src/main/java/org/apache/rocketmq/common/message/MessageConst.java b/common/src/main/java/org/apache/rocketmq/common/message/MessageConst.java index 72078f761d5..77ab3f2cb9f 100644 --- a/common/src/main/java/org/apache/rocketmq/common/message/MessageConst.java +++ b/common/src/main/java/org/apache/rocketmq/common/message/MessageConst.java @@ -64,6 +64,7 @@ public class MessageConst { public static final String PROPERTY_POP_CK_OFFSET = "POP_CK_OFFSET"; public static final String PROPERTY_FIRST_POP_TIME = "1ST_POP_TIME"; public static final String PROPERTY_SHARDING_KEY = "__SHARDINGKEY"; + public static final String PROPERTY_LITE_TOPIC = "__LITE_TOPIC"; public static final String PROPERTY_FORWARD_QUEUE_ID = "PROPERTY_FORWARD_QUEUE_ID"; public static final String PROPERTY_REDIRECT = "REDIRECT"; public static final String PROPERTY_INNER_MULTI_DISPATCH = "INNER_MULTI_DISPATCH"; @@ -171,5 +172,6 @@ public class MessageConst { STRING_HASH_SET.add(PROPERTY_DLQ_ORIGIN_MESSAGE_ID); STRING_HASH_SET.add(PROPERTY_CRC32); STRING_HASH_SET.add(PROPERTY_PRIORITY); + STRING_HASH_SET.add(PROPERTY_LITE_TOPIC); } } diff --git a/common/src/test/java/org/apache/rocketmq/common/attribute/TopicMessageTypeTest.java b/common/src/test/java/org/apache/rocketmq/common/attribute/TopicMessageTypeTest.java index 79402ca1b2a..1029e397781 100644 --- a/common/src/test/java/org/apache/rocketmq/common/attribute/TopicMessageTypeTest.java +++ b/common/src/test/java/org/apache/rocketmq/common/attribute/TopicMessageTypeTest.java @@ -52,7 +52,7 @@ public void setUp() { @Test public void testTopicMessageTypeSet() { Set expectedSet - = Sets.newHashSet("UNSPECIFIED", "NORMAL", "FIFO", "DELAY", "TRANSACTION", "PRIORITY", "MIXED"); + = Sets.newHashSet("UNSPECIFIED", "NORMAL", "FIFO", "DELAY", "TRANSACTION", "PRIORITY", "LITE", "MIXED"); Set actualSet = TopicMessageType.topicMessageTypeSet(); assertEquals(expectedSet, actualSet); } diff --git a/common/src/test/java/org/apache/rocketmq/common/utils/LiteUtilTest.java b/common/src/test/java/org/apache/rocketmq/common/utils/LiteUtilTest.java new file mode 100644 index 00000000000..c66330a8096 --- /dev/null +++ b/common/src/test/java/org/apache/rocketmq/common/utils/LiteUtilTest.java @@ -0,0 +1,110 @@ +/* + * 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.common.utils; + +import org.apache.rocketmq.common.Pair; +import org.apache.rocketmq.common.lite.LiteUtil; +import org.junit.Test; + +import static org.junit.Assert.assertEquals; +import static org.junit.Assert.assertFalse; +import static org.junit.Assert.assertNotNull; +import static org.junit.Assert.assertNull; +import static org.junit.Assert.assertTrue; + +public class LiteUtilTest { + + @Test + public void testToLmqName() { + String result = LiteUtil.toLmqName("parentTopic", "liteTopic"); + String expected = LiteUtil.LITE_TOPIC_PREFIX + "parentTopic" + LiteUtil.SEPARATOR + "liteTopic"; + assertEquals(expected, result); + + assertNull(LiteUtil.toLmqName(null, "liteTopic")); + assertNull(LiteUtil.toLmqName("parentTopic", null)); + assertNull(LiteUtil.toLmqName("", "liteTopic")); + assertNull(LiteUtil.toLmqName("parentTopic", "")); + } + + @Test + public void testIsLiteTopicQueue() { + assertTrue(LiteUtil.isLiteTopicQueue("%LMQ%$parentTopic$liteTopic")); + + assertFalse(LiteUtil.isLiteTopicQueue("%LMQ%parentTopic")); + assertFalse(LiteUtil.isLiteTopicQueue("parentTopic")); + assertFalse(LiteUtil.isLiteTopicQueue(null)); + assertFalse(LiteUtil.isLiteTopicQueue("%LMQ$")); + } + + @Test + public void testGetParentTopic() { + assertEquals("parentTopic", LiteUtil.getParentTopic("%LMQ%$parentTopic$liteTopic")); + + assertNull(LiteUtil.getParentTopic(null)); + assertNull(LiteUtil.getParentTopic("parentTopic")); + assertNull(LiteUtil.getParentTopic("%LMQ%parentTopic$liteTopic")); + assertNull(LiteUtil.getParentTopic("%LMQ%$$")); + assertNull(LiteUtil.getParentTopic("%LMQ%$parentTopic")); + assertNull(LiteUtil.getParentTopic("%LMQ%$parentTopic$")); + assertNull(LiteUtil.getParentTopic("%LMQ%$$liteTopic")); + assertNull(LiteUtil.getParentTopic("%LMQ%$parent$lite$extra")); + } + + @Test + public void testGetLiteTopic() { + assertEquals("liteTopic", LiteUtil.getLiteTopic("%LMQ%$parentTopic$liteTopic")); + + assertNull(LiteUtil.getLiteTopic(null)); + assertNull(LiteUtil.getLiteTopic("parentTopic")); + assertNull(LiteUtil.getParentTopic("%LMQ%parentTopic$liteTopic")); + assertNull(LiteUtil.getParentTopic("%LMQ%$$")); + assertNull(LiteUtil.getLiteTopic("%LMQ%$parentTopic")); + assertNull(LiteUtil.getLiteTopic("%LMQ%$parentTopic$")); + assertNull(LiteUtil.getLiteTopic("%LMQ%$$liteTopic")); + assertNull(LiteUtil.getLiteTopic("%LMQ%$parent$lite$extra")); + } + + @Test + public void testGetParentAndLiteTopic() { + Pair result = LiteUtil.getParentAndLiteTopic("%LMQ%$parentTopic$liteTopic"); + assertNotNull(result); + assertEquals("parentTopic", result.getObject1()); + assertEquals("liteTopic", result.getObject2()); + + assertNull(LiteUtil.getParentTopic(null)); + assertNull(LiteUtil.getParentTopic("parentTopic")); + assertNull(LiteUtil.getParentTopic("%LMQ%parentTopic$liteTopic")); + assertNull(LiteUtil.getParentTopic("%LMQ%$$")); + assertNull(LiteUtil.getParentTopic("%LMQ%$parentTopic")); + assertNull(LiteUtil.getParentTopic("%LMQ%$parentTopic$")); + assertNull(LiteUtil.getParentTopic("%LMQ%$$liteTopic")); + assertNull(LiteUtil.getParentTopic("%LMQ%$parent$lite$extra")); + } + + @Test + public void testBelongsTo() { + assertTrue(LiteUtil.belongsTo("%LMQ%$parentTopic$liteTopic", "parentTopic")); + assertTrue(LiteUtil.belongsTo("%LMQ%$parentTopic$", "parentTopic")); // only check prefix + assertTrue(LiteUtil.belongsTo("%LMQ%$parentTopic$liteTopic$xxx", "parentTopic")); // only check prefix + + assertFalse(LiteUtil.belongsTo("%LMQ%$parentTopic$liteTopic", "otherParent")); + assertFalse(LiteUtil.belongsTo("parentTopic", "parentTopic")); + assertFalse(LiteUtil.belongsTo(null, "parentTopic")); + assertFalse(LiteUtil.belongsTo("%LMQ%$parentTopic$liteTopic", null)); + } +} diff --git a/proxy/src/main/java/org/apache/rocketmq/proxy/common/MessageReceiptHandle.java b/proxy/src/main/java/org/apache/rocketmq/proxy/common/MessageReceiptHandle.java index c015e9f53f3..2f67d4ca14a 100644 --- a/proxy/src/main/java/org/apache/rocketmq/proxy/common/MessageReceiptHandle.java +++ b/proxy/src/main/java/org/apache/rocketmq/proxy/common/MessageReceiptHandle.java @@ -35,10 +35,16 @@ public class MessageReceiptHandle { private final AtomicInteger renewRetryTimes = new AtomicInteger(0); private final AtomicInteger renewTimes = new AtomicInteger(0); private final long consumeTimestamp; + private String liteTopic; private volatile String receiptHandleStr; public MessageReceiptHandle(String group, String topic, int queueId, String receiptHandleStr, String messageId, long queueOffset, int reconsumeTimes) { + this(group, topic, queueId, receiptHandleStr, messageId, queueOffset, reconsumeTimes, null); + } + + public MessageReceiptHandle(String group, String topic, int queueId, String receiptHandleStr, String messageId, + long queueOffset, int reconsumeTimes, String liteTopic) { this.originalReceiptHandle = ReceiptHandle.decode(receiptHandleStr); this.group = group; this.topic = topic; @@ -49,6 +55,7 @@ public MessageReceiptHandle(String group, String topic, int queueId, String rece this.queueOffset = queueOffset; this.reconsumeTimes = reconsumeTimes; this.consumeTimestamp = originalReceiptHandle.getRetrieveTime(); + this.liteTopic = liteTopic; } @Override @@ -86,6 +93,8 @@ public String toString() { .add("renewRetryTimes", renewRetryTimes) .add("firstConsumeTimestamp", consumeTimestamp) .add("receiptHandleStr", receiptHandleStr) + .add("liteTopic", liteTopic) + .omitNullValues() .toString(); } @@ -152,4 +161,12 @@ public int getRenewRetryTimes() { public ReceiptHandle getOriginalReceiptHandle() { return originalReceiptHandle; } + + public String getLiteTopic() { + return liteTopic; + } + + public void setLiteTopic(String liteTopic) { + this.liteTopic = liteTopic; + } } diff --git a/proxy/src/main/java/org/apache/rocketmq/proxy/common/ReceiptHandleGroup.java b/proxy/src/main/java/org/apache/rocketmq/proxy/common/ReceiptHandleGroup.java index a35eaa58868..005f18025dd 100644 --- a/proxy/src/main/java/org/apache/rocketmq/proxy/common/ReceiptHandleGroup.java +++ b/proxy/src/main/java/org/apache/rocketmq/proxy/common/ReceiptHandleGroup.java @@ -208,6 +208,10 @@ public long getHandleNum() { return handleNum; } + public int getMsgCount() { + return this.receiptHandleMap.size(); + } + public MessageReceiptHandle get(String msgID, String handle) { Map handleMap = this.receiptHandleMap.get(msgID); if (handleMap == null) { diff --git a/proxy/src/main/java/org/apache/rocketmq/proxy/config/ProxyConfig.java b/proxy/src/main/java/org/apache/rocketmq/proxy/config/ProxyConfig.java index bc1919c07a1..d21d55fe65e 100644 --- a/proxy/src/main/java/org/apache/rocketmq/proxy/config/ProxyConfig.java +++ b/proxy/src/main/java/org/apache/rocketmq/proxy/config/ProxyConfig.java @@ -118,6 +118,13 @@ public class ProxyConfig implements ConfigFile { * max message group size, 0 or negative number means no limit for proxy */ private int maxMessageGroupSize = 64; + /** + * max lite topic size + */ + private int maxLiteTopicSize = 64; + private int maxLiteRenewNumPerChannel = 100; + // syncLiteSubscription request rate limit per proxy + private int maxSyncLiteSubscriptionRate = 5000; /** * When a message pops, the message is invisible by default @@ -1538,6 +1545,30 @@ public void setEnableMessageBodyEmptyCheck(boolean enableMessageBodyEmptyCheck) this.enableMessageBodyEmptyCheck = enableMessageBodyEmptyCheck; } + public int getMaxLiteTopicSize() { + return maxLiteTopicSize; + } + + public void setMaxLiteTopicSize(int maxLiteTopicSize) { + this.maxLiteTopicSize = maxLiteTopicSize; + } + + public int getMaxLiteRenewNumPerChannel() { + return maxLiteRenewNumPerChannel; + } + + public void setMaxLiteRenewNumPerChannel(int maxLiteRenewNumPerChannel) { + this.maxLiteRenewNumPerChannel = maxLiteRenewNumPerChannel; + } + + public int getMaxSyncLiteSubscriptionRate() { + return maxSyncLiteSubscriptionRate; + } + + public void setMaxSyncLiteSubscriptionRate(int maxSyncLiteSubscriptionRate) { + this.maxSyncLiteSubscriptionRate = maxSyncLiteSubscriptionRate; + } + public int getReturnHandleGroupThreadPoolNums() { return returnHandleGroupThreadPoolNums; } diff --git a/proxy/src/main/java/org/apache/rocketmq/proxy/grpc/v2/AbstractMessagingActivity.java b/proxy/src/main/java/org/apache/rocketmq/proxy/grpc/v2/AbstractMessagingActivity.java index 3615c1515f7..7221c1eddbb 100644 --- a/proxy/src/main/java/org/apache/rocketmq/proxy/grpc/v2/AbstractMessagingActivity.java +++ b/proxy/src/main/java/org/apache/rocketmq/proxy/grpc/v2/AbstractMessagingActivity.java @@ -42,6 +42,10 @@ protected void validateTopic(Resource topic) { GrpcValidator.getInstance().validateTopic(topic); } + protected void validateLiteTopic(String liteTopic) { + GrpcValidator.getInstance().validateLiteTopic(liteTopic); + } + protected void validateConsumerGroup(Resource consumerGroup) { GrpcValidator.getInstance().validateConsumerGroup(consumerGroup); } diff --git a/proxy/src/main/java/org/apache/rocketmq/proxy/grpc/v2/DefaultGrpcMessagingActivity.java b/proxy/src/main/java/org/apache/rocketmq/proxy/grpc/v2/DefaultGrpcMessagingActivity.java index 90380735568..88099207b93 100644 --- a/proxy/src/main/java/org/apache/rocketmq/proxy/grpc/v2/DefaultGrpcMessagingActivity.java +++ b/proxy/src/main/java/org/apache/rocketmq/proxy/grpc/v2/DefaultGrpcMessagingActivity.java @@ -38,6 +38,8 @@ import apache.rocketmq.v2.ReceiveMessageResponse; import apache.rocketmq.v2.SendMessageRequest; import apache.rocketmq.v2.SendMessageResponse; +import apache.rocketmq.v2.SyncLiteSubscriptionRequest; +import apache.rocketmq.v2.SyncLiteSubscriptionResponse; import apache.rocketmq.v2.TelemetryCommand; import io.grpc.stub.StreamObserver; import java.util.concurrent.CompletableFuture; @@ -156,6 +158,12 @@ public CompletableFuture recallMessage(ProxyContext ctx, return this.recallMessageActivity.recallMessage(ctx, request); } + @Override + public CompletableFuture syncLiteSubscription(ProxyContext ctx, + SyncLiteSubscriptionRequest request) { + return this.clientActivity.syncLiteSubscription(ctx, request); + } + @Override public ContextStreamObserver telemetry(StreamObserver responseObserver) { return this.clientActivity.telemetry(responseObserver); diff --git a/proxy/src/main/java/org/apache/rocketmq/proxy/grpc/v2/GrpcMessagingActivity.java b/proxy/src/main/java/org/apache/rocketmq/proxy/grpc/v2/GrpcMessagingActivity.java index 9e3500fe53d..de68f0f8e50 100644 --- a/proxy/src/main/java/org/apache/rocketmq/proxy/grpc/v2/GrpcMessagingActivity.java +++ b/proxy/src/main/java/org/apache/rocketmq/proxy/grpc/v2/GrpcMessagingActivity.java @@ -39,6 +39,8 @@ import apache.rocketmq.v2.ReceiveMessageResponse; import apache.rocketmq.v2.SendMessageRequest; import apache.rocketmq.v2.SendMessageResponse; +import apache.rocketmq.v2.SyncLiteSubscriptionRequest; +import apache.rocketmq.v2.SyncLiteSubscriptionResponse; import apache.rocketmq.v2.TelemetryCommand; import io.grpc.stub.StreamObserver; import java.util.concurrent.CompletableFuture; @@ -73,5 +75,7 @@ CompletableFuture changeInvisibleDuration(Proxy CompletableFuture recallMessage(ProxyContext ctx, RecallMessageRequest request); + CompletableFuture syncLiteSubscription(ProxyContext ctx, SyncLiteSubscriptionRequest request); + ContextStreamObserver telemetry(StreamObserver responseObserver); } diff --git a/proxy/src/main/java/org/apache/rocketmq/proxy/grpc/v2/GrpcMessagingApplication.java b/proxy/src/main/java/org/apache/rocketmq/proxy/grpc/v2/GrpcMessagingApplication.java index 013d7f0dfb4..0b45dd7cf0c 100644 --- a/proxy/src/main/java/org/apache/rocketmq/proxy/grpc/v2/GrpcMessagingApplication.java +++ b/proxy/src/main/java/org/apache/rocketmq/proxy/grpc/v2/GrpcMessagingApplication.java @@ -42,6 +42,8 @@ import apache.rocketmq.v2.SendMessageRequest; import apache.rocketmq.v2.SendMessageResponse; import apache.rocketmq.v2.Status; +import apache.rocketmq.v2.SyncLiteSubscriptionRequest; +import apache.rocketmq.v2.SyncLiteSubscriptionResponse; import apache.rocketmq.v2.TelemetryCommand; import com.google.protobuf.GeneratedMessageV3; import io.grpc.Context; @@ -399,6 +401,26 @@ public void recallMessage(RecallMessageRequest request, StreamObserver responseObserver) { + Function statusResponseCreator = + status -> SyncLiteSubscriptionResponse.newBuilder().setStatus(status).build(); + ProxyContext context = createContext(); + try { + this.addExecutor(this.clientManagerThreadPoolExecutor, + context, + request, + () -> grpcMessagingActivity.syncLiteSubscription(context, request) + .whenComplete((response, throwable) -> + writeResponse(context, request, response, responseObserver, throwable, statusResponseCreator)), + responseObserver, + statusResponseCreator); + } catch (Throwable t) { + writeResponse(context, request, null, responseObserver, t, statusResponseCreator); + } + } + @Override public StreamObserver telemetry(StreamObserver responseObserver) { Function statusResponseCreator = status -> TelemetryCommand.newBuilder().setStatus(status).build(); diff --git a/proxy/src/main/java/org/apache/rocketmq/proxy/grpc/v2/channel/GrpcClientChannel.java b/proxy/src/main/java/org/apache/rocketmq/proxy/grpc/v2/channel/GrpcClientChannel.java index f05251c58c5..0135818fb3b 100644 --- a/proxy/src/main/java/org/apache/rocketmq/proxy/grpc/v2/channel/GrpcClientChannel.java +++ b/proxy/src/main/java/org/apache/rocketmq/proxy/grpc/v2/channel/GrpcClientChannel.java @@ -16,6 +16,7 @@ */ package org.apache.rocketmq.proxy.grpc.v2.channel; +import apache.rocketmq.v2.NotifyUnsubscribeLiteCommand; import apache.rocketmq.v2.PrintThreadStackTraceCommand; import apache.rocketmq.v2.RecoverOrphanedTransactionCommand; import apache.rocketmq.v2.Settings; @@ -56,6 +57,7 @@ import org.apache.rocketmq.remoting.protocol.header.CheckTransactionStateRequestHeader; import org.apache.rocketmq.remoting.protocol.header.ConsumeMessageDirectlyResultRequestHeader; import org.apache.rocketmq.remoting.protocol.header.GetConsumerRunningInfoRequestHeader; +import org.apache.rocketmq.remoting.protocol.header.NotifyUnsubscribeLiteRequestHeader; public class GrpcClientChannel extends ProxyChannel implements ChannelExtendAttributeGetter, RemoteChannelConverter { private static final Logger log = LoggerFactory.getLogger(LoggerName.PROXY_LOGGER_NAME); @@ -207,6 +209,25 @@ protected CompletableFuture processCheckTransaction(CheckTransactionStateR return writeFuture; } + @Override + protected CompletableFuture processNotifyUnsubscribeLite(NotifyUnsubscribeLiteRequestHeader header) { + final String group = header.getConsumerGroup(); + final String liteTopic = header.getLiteTopic(); + NotifyUnsubscribeLiteCommand unsubscribeLiteCommand = NotifyUnsubscribeLiteCommand.newBuilder() + .setLiteTopic(liteTopic) + .build(); + + TelemetryCommand telemetryCommand = TelemetryCommand.newBuilder() + .setNotifyUnsubscribeLiteCommand(unsubscribeLiteCommand) + .build(); + + this.writeTelemetryCommand(telemetryCommand); + + log.info("notifyUnsubscribeLite liteTopic:{} group:{} clientId:{}", liteTopic, group, clientId); + + return CompletableFuture.completedFuture(null); + } + @Override protected CompletableFuture processGetConsumerRunningInfo(RemotingCommand command, GetConsumerRunningInfoRequestHeader header, diff --git a/proxy/src/main/java/org/apache/rocketmq/proxy/grpc/v2/client/ClientActivity.java b/proxy/src/main/java/org/apache/rocketmq/proxy/grpc/v2/client/ClientActivity.java index 7c6eea47aba..13287f47c38 100644 --- a/proxy/src/main/java/org/apache/rocketmq/proxy/grpc/v2/client/ClientActivity.java +++ b/proxy/src/main/java/org/apache/rocketmq/proxy/grpc/v2/client/ClientActivity.java @@ -27,12 +27,17 @@ import apache.rocketmq.v2.Settings; import apache.rocketmq.v2.Status; import apache.rocketmq.v2.SubscriptionEntry; +import apache.rocketmq.v2.SyncLiteSubscriptionRequest; +import apache.rocketmq.v2.SyncLiteSubscriptionResponse; import apache.rocketmq.v2.TelemetryCommand; import apache.rocketmq.v2.ThreadStackTrace; import apache.rocketmq.v2.VerifyMessageResult; +import com.google.common.collect.ImmutableSet; import io.grpc.StatusRuntimeException; import io.grpc.stub.StreamObserver; import io.netty.channel.Channel; + +import java.time.Duration; import java.util.HashSet; import java.util.List; import java.util.Set; @@ -47,6 +52,9 @@ import org.apache.rocketmq.common.attribute.TopicMessageType; import org.apache.rocketmq.common.constant.LoggerName; import org.apache.rocketmq.common.consumer.ConsumeFromWhere; +import org.apache.rocketmq.common.lite.LiteSubscriptionAction; +import org.apache.rocketmq.common.lite.LiteSubscriptionDTO; +import org.apache.rocketmq.common.lite.OffsetOption; import org.apache.rocketmq.logging.org.slf4j.Logger; import org.apache.rocketmq.logging.org.slf4j.LoggerFactory; import org.apache.rocketmq.proxy.common.ProxyContext; @@ -107,6 +115,7 @@ public CompletableFuture heartbeat(ProxyContext ctx, Heartbea break; } case PUSH_CONSUMER: + case LITE_PUSH_CONSUMER: case SIMPLE_CONSUMER: { validateConsumerGroup(request.getGroup()); String consumerGroup = request.getGroup().getName(); @@ -157,6 +166,7 @@ public CompletableFuture notifyClientTerminatio } break; case PUSH_CONSUMER: + case LITE_PUSH_CONSUMER: case SIMPLE_CONSUMER: validateConsumerGroup(request.getGroup()); String consumerGroup = request.getGroup().getName(); @@ -164,6 +174,7 @@ public CompletableFuture notifyClientTerminatio if (channel != null) { ClientChannelInfo clientChannelInfo = new ClientChannelInfo(channel, clientId, languageCode, MQVersion.Version.V5_0_0.ordinal()); this.messagingProcessor.unRegisterConsumer(ctx, consumerGroup, clientChannelInfo); + this.grpcClientSettingsManager.offlineClientLiteSubscription(ctx, clientId, clientSettings); } break; default: @@ -181,10 +192,93 @@ public CompletableFuture notifyClientTerminatio return future; } + public CompletableFuture syncLiteSubscription(ProxyContext ctx, + SyncLiteSubscriptionRequest request) { + try { + validateTopicAndConsumerGroup(request.getTopic(), request.getGroup()); + + final LiteSubscriptionAction action = toLiteAction(request.getAction()); + final Set liteTopicSet = ImmutableSet.copyOf(request.getLiteTopicSetList()); + if (LiteSubscriptionAction.PARTIAL_ADD == action) { + for (String liteTopic : liteTopicSet) { + validateLiteTopic(liteTopic); + } + } + + final String group = request.getGroup().getName(); + final String topic = request.getTopic().getName(); + LiteSubscriptionDTO liteSubscriptionDTO = new LiteSubscriptionDTO() + .setAction(action) + .setClientId(ctx.getClientID()) + .setGroup(group) + .setTopic(topic) + .setLiteTopicSet(liteTopicSet) + .setVersion(request.getVersion()); + + if (LiteSubscriptionAction.PARTIAL_ADD == action) { + if (request.hasOffsetOption()) { + liteSubscriptionDTO.setOffsetOption(toOffsetOption(request.getOffsetOption())); + } + } + + return this.messagingProcessor + .syncLiteSubscription(ctx, liteSubscriptionDTO, Duration.ofSeconds(2).toMillis()) + .thenApply(v -> + SyncLiteSubscriptionResponse + .newBuilder() + .setStatus(ResponseBuilder.getInstance().buildStatus(Code.OK, null)) + .build() + ); + } catch (Throwable t) { + CompletableFuture future = new CompletableFuture<>(); + future.completeExceptionally(t); + return future; + } + } + + private static OffsetOption toOffsetOption(apache.rocketmq.v2.OffsetOption gRpcOffsetOption) { + OffsetOption offsetOption = new OffsetOption(); + switch (gRpcOffsetOption.getOffsetTypeCase()) { + case POLICY: + offsetOption.setType(OffsetOption.Type.POLICY); + offsetOption.setValue(toOffsetPolicy(gRpcOffsetOption.getPolicy())); + break; + case OFFSET: + offsetOption.setType(OffsetOption.Type.OFFSET); + offsetOption.setValue(gRpcOffsetOption.getOffset()); + break; + case TAIL_N: + offsetOption.setType(OffsetOption.Type.TAIL_N); + offsetOption.setValue(gRpcOffsetOption.getTailN()); + break; + case TIMESTAMP: + offsetOption.setType(OffsetOption.Type.TIMESTAMP); + offsetOption.setValue(gRpcOffsetOption.getTimestamp()); + break; + default: + throw new IllegalArgumentException("Unknown OffsetOption type: " + gRpcOffsetOption.getOffsetTypeCase()); + } + return offsetOption; + } + + private static long toOffsetPolicy(apache.rocketmq.v2.OffsetOption.Policy policy) { + switch (policy) { + case LAST: + return OffsetOption.POLICY_LAST_VALUE; + case MIN: + return OffsetOption.POLICY_MIN_VALUE; + case MAX: + return OffsetOption.POLICY_MAX_VALUE; + } + throw new IllegalArgumentException("Unknown OffsetOption.Policy value: " + policy); + } + public ContextStreamObserver telemetry(StreamObserver responseObserver) { return new ContextStreamObserver() { + private ProxyContext proxyCtx = null; @Override public void onNext(ProxyContext ctx, TelemetryCommand request) { + this.proxyCtx = ctx; try { switch (request.getCommandCase()) { case SETTINGS: { @@ -208,6 +302,7 @@ public void onNext(ProxyContext ctx, TelemetryCommand request) { @Override public void onError(Throwable t) { log.error("telemetry on error", t); + handleGrpcCancel(proxyCtx, t); } @Override @@ -217,6 +312,36 @@ public void onCompleted() { }; } + private static LiteSubscriptionAction toLiteAction(apache.rocketmq.v2.LiteSubscriptionAction gRpcAction) { + switch (gRpcAction) { + case PARTIAL_ADD: + return LiteSubscriptionAction.PARTIAL_ADD; + case PARTIAL_REMOVE: + return LiteSubscriptionAction.PARTIAL_REMOVE; + case COMPLETE_ADD: + return LiteSubscriptionAction.COMPLETE_ADD; + case COMPLETE_REMOVE: + return LiteSubscriptionAction.COMPLETE_REMOVE; + } + throw new IllegalArgumentException("unknown LiteSubscriptionAction: " + gRpcAction); + } + + private void handleGrpcCancel(ProxyContext ctx, Throwable t) { + final String clientId = ctx.getClientID(); + if (StringUtils.isBlank(clientId)) { + return; + } + if (!(t instanceof StatusRuntimeException)) { + return; + } + log.warn("handleGrpcCancel clientId:{}", clientId); + StatusRuntimeException statusException = (StatusRuntimeException) t; + if (io.grpc.Status.CANCELLED.getCode() == statusException.getStatus().getCode() || + io.grpc.Status.UNAVAILABLE.getCode() == statusException.getStatus().getCode()) { + this.grpcClientSettingsManager.offlineClientLiteSubscription(ctx, clientId, null); + } + } + protected void processTelemetryException(TelemetryCommand request, Throwable t, StreamObserver responseObserver) { StatusRuntimeException exception = io.grpc.Status.INTERNAL @@ -313,7 +438,7 @@ protected GrpcClientChannel registerConsumer(ProxyContext ctx, String consumerGr consumerGroup, clientChannelInfo, this.buildConsumeType(clientType), - MessageModel.CLUSTERING, + this.buildMessageModel(clientType), ConsumeFromWhere.CONSUME_FROM_LAST_OFFSET, this.buildSubscriptionDataSet(subscriptionEntryList), updateSubscription @@ -393,12 +518,20 @@ protected ConsumeType buildConsumeType(ClientType clientType) { case SIMPLE_CONSUMER: return ConsumeType.CONSUME_ACTIVELY; case PUSH_CONSUMER: + case LITE_PUSH_CONSUMER: return ConsumeType.CONSUME_PASSIVELY; default: throw new IllegalArgumentException("Client type is not consumer, type: " + clientType); } } + protected MessageModel buildMessageModel(ClientType clientType) { + if (clientType == ClientType.LITE_PUSH_CONSUMER) { + return MessageModel.LITE_SELECTIVE; + } + return MessageModel.CLUSTERING; + } + protected Set buildSubscriptionDataSet(List subscriptionEntryList) { Set subscriptionDataSet = new HashSet<>(); for (SubscriptionEntry sub : subscriptionEntryList) { diff --git a/proxy/src/main/java/org/apache/rocketmq/proxy/grpc/v2/common/GrpcClientSettingsManager.java b/proxy/src/main/java/org/apache/rocketmq/proxy/grpc/v2/common/GrpcClientSettingsManager.java index e741bd389d7..75cac21be4a 100644 --- a/proxy/src/main/java/org/apache/rocketmq/proxy/grpc/v2/common/GrpcClientSettingsManager.java +++ b/proxy/src/main/java/org/apache/rocketmq/proxy/grpc/v2/common/GrpcClientSettingsManager.java @@ -37,6 +37,8 @@ import org.apache.rocketmq.broker.client.ConsumerGroupInfo; import org.apache.rocketmq.common.ServiceThread; import org.apache.rocketmq.common.constant.LoggerName; +import org.apache.rocketmq.common.lite.LiteSubscriptionAction; +import org.apache.rocketmq.common.lite.LiteSubscriptionDTO; import org.apache.rocketmq.common.utils.StartAndShutdown; import org.apache.rocketmq.logging.org.slf4j.Logger; import org.apache.rocketmq.logging.org.slf4j.LoggerFactory; @@ -137,12 +139,15 @@ protected Settings mergeMetric(Settings settings) { protected static Settings mergeSubscriptionData(Settings settings, SubscriptionGroupConfig groupConfig) { Settings.Builder resultSettingsBuilder = settings.toBuilder(); - ProxyConfig config = ConfigurationManager.getProxyConfig(); + ProxyConfig proxyConfig = ConfigurationManager.getProxyConfig(); resultSettingsBuilder.getSubscriptionBuilder() - .setReceiveBatchSize(config.getGrpcClientConsumerLongPollingBatchSize()) - .setLongPollingTimeout(Durations.fromMillis(config.getGrpcClientConsumerMaxLongPollingTimeoutMillis())) - .setFifo(groupConfig.isConsumeMessageOrderly()); + .setReceiveBatchSize(proxyConfig.getGrpcClientConsumerLongPollingBatchSize()) + .setLongPollingTimeout(Durations.fromMillis(proxyConfig.getGrpcClientConsumerMaxLongPollingTimeoutMillis())) + .setFifo(groupConfig.isConsumeMessageOrderly()) + // client-side lite subscription quota limit + .setLiteSubscriptionQuota(groupConfig.getLiteSubClientQuota()) + .setMaxLiteTopicSize(proxyConfig.getMaxLiteTopicSize()); resultSettingsBuilder.getBackoffPolicyBuilder().setMaxAttempts(groupConfig.getRetryMaxTimes() + 1); @@ -213,6 +218,41 @@ public String getServiceName() { return "GrpcClientSettingsManagerCleaner"; } + /** + * Remove all lite subscriptions when client offline. + * + * @param ctx Proxy context + * @param clientId Client identifier + * @param settings Current client settings, if available + */ + public void offlineClientLiteSubscription(ProxyContext ctx, String clientId, Settings settings) { + if (settings == null) { + settings = getRawClientSettings(clientId); + } + if (settings == null || ClientType.LITE_PUSH_CONSUMER != settings.getClientType()) { + return; + } + try { + String topic = settings.getSubscription().getSubscriptions(0).getTopic().getName(); + String group = settings.getSubscription().getGroup().getName(); + log.info("offlineClientLiteSubscription, topic:{}, group:{}, clientId:{}", topic, group, clientId); + LiteSubscriptionDTO liteSubscriptionDTO = new LiteSubscriptionDTO() + .setAction(LiteSubscriptionAction.COMPLETE_REMOVE) + .setClientId(clientId) + .setGroup(group) + .setTopic(topic); + this.messagingProcessor.syncLiteSubscription(ctx, liteSubscriptionDTO, java.time.Duration.ofSeconds(2).toMillis()) + .whenComplete((result, throwable) -> { + if (throwable != null) { + log.error("offlineClientLiteSubscription failed, topic:{}, group:{}, clientId:{}", + topic, group, clientId, throwable); + } + }); + } catch (Exception e) { + log.error("offlineClientLiteSubscription error, clientId:{}, settings:{}", clientId, settings, e); + } + } + @Override public void run() { while (!this.isStopped()) { @@ -226,7 +266,9 @@ protected void onWaitEnd() { for (String clientId : clientIdSet) { try { CLIENT_SETTINGS_MAP.computeIfPresent(clientId, (clientIdKey, settings) -> { - if (!settings.getClientType().equals(ClientType.PUSH_CONSUMER) && !settings.getClientType().equals(ClientType.SIMPLE_CONSUMER)) { + if (!settings.getClientType().equals(ClientType.PUSH_CONSUMER) && + !settings.getClientType().equals(ClientType.SIMPLE_CONSUMER) && + !settings.getClientType().equals(ClientType.LITE_PUSH_CONSUMER)) { return settings; } String consumerGroup = settings.getSubscription().getGroup().getName(); diff --git a/proxy/src/main/java/org/apache/rocketmq/proxy/grpc/v2/common/GrpcConverter.java b/proxy/src/main/java/org/apache/rocketmq/proxy/grpc/v2/common/GrpcConverter.java index 4ce3dc831d4..04dab917ea4 100644 --- a/proxy/src/main/java/org/apache/rocketmq/proxy/grpc/v2/common/GrpcConverter.java +++ b/proxy/src/main/java/org/apache/rocketmq/proxy/grpc/v2/common/GrpcConverter.java @@ -170,6 +170,8 @@ protected SystemProperties buildSystemProperties(MessageExt messageExt) { systemPropertiesBuilder.setMessageType(MessageType.DELAY); } else if (messageExt.getProperty(MessageConst.PROPERTY_SHARDING_KEY) != null) { systemPropertiesBuilder.setMessageType(MessageType.FIFO); + } else if (messageExt.getProperty(MessageConst.PROPERTY_LITE_TOPIC) != null) { + systemPropertiesBuilder.setMessageType(MessageType.LITE); } else { systemPropertiesBuilder.setMessageType(MessageType.NORMAL); } @@ -224,6 +226,12 @@ protected SystemProperties buildSystemProperties(MessageExt messageExt) { systemPropertiesBuilder.setMessageGroup(shardingKey); } + // lite topic + String liteTopic = messageExt.getProperty(MessageConst.PROPERTY_LITE_TOPIC); + if (liteTopic != null) { + systemPropertiesBuilder.setLiteTopic(liteTopic); + } + // receipt_handle && invisible_period String handle = messageExt.getProperty(MessageConst.PROPERTY_POP_CK); if (handle != null) { diff --git a/proxy/src/main/java/org/apache/rocketmq/proxy/grpc/v2/common/GrpcValidator.java b/proxy/src/main/java/org/apache/rocketmq/proxy/grpc/v2/common/GrpcValidator.java index a556bfe2710..d11676bb5a5 100644 --- a/proxy/src/main/java/org/apache/rocketmq/proxy/grpc/v2/common/GrpcValidator.java +++ b/proxy/src/main/java/org/apache/rocketmq/proxy/grpc/v2/common/GrpcValidator.java @@ -20,6 +20,7 @@ import apache.rocketmq.v2.Code; import apache.rocketmq.v2.Resource; import com.google.common.base.CharMatcher; +import java.nio.charset.StandardCharsets; import org.apache.commons.lang3.StringUtils; import org.apache.rocketmq.client.Validators; import org.apache.rocketmq.client.exception.MQClientException; @@ -121,4 +122,33 @@ public boolean containControlCharacter(String data) { } return false; } + + public void validateLiteTopic(String liteTopic) { + if (StringUtils.isBlank(liteTopic)) { + throw new GrpcProxyException(Code.ILLEGAL_LITE_TOPIC, "lite topic cannot be the char sequence of whitespace"); + } + int maxSize = ConfigurationManager.getProxyConfig().getMaxLiteTopicSize(); + if (liteTopic.getBytes(StandardCharsets.UTF_8).length > maxSize) { + throw new GrpcProxyException(Code.ILLEGAL_LITE_TOPIC, "lite topic exceed the max size " + maxSize); + } + if (!isValidLiteTopic(liteTopic)) { + throw new GrpcProxyException(Code.ILLEGAL_LITE_TOPIC, "lite topic can only contain alphanumeric characters, hyphens(-), and underscores(_)"); + } + } + + /** + * alternative for regex "^[a-zA-Z0-9_-]+$" + */ + private boolean isValidLiteTopic(String liteTopic) { + for (int i = 0; i < liteTopic.length(); i++) { + char c = liteTopic.charAt(i); + if (!(c >= 'a' && c <= 'z') && + !(c >= 'A' && c <= 'Z') && + !(c >= '0' && c <= '9') && + c != '-' && c != '_') { + return false; + } + } + return true; + } } diff --git a/proxy/src/main/java/org/apache/rocketmq/proxy/grpc/v2/common/ResponseBuilder.java b/proxy/src/main/java/org/apache/rocketmq/proxy/grpc/v2/common/ResponseBuilder.java index ee5fc019e1a..97ade7de2c7 100644 --- a/proxy/src/main/java/org/apache/rocketmq/proxy/grpc/v2/common/ResponseBuilder.java +++ b/proxy/src/main/java/org/apache/rocketmq/proxy/grpc/v2/common/ResponseBuilder.java @@ -48,6 +48,8 @@ public class ResponseBuilder { RESPONSE_CODE_MAPPING.put(ResponseCode.SYSTEM_BUSY, Code.TOO_MANY_REQUESTS); RESPONSE_CODE_MAPPING.put(ResponseCode.REQUEST_CODE_NOT_SUPPORTED, Code.NOT_IMPLEMENTED); RESPONSE_CODE_MAPPING.put(ResponseCode.SUBSCRIPTION_GROUP_NOT_EXIST, Code.CONSUMER_GROUP_NOT_FOUND); + RESPONSE_CODE_MAPPING.put(ResponseCode.LMQ_QUOTA_EXCEEDED, Code.LITE_TOPIC_QUOTA_EXCEEDED); + RESPONSE_CODE_MAPPING.put(ResponseCode.LITE_SUBSCRIPTION_QUOTA_EXCEEDED, Code.LITE_SUBSCRIPTION_QUOTA_EXCEEDED); RESPONSE_CODE_MAPPING.put(ClientErrorCode.ACCESS_BROKER_TIMEOUT, Code.PROXY_TIMEOUT); } diff --git a/proxy/src/main/java/org/apache/rocketmq/proxy/grpc/v2/consumer/AckMessageActivity.java b/proxy/src/main/java/org/apache/rocketmq/proxy/grpc/v2/consumer/AckMessageActivity.java index 580f3b5f345..59de5abda63 100644 --- a/proxy/src/main/java/org/apache/rocketmq/proxy/grpc/v2/consumer/AckMessageActivity.java +++ b/proxy/src/main/java/org/apache/rocketmq/proxy/grpc/v2/consumer/AckMessageActivity.java @@ -55,7 +55,9 @@ public CompletableFuture ackMessage(ProxyContext ctx, AckMes validateTopicAndConsumerGroup(request.getTopic(), request.getGroup()); String group = request.getGroup().getName(); String topic = request.getTopic().getName(); - if (ConfigurationManager.getProxyConfig().isEnableBatchAck()) { + boolean isBatchAck = ConfigurationManager.getProxyConfig().isEnableBatchAck() + && !request.getEntries(0).hasLiteTopic(); + if (isBatchAck) { future = ackMessageInBatch(ctx, group, topic, request); } else { future = ackMessageOneByOne(ctx, group, topic, request); @@ -143,7 +145,8 @@ protected CompletableFuture processAckMessage(ProxyContex ReceiptHandle.decode(handleString), ackMessageEntry.getMessageId(), group, - topic + topic, + ackMessageEntry.hasLiteTopic() ? ackMessageEntry.getLiteTopic() : null ); ackResultFuture.thenAccept(result -> { future.complete(convertToAckMessageResultEntry(ctx, ackMessageEntry, result)); diff --git a/proxy/src/main/java/org/apache/rocketmq/proxy/grpc/v2/consumer/ReceiveMessageActivity.java b/proxy/src/main/java/org/apache/rocketmq/proxy/grpc/v2/consumer/ReceiveMessageActivity.java index 96afb4640ad..f5e1c7b76f3 100644 --- a/proxy/src/main/java/org/apache/rocketmq/proxy/grpc/v2/consumer/ReceiveMessageActivity.java +++ b/proxy/src/main/java/org/apache/rocketmq/proxy/grpc/v2/consumer/ReceiveMessageActivity.java @@ -16,6 +16,7 @@ */ package org.apache.rocketmq.proxy.grpc.v2.consumer; +import apache.rocketmq.v2.ClientType; import apache.rocketmq.v2.Code; import apache.rocketmq.v2.FilterExpression; import apache.rocketmq.v2.ReceiveMessageRequest; @@ -25,7 +26,9 @@ import com.google.protobuf.util.Durations; import io.grpc.stub.StreamObserver; import java.util.List; +import java.util.concurrent.CompletableFuture; import org.apache.commons.lang3.StringUtils; +import org.apache.rocketmq.client.consumer.PopResult; import org.apache.rocketmq.client.consumer.PopStatus; import org.apache.rocketmq.common.constant.ConsumeInitMode; import org.apache.rocketmq.common.message.MessageConst; @@ -62,6 +65,8 @@ public void receiveMessage(ProxyContext ctx, ReceiveMessageRequest request, try { Settings settings = this.grpcClientSettingsManager.getClientSettings(ctx); + final boolean isLite = ClientType.LITE_PUSH_CONSUMER.equals(settings.getClientType()); + Subscription subscription = settings.getSubscription(); boolean fifo = subscription.getFifo(); int maxAttempts = settings.getBackoffPolicy().getMaxAttempts(); @@ -118,7 +123,41 @@ public void receiveMessage(ProxyContext ctx, ReceiveMessageRequest request, return; } - this.messagingProcessor.popMessage( + CompletableFuture popFuture; + if (isLite) { + + GrpcClientChannel clientChannel = grpcChannelManager.getChannel(ctx.getClientID()); + if (clientChannel == null) { + writer.writeAndComplete(ctx, Code.BAD_REQUEST, + String.format("The client [%s] is disconnected.", ctx.getClientID())); + return; + } + // check lite consumer max unacked messages + int unackedMessageCount = messagingProcessor.getUnackedMessageCount(ctx, clientChannel, group); + if (proxyConfig.getMaxLiteRenewNumPerChannel() < unackedMessageCount) { + writer.writeAndComplete(ctx, Code.FORBIDDEN, + String.format("The client [%s] has too many unacked messages. Unacked count: %d", + ctx.getClientID(), unackedMessageCount)); + return; + } + + popFuture = this.messagingProcessor.popLiteMessage( + ctx, + new ReceiveMessageQueueSelector( + request.getMessageQueue().getBroker().getName() + ), + group, + topic, + request.getBatchSize(), + actualInvisibleTime, + pollingTime, + subscriptionData, + new PopMessageResultFilterImpl(maxAttempts), + request.hasAttemptId() ? request.getAttemptId() : null, + timeRemaining + ); + } else { + popFuture = this.messagingProcessor.popMessage( ctx, new ReceiveMessageQueueSelector( request.getMessageQueue().getBroker().getName() @@ -134,43 +173,54 @@ public void receiveMessage(ProxyContext ctx, ReceiveMessageRequest request, new PopMessageResultFilterImpl(maxAttempts), request.hasAttemptId() ? request.getAttemptId() : null, timeRemaining - ).thenAccept(popResult -> { - Runnable doAfterWrite = null; - if (proxyConfig.isEnableProxyAutoRenew() && request.getAutoRenew()) { - if (PopStatus.FOUND.equals(popResult.getPopStatus())) { - GrpcClientChannel clientChannel = grpcChannelManager.getChannel(ctx.getClientID()); - if (clientChannel == null) { - GrpcProxyException e = new GrpcProxyException(Code.MESSAGE_NOT_FOUND, - String.format("The client [%s] is disconnected.", ctx.getClientID())); - popResult.getMsgFoundList().forEach(messageExt -> - writer.processThrowableWhenWriteMessage(e, ctx, request, messageExt)); - throw e; - } - doAfterWrite = () -> { - List messageExtList = popResult.getMsgFoundList(); - for (MessageExt messageExt : messageExtList) { - String receiptHandle = messageExt.getProperty(MessageConst.PROPERTY_POP_CK); - if (receiptHandle != null) { - MessageReceiptHandle messageReceiptHandle = - new MessageReceiptHandle(group, topic, messageExt.getQueueId(), receiptHandle, messageExt.getMsgId(), - messageExt.getQueueOffset(), messageExt.getReconsumeTimes()); - messagingProcessor.addReceiptHandle(ctx, clientChannel, group, messageExt.getMsgId(), messageReceiptHandle); - } - } - }; - } - } - writer.writeAndComplete(ctx, request, popResult, doAfterWrite); - }) - .exceptionally(t -> { - writer.writeAndComplete(ctx, request, t); - return null; - }); + ); + } + + final boolean autoRenew = proxyConfig.isEnableProxyAutoRenew() && request.getAutoRenew(); + popFuture.thenAccept(popResult -> { + Runnable doAfterWrite = null; + if (autoRenew) { + doAfterWrite = handleAutoRenew(ctx, request, group, topic, popResult, writer); + } + writer.writeAndComplete(ctx, request, popResult, doAfterWrite); + }).exceptionally(t -> { + writer.writeAndComplete(ctx, request, t); + return null; + }); } catch (Throwable t) { writer.writeAndComplete(ctx, request, t); } } + private Runnable handleAutoRenew(ProxyContext ctx, ReceiveMessageRequest request, + String group, String topic, PopResult popResult, ReceiveMessageResponseStreamWriter writer + ) { + if (!PopStatus.FOUND.equals(popResult.getPopStatus())) { + return null; + } + + GrpcClientChannel clientChannel = grpcChannelManager.getChannel(ctx.getClientID()); + if (clientChannel == null) { + GrpcProxyException e = new GrpcProxyException(Code.MESSAGE_NOT_FOUND, + String.format("The client [%s] is disconnected.", ctx.getClientID())); + popResult.getMsgFoundList().forEach(messageExt -> + writer.processThrowableWhenWriteMessage(e, ctx, request, messageExt)); + throw e; + } + return () -> { + List messageExtList = popResult.getMsgFoundList(); + for (MessageExt messageExt : messageExtList) { + String receiptHandle = messageExt.getProperty(MessageConst.PROPERTY_POP_CK); + if (receiptHandle != null) { + MessageReceiptHandle messageReceiptHandle = + new MessageReceiptHandle(group, topic, messageExt.getQueueId(), receiptHandle, messageExt.getMsgId(), + messageExt.getQueueOffset(), messageExt.getReconsumeTimes()); + messagingProcessor.addReceiptHandle(ctx, clientChannel, group, messageExt.getMsgId(), messageReceiptHandle); + } + } + }; + } + protected ReceiveMessageResponseStreamWriter createWriter(ProxyContext ctx, StreamObserver responseObserver) { return new ReceiveMessageResponseStreamWriter( diff --git a/proxy/src/main/java/org/apache/rocketmq/proxy/grpc/v2/producer/ForwardMessageToDLQActivity.java b/proxy/src/main/java/org/apache/rocketmq/proxy/grpc/v2/producer/ForwardMessageToDLQActivity.java index 45e6638d507..af060d1860b 100644 --- a/proxy/src/main/java/org/apache/rocketmq/proxy/grpc/v2/producer/ForwardMessageToDLQActivity.java +++ b/proxy/src/main/java/org/apache/rocketmq/proxy/grpc/v2/producer/ForwardMessageToDLQActivity.java @@ -50,12 +50,15 @@ public CompletableFuture forwardMessage } ReceiptHandle receiptHandle = ReceiptHandle.decode(handleString); + String liteTopic = request.hasLiteTopic() ? request.getLiteTopic() : null; + return this.messagingProcessor.forwardMessageToDeadLetterQueue( ctx, receiptHandle, request.getMessageId(), request.getGroup().getName(), - request.getTopic().getName() + request.getTopic().getName(), + liteTopic ).thenApply(result -> convertToForwardMessageToDeadLetterQueueResponse(ctx, result)); } catch (Throwable t) { future.completeExceptionally(t); diff --git a/proxy/src/main/java/org/apache/rocketmq/proxy/grpc/v2/producer/SendMessageActivity.java b/proxy/src/main/java/org/apache/rocketmq/proxy/grpc/v2/producer/SendMessageActivity.java index 69bcaa27a0d..e0df3f1c909 100644 --- a/proxy/src/main/java/org/apache/rocketmq/proxy/grpc/v2/producer/SendMessageActivity.java +++ b/proxy/src/main/java/org/apache/rocketmq/proxy/grpc/v2/producer/SendMessageActivity.java @@ -279,6 +279,13 @@ protected Map buildMessageProperty(ProxyContext context, apache. validateMessageGroup(messageGroup); MessageAccessor.putProperty(messageWithHeader, MessageConst.PROPERTY_SHARDING_KEY, messageGroup); } + // set lite topic + String liteTopic = message.getSystemProperties().getLiteTopic(); + if (StringUtils.isNotEmpty(liteTopic)) { + validateLiteTopic(liteTopic); + MessageAccessor.setLiteTopic(messageWithHeader, liteTopic); + } + // set trace context String traceContext = message.getSystemProperties().getTraceContext(); if (!traceContext.isEmpty()) { @@ -385,6 +392,10 @@ public AddressableMessageQueue select(ProxyContext ctx, MessageQueueView message String shardingKey = null; if (request.getMessagesCount() == 1) { shardingKey = message.getSystemProperties().getMessageGroup(); + // lite topic + if (StringUtils.isBlank(shardingKey)) { + shardingKey = message.getSystemProperties().getLiteTopic(); + } } AddressableMessageQueue targetMessageQueue; if (StringUtils.isNotEmpty(shardingKey)) { diff --git a/proxy/src/main/java/org/apache/rocketmq/proxy/grpc/v2/route/RouteActivity.java b/proxy/src/main/java/org/apache/rocketmq/proxy/grpc/v2/route/RouteActivity.java index 7132b42953d..75f7089c5e0 100644 --- a/proxy/src/main/java/org/apache/rocketmq/proxy/grpc/v2/route/RouteActivity.java +++ b/proxy/src/main/java/org/apache/rocketmq/proxy/grpc/v2/route/RouteActivity.java @@ -38,6 +38,7 @@ import java.util.List; import java.util.Map; import java.util.concurrent.CompletableFuture; +import org.apache.commons.lang3.StringUtils; import org.apache.rocketmq.common.MixAll; import org.apache.rocketmq.common.attribute.TopicMessageType; import org.apache.rocketmq.common.constant.PermName; @@ -108,11 +109,13 @@ public CompletableFuture queryAssignment(ProxyContext c addressList, request.getTopic().getName()); - boolean fifo = false; - SubscriptionGroupConfig config = this.messagingProcessor.getSubscriptionGroupConfig(ctx, - request.getGroup().getName()); - if (config != null && config.isConsumeMessageOrderly()) { - fifo = true; + boolean isFifo = false; + boolean isLite = false; + SubscriptionGroupConfig groupConfig = this.messagingProcessor + .getSubscriptionGroupConfig(ctx, request.getGroup().getName()); + if (groupConfig != null) { + isFifo = groupConfig.isConsumeMessageOrderly(); + isLite = StringUtils.isNotEmpty(groupConfig.getLiteBindTopic()); } List assignments = new ArrayList<>(); @@ -123,7 +126,7 @@ public CompletableFuture queryAssignment(ProxyContext c if (brokerIdMap != null) { Broker broker = brokerIdMap.get(MixAll.MASTER_ID); Permission permission = this.convertToPermission(queueData.getPerm()); - if (fifo) { + if (isFifo && !isLite) { for (int i = 0; i < queueData.getReadQueueNums(); i++) { MessageQueue defaultMessageQueue = MessageQueue.newBuilder() .setTopic(request.getTopic()) @@ -302,6 +305,8 @@ private List parseTopicMessageType(TopicMessageType topicMessageTyp return Collections.singletonList(MessageType.NORMAL); case FIFO: return Collections.singletonList(MessageType.FIFO); + case LITE: + return Collections.singletonList(MessageType.LITE); case TRANSACTION: return Collections.singletonList(MessageType.TRANSACTION); case DELAY: diff --git a/proxy/src/main/java/org/apache/rocketmq/proxy/processor/ClientProcessor.java b/proxy/src/main/java/org/apache/rocketmq/proxy/processor/ClientProcessor.java index eeb9bf87e67..c73e66416da 100644 --- a/proxy/src/main/java/org/apache/rocketmq/proxy/processor/ClientProcessor.java +++ b/proxy/src/main/java/org/apache/rocketmq/proxy/processor/ClientProcessor.java @@ -16,24 +16,46 @@ */ package org.apache.rocketmq.proxy.processor; +import apache.rocketmq.v2.Code; +import com.google.common.util.concurrent.RateLimiter; import io.netty.channel.Channel; +import java.util.Objects; import java.util.Set; +import java.util.concurrent.CompletableFuture; +import org.apache.commons.collections.CollectionUtils; +import org.apache.commons.lang3.StringUtils; import org.apache.rocketmq.broker.client.ClientChannelInfo; import org.apache.rocketmq.broker.client.ConsumerGroupInfo; import org.apache.rocketmq.broker.client.ConsumerIdsChangeListener; import org.apache.rocketmq.broker.client.ProducerChangeListener; +import org.apache.rocketmq.common.constant.LoggerName; import org.apache.rocketmq.common.consumer.ConsumeFromWhere; +import org.apache.rocketmq.common.lite.LiteSubscriptionAction; +import org.apache.rocketmq.common.lite.LiteSubscriptionDTO; +import org.apache.rocketmq.logging.org.slf4j.Logger; +import org.apache.rocketmq.logging.org.slf4j.LoggerFactory; import org.apache.rocketmq.proxy.common.ProxyContext; +import org.apache.rocketmq.proxy.config.ConfigurationManager; +import org.apache.rocketmq.proxy.config.ProxyConfig; +import org.apache.rocketmq.proxy.grpc.v2.common.GrpcProxyException; import org.apache.rocketmq.proxy.service.ServiceManager; import org.apache.rocketmq.remoting.protocol.heartbeat.ConsumeType; import org.apache.rocketmq.remoting.protocol.heartbeat.MessageModel; import org.apache.rocketmq.remoting.protocol.heartbeat.SubscriptionData; +import org.apache.rocketmq.remoting.protocol.subscription.SubscriptionGroupConfig; +@SuppressWarnings("UnstableApiUsage") public class ClientProcessor extends AbstractProcessor { + private static final Logger log = LoggerFactory.getLogger(LoggerName.PROXY_LOGGER_NAME); + + private final RateLimiter syncLiteSubscriptionRateLimiter; public ClientProcessor(MessagingProcessor messagingProcessor, ServiceManager serviceManager) { super(messagingProcessor, serviceManager); + + ProxyConfig proxyConfig = ConfigurationManager.getProxyConfig(); + this.syncLiteSubscriptionRateLimiter = RateLimiter.create(proxyConfig.getMaxSyncLiteSubscriptionRate()); } public void registerProducer( @@ -74,6 +96,10 @@ public void registerConsumer( Set subList, boolean updateSubscription ) { + validateLiteMode(ctx, consumerGroup, messageModel); + if (MessageModel.LITE_SELECTIVE == messageModel) { + validateLiteSubTopic(ctx, consumerGroup, subList); + } this.serviceManager.getConsumerManager().registerConsumer( consumerGroup, clientChannelInfo, @@ -85,6 +111,34 @@ public void registerConsumer( updateSubscription); } + public CompletableFuture syncLiteSubscription(ProxyContext ctx, + LiteSubscriptionDTO liteSubscriptionDTO, long timeoutMillis + ) { + try { + validateLiteBindTopic(ctx, liteSubscriptionDTO.getGroup(), liteSubscriptionDTO.getTopic()); + if (CollectionUtils.isNotEmpty(liteSubscriptionDTO.getLiteTopicSet())) { + validateLiteSubscriptionQuota(ctx, liteSubscriptionDTO.getGroup(), liteSubscriptionDTO.getLiteTopicSet().size()); + } + + if (LiteSubscriptionAction.PARTIAL_ADD == liteSubscriptionDTO.getAction()) { + if (!syncLiteSubscriptionRateLimiter.tryAcquire()) { + String msg = String.format("Too many syncLiteSubscription requests, topic=%s, group=%s, clientId=%s", + liteSubscriptionDTO.getTopic(), liteSubscriptionDTO.getGroup(), ctx.getClientID()); + log.warn(msg); + throw new GrpcProxyException(Code.TOO_MANY_REQUESTS, msg); + } + } + + return this.serviceManager + .getLiteSubscriptionService() + .syncLiteSubscription(ctx, liteSubscriptionDTO, timeoutMillis); + } catch (Throwable t) { + CompletableFuture future = new CompletableFuture<>(); + future.completeExceptionally(t); + return future; + } + } + public ClientChannelInfo findConsumerChannel( ProxyContext ctx, String consumerGroup, @@ -113,4 +167,63 @@ public void registerConsumerIdsChangeListener(ConsumerIdsChangeListener listener public ConsumerGroupInfo getConsumerGroupInfo(ProxyContext ctx, String consumerGroup) { return this.serviceManager.getConsumerManager().getConsumerGroupInfo(consumerGroup); } + + /** + * Validates the message model for a given consumer group. + * Ensures that regular groups do not use LITE mode and LITE groups use LITE mode. + * + * @param ctx the proxy context + * @param group the consumer group name + * @param messageModel the message model to validate + */ + protected void validateLiteMode(ProxyContext ctx, String group, MessageModel messageModel) { + String bindTopic = getGroupOrException(ctx, group).getLiteBindTopic(); + if (StringUtils.isEmpty(bindTopic)) { + // regular group + if (MessageModel.LITE_SELECTIVE == messageModel) { + throw new GrpcProxyException(Code.ILLEGAL_CONSUMER_GROUP, + "regular group cannot use LITE mode: " + group); + } + } else { + // lite group + if (MessageModel.LITE_SELECTIVE != messageModel) { + throw new GrpcProxyException(Code.ILLEGAL_CONSUMER_GROUP, + "lite group must use LITE mode: " + group); + } + } + } + + protected void validateLiteSubTopic(ProxyContext ctx, String group, Set subList) { + if (CollectionUtils.isEmpty(subList)) { + return; + } + // check bindTopic for sub list + validateLiteBindTopic(ctx, group, subList.iterator().next().getTopic()); + } + + protected void validateLiteBindTopic(ProxyContext ctx, String group, String bindTopic) { + String expectedBindTopic = getGroupOrException(ctx, group).getLiteBindTopic(); + if (!Objects.equals(expectedBindTopic, bindTopic)) { + throw new GrpcProxyException(Code.ILLEGAL_TOPIC, + String.format("lite group %s is expected to bind topic %s, but actual is %s", + group, expectedBindTopic, bindTopic)); + } + } + + protected void validateLiteSubscriptionQuota(ProxyContext ctx, String group, int actual) { + int quota = getGroupOrException(ctx, group).getLiteSubClientQuota(); + int quotaBuffer = 300; + if (actual > quota + quotaBuffer) { + throw new GrpcProxyException(Code.LITE_SUBSCRIPTION_QUOTA_EXCEEDED, + "lite subscription quota exceeded: " + quota); + } + } + + protected SubscriptionGroupConfig getGroupOrException(ProxyContext ctx, String group) { + SubscriptionGroupConfig groupConfig = this.messagingProcessor.getSubscriptionGroupConfig(ctx, group); + if (groupConfig == null) { + throw new GrpcProxyException(Code.ILLEGAL_CONSUMER_GROUP, "group not found: " + group); + } + return groupConfig; + } } diff --git a/proxy/src/main/java/org/apache/rocketmq/proxy/processor/ConsumerProcessor.java b/proxy/src/main/java/org/apache/rocketmq/proxy/processor/ConsumerProcessor.java index ea2043b913c..cd93aed0f7a 100644 --- a/proxy/src/main/java/org/apache/rocketmq/proxy/processor/ConsumerProcessor.java +++ b/proxy/src/main/java/org/apache/rocketmq/proxy/processor/ConsumerProcessor.java @@ -27,6 +27,7 @@ import java.util.concurrent.CopyOnWriteArraySet; import java.util.concurrent.ExecutorService; import java.util.stream.Collectors; +import org.apache.commons.collections.CollectionUtils; import org.apache.commons.lang3.StringUtils; import org.apache.rocketmq.client.consumer.AckResult; import org.apache.rocketmq.client.consumer.PopResult; @@ -50,12 +51,14 @@ import org.apache.rocketmq.proxy.service.ServiceManager; import org.apache.rocketmq.proxy.service.message.ReceiptHandleMessage; import org.apache.rocketmq.proxy.service.route.AddressableMessageQueue; +import org.apache.rocketmq.remoting.CommandCustomHeader; import org.apache.rocketmq.remoting.protocol.body.LockBatchRequestBody; import org.apache.rocketmq.remoting.protocol.body.UnlockBatchRequestBody; import org.apache.rocketmq.remoting.protocol.header.AckMessageRequestHeader; import org.apache.rocketmq.remoting.protocol.header.ChangeInvisibleTimeRequestHeader; import org.apache.rocketmq.remoting.protocol.header.GetMaxOffsetRequestHeader; import org.apache.rocketmq.remoting.protocol.header.GetMinOffsetRequestHeader; +import org.apache.rocketmq.remoting.protocol.header.PopLiteMessageRequestHeader; import org.apache.rocketmq.remoting.protocol.header.PopMessageRequestHeader; import org.apache.rocketmq.remoting.protocol.header.PullMessageRequestHeader; import org.apache.rocketmq.remoting.protocol.header.QueryConsumerOffsetRequestHeader; @@ -143,59 +146,139 @@ public CompletableFuture popMessage( messageQueue, requestHeader, timeoutMillis) - .thenApplyAsync(popResult -> { - if (PopStatus.FOUND.equals(popResult.getPopStatus()) && - popResult.getMsgFoundList() != null && - !popResult.getMsgFoundList().isEmpty() && - popMessageResultFilter != null) { - - List messageExtList = new ArrayList<>(); - for (MessageExt messageExt : popResult.getMsgFoundList()) { - try { - fillUniqIDIfNeed(messageExt); - String handleString = createHandle(messageExt.getProperty(MessageConst.PROPERTY_POP_CK), messageExt.getCommitLogOffset()); - if (handleString == null) { - log.error("[BUG] pop message from broker but handle is empty. requestHeader:{}, msg:{}", requestHeader, messageExt); - messageExtList.add(messageExt); - continue; - } - MessageAccessor.putProperty(messageExt, MessageConst.PROPERTY_POP_CK, handleString); - - PopMessageResultFilter.FilterResult filterResult = - popMessageResultFilter.filterMessage(ctx, consumerGroup, subscriptionData, messageExt); - switch (filterResult) { - case NO_MATCH: - this.messagingProcessor.ackMessage( - ctx, - ReceiptHandle.decode(handleString), - messageExt.getMsgId(), - consumerGroup, - topic, - MessagingProcessor.DEFAULT_TIMEOUT_MILLS); - break; - case TO_DLQ: - this.messagingProcessor.forwardMessageToDeadLetterQueue( - ctx, - ReceiptHandle.decode(handleString), - messageExt.getMsgId(), - consumerGroup, - topic, - MessagingProcessor.DEFAULT_TIMEOUT_MILLS); - break; - case MATCH: - default: - messageExtList.add(messageExt); - break; - } - } catch (Throwable t) { - log.error("process filterMessage failed. requestHeader:{}, msg:{}", requestHeader, messageExt, t); - messageExtList.add(messageExt); - } - } - popResult.setMsgFoundList(messageExtList); + .thenApplyAsync(popResult -> filterPopResult(ctx, popResult, + requestHeader, consumerGroup, topic, subscriptionData, popMessageResultFilter), this.executor); + } catch (Throwable t) { + future.completeExceptionally(t); + } + return FutureUtils.addExecutor(future, this.executor); + } + + private PopResult filterPopResult(ProxyContext ctx, PopResult popResult, CommandCustomHeader requestHeader, + String consumerGroup, String topic, SubscriptionData subscriptionData, + PopMessageResultFilter popMessageResultFilter) { + if (PopStatus.FOUND.equals(popResult.getPopStatus()) && + !CollectionUtils.isEmpty(popResult.getMsgFoundList()) && + popMessageResultFilter != null) { + + List messageExtList = new ArrayList<>(); + for (MessageExt messageExt : popResult.getMsgFoundList()) { + try { + fillUniqIDIfNeed(messageExt); + String handleString = createHandle(messageExt.getProperty(MessageConst.PROPERTY_POP_CK), messageExt.getCommitLogOffset()); + if (handleString == null) { + log.error("[BUG] pop message from broker but handle is empty. requestHeader:{}, msg:{}", requestHeader, messageExt); + messageExtList.add(messageExt); + continue; } - return popResult; - }, this.executor); + MessageAccessor.putProperty(messageExt, MessageConst.PROPERTY_POP_CK, handleString); + + String liteTopic = messageExt.getProperty(MessageConst.PROPERTY_LITE_TOPIC); + + PopMessageResultFilter.FilterResult filterResult = + popMessageResultFilter.filterMessage(ctx, consumerGroup, subscriptionData, messageExt); + switch (filterResult) { + case NO_MATCH: + this.messagingProcessor.ackMessage( + ctx, + ReceiptHandle.decode(handleString), + messageExt.getMsgId(), + consumerGroup, + topic, + liteTopic, + MessagingProcessor.DEFAULT_TIMEOUT_MILLS); + break; + case TO_DLQ: + this.messagingProcessor.forwardMessageToDeadLetterQueue( + ctx, + ReceiptHandle.decode(handleString), + messageExt.getMsgId(), + consumerGroup, + topic, + liteTopic, + MessagingProcessor.DEFAULT_TIMEOUT_MILLS); + break; + case MATCH: + default: + messageExtList.add(messageExt); + break; + } + } catch (Throwable t) { + log.error("process filterMessage failed. requestHeader:{}, msg:{}", requestHeader, messageExt, t); + messageExtList.add(messageExt); + } + } + popResult.setMsgFoundList(messageExtList); + } + return popResult; + } + + public CompletableFuture popLiteMessage( + ProxyContext ctx, + QueueSelector queueSelector, + String consumerGroup, + String topic, + int maxMsgNums, + long invisibleTime, + long pollTime, + SubscriptionData subscriptionData, + PopMessageResultFilter popMessageResultFilter, + String attemptId, + long timeoutMillis + ) { + CompletableFuture future = new CompletableFuture<>(); + try { + AddressableMessageQueue messageQueue = queueSelector.select(ctx, + this.serviceManager.getTopicRouteService().getCurrentMessageQueueView(ctx, topic)); + if (messageQueue == null) { + throw new ProxyException(ProxyExceptionCode.FORBIDDEN, "no readable queue"); + } + return doPopLiteMessage(ctx, messageQueue, consumerGroup, topic, maxMsgNums, invisibleTime, pollTime, + subscriptionData, popMessageResultFilter, attemptId, timeoutMillis); + } catch (Throwable t) { + future.completeExceptionally(t); + } + return future; + } + + private CompletableFuture doPopLiteMessage( + ProxyContext ctx, + AddressableMessageQueue messageQueue, + String consumerGroup, + String topic, + int maxMsgNums, + long invisibleTime, + long pollTime, + SubscriptionData subscriptionData, + PopMessageResultFilter popMessageResultFilter, + String attemptId, + long timeoutMillis + ) { + CompletableFuture future = new CompletableFuture<>(); + try { + if (maxMsgNums > ProxyUtils.MAX_MSG_NUMS_FOR_POP_REQUEST) { + log.warn("change maxNums from {} to {} for pop request, with info: topic:{}, group:{}", + maxMsgNums, ProxyUtils.MAX_MSG_NUMS_FOR_POP_REQUEST, topic, consumerGroup); + maxMsgNums = ProxyUtils.MAX_MSG_NUMS_FOR_POP_REQUEST; + } + + PopLiteMessageRequestHeader requestHeader = new PopLiteMessageRequestHeader(); + requestHeader.setClientId(ctx.getClientID()); + requestHeader.setConsumerGroup(consumerGroup); + requestHeader.setTopic(topic); + requestHeader.setMaxMsgNum(maxMsgNums); + requestHeader.setInvisibleTime(invisibleTime); + requestHeader.setPollTime(pollTime); + requestHeader.setAttemptId(attemptId); + requestHeader.setBornTime(System.currentTimeMillis()); + + future = this.serviceManager.getMessageService().popLiteMessage( + ctx, + messageQueue, + requestHeader, + timeoutMillis) + .thenApplyAsync(popResult -> filterPopResult(ctx, popResult, + requestHeader, consumerGroup, topic, subscriptionData, popMessageResultFilter), this.executor); } catch (Throwable t) { future.completeExceptionally(t); FutureUtils.addExecutor(future, this.executor); @@ -218,6 +301,7 @@ public CompletableFuture ackMessage( String messageId, String consumerGroup, String topic, + String liteTopic, long timeoutMillis ) { CompletableFuture future = new CompletableFuture<>(); @@ -230,6 +314,7 @@ public CompletableFuture ackMessage( ackMessageRequestHeader.setQueueId(handle.getQueueId()); ackMessageRequestHeader.setExtraInfo(handle.getReceiptHandle()); ackMessageRequestHeader.setOffset(handle.getOffset()); + ackMessageRequestHeader.setLiteTopic(liteTopic); future = this.serviceManager.getMessageService().ackMessage( ctx, @@ -308,7 +393,7 @@ protected CompletableFuture> processBrokerHandle(ProxyConte } public CompletableFuture changeInvisibleTime(ProxyContext ctx, ReceiptHandle handle, - String messageId, String groupName, String topicName, long invisibleTime, long timeoutMillis) { + String messageId, String groupName, String topicName, long invisibleTime, String liteTopic, long timeoutMillis) { CompletableFuture future = new CompletableFuture<>(); try { this.validateReceiptHandle(handle); @@ -320,6 +405,7 @@ public CompletableFuture changeInvisibleTime(ProxyContext ctx, Receip changeInvisibleTimeRequestHeader.setExtraInfo(handle.getReceiptHandle()); changeInvisibleTimeRequestHeader.setOffset(handle.getOffset()); changeInvisibleTimeRequestHeader.setInvisibleTime(invisibleTime); + changeInvisibleTimeRequestHeader.setLiteTopic(liteTopic); long commitLogOffset = handle.getCommitLogOffset(); future = this.serviceManager.getMessageService().changeInvisibleTime( diff --git a/proxy/src/main/java/org/apache/rocketmq/proxy/processor/DefaultMessagingProcessor.java b/proxy/src/main/java/org/apache/rocketmq/proxy/processor/DefaultMessagingProcessor.java index fe25dfa511e..bc044ec7a13 100644 --- a/proxy/src/main/java/org/apache/rocketmq/proxy/processor/DefaultMessagingProcessor.java +++ b/proxy/src/main/java/org/apache/rocketmq/proxy/processor/DefaultMessagingProcessor.java @@ -40,6 +40,7 @@ import org.apache.rocketmq.common.MixAll; import org.apache.rocketmq.common.consumer.ConsumeFromWhere; import org.apache.rocketmq.common.consumer.ReceiptHandle; +import org.apache.rocketmq.common.lite.LiteSubscriptionDTO; import org.apache.rocketmq.common.message.Message; import org.apache.rocketmq.common.message.MessageQueue; import org.apache.rocketmq.common.thread.ThreadPoolMonitor; @@ -141,6 +142,7 @@ protected void init() { this.appendStartAndShutdown(this.receiptHandleProcessor); this.appendShutdown(this.producerProcessorExecutor::shutdown); this.appendShutdown(this.consumerProcessorExecutor::shutdown); + this.appendStartAndShutdown(this.receiptHandleProcessor); } @Override @@ -163,7 +165,15 @@ public CompletableFuture> sendMessage(ProxyContext ctx, QueueSe @Override public CompletableFuture forwardMessageToDeadLetterQueue(ProxyContext ctx, ReceiptHandle handle, String messageId, String groupName, String topicName, long timeoutMillis) { - return this.producerProcessor.forwardMessageToDeadLetterQueue(ctx, handle, messageId, groupName, topicName, timeoutMillis); + return this.producerProcessor.forwardMessageToDeadLetterQueue(ctx, + handle, messageId, groupName, topicName, null, timeoutMillis); + } + + @Override + public CompletableFuture forwardMessageToDeadLetterQueue(ProxyContext ctx, ReceiptHandle handle, + String messageId, String groupName, String topicName, String liteTopic, long timeoutMillis) { + return this.producerProcessor.forwardMessageToDeadLetterQueue(ctx, + handle, messageId, groupName, topicName, liteTopic, timeoutMillis); } @Override @@ -194,10 +204,26 @@ public CompletableFuture popMessage( invisibleTime, pollTime, initMode, subscriptionData, fifo, popMessageResultFilter, attemptId, timeoutMillis); } + @Override + public CompletableFuture popLiteMessage(ProxyContext ctx, QueueSelector queueSelector, + String consumerGroup, String topic, int maxMsgNums, long invisibleTime, long pollTime, + SubscriptionData subscriptionData, PopMessageResultFilter popMessageResultFilter, + String attemptId, long timeoutMillis) { + return this.consumerProcessor.popLiteMessage(ctx, queueSelector, + consumerGroup, topic, maxMsgNums, invisibleTime, pollTime, + subscriptionData, popMessageResultFilter, attemptId, timeoutMillis); + } + @Override public CompletableFuture ackMessage(ProxyContext ctx, ReceiptHandle handle, String messageId, String consumerGroup, String topic, long timeoutMillis) { - return this.consumerProcessor.ackMessage(ctx, handle, messageId, consumerGroup, topic, timeoutMillis); + return this.consumerProcessor.ackMessage(ctx, handle, messageId, consumerGroup, topic, null, timeoutMillis); + } + + @Override + public CompletableFuture ackMessage(ProxyContext ctx, ReceiptHandle handle, String messageId, + String consumerGroup, String topic, String liteTopic, long timeoutMillis) { + return this.consumerProcessor.ackMessage(ctx, handle, messageId, consumerGroup, topic, liteTopic, timeoutMillis); } @Override @@ -209,7 +235,15 @@ public CompletableFuture> batchAckMessage(ProxyContext ctx, @Override public CompletableFuture changeInvisibleTime(ProxyContext ctx, ReceiptHandle handle, String messageId, String groupName, String topicName, long invisibleTime, long timeoutMillis) { - return this.consumerProcessor.changeInvisibleTime(ctx, handle, messageId, groupName, topicName, invisibleTime, timeoutMillis); + return this.consumerProcessor.changeInvisibleTime(ctx, handle, messageId, groupName, topicName, + invisibleTime, null, timeoutMillis); + } + + @Override + public CompletableFuture changeInvisibleTime(ProxyContext ctx, ReceiptHandle handle, String messageId, + String groupName, String topicName, long invisibleTime, String liteTopic, long timeoutMillis) { + return this.consumerProcessor.changeInvisibleTime(ctx, handle, messageId, groupName, topicName, + invisibleTime, liteTopic, timeoutMillis); } @Override @@ -267,6 +301,12 @@ public CompletableFuture recallMessage(ProxyContext ctx, String topic, return this.producerProcessor.recallMessage(ctx, topic, recallHandle, timeoutMillis); } + @Override + public CompletableFuture syncLiteSubscription(ProxyContext ctx, + LiteSubscriptionDTO liteSubscriptionDTO, long timeoutMillis) { + return this.clientProcessor.syncLiteSubscription(ctx, liteSubscriptionDTO, timeoutMillis); + } + @Override public CompletableFuture request(ProxyContext ctx, String brokerName, RemotingCommand request, long timeoutMillis) { @@ -367,4 +407,8 @@ public MessageReceiptHandle removeReceiptHandle(ProxyContext ctx, Channel channe String receiptHandle) { return receiptHandleProcessor.removeReceiptHandle(ctx, channel, group, msgID, receiptHandle); } + + @Override public int getUnackedMessageCount(ProxyContext ctx, Channel channel, String group) { + return receiptHandleProcessor.getUnackedMessageCount(ctx, channel, group); + } } diff --git a/proxy/src/main/java/org/apache/rocketmq/proxy/processor/MessagingProcessor.java b/proxy/src/main/java/org/apache/rocketmq/proxy/processor/MessagingProcessor.java index fee0465e2bf..e2c3da67451 100644 --- a/proxy/src/main/java/org/apache/rocketmq/proxy/processor/MessagingProcessor.java +++ b/proxy/src/main/java/org/apache/rocketmq/proxy/processor/MessagingProcessor.java @@ -31,6 +31,7 @@ import org.apache.rocketmq.client.producer.SendResult; import org.apache.rocketmq.common.consumer.ConsumeFromWhere; import org.apache.rocketmq.common.consumer.ReceiptHandle; +import org.apache.rocketmq.common.lite.LiteSubscriptionDTO; import org.apache.rocketmq.common.message.Message; import org.apache.rocketmq.common.message.MessageQueue; import org.apache.rocketmq.common.utils.StartAndShutdown; @@ -100,6 +101,27 @@ CompletableFuture forwardMessageToDeadLetterQueue( long timeoutMillis ); + default CompletableFuture forwardMessageToDeadLetterQueue( + ProxyContext ctx, + ReceiptHandle handle, + String messageId, + String groupName, + String topicName, + String liteTopic + ) { + return forwardMessageToDeadLetterQueue(ctx, handle, messageId, groupName, topicName, liteTopic, DEFAULT_TIMEOUT_MILLS); + } + + CompletableFuture forwardMessageToDeadLetterQueue( + ProxyContext ctx, + ReceiptHandle handle, + String messageId, + String groupName, + String topicName, + String liteTopic, + long timeoutMillis + ); + default CompletableFuture endTransaction( ProxyContext ctx, String topic, @@ -139,6 +161,20 @@ CompletableFuture popMessage( long timeoutMillis ); + CompletableFuture popLiteMessage( + ProxyContext ctx, + QueueSelector queueSelector, + String consumerGroup, + String topic, + int maxMsgNums, + long invisibleTime, + long pollTime, + SubscriptionData subscriptionData, + PopMessageResultFilter popMessageResultFilter, + String attemptId, + long timeoutMillis + ); + default CompletableFuture ackMessage( ProxyContext ctx, ReceiptHandle handle, @@ -158,6 +194,27 @@ CompletableFuture ackMessage( long timeoutMillis ); + default CompletableFuture ackMessage( + ProxyContext ctx, + ReceiptHandle handle, + String messageId, + String consumerGroup, + String topic, + String liteTopic + ) { + return ackMessage(ctx, handle, messageId, consumerGroup, topic, liteTopic, DEFAULT_TIMEOUT_MILLS); + } + + CompletableFuture ackMessage( + ProxyContext ctx, + ReceiptHandle handle, + String messageId, + String consumerGroup, + String topic, + String liteTopic, + long timeoutMillis + ); + default CompletableFuture> batchAckMessage( ProxyContext ctx, List handleMessageList, @@ -196,6 +253,29 @@ CompletableFuture changeInvisibleTime( long timeoutMillis ); + default CompletableFuture changeInvisibleTime( + ProxyContext ctx, + ReceiptHandle handle, + String messageId, + String groupName, + String topicName, + long invisibleTime, + String liteTopic + ) { + return changeInvisibleTime(ctx, handle, messageId, groupName, topicName, invisibleTime, liteTopic, DEFAULT_TIMEOUT_MILLS); + } + + CompletableFuture changeInvisibleTime( + ProxyContext ctx, + ReceiptHandle handle, + String messageId, + String groupName, + String topicName, + long invisibleTime, + String liteTopic, + long timeoutMillis + ); + CompletableFuture pullMessage( ProxyContext ctx, MessageQueue messageQueue, @@ -267,6 +347,12 @@ CompletableFuture recallMessage( long timeoutMillis ); + CompletableFuture syncLiteSubscription( + ProxyContext ctx, + LiteSubscriptionDTO liteSubscriptionDTO, + long timeoutMillis + ); + CompletableFuture request(ProxyContext ctx, String brokerName, RemotingCommand request, long timeoutMillis); @@ -341,4 +427,6 @@ void addReceiptHandle(ProxyContext ctx, Channel channel, String group, String ms MessageReceiptHandle removeReceiptHandle(ProxyContext ctx, Channel channel, String group, String msgID, String receiptHandle); + + int getUnackedMessageCount(ProxyContext ctx, Channel channel, String group); } diff --git a/proxy/src/main/java/org/apache/rocketmq/proxy/processor/ProducerProcessor.java b/proxy/src/main/java/org/apache/rocketmq/proxy/processor/ProducerProcessor.java index 5aeb553f216..8c4907c588a 100644 --- a/proxy/src/main/java/org/apache/rocketmq/proxy/processor/ProducerProcessor.java +++ b/proxy/src/main/java/org/apache/rocketmq/proxy/processor/ProducerProcessor.java @@ -225,8 +225,14 @@ protected SendMessageRequestHeader buildSendMessageRequestHeader(List m return requestHeader; } - public CompletableFuture forwardMessageToDeadLetterQueue(ProxyContext ctx, ReceiptHandle handle, - String messageId, String groupName, String topicName, long timeoutMillis) { + public CompletableFuture forwardMessageToDeadLetterQueue(ProxyContext ctx, + ReceiptHandle handle, + String messageId, + String groupName, + String topicName, + String liteTopic, + long timeoutMillis + ) { CompletableFuture future = new CompletableFuture<>(); try { if (handle.getCommitLogOffset() < 0) { @@ -250,7 +256,7 @@ public CompletableFuture forwardMessageToDeadLetterQueue(ProxyC ).whenCompleteAsync((remotingCommand, t) -> { if (t == null && remotingCommand.getCode() == ResponseCode.SUCCESS) { this.messagingProcessor.ackMessage(ctx, handle, messageId, - groupName, topicName, timeoutMillis); + groupName, topicName, liteTopic, timeoutMillis); } }, this.executor); } catch (Throwable t) { diff --git a/proxy/src/main/java/org/apache/rocketmq/proxy/processor/ReceiptHandleProcessor.java b/proxy/src/main/java/org/apache/rocketmq/proxy/processor/ReceiptHandleProcessor.java index 9b010fad598..30386901094 100644 --- a/proxy/src/main/java/org/apache/rocketmq/proxy/processor/ReceiptHandleProcessor.java +++ b/proxy/src/main/java/org/apache/rocketmq/proxy/processor/ReceiptHandleProcessor.java @@ -41,7 +41,7 @@ public ReceiptHandleProcessor(MessagingProcessor messagingProcessor, ServiceMana MessageReceiptHandle messageReceiptHandle = event.getMessageReceiptHandle(); ReceiptHandle handle = ReceiptHandle.decode(messageReceiptHandle.getReceiptHandleStr()); messagingProcessor.changeInvisibleTime(context, handle, messageReceiptHandle.getMessageId(), - messageReceiptHandle.getGroup(), messageReceiptHandle.getTopic(), event.getRenewTime()) + messageReceiptHandle.getGroup(), messageReceiptHandle.getTopic(), event.getRenewTime(), messageReceiptHandle.getLiteTopic()) .whenComplete((v, t) -> { if (t != null) { event.getFuture().completeExceptionally(t); @@ -66,4 +66,8 @@ public MessageReceiptHandle removeReceiptHandle(ProxyContext ctx, Channel channe return receiptHandleManager.removeReceiptHandle(ctx, channel, group, msgID, receiptHandle); } + public int getUnackedMessageCount(ProxyContext ctx, Channel channel, String group) { + return receiptHandleManager.getUnackedMessageCount(ctx, channel, group); + } + } diff --git a/proxy/src/main/java/org/apache/rocketmq/proxy/remoting/channel/RemotingChannel.java b/proxy/src/main/java/org/apache/rocketmq/proxy/remoting/channel/RemotingChannel.java index ea8a519d5b9..2bdb6eb9bb6 100644 --- a/proxy/src/main/java/org/apache/rocketmq/proxy/remoting/channel/RemotingChannel.java +++ b/proxy/src/main/java/org/apache/rocketmq/proxy/remoting/channel/RemotingChannel.java @@ -25,6 +25,10 @@ import io.netty.channel.ChannelFuture; import io.netty.channel.ChannelFutureListener; import io.netty.channel.ChannelMetadata; +import java.time.Duration; +import java.util.Set; +import java.util.concurrent.CompletableFuture; +import org.apache.commons.lang3.NotImplementedException; import org.apache.rocketmq.common.constant.LoggerName; import org.apache.rocketmq.common.message.MessageExt; import org.apache.rocketmq.common.utils.ExceptionUtils; @@ -53,12 +57,9 @@ import org.apache.rocketmq.remoting.protocol.header.CheckTransactionStateRequestHeader; import org.apache.rocketmq.remoting.protocol.header.ConsumeMessageDirectlyResultRequestHeader; import org.apache.rocketmq.remoting.protocol.header.GetConsumerRunningInfoRequestHeader; +import org.apache.rocketmq.remoting.protocol.header.NotifyUnsubscribeLiteRequestHeader; import org.apache.rocketmq.remoting.protocol.heartbeat.SubscriptionData; -import java.time.Duration; -import java.util.Set; -import java.util.concurrent.CompletableFuture; - public class RemotingChannel extends ProxyChannel implements RemoteChannelConverter, ChannelExtendAttributeGetter { private static final Logger log = LoggerFactory.getLogger(LoggerName.PROXY_LOGGER_NAME); private static final long DEFAULT_MQ_CLIENT_TIMEOUT = Duration.ofSeconds(3).toMillis(); @@ -178,6 +179,11 @@ protected CompletableFuture processGetConsumerRunningInfo(RemotingCommand } } + @Override + protected CompletableFuture processNotifyUnsubscribeLite(NotifyUnsubscribeLiteRequestHeader header) { + throw new NotImplementedException(); + } + @Override protected CompletableFuture processConsumeMessageDirectly(RemotingCommand command, ConsumeMessageDirectlyResultRequestHeader header, MessageExt messageExt, diff --git a/proxy/src/main/java/org/apache/rocketmq/proxy/service/ClusterServiceManager.java b/proxy/src/main/java/org/apache/rocketmq/proxy/service/ClusterServiceManager.java index 33b65d2550e..8b1c20c0bdb 100644 --- a/proxy/src/main/java/org/apache/rocketmq/proxy/service/ClusterServiceManager.java +++ b/proxy/src/main/java/org/apache/rocketmq/proxy/service/ClusterServiceManager.java @@ -41,6 +41,7 @@ import org.apache.rocketmq.proxy.service.admin.DefaultAdminService; import org.apache.rocketmq.proxy.service.client.ClusterConsumerManager; import org.apache.rocketmq.proxy.service.client.ProxyClientRemotingProcessor; +import org.apache.rocketmq.proxy.service.lite.LiteSubscriptionService; import org.apache.rocketmq.proxy.service.message.ClusterMessageService; import org.apache.rocketmq.proxy.service.message.MessageService; import org.apache.rocketmq.proxy.service.metadata.ClusterMetadataService; @@ -65,11 +66,13 @@ public class ClusterServiceManager extends AbstractStartAndShutdown implements S protected ProxyRelayService proxyRelayService; protected ClusterMetadataService metadataService; protected AdminService adminService; + protected LiteSubscriptionService liteSubscriptionService; protected ScheduledExecutorService scheduledExecutorService; protected MQClientAPIFactory messagingClientAPIFactory; protected MQClientAPIFactory operationClientAPIFactory; protected MQClientAPIFactory transactionClientAPIFactory; + protected MQClientAPIFactory liteSubscriptionAPIFactory; public ClusterServiceManager(RPCHook rpcHook) { this(rpcHook, null); @@ -113,7 +116,7 @@ public ClusterServiceManager(RPCHook rpcHook, ObjectCreator remo nameserverAccessConfig, "ClusterTransaction_", 1, - new ProxyClientRemotingProcessor(producerManager), + new ProxyClientRemotingProcessor(producerManager, consumerManager), rpcHook, scheduledExecutorService, remotingClientCreator @@ -123,6 +126,16 @@ public ClusterServiceManager(RPCHook rpcHook, ObjectCreator remo this.transactionClientAPIFactory); this.proxyRelayService = new ClusterProxyRelayService(this.clusterTransactionService); + // Lite subscriptions use a separate channel + this.liteSubscriptionAPIFactory = new MQClientAPIFactory( + nameserverAccessConfig, + "LiteSubscription_", + 1, + new ProxyClientRemotingProcessor(producerManager, consumerManager), + rpcHook, + scheduledExecutorService); + this.liteSubscriptionService = new LiteSubscriptionService(this.topicRouteService, this.liteSubscriptionAPIFactory); + this.init(); } @@ -142,6 +155,7 @@ protected void init() { this.appendStartAndShutdown(this.messagingClientAPIFactory); this.appendStartAndShutdown(this.operationClientAPIFactory); this.appendStartAndShutdown(this.transactionClientAPIFactory); + this.appendStartAndShutdown(this.liteSubscriptionAPIFactory); this.appendStartAndShutdown(this.topicRouteService); this.appendStartAndShutdown(this.clusterTransactionService); this.appendStartAndShutdown(this.metadataService); @@ -188,6 +202,11 @@ public AdminService getAdminService() { return this.adminService; } + @Override + public LiteSubscriptionService getLiteSubscriptionService() { + return liteSubscriptionService; + } + protected static class ConsumerIdsChangeListenerImpl implements ConsumerIdsChangeListener { @Override diff --git a/proxy/src/main/java/org/apache/rocketmq/proxy/service/LocalServiceManager.java b/proxy/src/main/java/org/apache/rocketmq/proxy/service/LocalServiceManager.java index 59cd92685a3..8f5073bb3aa 100644 --- a/proxy/src/main/java/org/apache/rocketmq/proxy/service/LocalServiceManager.java +++ b/proxy/src/main/java/org/apache/rocketmq/proxy/service/LocalServiceManager.java @@ -33,6 +33,7 @@ import org.apache.rocketmq.proxy.service.admin.AdminService; import org.apache.rocketmq.proxy.service.admin.DefaultAdminService; import org.apache.rocketmq.proxy.service.channel.ChannelManager; +import org.apache.rocketmq.proxy.service.lite.LiteSubscriptionService; import org.apache.rocketmq.proxy.service.message.LocalMessageService; import org.apache.rocketmq.proxy.service.message.MessageService; import org.apache.rocketmq.proxy.service.metadata.LocalMetadataService; @@ -130,6 +131,11 @@ public AdminService getAdminService() { return this.adminService; } + @Override + public LiteSubscriptionService getLiteSubscriptionService() { + return null; + } + private class LocalServiceManagerStartAndShutdown implements StartAndShutdown { @Override public void start() throws Exception { diff --git a/proxy/src/main/java/org/apache/rocketmq/proxy/service/ServiceManager.java b/proxy/src/main/java/org/apache/rocketmq/proxy/service/ServiceManager.java index c271eca0a11..8e982ed8945 100644 --- a/proxy/src/main/java/org/apache/rocketmq/proxy/service/ServiceManager.java +++ b/proxy/src/main/java/org/apache/rocketmq/proxy/service/ServiceManager.java @@ -20,6 +20,7 @@ import org.apache.rocketmq.broker.client.ProducerManager; import org.apache.rocketmq.common.utils.StartAndShutdown; import org.apache.rocketmq.proxy.service.admin.AdminService; +import org.apache.rocketmq.proxy.service.lite.LiteSubscriptionService; import org.apache.rocketmq.proxy.service.message.MessageService; import org.apache.rocketmq.proxy.service.metadata.MetadataService; import org.apache.rocketmq.proxy.service.relay.ProxyRelayService; @@ -42,4 +43,6 @@ public interface ServiceManager extends StartAndShutdown { MetadataService getMetadataService(); AdminService getAdminService(); + + LiteSubscriptionService getLiteSubscriptionService(); } diff --git a/proxy/src/main/java/org/apache/rocketmq/proxy/service/client/ProxyClientRemotingProcessor.java b/proxy/src/main/java/org/apache/rocketmq/proxy/service/client/ProxyClientRemotingProcessor.java index 655ce7e64dd..10a8f3df50d 100644 --- a/proxy/src/main/java/org/apache/rocketmq/proxy/service/client/ProxyClientRemotingProcessor.java +++ b/proxy/src/main/java/org/apache/rocketmq/proxy/service/client/ProxyClientRemotingProcessor.java @@ -19,6 +19,8 @@ import io.netty.channel.Channel; import io.netty.channel.ChannelHandlerContext; import java.nio.ByteBuffer; +import org.apache.commons.lang3.StringUtils; +import org.apache.rocketmq.broker.client.ClientChannelInfo; import org.apache.rocketmq.broker.client.ProducerManager; import org.apache.rocketmq.client.impl.ClientRemotingProcessor; import org.apache.rocketmq.common.constant.LoggerName; @@ -33,14 +35,17 @@ import org.apache.rocketmq.remoting.protocol.RemotingCommand; import org.apache.rocketmq.remoting.protocol.RequestCode; import org.apache.rocketmq.remoting.protocol.header.CheckTransactionStateRequestHeader; +import org.apache.rocketmq.remoting.protocol.header.NotifyUnsubscribeLiteRequestHeader; public class ProxyClientRemotingProcessor extends ClientRemotingProcessor { private static final Logger log = LoggerFactory.getLogger(LoggerName.PROXY_LOGGER_NAME); private final ProducerManager producerManager; + private final ClusterConsumerManager consumerManager; - public ProxyClientRemotingProcessor(ProducerManager producerManager) { + public ProxyClientRemotingProcessor(ProducerManager producerManager, ClusterConsumerManager consumerManager) { super(null); this.producerManager = producerManager; + this.consumerManager = consumerManager; } @Override @@ -48,6 +53,8 @@ public RemotingCommand processRequest(ChannelHandlerContext ctx, RemotingCommand throws RemotingCommandException { if (request.getCode() == RequestCode.CHECK_TRANSACTION_STATE) { return this.checkTransactionState(ctx, request); + } else if (request.getCode() == RequestCode.NOTIFY_UNSUBSCRIBE_LITE) { + return this.notifyUnsubscribeLite(ctx, request); } return null; } @@ -74,4 +81,32 @@ public RemotingCommand checkTransactionState(ChannelHandlerContext ctx, } return null; } + + /** + * one way, return null response + */ + public RemotingCommand notifyUnsubscribeLite(ChannelHandlerContext ctx, + RemotingCommand request) throws RemotingCommandException { + NotifyUnsubscribeLiteRequestHeader requestHeader = + request.decodeCommandCustomHeader(NotifyUnsubscribeLiteRequestHeader.class); + request.writeCustomHeader(requestHeader); + final String clientId = requestHeader.getClientId(); + final String group = requestHeader.getConsumerGroup(); + if (StringUtils.isBlank(clientId) || StringUtils.isBlank(group)) { + log.warn("notifyUnsubscribeLite clientId or group is null. {}", requestHeader); + return null; + } + ClientChannelInfo channelInfo = consumerManager.findChannel(group, clientId); + if (channelInfo == null) { + log.warn("notifyUnsubscribeLite channelInfo is null. {}", requestHeader); + return null; + } + Channel channel = channelInfo.getChannel(); + if (channel == null) { + log.warn("notifyUnsubscribeLite channel is null. {}", requestHeader); + return null; + } + channel.writeAndFlush(request); + return null; + } } diff --git a/proxy/src/main/java/org/apache/rocketmq/proxy/service/lite/LiteSubscriptionService.java b/proxy/src/main/java/org/apache/rocketmq/proxy/service/lite/LiteSubscriptionService.java new file mode 100644 index 00000000000..b1990dba548 --- /dev/null +++ b/proxy/src/main/java/org/apache/rocketmq/proxy/service/lite/LiteSubscriptionService.java @@ -0,0 +1,70 @@ +/* + * 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.proxy.service.lite; + +import java.util.List; +import java.util.concurrent.CompletableFuture; +import org.apache.rocketmq.client.impl.mqclient.MQClientAPIFactory; +import org.apache.rocketmq.common.constant.LoggerName; +import org.apache.rocketmq.common.lite.LiteSubscriptionDTO; +import org.apache.rocketmq.logging.org.slf4j.Logger; +import org.apache.rocketmq.logging.org.slf4j.LoggerFactory; +import org.apache.rocketmq.proxy.common.ProxyContext; +import org.apache.rocketmq.proxy.service.route.AddressableMessageQueue; +import org.apache.rocketmq.proxy.service.route.MessageQueueView; +import org.apache.rocketmq.proxy.service.route.TopicRouteService; + +public class LiteSubscriptionService { + private static final Logger log = LoggerFactory.getLogger(LoggerName.PROXY_LOGGER_NAME); + + protected final TopicRouteService topicRouteService; + protected final MQClientAPIFactory mqClientAPIFactory; + + public LiteSubscriptionService(TopicRouteService topicRouteService, MQClientAPIFactory mqClientAPIFactory) { + this.topicRouteService = topicRouteService; + this.mqClientAPIFactory = mqClientAPIFactory; + } + + public CompletableFuture syncLiteSubscription(ProxyContext ctx, + LiteSubscriptionDTO liteSubscriptionDTO, long timeoutMillis) { + final String topic = liteSubscriptionDTO.getTopic(); + List readQueues; + try { + MessageQueueView messageQueueView = topicRouteService.getAllMessageQueueView(ctx, topic); + // Send subscriptions to all readable brokers. + readQueues = messageQueueView.getReadSelector().getBrokerActingQueues(); + } catch (Exception e) { + CompletableFuture future = new CompletableFuture<>(); + future.completeExceptionally(e); + return future; + } + + return CompletableFuture.allOf( + readQueues + .stream() + .map(writeQ -> + mqClientAPIFactory.getClient().syncLiteSubscriptionAsync( + writeQ.getBrokerAddr(), + liteSubscriptionDTO, + timeoutMillis + )) + .toArray(CompletableFuture[]::new) + ); + } + +} \ No newline at end of file diff --git a/proxy/src/main/java/org/apache/rocketmq/proxy/service/message/ClusterMessageService.java b/proxy/src/main/java/org/apache/rocketmq/proxy/service/message/ClusterMessageService.java index f6f3406ab4e..77c4ef60f14 100644 --- a/proxy/src/main/java/org/apache/rocketmq/proxy/service/message/ClusterMessageService.java +++ b/proxy/src/main/java/org/apache/rocketmq/proxy/service/message/ClusterMessageService.java @@ -44,6 +44,7 @@ import org.apache.rocketmq.remoting.protocol.header.EndTransactionRequestHeader; import org.apache.rocketmq.remoting.protocol.header.GetMaxOffsetRequestHeader; import org.apache.rocketmq.remoting.protocol.header.GetMinOffsetRequestHeader; +import org.apache.rocketmq.remoting.protocol.header.PopLiteMessageRequestHeader; import org.apache.rocketmq.remoting.protocol.header.PopMessageRequestHeader; import org.apache.rocketmq.remoting.protocol.header.PullMessageRequestHeader; import org.apache.rocketmq.remoting.protocol.header.QueryConsumerOffsetRequestHeader; @@ -118,6 +119,21 @@ public CompletableFuture popMessage(ProxyContext ctx, AddressableMess ); } + @Override + public CompletableFuture popLiteMessage( + ProxyContext ctx, + AddressableMessageQueue messageQueue, + PopLiteMessageRequestHeader requestHeader, + long timeoutMillis + ) { + return this.mqClientAPIFactory.getClient().popLiteMessageAsync( + messageQueue.getBrokerAddr(), + messageQueue.getBrokerName(), + requestHeader, + timeoutMillis + ); + } + @Override public CompletableFuture changeInvisibleTime(ProxyContext ctx, ReceiptHandle handle, String messageId, ChangeInvisibleTimeRequestHeader requestHeader, long timeoutMillis) { diff --git a/proxy/src/main/java/org/apache/rocketmq/proxy/service/message/LocalMessageService.java b/proxy/src/main/java/org/apache/rocketmq/proxy/service/message/LocalMessageService.java index f0d9f8c7b84..189fde7fd77 100644 --- a/proxy/src/main/java/org/apache/rocketmq/proxy/service/message/LocalMessageService.java +++ b/proxy/src/main/java/org/apache/rocketmq/proxy/service/message/LocalMessageService.java @@ -67,6 +67,7 @@ import org.apache.rocketmq.remoting.protocol.header.ExtraInfoUtil; import org.apache.rocketmq.remoting.protocol.header.GetMaxOffsetRequestHeader; import org.apache.rocketmq.remoting.protocol.header.GetMinOffsetRequestHeader; +import org.apache.rocketmq.remoting.protocol.header.PopLiteMessageRequestHeader; import org.apache.rocketmq.remoting.protocol.header.PopMessageRequestHeader; import org.apache.rocketmq.remoting.protocol.header.PopMessageResponseHeader; import org.apache.rocketmq.remoting.protocol.header.PullMessageRequestHeader; @@ -196,6 +197,12 @@ public CompletableFuture endTransactionOneway(ProxyContext ctx, String bro return future; } + @Override + public CompletableFuture popLiteMessage(ProxyContext ctx, AddressableMessageQueue messageQueue, + PopLiteMessageRequestHeader requestHeader, long timeoutMillis) { + throw new NotImplementedException(); + } + @Override public CompletableFuture popMessage(ProxyContext ctx, AddressableMessageQueue messageQueue, PopMessageRequestHeader requestHeader, long timeoutMillis) { diff --git a/proxy/src/main/java/org/apache/rocketmq/proxy/service/message/MessageService.java b/proxy/src/main/java/org/apache/rocketmq/proxy/service/message/MessageService.java index 80f5ae7217c..1e828c36fd9 100644 --- a/proxy/src/main/java/org/apache/rocketmq/proxy/service/message/MessageService.java +++ b/proxy/src/main/java/org/apache/rocketmq/proxy/service/message/MessageService.java @@ -37,6 +37,7 @@ import org.apache.rocketmq.remoting.protocol.header.EndTransactionRequestHeader; import org.apache.rocketmq.remoting.protocol.header.GetMaxOffsetRequestHeader; import org.apache.rocketmq.remoting.protocol.header.GetMinOffsetRequestHeader; +import org.apache.rocketmq.remoting.protocol.header.PopLiteMessageRequestHeader; import org.apache.rocketmq.remoting.protocol.header.PopMessageRequestHeader; import org.apache.rocketmq.remoting.protocol.header.PullMessageRequestHeader; import org.apache.rocketmq.remoting.protocol.header.QueryConsumerOffsetRequestHeader; @@ -76,6 +77,13 @@ CompletableFuture popMessage( long timeoutMillis ); + CompletableFuture popLiteMessage( + ProxyContext ctx, + AddressableMessageQueue messageQueue, + PopLiteMessageRequestHeader requestHeader, + long timeoutMillis + ); + CompletableFuture changeInvisibleTime( ProxyContext ctx, ReceiptHandle handle, diff --git a/proxy/src/main/java/org/apache/rocketmq/proxy/service/receipt/DefaultReceiptHandleManager.java b/proxy/src/main/java/org/apache/rocketmq/proxy/service/receipt/DefaultReceiptHandleManager.java index 522ab2b6dae..f9dfd825337 100644 --- a/proxy/src/main/java/org/apache/rocketmq/proxy/service/receipt/DefaultReceiptHandleManager.java +++ b/proxy/src/main/java/org/apache/rocketmq/proxy/service/receipt/DefaultReceiptHandleManager.java @@ -145,6 +145,11 @@ public MessageReceiptHandle removeReceiptHandle(ProxyContext context, Channel ch return handleGroup.remove(msgID, receiptHandle); } + public int getUnackedMessageCount(ProxyContext context, Channel channel, String group) { + ReceiptHandleGroup handleGroup = receiptHandleGroupMap.get(new ReceiptHandleGroupKey(channel, group)); + return handleGroup == null ? 0 : handleGroup.getMsgCount(); + } + protected boolean clientIsOffline(ReceiptHandleGroupKey groupKey) { return this.consumerManager.findChannel(groupKey.getGroup(), groupKey.getChannel()) == null; } diff --git a/proxy/src/main/java/org/apache/rocketmq/proxy/service/receipt/ReceiptHandleManager.java b/proxy/src/main/java/org/apache/rocketmq/proxy/service/receipt/ReceiptHandleManager.java index 6a8888e97ef..16ad57b07d9 100644 --- a/proxy/src/main/java/org/apache/rocketmq/proxy/service/receipt/ReceiptHandleManager.java +++ b/proxy/src/main/java/org/apache/rocketmq/proxy/service/receipt/ReceiptHandleManager.java @@ -25,4 +25,6 @@ public interface ReceiptHandleManager { void addReceiptHandle(ProxyContext context, Channel channel, String group, String msgID, MessageReceiptHandle messageReceiptHandle); MessageReceiptHandle removeReceiptHandle(ProxyContext context, Channel channel, String group, String msgID, String receiptHandle); + + int getUnackedMessageCount(ProxyContext context, Channel channel, String group); } diff --git a/proxy/src/main/java/org/apache/rocketmq/proxy/service/relay/ProxyChannel.java b/proxy/src/main/java/org/apache/rocketmq/proxy/service/relay/ProxyChannel.java index 5a1185a81e8..72fdfd0259a 100644 --- a/proxy/src/main/java/org/apache/rocketmq/proxy/service/relay/ProxyChannel.java +++ b/proxy/src/main/java/org/apache/rocketmq/proxy/service/relay/ProxyChannel.java @@ -46,6 +46,7 @@ import org.apache.rocketmq.remoting.protocol.header.CheckTransactionStateRequestHeader; import org.apache.rocketmq.remoting.protocol.header.ConsumeMessageDirectlyResultRequestHeader; import org.apache.rocketmq.remoting.protocol.header.GetConsumerRunningInfoRequestHeader; +import org.apache.rocketmq.remoting.protocol.header.NotifyUnsubscribeLiteRequestHeader; public abstract class ProxyChannel extends SimpleChannel { private static final Logger log = LoggerFactory.getLogger(LoggerName.PROXY_LOGGER_NAME); @@ -104,6 +105,11 @@ public ChannelFuture writeAndFlush(Object msg) { this.proxyRelayService.processConsumeMessageDirectly(context, command, header)); break; } + case RequestCode.NOTIFY_UNSUBSCRIBE_LITE: { + NotifyUnsubscribeLiteRequestHeader header = (NotifyUnsubscribeLiteRequestHeader) command.readCustomHeader(); + processFuture = this.processNotifyUnsubscribeLite(header); + break; + } default: break; } @@ -132,6 +138,8 @@ protected abstract CompletableFuture processCheckTransaction( TransactionData transactionData, CompletableFuture> responseFuture); + protected abstract CompletableFuture processNotifyUnsubscribeLite(NotifyUnsubscribeLiteRequestHeader header); + protected abstract CompletableFuture processGetConsumerRunningInfo( RemotingCommand command, GetConsumerRunningInfoRequestHeader header, diff --git a/proxy/src/test/java/org/apache/rocketmq/proxy/grpc/v2/client/ClientActivityTest.java b/proxy/src/test/java/org/apache/rocketmq/proxy/grpc/v2/client/ClientActivityTest.java index 0c1ebcdfae7..532c9795c87 100644 --- a/proxy/src/test/java/org/apache/rocketmq/proxy/grpc/v2/client/ClientActivityTest.java +++ b/proxy/src/test/java/org/apache/rocketmq/proxy/grpc/v2/client/ClientActivityTest.java @@ -23,6 +23,7 @@ import apache.rocketmq.v2.FilterType; import apache.rocketmq.v2.HeartbeatRequest; import apache.rocketmq.v2.HeartbeatResponse; +import apache.rocketmq.v2.LiteSubscriptionAction; import apache.rocketmq.v2.NotifyClientTerminationRequest; import apache.rocketmq.v2.NotifyClientTerminationResponse; import apache.rocketmq.v2.Publishing; @@ -30,6 +31,8 @@ import apache.rocketmq.v2.Settings; import apache.rocketmq.v2.Subscription; import apache.rocketmq.v2.SubscriptionEntry; +import apache.rocketmq.v2.SyncLiteSubscriptionRequest; +import apache.rocketmq.v2.SyncLiteSubscriptionResponse; import apache.rocketmq.v2.TelemetryCommand; import apache.rocketmq.v2.ThreadStackTrace; import apache.rocketmq.v2.VerifyMessageResult; @@ -41,11 +44,13 @@ import java.util.concurrent.ExecutionException; import org.apache.rocketmq.broker.client.ClientChannelInfo; import org.apache.rocketmq.common.attribute.TopicMessageType; +import org.apache.rocketmq.common.lite.LiteSubscriptionDTO; import org.apache.rocketmq.proxy.common.ProxyContext; import org.apache.rocketmq.proxy.grpc.v2.BaseActivityTest; import org.apache.rocketmq.proxy.grpc.v2.ContextStreamObserver; import org.apache.rocketmq.proxy.grpc.v2.channel.GrpcChannelManager; import org.apache.rocketmq.proxy.grpc.v2.channel.GrpcClientChannel; +import org.apache.rocketmq.proxy.grpc.v2.common.GrpcValidator; import org.apache.rocketmq.proxy.grpc.v2.common.ResponseBuilder; import org.apache.rocketmq.proxy.service.relay.ProxyRelayResult; import org.apache.rocketmq.remoting.protocol.LanguageCode; @@ -61,6 +66,7 @@ import org.mockito.ArgumentCaptor; import org.mockito.Captor; import org.mockito.Mock; +import org.mockito.MockedStatic; import org.mockito.junit.MockitoJUnitRunner; import static org.assertj.core.api.Assertions.assertThat; @@ -69,8 +75,12 @@ import static org.junit.Assert.fail; import static org.mockito.ArgumentMatchers.any; import static org.mockito.ArgumentMatchers.anyBoolean; +import static org.mockito.ArgumentMatchers.anyLong; import static org.mockito.ArgumentMatchers.anyString; import static org.mockito.Mockito.doNothing; +import static org.mockito.Mockito.doThrow; +import static org.mockito.Mockito.mock; +import static org.mockito.Mockito.mockStatic; import static org.mockito.Mockito.times; import static org.mockito.Mockito.verify; import static org.mockito.Mockito.when; @@ -425,4 +435,98 @@ public void onCompleted() { .build()); return future; } + + @Test + public void testSyncLiteSubscription_Success() { + ProxyContext proxyContext = createContext(); + proxyContext.setClientID("client-id"); + Resource topic = Resource.newBuilder().setName("test-topic").build(); + Resource group = Resource.newBuilder().setName("test-group").build(); + SyncLiteSubscriptionRequest request = SyncLiteSubscriptionRequest.newBuilder() + .setTopic(topic) + .setGroup(group) + .setAction(LiteSubscriptionAction.PARTIAL_ADD) + .addAllLiteTopicSet(java.util.Collections.emptyList()) + .setVersion(1L) + .build(); + + when(messagingProcessor.syncLiteSubscription(any(), any(LiteSubscriptionDTO.class), anyLong())) + .thenReturn(CompletableFuture.completedFuture(null)); + + CompletableFuture future = clientActivity.syncLiteSubscription(proxyContext, request); + + SyncLiteSubscriptionResponse response = future.join(); + assertEquals(Code.OK, response.getStatus().getCode()); + } + + @Test + public void testSyncLiteSubscription_ValidationFailure() { + ProxyContext proxyContext = createContext(); + Resource topic = Resource.newBuilder().setName("test-topic").build(); + Resource group = Resource.newBuilder().setName("test-group").build(); + SyncLiteSubscriptionRequest request = SyncLiteSubscriptionRequest.newBuilder() + .setTopic(topic) + .setGroup(group) + .build(); + + // Mock the GrpcValidator singleton + GrpcValidator mockValidator = mock(GrpcValidator.class); + try (MockedStatic mocked = mockStatic(GrpcValidator.class)) { + mocked.when(GrpcValidator::getInstance).thenReturn(mockValidator); + + doThrow(new IllegalArgumentException("Invalid topic")) + .when(mockValidator).validateTopicAndConsumerGroup(topic, group); + + CompletableFuture future = clientActivity.syncLiteSubscription(proxyContext, request); + + assertTrue(future.isCompletedExceptionally()); + } + } + + @Test + public void testSyncLiteSubscription_ProcessingFailure() { + ProxyContext proxyContext = createContext(); + proxyContext.setClientID("client-id"); + Resource topic = Resource.newBuilder().setName("test-topic").build(); + Resource group = Resource.newBuilder().setName("test-group").build(); + SyncLiteSubscriptionRequest request = SyncLiteSubscriptionRequest.newBuilder() + .setTopic(topic) + .setGroup(group) + .setAction(LiteSubscriptionAction.PARTIAL_ADD) + .addAllLiteTopicSet(java.util.Collections.emptyList()) + .setVersion(1L) + .build(); + + CompletableFuture failedFuture = new CompletableFuture<>(); + failedFuture.completeExceptionally(new RuntimeException("Processing failed")); + when(messagingProcessor.syncLiteSubscription(any(), any(LiteSubscriptionDTO.class), anyLong())) + .thenReturn(failedFuture); + + CompletableFuture future = clientActivity.syncLiteSubscription(proxyContext, request); + + assertTrue(future.isCompletedExceptionally()); + } + + @Test + public void testSyncLiteSubscription_NullContext() { + Resource topic = Resource.newBuilder().setName("test-topic").build(); + Resource group = Resource.newBuilder().setName("test-group").build(); + SyncLiteSubscriptionRequest request = SyncLiteSubscriptionRequest.newBuilder() + .setTopic(topic) + .setGroup(group) + .build(); + + CompletableFuture future = clientActivity.syncLiteSubscription(null, request); + + assertTrue(future.isCompletedExceptionally()); + } + + @Test + public void testSyncLiteSubscription_NullRequest() { + ProxyContext proxyContext = createContext(); + + CompletableFuture future = clientActivity.syncLiteSubscription(proxyContext, null); + + assertTrue(future.isCompletedExceptionally()); + } } diff --git a/proxy/src/test/java/org/apache/rocketmq/proxy/grpc/v2/common/GrpcClientSettingsManagerTest.java b/proxy/src/test/java/org/apache/rocketmq/proxy/grpc/v2/common/GrpcClientSettingsManagerTest.java index 6742f094c82..4d0037a272a 100644 --- a/proxy/src/test/java/org/apache/rocketmq/proxy/grpc/v2/common/GrpcClientSettingsManagerTest.java +++ b/proxy/src/test/java/org/apache/rocketmq/proxy/grpc/v2/common/GrpcClientSettingsManagerTest.java @@ -17,6 +17,7 @@ package org.apache.rocketmq.proxy.grpc.v2.common; +import apache.rocketmq.v2.ClientType; import apache.rocketmq.v2.CustomizedBackoff; import apache.rocketmq.v2.ExponentialBackoff; import apache.rocketmq.v2.Publishing; @@ -24,7 +25,10 @@ import apache.rocketmq.v2.RetryPolicy; import apache.rocketmq.v2.Settings; import apache.rocketmq.v2.Subscription; +import apache.rocketmq.v2.SubscriptionEntry; import com.google.protobuf.util.Durations; +import java.util.concurrent.CompletableFuture; +import org.apache.rocketmq.common.lite.LiteSubscriptionDTO; import org.apache.rocketmq.proxy.common.ContextVariable; import org.apache.rocketmq.proxy.common.ProxyContext; import org.apache.rocketmq.proxy.grpc.v2.BaseActivityTest; @@ -39,15 +43,24 @@ import static org.junit.Assert.assertNotEquals; import static org.junit.Assert.assertNull; import static org.mockito.ArgumentMatchers.any; +import static org.mockito.ArgumentMatchers.anyLong; +import static org.mockito.ArgumentMatchers.anyString; +import static org.mockito.Mockito.doReturn; +import static org.mockito.Mockito.never; +import static org.mockito.Mockito.spy; +import static org.mockito.Mockito.times; +import static org.mockito.Mockito.verify; import static org.mockito.Mockito.when; public class GrpcClientSettingsManagerTest extends BaseActivityTest { - private GrpcClientSettingsManager grpcClientSettingsManager; + + private final ProxyContext ctx = ProxyContext.create(); + private final String clientId = "testClientId"; @Before public void before() throws Throwable { super.before(); - this.grpcClientSettingsManager = new GrpcClientSettingsManager(this.messagingProcessor); + grpcClientSettingsManager = spy(new GrpcClientSettingsManager(messagingProcessor)); } @Test @@ -110,4 +123,82 @@ public void testGetSubscriptionData() { assertNull(this.grpcClientSettingsManager.getClientSettings(context)); assertNull(this.grpcClientSettingsManager.removeAndGetClientSettings(context)); } + + @Test + public void testOfflineClientLiteSubscription_SettingsNullAndNoCachedSettings() { + doReturn(null).when(grpcClientSettingsManager).getRawClientSettings(anyString()); + + grpcClientSettingsManager.offlineClientLiteSubscription(ctx, clientId, null); + + verify(messagingProcessor, never()).syncLiteSubscription(any(), any(), anyLong()); + } + + @Test + public void testOfflineClientLiteSubscription_SettingsNull_CachedSettingsNotLite() { + Settings cachedSettings = Settings.newBuilder() + .setClientType(ClientType.PRODUCER) + .build(); + doReturn(cachedSettings).when(grpcClientSettingsManager).getRawClientSettings(anyString()); + + grpcClientSettingsManager.offlineClientLiteSubscription(ctx, clientId, null); + + verify(messagingProcessor, never()).syncLiteSubscription(any(), any(), anyLong()); + } + + @Test + public void testOfflineClientLiteSubscription_SettingsNotNull_NotLiteConsumer() { + Settings settings = Settings.newBuilder() + .setClientType(ClientType.PUSH_CONSUMER) + .build(); + + grpcClientSettingsManager.offlineClientLiteSubscription(ctx, clientId, settings); + + verify(messagingProcessor, never()).syncLiteSubscription(any(), any(), anyLong()); + } + + @Test + public void testOfflineClientLiteSubscription_ValidLiteConsumer_Success() { + Subscription subscription = Subscription.newBuilder() + .setGroup(Resource.newBuilder().setName("testGroup").build()) + .addSubscriptions(SubscriptionEntry.newBuilder() + .setTopic(Resource.newBuilder().setName("testTopic").build()) + .build()) + .build(); + + Settings settings = Settings.newBuilder() + .setClientType(ClientType.LITE_PUSH_CONSUMER) + .setSubscription(subscription) + .build(); + + when(messagingProcessor.syncLiteSubscription(any(), any(LiteSubscriptionDTO.class), anyLong())) + .thenReturn(CompletableFuture.completedFuture(null)); + + grpcClientSettingsManager.offlineClientLiteSubscription(ctx, clientId, settings); + + verify(messagingProcessor, times(1)).syncLiteSubscription(any(), any(LiteSubscriptionDTO.class), anyLong()); + } + + @Test + public void testOfflineClientLiteSubscription_ValidLiteConsumer_SyncThrowsException() { + Subscription subscription = Subscription.newBuilder() + .setGroup(Resource.newBuilder().setName("testGroup").build()) + .addSubscriptions(SubscriptionEntry.newBuilder() + .setTopic(Resource.newBuilder().setName("testTopic").build()) + .build()) + .build(); + + Settings settings = Settings.newBuilder() + .setClientType(ClientType.LITE_PUSH_CONSUMER) + .setSubscription(subscription) + .build(); + + CompletableFuture future = new CompletableFuture<>(); + future.completeExceptionally(new RuntimeException("Simulated error")); + when(messagingProcessor.syncLiteSubscription(any(), any(LiteSubscriptionDTO.class), anyLong())) + .thenReturn(future); + + grpcClientSettingsManager.offlineClientLiteSubscription(ctx, clientId, settings); + + verify(messagingProcessor, times(1)).syncLiteSubscription(any(), any(LiteSubscriptionDTO.class), anyLong()); + } } diff --git a/proxy/src/test/java/org/apache/rocketmq/proxy/grpc/v2/common/GrpcConverterTest.java b/proxy/src/test/java/org/apache/rocketmq/proxy/grpc/v2/common/GrpcConverterTest.java index bc9b8a60b40..48d1596164e 100644 --- a/proxy/src/test/java/org/apache/rocketmq/proxy/grpc/v2/common/GrpcConverterTest.java +++ b/proxy/src/test/java/org/apache/rocketmq/proxy/grpc/v2/common/GrpcConverterTest.java @@ -18,10 +18,17 @@ package org.apache.rocketmq.proxy.grpc.v2.common; import apache.rocketmq.v2.MessageQueue; +import apache.rocketmq.v2.MessageType; +import java.net.InetSocketAddress; +import java.nio.charset.StandardCharsets; +import org.apache.rocketmq.common.message.MessageAccessor; import org.apache.rocketmq.common.message.MessageExt; import org.junit.Test; import static org.assertj.core.api.Assertions.assertThat; +import static org.junit.Assert.assertEquals; +import static org.junit.Assert.assertNotNull; +import static org.junit.Assert.assertTrue; public class GrpcConverterTest { @Test @@ -38,4 +45,42 @@ public void testBuildMessageQueue() { assertThat(messageQueue.getBroker().getName()).isEqualTo(brokerName); assertThat(messageQueue.getId()).isEqualTo(queueId); } + + @Test + public void testBuildMessageWithLiteTopic() { + final String topic = "test-topic"; + final String liteTopic = "test-lite-topic"; + // Build a message with lite topic properties + MessageExt messageExt = new MessageExt(); + messageExt.setTopic(topic); + messageExt.setBody("test-body".getBytes(StandardCharsets.UTF_8)); + messageExt.setQueueId(1); + messageExt.setQueueOffset(100L); + messageExt.setBornTimestamp(System.currentTimeMillis()); + messageExt.setStoreTimestamp(System.currentTimeMillis()); + messageExt.setBornHost(new InetSocketAddress("127.0.0.1", 1234)); + messageExt.setStoreHost(new InetSocketAddress("127.0.0.1", 5678)); + messageExt.setReconsumeTimes(0); + messageExt.setMsgId("test-msg-id"); + + // Set lite topic property + MessageAccessor.setLiteTopic(messageExt, liteTopic); + + // Convert message + GrpcConverter grpcConverter = GrpcConverter.getInstance(); + apache.rocketmq.v2.Message grpcMessage = grpcConverter.buildMessage(messageExt); + + // Verify basic properties + assertNotNull(grpcMessage); + assertEquals(topic, grpcMessage.getTopic().getName()); + assertEquals("test-body", grpcMessage.getBody().toString(StandardCharsets.UTF_8)); + + // Verify lite topic in system properties + assertNotNull(grpcMessage.getSystemProperties()); + assertTrue(grpcMessage.getSystemProperties().hasLiteTopic()); + assertEquals(liteTopic, grpcMessage.getSystemProperties().getLiteTopic()); + + // Verify message type is LITE + assertEquals(MessageType.LITE, grpcMessage.getSystemProperties().getMessageType()); + } } \ No newline at end of file diff --git a/proxy/src/test/java/org/apache/rocketmq/proxy/grpc/v2/common/GrpcValidatorTest.java b/proxy/src/test/java/org/apache/rocketmq/proxy/grpc/v2/common/GrpcValidatorTest.java index df42844e95e..225c6c87a6b 100644 --- a/proxy/src/test/java/org/apache/rocketmq/proxy/grpc/v2/common/GrpcValidatorTest.java +++ b/proxy/src/test/java/org/apache/rocketmq/proxy/grpc/v2/common/GrpcValidatorTest.java @@ -17,10 +17,17 @@ package org.apache.rocketmq.proxy.grpc.v2.common; +import org.apache.commons.lang3.RandomStringUtils; +import org.apache.rocketmq.proxy.config.ConfigurationManager; +import org.apache.rocketmq.proxy.config.ProxyConfig; import org.junit.Before; import org.junit.Test; +import org.mockito.MockedStatic; import static org.junit.Assert.assertThrows; +import static org.mockito.Mockito.mock; +import static org.mockito.Mockito.mockStatic; +import static org.mockito.Mockito.when; public class GrpcValidatorTest { @@ -44,4 +51,62 @@ public void testValidateConsumerGroup() { assertThrows(GrpcProxyException.class, () -> grpcValidator.validateConsumerGroup("CID_RMQ_SYS_xxxx")); grpcValidator.validateConsumerGroup("consumerGroupName"); } + + + @Test + public void testValidateLiteTopic_Null() { + assertThrows(GrpcProxyException.class, () -> grpcValidator.validateLiteTopic(null)); + } + + @Test + public void testValidateLiteTopic_Blank() { + assertThrows(GrpcProxyException.class, () -> grpcValidator.validateLiteTopic(" ")); + } + + @Test + public void testValidateLiteTopic_TooLong() { + try (MockedStatic mockedConfig = mockStatic(ConfigurationManager.class)) { + ProxyConfig proxyConfig = mock(ProxyConfig.class); + when(proxyConfig.getMaxLiteTopicSize()).thenReturn(5); + mockedConfig.when(ConfigurationManager::getProxyConfig).thenReturn(proxyConfig); + + assertThrows(GrpcProxyException.class, () -> grpcValidator.validateLiteTopic("toolongtopic")); + } + } + + @Test + public void testValidateLiteTopic_IllegalCharacter() { + try (MockedStatic mockedConfig = mockStatic(ConfigurationManager.class)) { + ProxyConfig proxyConfig = mock(ProxyConfig.class); + when(proxyConfig.getMaxLiteTopicSize()).thenReturn(100); + mockedConfig.when(ConfigurationManager::getProxyConfig).thenReturn(proxyConfig); + + assertThrows(GrpcProxyException.class, () -> grpcValidator.validateLiteTopic("invalid@topic")); + + assertThrows(GrpcProxyException.class, () -> grpcValidator.validateLiteTopic("invalid$topic")); + + assertThrows(GrpcProxyException.class, () -> grpcValidator.validateLiteTopic("invalid%topic")); + + assertThrows(GrpcProxyException.class, () -> grpcValidator.validateLiteTopic("invalid\ttopic")); + + assertThrows(GrpcProxyException.class, () -> grpcValidator.validateLiteTopic("invalid\ntopic")); + + assertThrows(GrpcProxyException.class, () -> grpcValidator.validateLiteTopic("invalid\0topic")); + } + } + + @Test + public void testValidateLiteTopic_Valid() { + try (MockedStatic mockedConfig = mockStatic(ConfigurationManager.class)) { + ProxyConfig proxyConfig = mock(ProxyConfig.class); + when(proxyConfig.getMaxLiteTopicSize()).thenReturn(64); + mockedConfig.when(ConfigurationManager::getProxyConfig).thenReturn(proxyConfig); + + grpcValidator.validateLiteTopic("Valid_Topic-123"); + + grpcValidator.validateLiteTopic(RandomStringUtils.randomAlphanumeric(64)); + + grpcValidator.validateLiteTopic(RandomStringUtils.randomAlphanumeric(63)); + } + } } diff --git a/proxy/src/test/java/org/apache/rocketmq/proxy/grpc/v2/consumer/AckMessageActivityTest.java b/proxy/src/test/java/org/apache/rocketmq/proxy/grpc/v2/consumer/AckMessageActivityTest.java index 3c474610518..5dd4c6b3610 100644 --- a/proxy/src/test/java/org/apache/rocketmq/proxy/grpc/v2/consumer/AckMessageActivityTest.java +++ b/proxy/src/test/java/org/apache/rocketmq/proxy/grpc/v2/consumer/AckMessageActivityTest.java @@ -69,17 +69,17 @@ public void testAckMessage() throws Throwable { String msg2 = "msg2"; String msg3 = "msg3"; - when(this.messagingProcessor.ackMessage(any(), any(), eq(msg1), anyString(), anyString())) + when(this.messagingProcessor.ackMessage(any(), any(), eq(msg1), anyString(), anyString(), any())) .thenThrow(new ProxyException(ProxyExceptionCode.INVALID_RECEIPT_HANDLE, "receipt handle is expired")); AckResult msg2AckResult = new AckResult(); msg2AckResult.setStatus(AckStatus.OK); - when(this.messagingProcessor.ackMessage(any(), any(), eq(msg2), anyString(), anyString())) + when(this.messagingProcessor.ackMessage(any(), any(), eq(msg2), anyString(), anyString(), any())) .thenReturn(CompletableFuture.completedFuture(msg2AckResult)); AckResult msg3AckResult = new AckResult(); msg3AckResult.setStatus(AckStatus.NO_EXIST); - when(this.messagingProcessor.ackMessage(any(), any(), eq(msg3), anyString(), anyString())) + when(this.messagingProcessor.ackMessage(any(), any(), eq(msg3), anyString(), anyString(), any())) .thenReturn(CompletableFuture.completedFuture(msg3AckResult)); { diff --git a/proxy/src/test/java/org/apache/rocketmq/proxy/grpc/v2/producer/ForwardMessageToDLQActivityTest.java b/proxy/src/test/java/org/apache/rocketmq/proxy/grpc/v2/producer/ForwardMessageToDLQActivityTest.java index 87824e5b4bc..61fe605899f 100644 --- a/proxy/src/test/java/org/apache/rocketmq/proxy/grpc/v2/producer/ForwardMessageToDLQActivityTest.java +++ b/proxy/src/test/java/org/apache/rocketmq/proxy/grpc/v2/producer/ForwardMessageToDLQActivityTest.java @@ -50,7 +50,7 @@ public void before() throws Throwable { @Test public void testForwardMessageToDeadLetterQueue() throws Throwable { ArgumentCaptor receiptHandleCaptor = ArgumentCaptor.forClass(ReceiptHandle.class); - when(this.messagingProcessor.forwardMessageToDeadLetterQueue(any(), receiptHandleCaptor.capture(), anyString(), anyString(), anyString())) + when(this.messagingProcessor.forwardMessageToDeadLetterQueue(any(), receiptHandleCaptor.capture(), anyString(), anyString(), anyString(), any())) .thenReturn(CompletableFuture.completedFuture(RemotingCommand.createResponseCommand(ResponseCode.SUCCESS, ""))); String handleStr = buildReceiptHandle("topic", System.currentTimeMillis(), 3000); @@ -71,7 +71,7 @@ public void testForwardMessageToDeadLetterQueue() throws Throwable { @Test public void testForwardMessageToDeadLetterQueueWhenHasMappingHandle() throws Throwable { ArgumentCaptor receiptHandleCaptor = ArgumentCaptor.forClass(ReceiptHandle.class); - when(this.messagingProcessor.forwardMessageToDeadLetterQueue(any(), receiptHandleCaptor.capture(), anyString(), anyString(), anyString())) + when(this.messagingProcessor.forwardMessageToDeadLetterQueue(any(), receiptHandleCaptor.capture(), anyString(), anyString(), anyString(), any())) .thenReturn(CompletableFuture.completedFuture(RemotingCommand.createResponseCommand(ResponseCode.SUCCESS, ""))); String savedHandleStr = buildReceiptHandle("topic", System.currentTimeMillis(),3000); diff --git a/proxy/src/test/java/org/apache/rocketmq/proxy/grpc/v2/producer/SendMessageActivityTest.java b/proxy/src/test/java/org/apache/rocketmq/proxy/grpc/v2/producer/SendMessageActivityTest.java index a64867ddfe1..fbb0d218ca1 100644 --- a/proxy/src/test/java/org/apache/rocketmq/proxy/grpc/v2/producer/SendMessageActivityTest.java +++ b/proxy/src/test/java/org/apache/rocketmq/proxy/grpc/v2/producer/SendMessageActivityTest.java @@ -61,6 +61,7 @@ import static org.junit.Assert.assertEquals; import static org.junit.Assert.assertNotEquals; +import static org.junit.Assert.assertNull; import static org.junit.Assert.assertThrows; import static org.mockito.ArgumentMatchers.any; import static org.mockito.ArgumentMatchers.anyInt; @@ -239,6 +240,35 @@ public void testBuildMessage() { assertEquals(deliveryTime, Long.parseLong(messageExt.getProperty(MessageConst.PROPERTY_TIMER_DELIVER_MS))); } + @Test + public void testBuildMessageWithLiteTopic() { + String msgId = MessageClientIDSetter.createUniqID(); + String liteTopic = "build-test-lite-topic"; + String topic = "build-test-topic"; + + org.apache.rocketmq.common.message.Message messageExt = this.sendMessageActivity.buildMessage( + ProxyContext.create(), + Message.newBuilder() + .setTopic(Resource.newBuilder() + .setName(topic) + .build()) + .setSystemProperties(SystemProperties.newBuilder() + .setMessageId(msgId) + .setQueueId(0) + .setMessageType(MessageType.LITE) + .setLiteTopic(liteTopic) + .setBornTimestamp(Timestamps.fromMillis(System.currentTimeMillis())) + .setBornHost(StringUtils.defaultString(NetworkUtil.getLocalAddress(), "127.0.0.1:1234")) + .build()) + .setBody(ByteString.copyFromUtf8("test body")) + .build(), + "test-producer-group" + ); + + assertEquals(liteTopic, messageExt.getProperty(MessageConst.PROPERTY_LITE_TOPIC)); + assertNull(messageExt.getProperty(MessageConst.PROPERTY_INNER_MULTI_DISPATCH)); + } + @Test public void testTxMessage() { String msgId = MessageClientIDSetter.createUniqID(); diff --git a/proxy/src/test/java/org/apache/rocketmq/proxy/processor/ClientProcessorTest.java b/proxy/src/test/java/org/apache/rocketmq/proxy/processor/ClientProcessorTest.java new file mode 100644 index 00000000000..6644341e551 --- /dev/null +++ b/proxy/src/test/java/org/apache/rocketmq/proxy/processor/ClientProcessorTest.java @@ -0,0 +1,218 @@ +/* + * 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.proxy.processor; + +import java.util.HashSet; +import java.util.Set; +import org.apache.rocketmq.proxy.common.ProxyContext; +import org.apache.rocketmq.proxy.config.ConfigurationManager; +import org.apache.rocketmq.proxy.grpc.v2.common.GrpcProxyException; +import org.apache.rocketmq.proxy.service.ServiceManager; +import org.apache.rocketmq.remoting.protocol.heartbeat.MessageModel; +import org.apache.rocketmq.remoting.protocol.heartbeat.SubscriptionData; +import org.apache.rocketmq.remoting.protocol.subscription.SubscriptionGroupConfig; +import org.junit.Before; +import org.junit.Test; +import org.junit.runner.RunWith; +import org.mockito.Mock; +import org.mockito.junit.MockitoJUnitRunner; + +import static org.junit.jupiter.api.Assertions.assertDoesNotThrow; +import static org.junit.jupiter.api.Assertions.assertEquals; +import static org.junit.jupiter.api.Assertions.assertThrows; +import static org.junit.jupiter.api.Assertions.assertTrue; +import static org.mockito.Mockito.when; + +@RunWith(MockitoJUnitRunner.class) +public class ClientProcessorTest { + + @Mock + private MessagingProcessor messagingProcessor; + + @Mock + private ServiceManager serviceManager; + + @Mock + private ProxyContext ctx; + + @Mock + private SubscriptionGroupConfig groupConfig; + + private ClientProcessor clientProcessor; + + @Before + public void setUp() throws Exception { + ConfigurationManager.initConfig(); + clientProcessor = new ClientProcessor(messagingProcessor, serviceManager); + } + + @Test + public void testValidateLiteMode_regularGroupWithLiteMode_throwsException() { + String group = "regularGroup"; + when(groupConfig.getLiteBindTopic()).thenReturn(""); + when(messagingProcessor.getSubscriptionGroupConfig(ctx, group)).thenReturn(groupConfig); + + GrpcProxyException exception = assertThrows(GrpcProxyException.class, () -> { + clientProcessor.validateLiteMode(ctx, group, MessageModel.LITE_SELECTIVE); + }); + + assertEquals("regular group cannot use LITE mode: " + group, exception.getMessage()); + } + + @Test + public void testValidateLiteMode_liteGroupWithoutLiteMode_throwsException() { + String group = "liteGroup"; + when(groupConfig.getLiteBindTopic()).thenReturn("topic1"); + when(messagingProcessor.getSubscriptionGroupConfig(ctx, group)).thenReturn(groupConfig); + + GrpcProxyException exception = assertThrows(GrpcProxyException.class, () -> { + clientProcessor.validateLiteMode(ctx, group, MessageModel.CLUSTERING); + }); + + assertEquals("lite group must use LITE mode: " + group, exception.getMessage()); + } + + @Test + public void testValidateLiteMode_regularGroupWithoutLiteMode_noException() { + String group = "regularGroup"; + when(groupConfig.getLiteBindTopic()).thenReturn(""); + when(messagingProcessor.getSubscriptionGroupConfig(ctx, group)).thenReturn(groupConfig); + + assertDoesNotThrow(() -> { + clientProcessor.validateLiteMode(ctx, group, MessageModel.CLUSTERING); + }); + } + + @Test + public void testValidateLiteMode_liteGroupWithLiteMode_noException() { + String group = "liteGroup"; + when(groupConfig.getLiteBindTopic()).thenReturn("topic1"); + when(messagingProcessor.getSubscriptionGroupConfig(ctx, group)).thenReturn(groupConfig); + + assertDoesNotThrow(() -> { + clientProcessor.validateLiteMode(ctx, group, MessageModel.LITE_SELECTIVE); + }); + } + + @Test + public void testValidateLiteSubTopic_emptySubList_noException() { + String group = "group"; + Set subList = new HashSet<>(); + + assertDoesNotThrow(() -> { + clientProcessor.validateLiteSubTopic(ctx, group, subList); + }); + } + + @Test + public void testValidateLiteSubTopic_validSubList_noException() { + String group = "group"; + String topic = "topic1"; + SubscriptionData subscriptionData = new SubscriptionData(); + subscriptionData.setTopic(topic); + Set subList = new HashSet<>(); + subList.add(subscriptionData); + + when(groupConfig.getLiteBindTopic()).thenReturn(topic); + when(messagingProcessor.getSubscriptionGroupConfig(ctx, group)).thenReturn(groupConfig); + + assertDoesNotThrow(() -> { + clientProcessor.validateLiteSubTopic(ctx, group, subList); + }); + } + + @Test + public void testValidateLiteBindTopic_matchingTopics_noException() { + String group = "group"; + String bindTopic = "topic1"; + + when(groupConfig.getLiteBindTopic()).thenReturn(bindTopic); + when(messagingProcessor.getSubscriptionGroupConfig(ctx, group)).thenReturn(groupConfig); + + assertDoesNotThrow(() -> { + clientProcessor.validateLiteBindTopic(ctx, group, bindTopic); + }); + } + + @Test + public void testValidateLiteBindTopic_mismatchedTopics_throwsException() { + String group = "group"; + String expectedTopic = "expectedTopic"; + String actualTopic = "actualTopic"; + + when(groupConfig.getLiteBindTopic()).thenReturn(expectedTopic); + when(messagingProcessor.getSubscriptionGroupConfig(ctx, group)).thenReturn(groupConfig); + + GrpcProxyException exception = assertThrows(GrpcProxyException.class, () -> { + clientProcessor.validateLiteBindTopic(ctx, group, actualTopic); + }); + + assertTrue(exception.getMessage().contains("expected to bind topic")); + } + + @Test + public void testValidateLiteSubscriptionQuota_withinQuota_noException() { + String group = "group"; + int quota = 10; + int actual = 5; + + when(groupConfig.getLiteSubClientQuota()).thenReturn(quota); + when(messagingProcessor.getSubscriptionGroupConfig(ctx, group)).thenReturn(groupConfig); + + assertDoesNotThrow(() -> { + clientProcessor.validateLiteSubscriptionQuota(ctx, group, actual); + }); + } + + @Test + public void testValidateLiteSubscriptionQuota_exceedsQuota_throwsException() { + String group = "group"; + int quota = 10; + int actual = 15 + 300 /*quota buffer*/; + + when(groupConfig.getLiteSubClientQuota()).thenReturn(quota); + when(messagingProcessor.getSubscriptionGroupConfig(ctx, group)).thenReturn(groupConfig); + + GrpcProxyException exception = assertThrows(GrpcProxyException.class, () -> { + clientProcessor.validateLiteSubscriptionQuota(ctx, group, actual); + }); + + assertTrue(exception.getMessage().contains("lite subscription quota exceeded")); + } + + @Test + public void testGetGroupOrException_groupExists_returnsConfig() { + String group = "group"; + when(messagingProcessor.getSubscriptionGroupConfig(ctx, group)).thenReturn(groupConfig); + + SubscriptionGroupConfig result = clientProcessor.getGroupOrException(ctx, group); + assertEquals(groupConfig, result); + } + + @Test + public void testGetGroupOrException_groupNotExists_throwsException() { + String group = "nonExistentGroup"; + when(messagingProcessor.getSubscriptionGroupConfig(ctx, group)).thenReturn(null); + + GrpcProxyException exception = assertThrows(GrpcProxyException.class, () -> { + clientProcessor.getGroupOrException(ctx, group); + }); + + assertEquals("group not found: " + group, exception.getMessage()); + } +} diff --git a/proxy/src/test/java/org/apache/rocketmq/proxy/processor/ConsumerProcessorTest.java b/proxy/src/test/java/org/apache/rocketmq/proxy/processor/ConsumerProcessorTest.java index 9720938cf9e..9b203ef1f68 100644 --- a/proxy/src/test/java/org/apache/rocketmq/proxy/processor/ConsumerProcessorTest.java +++ b/proxy/src/test/java/org/apache/rocketmq/proxy/processor/ConsumerProcessorTest.java @@ -108,11 +108,11 @@ public void testPopMessage() throws Throwable { .thenReturn(mock(MessageQueueView.class)); ArgumentCaptor ackMessageIdArgumentCaptor = ArgumentCaptor.forClass(String.class); - when(this.messagingProcessor.ackMessage(any(), any(), ackMessageIdArgumentCaptor.capture(), anyString(), anyString(), anyLong())) + when(this.messagingProcessor.ackMessage(any(), any(), ackMessageIdArgumentCaptor.capture(), anyString(), anyString(), any(), anyLong())) .thenReturn(CompletableFuture.completedFuture(mock(AckResult.class))); ArgumentCaptor toDLQMessageIdArgumentCaptor = ArgumentCaptor.forClass(String.class); - when(this.messagingProcessor.forwardMessageToDeadLetterQueue(any(), any(), toDLQMessageIdArgumentCaptor.capture(), anyString(), anyString(), anyLong())) + when(this.messagingProcessor.forwardMessageToDeadLetterQueue(any(), any(), toDLQMessageIdArgumentCaptor.capture(), anyString(), anyString(), any(), anyLong())) .thenReturn(CompletableFuture.completedFuture(mock(RemotingCommand.class))); AddressableMessageQueue messageQueue = mock(AddressableMessageQueue.class); @@ -167,7 +167,7 @@ public void testAckMessage() throws Throwable { .thenReturn(CompletableFuture.completedFuture(innerAckResult)); AckResult ackResult = this.consumerProcessor.ackMessage(createContext(), handle, MessageClientIDSetter.createUniqID(), - CONSUMER_GROUP, TOPIC, 3000).get(); + CONSUMER_GROUP, TOPIC, null, 3000).get(); assertEquals(AckStatus.OK, ackResult.getStatus()); assertEquals(KeyBuilder.buildPopRetryTopic(TOPIC, CONSUMER_GROUP, new BrokerConfig().isEnableRetryTopicV2()), requestHeaderArgumentCaptor.getValue().getTopic()); @@ -290,7 +290,7 @@ public void testChangeInvisibleTime() throws Throwable { .thenReturn(CompletableFuture.completedFuture(innerAckResult)); AckResult ackResult = this.consumerProcessor.changeInvisibleTime(createContext(), handle, MessageClientIDSetter.createUniqID(), - CONSUMER_GROUP, TOPIC, 1000, 3000).get(); + CONSUMER_GROUP, TOPIC, 1000, null, 3000).get(); assertEquals(AckStatus.OK, ackResult.getStatus()); assertEquals(KeyBuilder.buildPopRetryTopic(TOPIC, CONSUMER_GROUP, new BrokerConfig().isEnableRetryTopicV2()), requestHeaderArgumentCaptor.getValue().getTopic()); diff --git a/proxy/src/test/java/org/apache/rocketmq/proxy/processor/ProducerProcessorTest.java b/proxy/src/test/java/org/apache/rocketmq/proxy/processor/ProducerProcessorTest.java index 6729ef0c4b3..e6a90df36be 100644 --- a/proxy/src/test/java/org/apache/rocketmq/proxy/processor/ProducerProcessorTest.java +++ b/proxy/src/test/java/org/apache/rocketmq/proxy/processor/ProducerProcessorTest.java @@ -201,6 +201,7 @@ public void testForwardMessageToDeadLetterQueue() throws Throwable { messageExt.getMsgId(), CONSUMER_GROUP, TOPIC, + null, 3000 ).get(); diff --git a/proxy/src/test/java/org/apache/rocketmq/proxy/processor/ReceiptHandleProcessorTest.java b/proxy/src/test/java/org/apache/rocketmq/proxy/processor/ReceiptHandleProcessorTest.java index a25ebc1fe31..62e5e64eb42 100644 --- a/proxy/src/test/java/org/apache/rocketmq/proxy/processor/ReceiptHandleProcessorTest.java +++ b/proxy/src/test/java/org/apache/rocketmq/proxy/processor/ReceiptHandleProcessorTest.java @@ -95,7 +95,7 @@ public void testStart() throws Exception { Mockito.when(consumerManager.findChannel(Mockito.eq(CONSUMER_GROUP), Mockito.eq(PROXY_CONTEXT.getChannel()))).thenReturn(Mockito.mock(ClientChannelInfo.class)); Mockito.verify(messagingProcessor, Mockito.timeout(10000).times(1)) .changeInvisibleTime(Mockito.any(ProxyContext.class), Mockito.any(ReceiptHandle.class), Mockito.eq(MESSAGE_ID), - Mockito.eq(CONSUMER_GROUP), Mockito.eq(TOPIC), Mockito.eq(ConfigurationManager.getProxyConfig().getDefaultInvisibleTimeMills())); + Mockito.eq(CONSUMER_GROUP), Mockito.eq(TOPIC), Mockito.eq(ConfigurationManager.getProxyConfig().getDefaultInvisibleTimeMills()), Mockito.eq(null)); } } diff --git a/proxy/src/test/java/org/apache/rocketmq/proxy/service/lite/LiteSubscriptionServiceTest.java b/proxy/src/test/java/org/apache/rocketmq/proxy/service/lite/LiteSubscriptionServiceTest.java new file mode 100644 index 00000000000..e89066fabe7 --- /dev/null +++ b/proxy/src/test/java/org/apache/rocketmq/proxy/service/lite/LiteSubscriptionServiceTest.java @@ -0,0 +1,157 @@ +/* + * 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.proxy.service.lite; + +import java.util.Arrays; +import java.util.List; +import java.util.concurrent.CompletableFuture; +import org.apache.rocketmq.client.impl.mqclient.MQClientAPIExt; +import org.apache.rocketmq.client.impl.mqclient.MQClientAPIFactory; +import org.apache.rocketmq.common.lite.LiteSubscriptionDTO; +import org.apache.rocketmq.proxy.common.ProxyContext; +import org.apache.rocketmq.proxy.service.route.AddressableMessageQueue; +import org.apache.rocketmq.proxy.service.route.MessageQueueSelector; +import org.apache.rocketmq.proxy.service.route.MessageQueueView; +import org.apache.rocketmq.proxy.service.route.TopicRouteService; +import org.junit.Before; +import org.junit.Test; +import org.junit.runner.RunWith; +import org.mockito.Mock; +import org.mockito.junit.MockitoJUnitRunner; + +import static org.junit.jupiter.api.Assertions.assertDoesNotThrow; +import static org.junit.jupiter.api.Assertions.assertTrue; +import static org.mockito.Mockito.any; +import static org.mockito.Mockito.anyLong; +import static org.mockito.Mockito.anyString; +import static org.mockito.Mockito.mock; +import static org.mockito.Mockito.never; +import static org.mockito.Mockito.times; +import static org.mockito.Mockito.verify; +import static org.mockito.Mockito.when; + +@RunWith(MockitoJUnitRunner.class) +public class LiteSubscriptionServiceTest { + + @Mock + private TopicRouteService topicRouteService; + + @Mock + private MQClientAPIFactory mqClientAPIFactory; + + @Mock + private MQClientAPIExt mqClientAPIExt; + + private LiteSubscriptionService liteSubscriptionService; + + @Before + public void setUp() { + liteSubscriptionService = new LiteSubscriptionService(topicRouteService, mqClientAPIFactory); + } + + /** + * Test successful case: all brokers sync successfully + */ + @Test + public void testSyncLiteSubscription_Success() throws Exception { + ProxyContext ctx = ProxyContext.create(); + LiteSubscriptionDTO liteSubscriptionDTO = new LiteSubscriptionDTO(); + liteSubscriptionDTO.setTopic("testTopic"); + long timeoutMillis = 3000L; + + MessageQueueView messageQueueView = mock(MessageQueueView.class); + MessageQueueSelector readSelector = mock(MessageQueueSelector.class); + when(messageQueueView.getReadSelector()).thenReturn(readSelector); + + AddressableMessageQueue queue1 = mock(AddressableMessageQueue.class); + AddressableMessageQueue queue2 = mock(AddressableMessageQueue.class); + when(queue1.getBrokerAddr()).thenReturn("broker1:10911"); + when(queue2.getBrokerAddr()).thenReturn("broker2:10911"); + List readQueues = Arrays.asList(queue1, queue2); + when(readSelector.getBrokerActingQueues()).thenReturn(readQueues); + + when(topicRouteService.getAllMessageQueueView(ctx, "testTopic")).thenReturn(messageQueueView); + + when(mqClientAPIFactory.getClient()).thenReturn(mqClientAPIExt); + + when(mqClientAPIExt.syncLiteSubscriptionAsync(anyString(), any(LiteSubscriptionDTO.class), anyLong())) + .thenReturn(CompletableFuture.completedFuture(null)) + .thenReturn(CompletableFuture.completedFuture(null)); + + CompletableFuture future = liteSubscriptionService.syncLiteSubscription(ctx, liteSubscriptionDTO, timeoutMillis); + + assertDoesNotThrow(() -> future.get()); + verify(mqClientAPIExt, times(2)).syncLiteSubscriptionAsync(anyString(), any(LiteSubscriptionDTO.class), anyLong()); + } + + /** + * Test exception case: topicRouteService throws exception + */ + @Test + public void testSyncLiteSubscription_TopicRouteServiceException() throws Exception { + ProxyContext ctx = ProxyContext.create(); + LiteSubscriptionDTO liteSubscriptionDTO = new LiteSubscriptionDTO(); + liteSubscriptionDTO.setTopic("testTopic"); + long timeoutMillis = 3000L; + + when(topicRouteService.getAllMessageQueueView(ctx, "testTopic")) + .thenThrow(new RuntimeException("Topic route error")); + + CompletableFuture future = liteSubscriptionService.syncLiteSubscription(ctx, liteSubscriptionDTO, timeoutMillis); + + assertTrue(future.isCompletedExceptionally()); + verify(mqClientAPIFactory, never()).getClient(); + } + + /** + * Test exception case: some broker sync fails + */ + @Test + public void testSyncLiteSubscription_SomeBrokerFail() throws Exception { + ProxyContext ctx = ProxyContext.create(); + LiteSubscriptionDTO liteSubscriptionDTO = new LiteSubscriptionDTO(); + liteSubscriptionDTO.setTopic("testTopic"); + long timeoutMillis = 3000L; + + MessageQueueView messageQueueView = mock(MessageQueueView.class); + MessageQueueSelector readSelector = mock(MessageQueueSelector.class); + when(messageQueueView.getReadSelector()).thenReturn(readSelector); + + AddressableMessageQueue queue1 = mock(AddressableMessageQueue.class); + AddressableMessageQueue queue2 = mock(AddressableMessageQueue.class); + when(queue1.getBrokerAddr()).thenReturn("broker1:10911"); + when(queue2.getBrokerAddr()).thenReturn("broker2:10911"); + List readQueues = Arrays.asList(queue1, queue2); + when(readSelector.getBrokerActingQueues()).thenReturn(readQueues); + + when(topicRouteService.getAllMessageQueueView(ctx, "testTopic")).thenReturn(messageQueueView); + + when(mqClientAPIFactory.getClient()).thenReturn(mqClientAPIExt); + + CompletableFuture failedFuture = new CompletableFuture<>(); + failedFuture.completeExceptionally(new RuntimeException("Broker sync failed")); + + when(mqClientAPIExt.syncLiteSubscriptionAsync(anyString(), any(LiteSubscriptionDTO.class), anyLong())) + .thenReturn(failedFuture); + + CompletableFuture future = liteSubscriptionService.syncLiteSubscription(ctx, liteSubscriptionDTO, timeoutMillis); + + assertTrue(future.isCompletedExceptionally()); + verify(mqClientAPIExt, times(2)).syncLiteSubscriptionAsync(anyString(), any(LiteSubscriptionDTO.class), anyLong()); + } +} diff --git a/proxy/src/test/java/org/apache/rocketmq/proxy/service/mqclient/MQClientAPIExtTest.java b/proxy/src/test/java/org/apache/rocketmq/proxy/service/mqclient/MQClientAPIExtTest.java index e2d05b0f5a8..1680c8732a6 100644 --- a/proxy/src/test/java/org/apache/rocketmq/proxy/service/mqclient/MQClientAPIExtTest.java +++ b/proxy/src/test/java/org/apache/rocketmq/proxy/service/mqclient/MQClientAPIExtTest.java @@ -37,6 +37,7 @@ 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.impl.CommunicationMode; import org.apache.rocketmq.client.impl.MQClientAPIImpl; import org.apache.rocketmq.client.impl.consumer.PullResultExt; @@ -44,6 +45,7 @@ import org.apache.rocketmq.client.impl.mqclient.MQClientAPIExt; import org.apache.rocketmq.client.producer.SendResult; import org.apache.rocketmq.client.producer.SendStatus; +import org.apache.rocketmq.common.lite.LiteSubscriptionDTO; import org.apache.rocketmq.common.message.MessageClientIDSetter; import org.apache.rocketmq.common.message.MessageDecoder; import org.apache.rocketmq.common.message.MessageExt; @@ -54,6 +56,7 @@ import org.apache.rocketmq.remoting.netty.ResponseFuture; import org.apache.rocketmq.remoting.protocol.RemotingCommand; import org.apache.rocketmq.remoting.protocol.ResponseCode; +import org.apache.rocketmq.remoting.protocol.body.GetLiteTopicInfoResponseBody; import org.apache.rocketmq.remoting.protocol.header.AckMessageRequestHeader; import org.apache.rocketmq.remoting.protocol.header.ChangeInvisibleTimeRequestHeader; import org.apache.rocketmq.remoting.protocol.header.ConsumerSendMsgBackRequestHeader; @@ -62,6 +65,7 @@ import org.apache.rocketmq.remoting.protocol.header.GetConsumerListByGroupResponseHeader; import org.apache.rocketmq.remoting.protocol.header.GetMaxOffsetRequestHeader; import org.apache.rocketmq.remoting.protocol.header.GetMaxOffsetResponseHeader; +import org.apache.rocketmq.remoting.protocol.header.PopLiteMessageRequestHeader; import org.apache.rocketmq.remoting.protocol.header.PopMessageRequestHeader; import org.apache.rocketmq.remoting.protocol.header.PullMessageRequestHeader; import org.apache.rocketmq.remoting.protocol.header.SearchOffsetRequestHeader; @@ -194,6 +198,31 @@ public void testPopMessageAsync() throws Exception { assertSame(popResult, mqClientAPI.popMessageAsync(BROKER_ADDR, BROKER_NAME, new PopMessageRequestHeader(), TIMEOUT).get()); } + @Test + public void testPopLiteMessageAsync() throws Exception { + PopResult popResult = new PopResult(PopStatus.FOUND, new ArrayList<>()); + doAnswer((Answer) mock -> { + PopCallback popCallback = mock.getArgument(4); + popCallback.onSuccess(popResult); + return null; + }).when(mqClientAPI).popLiteMessageAsync(anyString(), anyString(), any(), anyLong(), any()); + + assertSame(popResult, mqClientAPI.popLiteMessageAsync(BROKER_ADDR, BROKER_NAME, new PopLiteMessageRequestHeader(), TIMEOUT).get()); + } + + @Test + public void testPopLiteMessageAsync_Exception() throws Exception { + Throwable throwable = new RuntimeException("test exception"); + doAnswer((Answer) mock -> { + PopCallback popCallback = mock.getArgument(4); + popCallback.onException(throwable); + return null; + }).when(mqClientAPI).popLiteMessageAsync(anyString(), anyString(), any(), anyLong(), any()); + + CompletableFuture future = mqClientAPI.popLiteMessageAsync(BROKER_ADDR, BROKER_NAME, new PopLiteMessageRequestHeader(), TIMEOUT); + assertTrue(future.isCompletedExceptionally()); + } + @Test public void testAckMessageAsync() throws Exception { AckResult ackResult = new AckResult(); @@ -347,4 +376,139 @@ protected MessageExt createMessage() { MessageClientIDSetter.setUniqID(messageExt); return messageExt; } + + @Test + public void testSyncLiteSubscriptionAsync_Success() throws Exception { + LiteSubscriptionDTO liteSubscriptionDTO = new LiteSubscriptionDTO(); + liteSubscriptionDTO.setTopic("test-topic"); + liteSubscriptionDTO.setGroup("test-group"); + + CompletableFuture future = new CompletableFuture<>(); + RemotingCommand response = RemotingCommand.createResponseCommand(ResponseCode.SUCCESS, ""); + future.complete(response); + + doReturn(future).when(remotingClient).invoke(anyString(), any(RemotingCommand.class), anyLong()); + + CompletableFuture result = mqClientAPI.syncLiteSubscriptionAsync(BROKER_ADDR, liteSubscriptionDTO, TIMEOUT); + + assertNotNull(result); + result.get(); + } + + @Test + public void testSyncLiteSubscriptionAsync_Failure() throws Exception { + LiteSubscriptionDTO liteSubscriptionDTO = new LiteSubscriptionDTO(); + liteSubscriptionDTO.setTopic("test-topic"); + liteSubscriptionDTO.setGroup("test-group"); + + CompletableFuture future = new CompletableFuture<>(); + RemotingCommand response = RemotingCommand.createResponseCommand(ResponseCode.SYSTEM_ERROR, "System error"); + future.complete(response); + + doReturn(future).when(remotingClient).invoke(anyString(), any(RemotingCommand.class), anyLong()); + + CompletableFuture result = mqClientAPI.syncLiteSubscriptionAsync(BROKER_ADDR, liteSubscriptionDTO, TIMEOUT); + + assertNotNull(result); + assertTrue(result.isCompletedExceptionally()); + + try { + result.get(); + } catch (Exception e) { + assertTrue(e.getCause() instanceof MQBrokerException); + MQBrokerException brokerException = (MQBrokerException) e.getCause(); + assertEquals(ResponseCode.SYSTEM_ERROR, brokerException.getResponseCode()); + } + } + + @Test + public void testSyncLiteSubscriptionAsync_Exception() throws Exception { + LiteSubscriptionDTO liteSubscriptionDTO = new LiteSubscriptionDTO(); + liteSubscriptionDTO.setTopic("test-topic"); + liteSubscriptionDTO.setGroup("test-group"); + + CompletableFuture future = new CompletableFuture<>(); + future.completeExceptionally(new RuntimeException("Network error")); + + doReturn(future).when(remotingClient).invoke(anyString(), any(RemotingCommand.class), anyLong()); + + CompletableFuture result = mqClientAPI.syncLiteSubscriptionAsync(BROKER_ADDR, liteSubscriptionDTO, TIMEOUT); + + assertNotNull(result); + assertTrue(result.isCompletedExceptionally()); + + try { + result.get(); + } catch (Exception e) { + assertTrue(e.getCause() instanceof RuntimeException); + assertEquals("Network error", e.getCause().getMessage()); + } + } + + @Test + public void testSyncLiteSubscriptionAsync_EmptySubscription() throws Exception { + LiteSubscriptionDTO liteSubscriptionDTO = new LiteSubscriptionDTO(); + + CompletableFuture future = new CompletableFuture<>(); + RemotingCommand response = RemotingCommand.createResponseCommand(ResponseCode.SUCCESS, ""); + future.complete(response); + + doReturn(future).when(remotingClient).invoke(anyString(), any(RemotingCommand.class), anyLong()); + + CompletableFuture result = mqClientAPI.syncLiteSubscriptionAsync(BROKER_ADDR, liteSubscriptionDTO, TIMEOUT); + + assertNotNull(result); + result.get(); + } + + @Test + public void testGetLiteTopicInfoAsync_Success() throws Exception { + String parentTopic = "parentTopic"; + String liteTopic = "liteTopic"; + + GetLiteTopicInfoResponseBody responseBody = new GetLiteTopicInfoResponseBody(); + responseBody.setLiteTopic(liteTopic); + responseBody.setParentTopic(parentTopic); + + CompletableFuture future = new CompletableFuture<>(); + RemotingCommand response = RemotingCommand.createResponseCommand(ResponseCode.SUCCESS, ""); + response.setBody(responseBody.encode()); + future.complete(response); + + doReturn(future).when(remotingClient).invoke(anyString(), any(RemotingCommand.class), anyLong()); + + CompletableFuture result = + mqClientAPI.getLiteTopicInfoAsync(BROKER_ADDR, parentTopic, liteTopic, TIMEOUT); + + assertNotNull(result); + GetLiteTopicInfoResponseBody actualBody = result.get(); + assertNotNull(actualBody); + assertEquals(liteTopic, actualBody.getLiteTopic()); + assertEquals(parentTopic, actualBody.getParentTopic()); + } + + @Test + public void testGetLiteTopicInfoAsync_Failure() throws Exception { + String parentTopic = "parentTopic"; + String liteTopic = "liteTopic"; + + CompletableFuture future = new CompletableFuture<>(); + RemotingCommand response = RemotingCommand.createResponseCommand(ResponseCode.SYSTEM_ERROR, "System error"); + future.complete(response); + + doReturn(future).when(remotingClient).invoke(anyString(), any(RemotingCommand.class), anyLong()); + + CompletableFuture result = + mqClientAPI.getLiteTopicInfoAsync(BROKER_ADDR, parentTopic, liteTopic, TIMEOUT); + + assertNotNull(result); + assertTrue(result.isCompletedExceptionally()); + + try { + result.get(); + } catch (Exception e) { + assertTrue(e.getCause() instanceof MQBrokerException); + } + } + } diff --git a/proxy/src/test/java/org/apache/rocketmq/proxy/service/mqclient/ProxyClientRemotingProcessorTest.java b/proxy/src/test/java/org/apache/rocketmq/proxy/service/mqclient/ProxyClientRemotingProcessorTest.java index 2cdd92ba5be..441d3c04012 100644 --- a/proxy/src/test/java/org/apache/rocketmq/proxy/service/mqclient/ProxyClientRemotingProcessorTest.java +++ b/proxy/src/test/java/org/apache/rocketmq/proxy/service/mqclient/ProxyClientRemotingProcessorTest.java @@ -88,7 +88,7 @@ public void testTransactionCheck() throws Exception { when(producerManager.getAvailableChannel(anyString())) .thenReturn(grpcClientChannel); - ProxyClientRemotingProcessor processor = new ProxyClientRemotingProcessor(producerManager); + ProxyClientRemotingProcessor processor = new ProxyClientRemotingProcessor(producerManager, null); CheckTransactionStateRequestHeader requestHeader = new CheckTransactionStateRequestHeader(); RemotingCommand command = RemotingCommand.createRequestCommand(RequestCode.CHECK_TRANSACTION_STATE, requestHeader); MessageExt message = new MessageExt(); diff --git a/proxy/src/test/java/org/apache/rocketmq/proxy/service/relay/ProxyChannelTest.java b/proxy/src/test/java/org/apache/rocketmq/proxy/service/relay/ProxyChannelTest.java index 947ae2c24f5..03be5cdb018 100644 --- a/proxy/src/test/java/org/apache/rocketmq/proxy/service/relay/ProxyChannelTest.java +++ b/proxy/src/test/java/org/apache/rocketmq/proxy/service/relay/ProxyChannelTest.java @@ -21,6 +21,7 @@ import java.nio.charset.StandardCharsets; import java.util.UUID; import java.util.concurrent.CompletableFuture; +import org.apache.commons.lang3.NotImplementedException; import org.apache.rocketmq.common.message.MessageClientIDSetter; import org.apache.rocketmq.common.message.MessageDecoder; import org.apache.rocketmq.common.message.MessageExt; @@ -33,6 +34,7 @@ import org.apache.rocketmq.remoting.protocol.header.CheckTransactionStateRequestHeader; import org.apache.rocketmq.remoting.protocol.header.ConsumeMessageDirectlyResultRequestHeader; import org.apache.rocketmq.remoting.protocol.header.GetConsumerRunningInfoRequestHeader; +import org.apache.rocketmq.remoting.protocol.header.NotifyUnsubscribeLiteRequestHeader; import org.junit.Test; import org.junit.runner.RunWith; import org.mockito.ArgumentCaptor; @@ -147,6 +149,11 @@ protected CompletableFuture processConsumeMessageDirectly(RemotingCommand assertArrayEquals(consumeMessageDirectlyMessageExt.getBody(), messageExt.getBody()); return CompletableFuture.completedFuture(null); } + + @Override + protected CompletableFuture processNotifyUnsubscribeLite(NotifyUnsubscribeLiteRequestHeader header) { + throw new NotImplementedException(); + } }; assertTrue(channel.writeAndFlush(checkTransactionRequest).isSuccess()); diff --git a/remoting/src/main/java/org/apache/rocketmq/remoting/protocol/RequestCode.java b/remoting/src/main/java/org/apache/rocketmq/remoting/protocol/RequestCode.java index 89c9acff94b..b32dbbc87ea 100644 --- a/remoting/src/main/java/org/apache/rocketmq/remoting/protocol/RequestCode.java +++ b/remoting/src/main/java/org/apache/rocketmq/remoting/protocol/RequestCode.java @@ -86,6 +86,18 @@ public class RequestCode { public static final int POLLING_INFO = 200055; public static final int POP_ROLLBACK = 200056; + public static final int POP_LITE_MESSAGE = 200070; + public static final int LITE_SUBSCRIPTION_CTL = 200071; + public static final int ACK_LITE_MESSAGE = 200072; + public static final int NOTIFY_UNSUBSCRIBE_LITE = 200073; + // lite admin api + public static final int GET_BROKER_LITE_INFO = 200074; + public static final int GET_PARENT_TOPIC_INFO = 200075; + public static final int GET_LITE_TOPIC_INFO = 200076; + public static final int GET_LITE_CLIENT_INFO = 200077; + public static final int GET_LITE_GROUP_INFO = 200078; + public static final int TRIGGER_LITE_DISPATCH = 200079; + public static final int PUT_KV_CONFIG = 100; public static final int GET_KV_CONFIG = 101; diff --git a/remoting/src/main/java/org/apache/rocketmq/remoting/protocol/ResponseCode.java b/remoting/src/main/java/org/apache/rocketmq/remoting/protocol/ResponseCode.java index 68f77ab31be..e29d2e91f94 100644 --- a/remoting/src/main/java/org/apache/rocketmq/remoting/protocol/ResponseCode.java +++ b/remoting/src/main/java/org/apache/rocketmq/remoting/protocol/ResponseCode.java @@ -127,6 +127,10 @@ public class ResponseCode extends RemotingSysResponseCode { public static final int CONTROLLER_BROKER_LIVE_INFO_NOT_EXISTS = 2016; + public static final int LMQ_QUOTA_EXCEEDED = 2017; + + public static final int LITE_SUBSCRIPTION_QUOTA_EXCEEDED = 2018; + public static final int USER_NOT_EXIST = 3001; public static final int POLICY_NOT_EXIST = 3002; diff --git a/remoting/src/main/java/org/apache/rocketmq/remoting/protocol/body/GetBrokerLiteInfoResponseBody.java b/remoting/src/main/java/org/apache/rocketmq/remoting/protocol/body/GetBrokerLiteInfoResponseBody.java new file mode 100644 index 00000000000..e7c1d08cbe5 --- /dev/null +++ b/remoting/src/main/java/org/apache/rocketmq/remoting/protocol/body/GetBrokerLiteInfoResponseBody.java @@ -0,0 +1,117 @@ +/* + * 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.remoting.protocol.body; + +import org.apache.rocketmq.remoting.protocol.RemotingSerializable; + +import java.util.Set; +import java.util.Map; + +public class GetBrokerLiteInfoResponseBody extends RemotingSerializable { + + private String storeType; + private int maxLmqNum; + private int currentLmqNum; + private int liteSubscriptionCount; + private int orderInfoCount; + private int cqTableSize; + private int offsetTableSize; + private int eventMapSize; + private Map topicMeta; + private Map> groupMeta; + + public String getStoreType() { + return storeType; + } + + public void setStoreType(String storeType) { + this.storeType = storeType; + } + + public int getMaxLmqNum() { + return maxLmqNum; + } + + public void setMaxLmqNum(int maxLmqNum) { + this.maxLmqNum = maxLmqNum; + } + + public int getCurrentLmqNum() { + return currentLmqNum; + } + + public void setCurrentLmqNum(int currentLmqNum) { + this.currentLmqNum = currentLmqNum; + } + + public int getLiteSubscriptionCount() { + return liteSubscriptionCount; + } + + public void setLiteSubscriptionCount(int liteSubscriptionCount) { + this.liteSubscriptionCount = liteSubscriptionCount; + } + + public int getOrderInfoCount() { + return orderInfoCount; + } + + public void setOrderInfoCount(int orderInfoCount) { + this.orderInfoCount = orderInfoCount; + } + + public int getCqTableSize() { + return cqTableSize; + } + + public void setCqTableSize(int cqTableSize) { + this.cqTableSize = cqTableSize; + } + + public int getOffsetTableSize() { + return offsetTableSize; + } + + public void setOffsetTableSize(int offsetTableSize) { + this.offsetTableSize = offsetTableSize; + } + + public int getEventMapSize() { + return eventMapSize; + } + + public void setEventMapSize(int eventMapSize) { + this.eventMapSize = eventMapSize; + } + + public Map getTopicMeta() { + return topicMeta; + } + + public void setTopicMeta(Map topicMeta) { + this.topicMeta = topicMeta; + } + + public Map> getGroupMeta() { + return groupMeta; + } + + public void setGroupMeta(Map> groupMeta) { + this.groupMeta = groupMeta; + } +} diff --git a/remoting/src/main/java/org/apache/rocketmq/remoting/protocol/body/GetLiteClientInfoResponseBody.java b/remoting/src/main/java/org/apache/rocketmq/remoting/protocol/body/GetLiteClientInfoResponseBody.java new file mode 100644 index 00000000000..d652b009672 --- /dev/null +++ b/remoting/src/main/java/org/apache/rocketmq/remoting/protocol/body/GetLiteClientInfoResponseBody.java @@ -0,0 +1,89 @@ +/* + * 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.remoting.protocol.body; + +import org.apache.rocketmq.remoting.protocol.RemotingSerializable; + +import java.util.Set; + +public class GetLiteClientInfoResponseBody extends RemotingSerializable { + + private String parentTopic; + private String group; + private String clientId; + private long lastAccessTime; + private long lastConsumeTime; + private int liteTopicCount; + private Set liteTopicSet; + + public String getParentTopic() { + return parentTopic; + } + + public void setParentTopic(String parentTopic) { + this.parentTopic = parentTopic; + } + + public String getGroup() { + return group; + } + + public void setGroup(String group) { + this.group = group; + } + + public String getClientId() { + return clientId; + } + + public void setClientId(String clientId) { + this.clientId = clientId; + } + + public long getLastAccessTime() { + return lastAccessTime; + } + + public void setLastAccessTime(long lastAccessTime) { + this.lastAccessTime = lastAccessTime; + } + + public long getLastConsumeTime() { + return lastConsumeTime; + } + + public void setLastConsumeTime(long lastConsumeTime) { + this.lastConsumeTime = lastConsumeTime; + } + + public int getLiteTopicCount() { + return liteTopicCount; + } + + public void setLiteTopicCount(int liteTopicCount) { + this.liteTopicCount = liteTopicCount; + } + + public Set getLiteTopicSet() { + return liteTopicSet; + } + + public void setLiteTopicSet(Set liteTopicSet) { + this.liteTopicSet = liteTopicSet; + } +} diff --git a/remoting/src/main/java/org/apache/rocketmq/remoting/protocol/body/GetLiteGroupInfoResponseBody.java b/remoting/src/main/java/org/apache/rocketmq/remoting/protocol/body/GetLiteGroupInfoResponseBody.java new file mode 100644 index 00000000000..064fb2df3ad --- /dev/null +++ b/remoting/src/main/java/org/apache/rocketmq/remoting/protocol/body/GetLiteGroupInfoResponseBody.java @@ -0,0 +1,100 @@ +/* + * 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.remoting.protocol.body; + +import java.util.List; +import org.apache.rocketmq.common.lite.LiteLagInfo; +import org.apache.rocketmq.remoting.protocol.RemotingSerializable; +import org.apache.rocketmq.remoting.protocol.admin.OffsetWrapper; + +public class GetLiteGroupInfoResponseBody extends RemotingSerializable { + private String group; + private String parentTopic; + private String liteTopic; + // total log info + private long earliestUnconsumedTimestamp = -1; + private long totalLagCount; + // lite topic detail info + private OffsetWrapper liteTopicOffsetWrapper; // if lite topic specified + // topK info + private List lagCountTopK; + private List lagTimestampTopK; + + public String getGroup() { + return group; + } + + public void setGroup(String group) { + this.group = group; + } + + public String getParentTopic() { + return parentTopic; + } + + public void setParentTopic(String parentTopic) { + this.parentTopic = parentTopic; + } + + public String getLiteTopic() { + return liteTopic; + } + + public void setLiteTopic(String liteTopic) { + this.liteTopic = liteTopic; + } + + public long getEarliestUnconsumedTimestamp() { + return earliestUnconsumedTimestamp; + } + + public void setEarliestUnconsumedTimestamp(long earliestUnconsumedTimestamp) { + this.earliestUnconsumedTimestamp = earliestUnconsumedTimestamp; + } + + public long getTotalLagCount() { + return totalLagCount; + } + + public void setTotalLagCount(long totalLagCount) { + this.totalLagCount = totalLagCount; + } + + public OffsetWrapper getLiteTopicOffsetWrapper() { + return liteTopicOffsetWrapper; + } + + public void setLiteTopicOffsetWrapper(OffsetWrapper liteTopicOffsetWrapper) { + this.liteTopicOffsetWrapper = liteTopicOffsetWrapper; + } + + public List getLagCountTopK() { + return lagCountTopK; + } + + public void setLagCountTopK(List lagCountTopK) { + this.lagCountTopK = lagCountTopK; + } + + public List getLagTimestampTopK() { + return lagTimestampTopK; + } + + public void setLagTimestampTopK(List lagTimestampTopK) { + this.lagTimestampTopK = lagTimestampTopK; + } +} diff --git a/remoting/src/main/java/org/apache/rocketmq/remoting/protocol/body/GetLiteTopicInfoResponseBody.java b/remoting/src/main/java/org/apache/rocketmq/remoting/protocol/body/GetLiteTopicInfoResponseBody.java new file mode 100644 index 00000000000..93118ace323 --- /dev/null +++ b/remoting/src/main/java/org/apache/rocketmq/remoting/protocol/body/GetLiteTopicInfoResponseBody.java @@ -0,0 +1,73 @@ +/* + * 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.remoting.protocol.body; + +import org.apache.rocketmq.common.entity.ClientGroup; +import org.apache.rocketmq.remoting.protocol.RemotingSerializable; +import org.apache.rocketmq.remoting.protocol.admin.TopicOffset; + +import java.util.Set; + +public class GetLiteTopicInfoResponseBody extends RemotingSerializable { + + private String parentTopic; + private String liteTopic; + private Set subscriber; + private TopicOffset topicOffset; + private boolean shardingToBroker; + + public String getParentTopic() { + return parentTopic; + } + + public void setParentTopic(String parentTopic) { + this.parentTopic = parentTopic; + } + + public String getLiteTopic() { + return liteTopic; + } + + public void setLiteTopic(String liteTopic) { + this.liteTopic = liteTopic; + } + + public Set getSubscriber() { + return subscriber; + } + + public void setSubscriber(Set subscriber) { + this.subscriber = subscriber; + } + + public TopicOffset getTopicOffset() { + return topicOffset; + } + + public void setTopicOffset(TopicOffset topicOffset) { + this.topicOffset = topicOffset; + } + + public boolean isShardingToBroker() { + return shardingToBroker; + } + + public void setShardingToBroker(boolean shardingToBroker) { + this.shardingToBroker = shardingToBroker; + } +} diff --git a/remoting/src/main/java/org/apache/rocketmq/remoting/protocol/body/GetParentTopicInfoResponseBody.java b/remoting/src/main/java/org/apache/rocketmq/remoting/protocol/body/GetParentTopicInfoResponseBody.java new file mode 100644 index 00000000000..3f2bfeac96c --- /dev/null +++ b/remoting/src/main/java/org/apache/rocketmq/remoting/protocol/body/GetParentTopicInfoResponseBody.java @@ -0,0 +1,71 @@ +/* + * 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.remoting.protocol.body; + +import org.apache.rocketmq.remoting.protocol.RemotingSerializable; + +import java.util.Set; + +public class GetParentTopicInfoResponseBody extends RemotingSerializable { + + private String topic; + private int ttl; + private Set groups; + private int lmqNum; + private int liteTopicCount; + + public String getTopic() { + return topic; + } + + public void setTopic(String topic) { + this.topic = topic; + } + + public int getTtl() { + return ttl; + } + + public void setTtl(int ttl) { + this.ttl = ttl; + } + + public Set getGroups() { + return groups; + } + + public void setGroups(Set groups) { + this.groups = groups; + } + + public int getLmqNum() { + return lmqNum; + } + + public void setLmqNum(int lmqNum) { + this.lmqNum = lmqNum; + } + + public int getLiteTopicCount() { + return liteTopicCount; + } + + public void setLiteTopicCount(int liteTopicCount) { + this.liteTopicCount = liteTopicCount; + } +} diff --git a/remoting/src/main/java/org/apache/rocketmq/remoting/protocol/body/LiteSubscriptionCtlRequestBody.java b/remoting/src/main/java/org/apache/rocketmq/remoting/protocol/body/LiteSubscriptionCtlRequestBody.java new file mode 100644 index 00000000000..fdfe90be22b --- /dev/null +++ b/remoting/src/main/java/org/apache/rocketmq/remoting/protocol/body/LiteSubscriptionCtlRequestBody.java @@ -0,0 +1,35 @@ +/* + * 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.remoting.protocol.body; + +import java.util.Set; +import org.apache.rocketmq.common.lite.LiteSubscriptionDTO; +import org.apache.rocketmq.remoting.protocol.RemotingSerializable; + +public class LiteSubscriptionCtlRequestBody extends RemotingSerializable { + + private Set subscriptionSet; + + public void setSubscriptionSet(Set subscriptionSet) { + this.subscriptionSet = subscriptionSet; + } + + public Set getSubscriptionSet() { + return subscriptionSet; + } +} diff --git a/remoting/src/main/java/org/apache/rocketmq/remoting/protocol/header/AckMessageRequestHeader.java b/remoting/src/main/java/org/apache/rocketmq/remoting/protocol/header/AckMessageRequestHeader.java index 28313fab9f0..19929931ab6 100644 --- a/remoting/src/main/java/org/apache/rocketmq/remoting/protocol/header/AckMessageRequestHeader.java +++ b/remoting/src/main/java/org/apache/rocketmq/remoting/protocol/header/AckMessageRequestHeader.java @@ -42,6 +42,7 @@ public class AckMessageRequestHeader extends TopicQueueRequestHeader { @CFNotNull private Long offset; + private String liteTopic; @Override public void checkFields() throws RemotingCommandException { @@ -87,6 +88,14 @@ public void setQueueId(Integer queueId) { this.queueId = queueId; } + public String getLiteTopic() { + return liteTopic; + } + + public void setLiteTopic(String liteTopic) { + this.liteTopic = liteTopic; + } + @Override public String toString() { return MoreObjects.toStringHelper(this) @@ -95,6 +104,8 @@ public String toString() { .add("queueId", queueId) .add("extraInfo", extraInfo) .add("offset", offset) + .add("liteTopic", liteTopic) + .omitNullValues() .toString(); } } diff --git a/remoting/src/main/java/org/apache/rocketmq/remoting/protocol/header/ChangeInvisibleTimeRequestHeader.java b/remoting/src/main/java/org/apache/rocketmq/remoting/protocol/header/ChangeInvisibleTimeRequestHeader.java index ebd32cc534c..9d44590da32 100644 --- a/remoting/src/main/java/org/apache/rocketmq/remoting/protocol/header/ChangeInvisibleTimeRequestHeader.java +++ b/remoting/src/main/java/org/apache/rocketmq/remoting/protocol/header/ChangeInvisibleTimeRequestHeader.java @@ -48,6 +48,8 @@ public class ChangeInvisibleTimeRequestHeader extends TopicQueueRequestHeader { @CFNotNull private Long invisibleTime; + private String liteTopic; + @Override public void checkFields() throws RemotingCommandException { } @@ -103,6 +105,14 @@ public void setQueueId(Integer queueId) { this.queueId = queueId; } + public String getLiteTopic() { + return liteTopic; + } + + public void setLiteTopic(String liteTopic) { + this.liteTopic = liteTopic; + } + @Override public String toString() { return MoreObjects.toStringHelper(this) @@ -112,6 +122,8 @@ public String toString() { .add("extraInfo", extraInfo) .add("offset", offset) .add("invisibleTime", invisibleTime) + .add("liteTopic", liteTopic) + .omitNullValues() .toString(); } } diff --git a/remoting/src/main/java/org/apache/rocketmq/remoting/protocol/header/ExtraInfoUtil.java b/remoting/src/main/java/org/apache/rocketmq/remoting/protocol/header/ExtraInfoUtil.java index a6a4a777675..bba6063f61f 100644 --- a/remoting/src/main/java/org/apache/rocketmq/remoting/protocol/header/ExtraInfoUtil.java +++ b/remoting/src/main/java/org/apache/rocketmq/remoting/protocol/header/ExtraInfoUtil.java @@ -17,9 +17,14 @@ package org.apache.rocketmq.remoting.protocol.header; import java.util.ArrayList; +import java.util.Arrays; import java.util.HashMap; import java.util.List; import java.util.Map; +import java.util.stream.Collectors; + +import org.apache.commons.lang3.StringUtils; +import org.apache.commons.lang3.math.NumberUtils; import org.apache.rocketmq.common.KeyBuilder; import org.apache.rocketmq.common.MixAll; import org.apache.rocketmq.common.message.MessageConst; @@ -283,6 +288,28 @@ public static Map parseOrderCountInfo(String orderCountInfo) { return startOffsetMap; } + public static List parseLiteOrderCountInfo(String orderCountInfo, int msgCount) { + if (StringUtils.isEmpty(orderCountInfo)) { + return null; + } + String[] infos = orderCountInfo.split(";"); + if (infos.length != msgCount) { + return null; + } + return Arrays.stream(infos).map(ExtraInfoUtil::parseLiteOrderCount).collect(Collectors.toList()); + } + + private static int parseLiteOrderCount(String info) { + if (StringUtils.isBlank(info)) { + return 0; + } + if (!info.contains(QUEUE_OFFSET)) { + return NumberUtils.toInt(info, 0); + } + String[] split = info.split(MessageConst.KEY_SEPARATOR); + return split.length != 3 ? 0 : NumberUtils.toInt(split[2], 0); + } + public static String getStartOffsetInfoMapKey(String topic, long key) { return getRetry(topic) + "@" + key; } diff --git a/remoting/src/main/java/org/apache/rocketmq/remoting/protocol/header/GetLiteClientInfoRequestHeader.java b/remoting/src/main/java/org/apache/rocketmq/remoting/protocol/header/GetLiteClientInfoRequestHeader.java new file mode 100644 index 00000000000..9b0f99c67f8 --- /dev/null +++ b/remoting/src/main/java/org/apache/rocketmq/remoting/protocol/header/GetLiteClientInfoRequestHeader.java @@ -0,0 +1,68 @@ +/* + * 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.remoting.protocol.header; + +import org.apache.rocketmq.remoting.CommandCustomHeader; +import org.apache.rocketmq.remoting.exception.RemotingCommandException; + +public class GetLiteClientInfoRequestHeader implements CommandCustomHeader { + + private String parentTopic; + private String group; + private String clientId; + private int maxCount = 1000; + + @Override + public void checkFields() throws RemotingCommandException { + if (maxCount <= 0) { + throw new RemotingCommandException("[maxCount] field invalid"); + } + } + + public String getParentTopic() { + return parentTopic; + } + + public void setParentTopic(String parentTopic) { + this.parentTopic = parentTopic; + } + + public String getGroup() { + return group; + } + + public void setGroup(String group) { + this.group = group; + } + + public String getClientId() { + return clientId; + } + + public void setClientId(String clientId) { + this.clientId = clientId; + } + + public int getMaxCount() { + return maxCount; + } + + public void setMaxCount(int maxCount) { + this.maxCount = maxCount; + } +} diff --git a/remoting/src/main/java/org/apache/rocketmq/remoting/protocol/header/GetLiteGroupInfoRequestHeader.java b/remoting/src/main/java/org/apache/rocketmq/remoting/protocol/header/GetLiteGroupInfoRequestHeader.java new file mode 100644 index 00000000000..db1dbd2a4c3 --- /dev/null +++ b/remoting/src/main/java/org/apache/rocketmq/remoting/protocol/header/GetLiteGroupInfoRequestHeader.java @@ -0,0 +1,62 @@ +/* + * 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.remoting.protocol.header; + +import org.apache.rocketmq.common.resource.ResourceType; +import org.apache.rocketmq.common.resource.RocketMQResource; +import org.apache.rocketmq.remoting.CommandCustomHeader; +import org.apache.rocketmq.remoting.annotation.CFNotNull; +import org.apache.rocketmq.remoting.exception.RemotingCommandException; + +public class GetLiteGroupInfoRequestHeader implements CommandCustomHeader { + + @CFNotNull + @RocketMQResource(ResourceType.GROUP) + private String group; + + private String liteTopic; + + private int topK; + + public String getGroup() { + return group; + } + + public void setGroup(String group) { + this.group = group; + } + + public String getLiteTopic() { + return liteTopic; + } + + public void setLiteTopic(String liteTopic) { + this.liteTopic = liteTopic; + } + + public int getTopK() { + return topK; + } + + public void setTopK(int topK) { + this.topK = topK; + } + + @Override + public void checkFields() throws RemotingCommandException { + } +} diff --git a/remoting/src/main/java/org/apache/rocketmq/remoting/protocol/header/GetLiteTopicInfoRequestHeader.java b/remoting/src/main/java/org/apache/rocketmq/remoting/protocol/header/GetLiteTopicInfoRequestHeader.java new file mode 100644 index 00000000000..bdb9c8408f3 --- /dev/null +++ b/remoting/src/main/java/org/apache/rocketmq/remoting/protocol/header/GetLiteTopicInfoRequestHeader.java @@ -0,0 +1,48 @@ +/* + * 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.remoting.protocol.header; + +import org.apache.rocketmq.remoting.CommandCustomHeader; +import org.apache.rocketmq.remoting.exception.RemotingCommandException; + +public class GetLiteTopicInfoRequestHeader implements CommandCustomHeader { + + private String parentTopic; + private String liteTopic; + + @Override + public void checkFields() throws RemotingCommandException { + + } + + public String getParentTopic() { + return parentTopic; + } + + public void setParentTopic(String parentTopic) { + this.parentTopic = parentTopic; + } + + public String getLiteTopic() { + return liteTopic; + } + + public void setLiteTopic(String liteTopic) { + this.liteTopic = liteTopic; + } +} diff --git a/remoting/src/main/java/org/apache/rocketmq/remoting/protocol/header/GetParentTopicInfoRequestHeader.java b/remoting/src/main/java/org/apache/rocketmq/remoting/protocol/header/GetParentTopicInfoRequestHeader.java new file mode 100644 index 00000000000..885e9cb66b7 --- /dev/null +++ b/remoting/src/main/java/org/apache/rocketmq/remoting/protocol/header/GetParentTopicInfoRequestHeader.java @@ -0,0 +1,44 @@ +/* + * 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.remoting.protocol.header; + +import org.apache.rocketmq.common.resource.ResourceType; +import org.apache.rocketmq.common.resource.RocketMQResource; +import org.apache.rocketmq.remoting.CommandCustomHeader; +import org.apache.rocketmq.remoting.annotation.CFNotNull; +import org.apache.rocketmq.remoting.exception.RemotingCommandException; + +public class GetParentTopicInfoRequestHeader implements CommandCustomHeader { + + @CFNotNull + @RocketMQResource(ResourceType.TOPIC) + private String topic; + + @Override + public void checkFields() throws RemotingCommandException { + + } + + public String getTopic() { + return topic; + } + + public void setTopic(String topic) { + this.topic = topic; + } +} diff --git a/remoting/src/main/java/org/apache/rocketmq/remoting/protocol/header/LiteSubscriptionCtlRequestHeader.java b/remoting/src/main/java/org/apache/rocketmq/remoting/protocol/header/LiteSubscriptionCtlRequestHeader.java new file mode 100644 index 00000000000..55b7adf0442 --- /dev/null +++ b/remoting/src/main/java/org/apache/rocketmq/remoting/protocol/header/LiteSubscriptionCtlRequestHeader.java @@ -0,0 +1,29 @@ +/* + * 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.remoting.protocol.header; + +import org.apache.rocketmq.remoting.exception.RemotingCommandException; +import org.apache.rocketmq.remoting.rpc.RpcRequestHeader; + +public class LiteSubscriptionCtlRequestHeader extends RpcRequestHeader { + + @Override + public void checkFields() throws RemotingCommandException { + + } +} diff --git a/remoting/src/main/java/org/apache/rocketmq/remoting/protocol/header/NotifyUnsubscribeLiteRequestHeader.java b/remoting/src/main/java/org/apache/rocketmq/remoting/protocol/header/NotifyUnsubscribeLiteRequestHeader.java new file mode 100644 index 00000000000..5347b6bf017 --- /dev/null +++ b/remoting/src/main/java/org/apache/rocketmq/remoting/protocol/header/NotifyUnsubscribeLiteRequestHeader.java @@ -0,0 +1,78 @@ +/* + * 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.remoting.protocol.header; + +import org.apache.rocketmq.common.action.Action; +import org.apache.rocketmq.common.action.RocketMQAction; +import org.apache.rocketmq.common.resource.ResourceType; +import org.apache.rocketmq.common.resource.RocketMQResource; +import org.apache.rocketmq.remoting.annotation.CFNotNull; +import org.apache.rocketmq.remoting.exception.RemotingCommandException; +import org.apache.rocketmq.remoting.protocol.RequestCode; +import org.apache.rocketmq.remoting.rpc.RpcRequestHeader; + +@RocketMQAction(value = RequestCode.NOTIFY_UNSUBSCRIBE_LITE, action = Action.SUB) +public class NotifyUnsubscribeLiteRequestHeader extends RpcRequestHeader { + + @CFNotNull + private String liteTopic; + + @RocketMQResource(ResourceType.GROUP) + @CFNotNull + private String consumerGroup; + + @CFNotNull + private String clientId; + + @Override + public void checkFields() throws RemotingCommandException { + } + + public String getLiteTopic() { + return liteTopic; + } + + public void setLiteTopic(String liteTopic) { + this.liteTopic = liteTopic; + } + + public String getConsumerGroup() { + return consumerGroup; + } + + public void setConsumerGroup(String consumerGroup) { + this.consumerGroup = consumerGroup; + } + + public String getClientId() { + return clientId; + } + + public void setClientId(String clientId) { + this.clientId = clientId; + } + + @Override + public String toString() { + return "NotifyUnsubscribeLiteRequestHeader{" + + "liteTopic='" + liteTopic + '\'' + + ", consumerGroup='" + consumerGroup + '\'' + + ", clientId='" + clientId + '\'' + + '}'; + } +} diff --git a/remoting/src/main/java/org/apache/rocketmq/remoting/protocol/header/PopLiteMessageRequestHeader.java b/remoting/src/main/java/org/apache/rocketmq/remoting/protocol/header/PopLiteMessageRequestHeader.java new file mode 100644 index 00000000000..e42606f3148 --- /dev/null +++ b/remoting/src/main/java/org/apache/rocketmq/remoting/protocol/header/PopLiteMessageRequestHeader.java @@ -0,0 +1,134 @@ +/* + * 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.remoting.protocol.header; + +import com.google.common.base.MoreObjects; +import org.apache.rocketmq.common.resource.ResourceType; +import org.apache.rocketmq.common.resource.RocketMQResource; +import org.apache.rocketmq.remoting.annotation.CFNotNull; +import org.apache.rocketmq.remoting.exception.RemotingCommandException; +import org.apache.rocketmq.remoting.rpc.RpcRequestHeader; + +public class PopLiteMessageRequestHeader extends RpcRequestHeader { + + @CFNotNull + private String clientId; + @CFNotNull + @RocketMQResource(ResourceType.GROUP) + private String consumerGroup; + @CFNotNull + @RocketMQResource(ResourceType.TOPIC) + private String topic; + @CFNotNull + private int maxMsgNum; + @CFNotNull + private long invisibleTime; + @CFNotNull + private long pollTime; + @CFNotNull + private long bornTime; + + private String attemptId; + + @Override + public void checkFields() throws RemotingCommandException { + + } + + public String getClientId() { + return clientId; + } + + public void setClientId(String clientId) { + this.clientId = clientId; + } + + public String getConsumerGroup() { + return consumerGroup; + } + + public void setConsumerGroup(String consumerGroup) { + this.consumerGroup = consumerGroup; + } + + public String getTopic() { + return topic; + } + + public void setTopic(String topic) { + this.topic = topic; + } + + public int getMaxMsgNum() { + return maxMsgNum; + } + + public void setMaxMsgNum(int maxMsgNum) { + this.maxMsgNum = maxMsgNum; + } + + public long getInvisibleTime() { + return invisibleTime; + } + + public void setInvisibleTime(long invisibleTime) { + this.invisibleTime = invisibleTime; + } + + public long getPollTime() { + return pollTime; + } + + public void setPollTime(long pollTime) { + this.pollTime = pollTime; + } + + public long getBornTime() { + return bornTime; + } + + public void setBornTime(long bornTime) { + this.bornTime = bornTime; + } + + public String getAttemptId() { + return attemptId; + } + + public void setAttemptId(String attemptId) { + this.attemptId = attemptId; + } + + public boolean isTimeoutTooMuch() { + return System.currentTimeMillis() - bornTime - pollTime > 500; + } + + @Override + public String toString() { + return MoreObjects.toStringHelper(this) + .add("consumerGroup", consumerGroup) + .add("topic", topic) + .add("maxMsgNum", maxMsgNum) + .add("invisibleTime", invisibleTime) + .add("pollTime", pollTime) + .add("bornTime", bornTime) + .add("attemptId", attemptId) + .add("clientId", clientId) + .toString(); + } +} diff --git a/remoting/src/main/java/org/apache/rocketmq/remoting/protocol/header/PopLiteMessageResponseHeader.java b/remoting/src/main/java/org/apache/rocketmq/remoting/protocol/header/PopLiteMessageResponseHeader.java new file mode 100644 index 00000000000..396c221bede --- /dev/null +++ b/remoting/src/main/java/org/apache/rocketmq/remoting/protocol/header/PopLiteMessageResponseHeader.java @@ -0,0 +1,87 @@ +/* + * 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.remoting.protocol.header; + +import org.apache.rocketmq.remoting.CommandCustomHeader; +import org.apache.rocketmq.remoting.annotation.CFNotNull; +import org.apache.rocketmq.remoting.exception.RemotingCommandException; + +public class PopLiteMessageResponseHeader implements CommandCustomHeader { + + @CFNotNull + private long popTime; + @CFNotNull + private long invisibleTime; + @CFNotNull + private int reviveQid; // reuse current ack implementation + + private String startOffsetInfo; + private String msgOffsetInfo; + private String orderCountInfo; + + @Override + public void checkFields() throws RemotingCommandException { + } + + public long getPopTime() { + return popTime; + } + + public void setPopTime(long popTime) { + this.popTime = popTime; + } + + public long getInvisibleTime() { + return invisibleTime; + } + + public void setInvisibleTime(long invisibleTime) { + this.invisibleTime = invisibleTime; + } + + public int getReviveQid() { + return reviveQid; + } + + public void setReviveQid(int reviveQid) { + this.reviveQid = reviveQid; + } + + public String getStartOffsetInfo() { + return startOffsetInfo; + } + + public void setStartOffsetInfo(String startOffsetInfo) { + this.startOffsetInfo = startOffsetInfo; + } + + public String getMsgOffsetInfo() { + return msgOffsetInfo; + } + + public void setMsgOffsetInfo(String msgOffsetInfo) { + this.msgOffsetInfo = msgOffsetInfo; + } + + public String getOrderCountInfo() { + return orderCountInfo; + } + + public void setOrderCountInfo(String orderCountInfo) { + this.orderCountInfo = orderCountInfo; + } +} diff --git a/remoting/src/main/java/org/apache/rocketmq/remoting/protocol/header/PullMessageRequestHeader.java b/remoting/src/main/java/org/apache/rocketmq/remoting/protocol/header/PullMessageRequestHeader.java index 5785615b204..af605884122 100644 --- a/remoting/src/main/java/org/apache/rocketmq/remoting/protocol/header/PullMessageRequestHeader.java +++ b/remoting/src/main/java/org/apache/rocketmq/remoting/protocol/header/PullMessageRequestHeader.java @@ -43,6 +43,7 @@ public class PullMessageRequestHeader extends TopicQueueRequestHeader implements @CFNotNull @RocketMQResource(ResourceType.TOPIC) private String topic; + private String liteTopic; @CFNotNull private Integer queueId; @CFNotNull @@ -82,6 +83,7 @@ public void checkFields() throws RemotingCommandException { public void encode(ByteBuf out) { writeIfNotNull(out, "consumerGroup", consumerGroup); writeIfNotNull(out, "topic", topic); + writeIfNotNull(out, "liteTopic", liteTopic); writeIfNotNull(out, "queueId", queueId); writeIfNotNull(out, "queueOffset", queueOffset); writeIfNotNull(out, "maxMsgNums", maxMsgNums); @@ -113,6 +115,11 @@ public void decode(HashMap fields) throws RemotingCommandExcepti this.topic = str; } + str = fields.get("liteTopic"); + if (str != null) { + this.liteTopic = str; + } + str = getAndCheckNotNull(fields, "queueId"); if (str != null) { this.queueId = Integer.parseInt(str); @@ -217,6 +224,14 @@ public void setTopic(String topic) { this.topic = topic; } + public String getLiteTopic() { + return liteTopic; + } + + public void setLiteTopic(String liteTopic) { + this.liteTopic = liteTopic; + } + @Override public Integer getQueueId() { return queueId; @@ -320,6 +335,7 @@ public String toString() { return MoreObjects.toStringHelper(this) .add("consumerGroup", consumerGroup) .add("topic", topic) + .add("liteTopic", liteTopic) .add("queueId", queueId) .add("queueOffset", queueOffset) .add("maxMsgBytes", maxMsgBytes) diff --git a/remoting/src/main/java/org/apache/rocketmq/remoting/protocol/header/SearchOffsetRequestHeader.java b/remoting/src/main/java/org/apache/rocketmq/remoting/protocol/header/SearchOffsetRequestHeader.java index bbefa8c1e5b..c66098159eb 100644 --- a/remoting/src/main/java/org/apache/rocketmq/remoting/protocol/header/SearchOffsetRequestHeader.java +++ b/remoting/src/main/java/org/apache/rocketmq/remoting/protocol/header/SearchOffsetRequestHeader.java @@ -36,6 +36,7 @@ public class SearchOffsetRequestHeader extends TopicQueueRequestHeader { @CFNotNull @RocketMQResource(ResourceType.TOPIC) private String topic; + private String liteTopic; @CFNotNull private Integer queueId; @CFNotNull @@ -58,6 +59,14 @@ public void setTopic(String topic) { this.topic = topic; } + public String getLiteTopic() { + return liteTopic; + } + + public void setLiteTopic(String liteTopic) { + this.liteTopic = liteTopic; + } + @Override public Integer getQueueId() { return queueId; @@ -89,6 +98,7 @@ public void setBoundaryType(BoundaryType boundaryType) { public String toString() { return MoreObjects.toStringHelper(this) .add("topic", topic) + .add("liteTopic", liteTopic) .add("queueId", queueId) .add("timestamp", timestamp) .add("boundaryType", boundaryType.getName()) diff --git a/remoting/src/main/java/org/apache/rocketmq/remoting/protocol/header/TriggerLiteDispatchRequestHeader.java b/remoting/src/main/java/org/apache/rocketmq/remoting/protocol/header/TriggerLiteDispatchRequestHeader.java new file mode 100644 index 00000000000..e39725e2074 --- /dev/null +++ b/remoting/src/main/java/org/apache/rocketmq/remoting/protocol/header/TriggerLiteDispatchRequestHeader.java @@ -0,0 +1,52 @@ +/* + * 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.remoting.protocol.header; + +import org.apache.rocketmq.common.resource.ResourceType; +import org.apache.rocketmq.common.resource.RocketMQResource; +import org.apache.rocketmq.remoting.CommandCustomHeader; +import org.apache.rocketmq.remoting.annotation.CFNotNull; +import org.apache.rocketmq.remoting.exception.RemotingCommandException; + +public class TriggerLiteDispatchRequestHeader implements CommandCustomHeader { + + @CFNotNull + @RocketMQResource(ResourceType.GROUP) + private String group; + + private String clientId; + + public String getGroup() { + return group; + } + + public void setGroup(String group) { + this.group = group; + } + + public String getClientId() { + return clientId; + } + + public void setClientId(String clientId) { + this.clientId = clientId; + } + + @Override + public void checkFields() throws RemotingCommandException { + } +} diff --git a/remoting/src/main/java/org/apache/rocketmq/remoting/protocol/heartbeat/MessageModel.java b/remoting/src/main/java/org/apache/rocketmq/remoting/protocol/heartbeat/MessageModel.java index 11f2e6c9ec4..194e7520e5b 100644 --- a/remoting/src/main/java/org/apache/rocketmq/remoting/protocol/heartbeat/MessageModel.java +++ b/remoting/src/main/java/org/apache/rocketmq/remoting/protocol/heartbeat/MessageModel.java @@ -31,7 +31,11 @@ public enum MessageModel { /** * clustering */ - CLUSTERING("CLUSTERING"); + CLUSTERING("CLUSTERING"), + /** + * for lite consumer + */ + LITE_SELECTIVE("LITE_SELECTIVE"); private String modeCN; diff --git a/remoting/src/main/java/org/apache/rocketmq/remoting/protocol/subscription/SubscriptionGroupConfig.java b/remoting/src/main/java/org/apache/rocketmq/remoting/protocol/subscription/SubscriptionGroupConfig.java index 2c3738a464f..fa8a9804f4e 100644 --- a/remoting/src/main/java/org/apache/rocketmq/remoting/protocol/subscription/SubscriptionGroupConfig.java +++ b/remoting/src/main/java/org/apache/rocketmq/remoting/protocol/subscription/SubscriptionGroupConfig.java @@ -21,10 +21,19 @@ import com.google.common.base.MoreObjects; import java.util.HashMap; import java.util.Map; +import java.util.Objects; import java.util.Set; import org.apache.commons.lang3.builder.EqualsBuilder; import org.apache.commons.lang3.math.NumberUtils; import org.apache.rocketmq.common.MixAll; +import org.apache.rocketmq.common.attribute.LiteSubModel; + +import static org.apache.rocketmq.common.SubscriptionGroupAttributes.LITE_SUB_CLIENT_MAX_EVENT_COUNT; +import static org.apache.rocketmq.common.SubscriptionGroupAttributes.LITE_SUB_CLIENT_QUOTA_ATTRIBUTE; +import static org.apache.rocketmq.common.SubscriptionGroupAttributes.LITE_SUB_MODEL_ATTRIBUTE; +import static org.apache.rocketmq.common.SubscriptionGroupAttributes.LITE_SUB_RESET_OFFSET_EXCLUSIVE_ATTRIBUTE; +import static org.apache.rocketmq.common.SubscriptionGroupAttributes.LITE_BIND_TOPIC_ATTRIBUTE; +import static org.apache.rocketmq.common.SubscriptionGroupAttributes.LITE_SUB_RESET_OFFSET_UNSUBSCRIBE_ATTRIBUTE; import static org.apache.rocketmq.common.SubscriptionGroupAttributes.PRIORITY_FACTOR_ATTRIBUTE; @@ -183,6 +192,58 @@ public long getPriorityFactor() { return NumberUtils.toLong(factorStr, PRIORITY_FACTOR_ATTRIBUTE.getDefaultValue()); } + @JSONField(serialize = false, deserialize = false) + public void setLiteBindTopic(String liteBindTopic) { + if (liteBindTopic != null) { + attributes.put(LITE_BIND_TOPIC_ATTRIBUTE.getName(), liteBindTopic); + } + } + + @JSONField(serialize = false, deserialize = false) + public String getLiteBindTopic() { + return attributes.get(LITE_BIND_TOPIC_ATTRIBUTE.getName()); + } + + @JSONField(serialize = false, deserialize = false) + public int getLiteSubClientQuota() { + long quota = LITE_SUB_CLIENT_QUOTA_ATTRIBUTE.getDefaultValue(); + String quotaStr = attributes.get(LITE_SUB_CLIENT_QUOTA_ATTRIBUTE.getName()); + if (quotaStr != null) { + quota = Long.parseLong(quotaStr); + } + return Math.toIntExact(quota); + } + + @JSONField(serialize = false, deserialize = false) + public boolean isLiteSubExclusive() { + String subLiteModel = attributes.get(LITE_SUB_MODEL_ATTRIBUTE.getName()); + return Objects.equals(LiteSubModel.Exclusive.name(), subLiteModel); + } + + /** + * Whether to reset offset in exclusive mode + */ + @JSONField(serialize = false, deserialize = false) + public boolean isResetOffsetInExclusiveMode() { + String boolStr = attributes.get(LITE_SUB_RESET_OFFSET_EXCLUSIVE_ATTRIBUTE.getName()); + return Boolean.parseBoolean(boolStr); + } + + @JSONField(serialize = false, deserialize = false) + public boolean isResetOffsetOnUnsubscribe() { + String boolStr = attributes.get(LITE_SUB_RESET_OFFSET_UNSUBSCRIBE_ATTRIBUTE.getName()); + return Boolean.parseBoolean(boolStr); + } + + @JSONField(serialize = false, deserialize = false) + public int getMaxClientEventCount() { + String content = attributes.get(LITE_SUB_CLIENT_MAX_EVENT_COUNT.getName()); + if (content == null) { + return -1; + } + return NumberUtils.toInt(content, -1); + } + @Override public int hashCode() { final int prime = 31; 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 aae6d50da97..0dbb207af68 100644 --- a/store/src/main/java/org/apache/rocketmq/store/DefaultMessageStore.java +++ b/store/src/main/java/org/apache/rocketmq/store/DefaultMessageStore.java @@ -1528,7 +1528,9 @@ public int deleteTopics(final Set deleteTopics) { } if (this.brokerConfig.isAutoDeleteUnusedStats()) { - this.brokerStatsManager.onTopicDeleted(topic); + if (!MixAll.isLmq(topic)) { + this.brokerStatsManager.onTopicDeleted(topic); + } } // destroy consume queue dir diff --git a/store/src/main/java/org/apache/rocketmq/store/config/MessageStoreConfig.java b/store/src/main/java/org/apache/rocketmq/store/config/MessageStoreConfig.java index ad773192644..65dba5390dd 100644 --- a/store/src/main/java/org/apache/rocketmq/store/config/MessageStoreConfig.java +++ b/store/src/main/java/org/apache/rocketmq/store/config/MessageStoreConfig.java @@ -282,6 +282,7 @@ public class MessageStoreConfig { private boolean enableLmq = false; private boolean enableMultiDispatch = false; private int maxLmqConsumeQueueNum = 20000; + private boolean enableLmqQuota = false; private boolean enableScheduleAsyncDeliver = false; private int scheduleAsyncDeliverMaxPendingLimit = 2000; @@ -1654,6 +1655,14 @@ public void setMaxLmqConsumeQueueNum(int maxLmqConsumeQueueNum) { this.maxLmqConsumeQueueNum = maxLmqConsumeQueueNum; } + public boolean isEnableLmqQuota() { + return enableLmqQuota; + } + + public void setEnableLmqQuota(boolean enableLmqQuota) { + this.enableLmqQuota = enableLmqQuota; + } + public boolean isEnableScheduleAsyncDeliver() { return enableScheduleAsyncDeliver; } diff --git a/store/src/main/java/org/apache/rocketmq/store/queue/CombineConsumeQueueStore.java b/store/src/main/java/org/apache/rocketmq/store/queue/CombineConsumeQueueStore.java index 5de1664cdbf..ffb0851e0d6 100644 --- a/store/src/main/java/org/apache/rocketmq/store/queue/CombineConsumeQueueStore.java +++ b/store/src/main/java/org/apache/rocketmq/store/queue/CombineConsumeQueueStore.java @@ -390,6 +390,16 @@ public long getTotalSize() { return result; } + @Override + public int getLmqNum() { + return currentReadStore.getLmqNum(); + } + + @Override + public boolean isLmqExist(String lmqTopic) { + return currentReadStore.isLmqExist(lmqTopic); + } + public RocksDBConsumeQueueStore getRocksDBConsumeQueueStore() { return rocksDBConsumeQueueStore; } diff --git a/store/src/main/java/org/apache/rocketmq/store/queue/ConsumeQueueStore.java b/store/src/main/java/org/apache/rocketmq/store/queue/ConsumeQueueStore.java index a91fc2e57bb..e9b0312c01c 100644 --- a/store/src/main/java/org/apache/rocketmq/store/queue/ConsumeQueueStore.java +++ b/store/src/main/java/org/apache/rocketmq/store/queue/ConsumeQueueStore.java @@ -33,7 +33,10 @@ import java.util.concurrent.FutureTask; import java.util.concurrent.LinkedBlockingQueue; import java.util.concurrent.TimeUnit; +import java.util.concurrent.atomic.AtomicInteger; + import org.apache.rocketmq.common.BoundaryType; +import org.apache.rocketmq.common.MixAll; import org.apache.rocketmq.common.ServiceThread; import org.apache.rocketmq.common.ThreadFactoryImpl; import org.apache.rocketmq.common.TopicConfig; @@ -61,6 +64,7 @@ public class ConsumeQueueStore extends AbstractConsumeQueueStore { private long dispatchFromPhyOffset; private long dispatchFromStoreTimestamp; + private final AtomicInteger lmqCounter = new AtomicInteger(0); public ConsumeQueueStore(DefaultMessageStore messageStore) { super(messageStore); @@ -336,6 +340,9 @@ public void flush() throws StoreException { public void destroy(ConsumeQueueInterface consumeQueue) { FileQueueLifeCycle fileQueueLifeCycle = getLifeCycle(consumeQueue.getTopic(), consumeQueue.getQueueId()); fileQueueLifeCycle.destroy(); + if (MixAll.isLmq(consumeQueue.getTopic())) { + lmqCounter.decrementAndGet(); + } } public int deleteExpiredFile(ConsumeQueueInterface consumeQueue, long minCommitLogPos) { @@ -412,6 +419,9 @@ public ConsumeQueueInterface findOrCreateConsumeQueue(String topic, int queueId) logic = oldLogic; } else { logic = newLogic; + if (MixAll.isLmq(topic)) { + lmqCounter.incrementAndGet(); + } } return logic; @@ -441,8 +451,14 @@ private void putConsumeQueue(final String topic, final int queueId, final Consum map = new ConcurrentHashMap<>(); map.put(queueId, consumeQueue); this.consumeQueueTable.put(topic, map); + if (MixAll.isLmq(topic)) { + lmqCounter.incrementAndGet(); + } } else { - map.put(queueId, consumeQueue); + ConsumeQueueInterface prev = map.put(queueId, consumeQueue); + if (null == prev && MixAll.isLmq(topic)) { + lmqCounter.incrementAndGet(); + } } } @@ -603,6 +619,16 @@ public boolean isMappedFileMatchedRecover(long phyOffset, long storeTimestamp, b } } + @Override + public int getLmqNum() { + return lmqCounter.get(); + } + + @Override + public boolean isLmqExist(String lmqTopic) { + return getConsumeQueue(lmqTopic, 0) != null; + } + public class FlushConsumeQueueService extends ServiceThread { private static final int RETRY_TIMES_OVER = 3; private long lastFlushTimestamp = 0; diff --git a/store/src/main/java/org/apache/rocketmq/store/queue/ConsumeQueueStoreInterface.java b/store/src/main/java/org/apache/rocketmq/store/queue/ConsumeQueueStoreInterface.java index c6dfa8a2a70..d3f1f24612f 100644 --- a/store/src/main/java/org/apache/rocketmq/store/queue/ConsumeQueueStoreInterface.java +++ b/store/src/main/java/org/apache/rocketmq/store/queue/ConsumeQueueStoreInterface.java @@ -205,4 +205,17 @@ boolean isMappedFileMatchedRecover(long phyOffset, long storeTimestamp, */ long getTotalSize(); + /** + * get lmq consume queue count + * @return the count of lmq + */ + int getLmqNum(); + + /** + * Check if the LMQ exists, this is different from getConsumeQueue() + * @param lmqTopic + * @return exist or not + */ + boolean isLmqExist(String lmqTopic); + } diff --git a/store/src/main/java/org/apache/rocketmq/store/queue/RocksDBConsumeQueueOffsetTable.java b/store/src/main/java/org/apache/rocketmq/store/queue/RocksDBConsumeQueueOffsetTable.java index da898cf78bd..dc3712663c7 100644 --- a/store/src/main/java/org/apache/rocketmq/store/queue/RocksDBConsumeQueueOffsetTable.java +++ b/store/src/main/java/org/apache/rocketmq/store/queue/RocksDBConsumeQueueOffsetTable.java @@ -28,6 +28,7 @@ import java.util.Set; import java.util.concurrent.ConcurrentHashMap; import java.util.concurrent.ConcurrentMap; +import java.util.concurrent.atomic.AtomicInteger; import java.util.function.Consumer; import java.util.function.Function; import org.apache.rocketmq.common.MixAll; @@ -130,6 +131,7 @@ public class RocksDBConsumeQueueOffsetTable { */ private final ConcurrentMap topicQueueMinOffset; private final ConcurrentMap topicQueueMaxCqOffset; + private final AtomicInteger lmqCounter = new AtomicInteger(0); public RocksDBConsumeQueueOffsetTable(RocksDBConsumeQueueTable rocksDBConsumeQueueTable, ConsumeQueueRocksDBStorage rocksDBStorage, DefaultMessageStore messageStore) { @@ -164,13 +166,18 @@ public Set scanAllQueueIdInTopic(String topic) throws RocksDBException } private void loadMaxConsumeQueueOffsets() { + lmqCounter.set(0); Function predicate = entry -> entry.type == OffsetEntryType.MAXIMUM; Consumer fn = entry -> { topicQueueMaxCqOffset.putIfAbsent(entry.topic + "-" + entry.queueId, entry.offset); + if (MixAll.isLmq(entry.topic)) { + lmqCounter.incrementAndGet(); + } log.info("LoadMaxConsumeQueueOffsets Max {}:{} --> {}|{}", entry.topic, entry.queueId, entry.offset, entry.commitLogOffset); }; try { forEach(predicate, fn); + log.info("lmq count from maxConsumeQueueOffset table. {}", lmqCounter.get()); } catch (RocksDBException e) { log.error("Failed to maximum consume queue offset", e); } @@ -567,6 +574,14 @@ private void putHeapMaxCqOffset(final String topic, final int queueId, final lon ERROR_LOG.error("Max offset of consume-queue[topic={}, queue-id={}] regressed. prev-max={}, current-max={}", topic, queueId, prev, maxOffset); } + if (prev != null && prev == -1 && MixAll.isLmq(topic)) { + lmqCounter.incrementAndGet(); + } + if (null == prev && MixAll.isLmq(topic)) { + // this usually happens when broker exits abnormally, do nothing here and wait for the next scan to delete it. + ERROR_LOG.error("probably recover a lmq which was already deleted. lmq:{}, maxOffset:{}", topic, maxOffset); + lmqCounter.incrementAndGet(); + } } private PhyAndCQOffset getHeapMinOffset(final String topic, final int queueId) { @@ -583,7 +598,11 @@ private PhyAndCQOffset removeHeapMinCqOffset(String topicQueueId) { } private Long removeHeapMaxCqOffset(String topicQueueId) { - return this.topicQueueMaxCqOffset.remove(topicQueueId); + Long prev = this.topicQueueMaxCqOffset.remove(topicQueueId); + if (prev != null && topicQueueId.startsWith(MixAll.LMQ_PREFIX)) { + lmqCounter.decrementAndGet(); + } + return prev; } public void updateCqOffset(final String topic, final int queueId, final long phyOffset, @@ -616,6 +635,14 @@ public void updateCqOffset(final String topic, final int queueId, final long phy } } + public int getLmqNum() { + return lmqCounter.get(); + } + + public boolean isLmqExist(String lmqTopic) { + return this.topicQueueMaxCqOffset.containsKey(buildTopicQueueId(lmqTopic, 0)); + } + private boolean correctMaxCqOffset(final String topic, final int queueId, final long maxCQOffset, final long maxPhyOffsetInCQ) throws RocksDBException { // 'getMinOffsetInQueue' may correct minCqOffset and put it into heap diff --git a/store/src/main/java/org/apache/rocketmq/store/queue/RocksDBConsumeQueueStore.java b/store/src/main/java/org/apache/rocketmq/store/queue/RocksDBConsumeQueueStore.java index 84ac6833fce..299f4458d92 100644 --- a/store/src/main/java/org/apache/rocketmq/store/queue/RocksDBConsumeQueueStore.java +++ b/store/src/main/java/org/apache/rocketmq/store/queue/RocksDBConsumeQueueStore.java @@ -578,6 +578,16 @@ public Long getMaxOffset(String topic, int queueId) throws ConsumeQueueException return super.getMaxOffset(topic, queueId); } + @Override + public int getLmqNum() { + return this.rocksDBConsumeQueueOffsetTable.getLmqNum(); + } + + @Override + public boolean isLmqExist(String lmqTopic) { + return MixAll.isLmq(lmqTopic) ? this.rocksDBConsumeQueueOffsetTable.isLmqExist(lmqTopic) : false; + } + public boolean isStopped() { return ServiceState.SHUTDOWN_ALREADY == serviceState.get(); } diff --git a/store/src/test/java/org/apache/rocketmq/store/ConsumeQueueTest.java b/store/src/test/java/org/apache/rocketmq/store/ConsumeQueueTest.java index 937e8b99558..00fbe60a3c1 100644 --- a/store/src/test/java/org/apache/rocketmq/store/ConsumeQueueTest.java +++ b/store/src/test/java/org/apache/rocketmq/store/ConsumeQueueTest.java @@ -32,6 +32,7 @@ import org.apache.rocketmq.common.BrokerConfig; import org.apache.rocketmq.common.MixAll; import org.apache.rocketmq.common.UtilAll; +import org.apache.rocketmq.common.message.MessageAccessor; import org.apache.rocketmq.common.message.MessageConst; import org.apache.rocketmq.common.message.MessageDecoder; import org.apache.rocketmq.common.message.MessageExtBrokerInner; @@ -221,7 +222,7 @@ private MessageExtBrokerInner buildMessageMultiQueue() { msg.setStoreHost(storeHost); msg.setBornHost(bornHost); for (int i = 0; i < 1; i++) { - msg.putUserProperty(MessageConst.PROPERTY_INNER_MULTI_DISPATCH, "%LMQ%123,%LMQ%456"); + MessageAccessor.putProperty(msg, MessageConst.PROPERTY_INNER_MULTI_DISPATCH, "%LMQ%123,%LMQ%456"); msg.putUserProperty(String.valueOf(i), "imagoodperson" + i); } msg.setPropertiesString(MessageDecoder.messageProperties2String(msg.getProperties())); diff --git a/store/src/test/java/org/apache/rocketmq/store/queue/ConsumeQueueStoreTest.java b/store/src/test/java/org/apache/rocketmq/store/queue/ConsumeQueueStoreTest.java index 59e1d08791f..ca059cec845 100644 --- a/store/src/test/java/org/apache/rocketmq/store/queue/ConsumeQueueStoreTest.java +++ b/store/src/test/java/org/apache/rocketmq/store/queue/ConsumeQueueStoreTest.java @@ -16,14 +16,23 @@ */ package org.apache.rocketmq.store.queue; +import java.util.List; import java.util.concurrent.ConcurrentHashMap; import java.util.concurrent.ConcurrentMap; + +import com.google.common.collect.Sets; +import org.apache.rocketmq.common.MixAll; import org.apache.rocketmq.common.TopicConfig; import org.apache.rocketmq.common.UtilAll; import org.apache.rocketmq.common.attribute.CQType; +import org.apache.rocketmq.common.message.MessageAccessor; +import org.apache.rocketmq.common.message.MessageConst; +import org.apache.rocketmq.common.message.MessageExtBrokerInner; +import org.apache.rocketmq.store.LmqDispatch; import org.apache.rocketmq.store.MessageStore; import org.apache.rocketmq.store.PutMessageResult; import org.apache.rocketmq.store.PutMessageStatus; +import org.apache.rocketmq.store.exception.ConsumeQueueException; import org.junit.After; import org.junit.Assert; import org.junit.Before; @@ -31,17 +40,19 @@ import java.io.File; import java.util.UUID; +import java.util.stream.IntStream; import static java.util.concurrent.TimeUnit.SECONDS; import static org.awaitility.Awaitility.await; import static org.junit.Assert.assertEquals; +import static org.junit.Assert.assertNotNull; +import static org.junit.Assert.assertNull; public class ConsumeQueueStoreTest extends QueueTestBase { + private MessageStore messageStore; private ConcurrentMap topicConfigTableMap; - - @Before public void init() throws Exception { this.topicConfigTableMap = new ConcurrentHashMap<>(); @@ -106,4 +117,68 @@ public void testLoadBatchConsumeQueuesWithWrongAttribute() { Assert.assertTrue(runtimeException.getMessage().endsWith("should be BatchCQ, but is SimpleCQ")); } + @Test + public void testLmqCounter_running() throws ConsumeQueueException { + messageStore.getMessageStoreConfig().setEnableMultiDispatch(true); + messageStore.getMessageStoreConfig().setEnableLmq(true); + messageStore.getMessageStoreConfig().setEnableCompaction(false); + int num = 5; + String topic = "topic"; + List lmqNameList = IntStream.range(0, num) + .mapToObj(i -> MixAll.LMQ_PREFIX + UUID.randomUUID()) + .collect(java.util.stream.Collectors.toList()); + assertEquals(0, messageStore.getQueueStore().getLmqNum()); + + lmqNameList.forEach(lmqName -> assertNull(messageStore.getConsumeQueue(lmqName, 0))); + assertEquals(0, messageStore.getQueueStore().getLmqNum()); + + for (String lmqName : lmqNameList) { + MessageExtBrokerInner message = buildMessage(topic, -1); + MessageAccessor.putProperty(message, MessageConst.PROPERTY_INNER_MULTI_DISPATCH, lmqName); + LmqDispatch.wrapLmqDispatch(messageStore, message); + PutMessageResult putMessageResult = messageStore.putMessage(message); + assertEquals(PutMessageStatus.PUT_OK, putMessageResult.getPutMessageStatus()); + } + await().atMost(5, SECONDS).until(fullyDispatched(messageStore)); + + lmqNameList.forEach(lmqName -> assertNotNull(messageStore.getConsumeQueue(lmqName, 0))); + assertEquals(num, messageStore.getQueueStore().getLmqNum()); + + lmqNameList.forEach(lmqName -> messageStore.deleteTopics(Sets.newHashSet(lmqName))); + assertEquals(0, messageStore.getQueueStore().getLmqNum()); + } + + @Test + public void testLmqCounter_reload() throws Exception { + messageStore.getMessageStoreConfig().setEnableMultiDispatch(true); + messageStore.getMessageStoreConfig().setEnableLmq(true); + int num = 5; + String topic = "topic"; + List lmqNameList = IntStream.range(0, num) + .mapToObj(i -> MixAll.LMQ_PREFIX + UUID.randomUUID()) + .collect(java.util.stream.Collectors.toList()); + assertEquals(0, messageStore.getQueueStore().getLmqNum()); + + for (String lmqName : lmqNameList) { + MessageExtBrokerInner message = buildMessage(topic, -1); + MessageAccessor.putProperty(message, MessageConst.PROPERTY_INNER_MULTI_DISPATCH, lmqName); + LmqDispatch.wrapLmqDispatch(messageStore, message); + PutMessageResult putMessageResult = messageStore.putMessage(message); + assertEquals(PutMessageStatus.PUT_OK, putMessageResult.getPutMessageStatus()); + } + await().atMost(5, SECONDS).until(fullyDispatched(messageStore)); + assertEquals(num, messageStore.getQueueStore().getLmqNum()); + messageStore.shutdown(); + + // create new one based on current store + MessageStore newStore = createMessageStore(messageStore.getMessageStoreConfig().getStorePathRootDir(), + true, topicConfigTableMap, messageStore.getMessageStoreConfig()); + newStore.load(); + newStore.start(); + + assertEquals(num, newStore.getQueueStore().getLmqNum()); + lmqNameList.forEach(lmqName -> assertNotNull(newStore.getConsumeQueue(lmqName, 0))); + newStore.shutdown(); + } + } diff --git a/store/src/test/java/org/apache/rocketmq/store/queue/QueueTestBase.java b/store/src/test/java/org/apache/rocketmq/store/queue/QueueTestBase.java index df3da24ccb0..92d89e6beec 100644 --- a/store/src/test/java/org/apache/rocketmq/store/queue/QueueTestBase.java +++ b/store/src/test/java/org/apache/rocketmq/store/queue/QueueTestBase.java @@ -34,6 +34,7 @@ import org.apache.rocketmq.store.ConsumeQueue; import org.apache.rocketmq.store.DefaultMessageStore; import org.apache.rocketmq.store.MessageStore; +import org.apache.rocketmq.store.RocksDBMessageStore; import org.apache.rocketmq.store.StoreTestBase; import org.apache.rocketmq.store.config.MessageStoreConfig; import org.apache.rocketmq.store.stats.BrokerStatsManager; @@ -85,12 +86,24 @@ protected MessageStore createMessageStore(String baseDir, boolean extent, messageStoreConfig.setFlushIntervalCommitLog(1); messageStoreConfig.setFlushCommitLogThoroughInterval(2); - return new DefaultMessageStore( - messageStoreConfig, - new BrokerStatsManager("simpleTest", true), - (topic, queueId, logicOffset, tagsCode, msgStoreTime, filterBitMap, properties) -> { - }, - new BrokerConfig(), topicConfigTable); + + MessageStore messageStore; + if (messageStoreConfig.isEnableRocksDBStore()) { + messageStore = new RocksDBMessageStore( + messageStoreConfig, + new BrokerStatsManager("simpleTest", true), + (topic, queueId, logicOffset, tagsCode, msgStoreTime, filterBitMap, properties) -> { + }, + new BrokerConfig(), topicConfigTable); + } else { + messageStore = new DefaultMessageStore( + messageStoreConfig, + new BrokerStatsManager("simpleTest", true), + (topic, queueId, logicOffset, tagsCode, msgStoreTime, filterBitMap, properties) -> { + }, + new BrokerConfig(), topicConfigTable); + } + return messageStore; } public MessageExtBrokerInner buildMessage(String topic, int batchNum) { diff --git a/store/src/test/java/org/apache/rocketmq/store/queue/RocksDBConsumeQueueOffsetTableTest.java b/store/src/test/java/org/apache/rocketmq/store/queue/RocksDBConsumeQueueOffsetTableTest.java index b1e12d49468..b8f415537e1 100644 --- a/store/src/test/java/org/apache/rocketmq/store/queue/RocksDBConsumeQueueOffsetTableTest.java +++ b/store/src/test/java/org/apache/rocketmq/store/queue/RocksDBConsumeQueueOffsetTableTest.java @@ -21,7 +21,10 @@ import java.io.IOException; import java.nio.ByteBuffer; import java.nio.charset.StandardCharsets; +import java.util.UUID; import java.util.concurrent.atomic.AtomicBoolean; + +import org.apache.rocketmq.common.MixAll; import org.apache.rocketmq.store.DefaultMessageStore; import org.apache.rocketmq.store.queue.offset.OffsetEntryType; import org.apache.rocketmq.store.rocksdb.ConsumeQueueRocksDBStorage; @@ -74,26 +77,7 @@ public static void initDB() throws IOException, RocksDBException { topicBuilder.append("topic"); } topicName = topicBuilder.toString(); - byte[] topicInBytes = topicName.getBytes(StandardCharsets.UTF_8); - - ByteBuffer keyBuffer = ByteBuffer.allocateDirect(RocksDBConsumeQueueOffsetTable.OFFSET_KEY_LENGTH_WITHOUT_TOPIC_BYTES + topicInBytes.length); - RocksDBConsumeQueueOffsetTable.buildOffsetKeyByteBuffer(keyBuffer, topicInBytes, 1, true); - Assert.assertEquals(0, keyBuffer.position()); - Assert.assertEquals(RocksDBConsumeQueueOffsetTable.OFFSET_KEY_LENGTH_WITHOUT_TOPIC_BYTES + topicInBytes.length, keyBuffer.limit()); - - ByteBuffer valueBuffer = ByteBuffer.allocateDirect(Long.BYTES + Long.BYTES); - valueBuffer.putLong(100); - valueBuffer.putLong(2); - valueBuffer.flip(); - - try (WriteBatch writeBatch = new WriteBatch(); - WriteOptions writeOptions = new WriteOptions()) { - writeOptions.setDisableWAL(false); - writeOptions.setSync(true); - writeBatch.put(keyBuffer, valueBuffer); - db.write(writeOptions, writeBatch); - } - + writeOffset(topicName, 1, 100, 2, true); } @AfterClass @@ -128,4 +112,51 @@ public void testForEach() throws RocksDBException { }); Assert.assertTrue(called.get()); } + + @Test + public void testLmqCounter() throws RocksDBException { + Assert.assertEquals(0, offsetTable.getLmqNum()); + offsetTable.load(); + int initCount = offsetTable.getLmqNum(); + int lmqCount = 2; + int repeatCount = 3; + for (int i = 0; i < lmqCount; i++) { + String lmqName = MixAll.LMQ_PREFIX + UUID.randomUUID(); + String normalTopic = UUID.randomUUID().toString(); + for (int j = 0; j < repeatCount; j++) { + writeOffset(lmqName, 0, 100, j, true); + writeOffset(lmqName, 0, 100, j, false); + writeOffset(normalTopic, 0, 100, j, true); + writeOffset(normalTopic, 0, 100, j, false); + } + } + + Mockito.doReturn(db.newIterator()).when(rocksDBStorage).seekOffsetCF(); + offsetTable.load(); + Assert.assertEquals(initCount + lmqCount, offsetTable.getLmqNum()); + } + + private static void writeOffset(String topic, int queueId, long phyOffset, + long cqOffset, boolean max) throws RocksDBException { + byte[] topicInBytes = topic.getBytes(StandardCharsets.UTF_8); + ByteBuffer keyBuffer = ByteBuffer.allocateDirect( + RocksDBConsumeQueueOffsetTable.OFFSET_KEY_LENGTH_WITHOUT_TOPIC_BYTES + topicInBytes.length); + RocksDBConsumeQueueOffsetTable.buildOffsetKeyByteBuffer(keyBuffer, topicInBytes, 1, max); + Assert.assertEquals(0, keyBuffer.position()); + Assert.assertEquals(RocksDBConsumeQueueOffsetTable.OFFSET_KEY_LENGTH_WITHOUT_TOPIC_BYTES + + topicInBytes.length, keyBuffer.limit()); + + ByteBuffer valueBuffer = ByteBuffer.allocateDirect(Long.BYTES + Long.BYTES); + valueBuffer.putLong(phyOffset); + valueBuffer.putLong(cqOffset); + valueBuffer.flip(); + + try (WriteBatch writeBatch = new WriteBatch(); + WriteOptions writeOptions = new WriteOptions()) { + writeOptions.setDisableWAL(false); + writeOptions.setSync(true); + writeBatch.put(keyBuffer, valueBuffer); + db.write(writeOptions, writeBatch); + } + } } diff --git a/store/src/test/java/org/apache/rocketmq/store/queue/RocksDBConsumeQueueTest.java b/store/src/test/java/org/apache/rocketmq/store/queue/RocksDBConsumeQueueTest.java index acf4a6a8023..702d91fb075 100644 --- a/store/src/test/java/org/apache/rocketmq/store/queue/RocksDBConsumeQueueTest.java +++ b/store/src/test/java/org/apache/rocketmq/store/queue/RocksDBConsumeQueueTest.java @@ -16,16 +16,42 @@ */ package org.apache.rocketmq.store.queue; +import java.io.File; import java.nio.ByteBuffer; +import java.util.List; +import java.util.UUID; +import java.util.concurrent.ConcurrentHashMap; +import java.util.concurrent.ConcurrentMap; +import java.util.stream.IntStream; + +import com.google.common.collect.Sets; import org.apache.rocketmq.common.MixAll; +import org.apache.rocketmq.common.TopicConfig; +import org.apache.rocketmq.common.UtilAll; +import org.apache.rocketmq.common.message.MessageAccessor; +import org.apache.rocketmq.common.message.MessageConst; +import org.apache.rocketmq.common.message.MessageExtBrokerInner; import org.apache.rocketmq.store.DefaultMessageStore; +import org.apache.rocketmq.store.LmqDispatch; +import org.apache.rocketmq.store.MessageStore; +import org.apache.rocketmq.store.PutMessageResult; +import org.apache.rocketmq.store.PutMessageStatus; +import org.apache.rocketmq.store.StoreType; +import org.apache.rocketmq.store.config.MessageStoreConfig; +import org.apache.rocketmq.store.exception.ConsumeQueueException; +import org.junit.After; +import org.junit.Before; import org.junit.Test; import org.mockito.invocation.InvocationOnMock; import org.mockito.stubbing.Answer; +import static java.util.concurrent.TimeUnit.SECONDS; import static org.apache.rocketmq.store.queue.RocksDBConsumeQueueTable.CQ_UNIT_SIZE; +import static org.awaitility.Awaitility.await; import static org.junit.Assert.assertEquals; import static org.junit.Assert.assertFalse; +import static org.junit.Assert.assertNotNull; +import static org.junit.Assert.assertNull; import static org.junit.Assert.assertTrue; import static org.mockito.ArgumentMatchers.anyInt; import static org.mockito.ArgumentMatchers.anyLong; @@ -35,6 +61,30 @@ public class RocksDBConsumeQueueTest extends QueueTestBase { + private MessageStore messageStore; + private ConcurrentMap topicConfigTableMap; + + @Before + public void init() throws Exception { + MessageStoreConfig storeConfig = new MessageStoreConfig(); + storeConfig.setStoreType(StoreType.DEFAULT_ROCKSDB.getStoreType()); + storeConfig.setEnableCompaction(false); + this.topicConfigTableMap = new ConcurrentHashMap<>(); + + messageStore = createMessageStore(null, true, topicConfigTableMap, storeConfig); + messageStore.load(); + messageStore.start(); + } + + @After + public void destroy() { + messageStore.shutdown(); + messageStore.destroy(); + + File file = new File(messageStore.getMessageStoreConfig().getStorePathRootDir()); + UtilAll.deleteFile(file); + } + @Test public void testIterator() throws Exception { if (MixAll.isMac()) { @@ -69,4 +119,67 @@ public ByteBuffer answer(InvocationOnMock mock) throws Throwable { } assertFalse(it.hasNext()); } + + @Test + public void testLmqCounter_running() throws ConsumeQueueException { + messageStore.getMessageStoreConfig().setEnableMultiDispatch(true); + messageStore.getMessageStoreConfig().setEnableLmq(true); + int num = 5; + String topic = "topic"; + List lmqNameList = IntStream.range(0, num) + .mapToObj(i -> MixAll.LMQ_PREFIX + UUID.randomUUID()) + .collect(java.util.stream.Collectors.toList()); + assertEquals(0, messageStore.getQueueStore().getLmqNum()); + + lmqNameList.forEach(lmqName -> assertNotNull(messageStore.getConsumeQueue(lmqName, 0))); // create if not exist + assertEquals(0, messageStore.getQueueStore().getLmqNum()); + + for (String lmqName : lmqNameList) { + MessageExtBrokerInner message = buildMessage(topic, -1); + MessageAccessor.putProperty(message, MessageConst.PROPERTY_INNER_MULTI_DISPATCH, lmqName); + LmqDispatch.wrapLmqDispatch(messageStore, message); + PutMessageResult putMessageResult = messageStore.putMessage(message); + assertEquals(PutMessageStatus.PUT_OK, putMessageResult.getPutMessageStatus()); + } + await().atMost(5, SECONDS).until(fullyDispatched(messageStore)); + + lmqNameList.forEach(lmqName -> assertNotNull(messageStore.getConsumeQueue(lmqName, 0))); + assertEquals(num, messageStore.getQueueStore().getLmqNum()); + + lmqNameList.forEach(lmqName -> messageStore.deleteTopics(Sets.newHashSet(lmqName))); + assertEquals(0, messageStore.getQueueStore().getLmqNum()); + } + + @Test + public void testLmqCounter_reload() throws Exception { + messageStore.getMessageStoreConfig().setEnableMultiDispatch(true); + messageStore.getMessageStoreConfig().setEnableLmq(true); + int num = 5; + String topic = "topic"; + List lmqNameList = IntStream.range(0, num) + .mapToObj(i -> MixAll.LMQ_PREFIX + UUID.randomUUID()) + .collect(java.util.stream.Collectors.toList()); + assertEquals(0, messageStore.getQueueStore().getLmqNum()); + + for (String lmqName : lmqNameList) { + MessageExtBrokerInner message = buildMessage(topic, -1); + MessageAccessor.putProperty(message, MessageConst.PROPERTY_INNER_MULTI_DISPATCH, lmqName); + LmqDispatch.wrapLmqDispatch(messageStore, message); + PutMessageResult putMessageResult = messageStore.putMessage(message); + assertEquals(PutMessageStatus.PUT_OK, putMessageResult.getPutMessageStatus()); + } + await().atMost(5, SECONDS).until(fullyDispatched(messageStore)); + assertEquals(num, messageStore.getQueueStore().getLmqNum()); + messageStore.shutdown(); + + // create new one based on current store + MessageStore newStore = createMessageStore(messageStore.getMessageStoreConfig().getStorePathRootDir(), + true, topicConfigTableMap, messageStore.getMessageStoreConfig()); + newStore.load(); + newStore.start(); + + assertEquals(num, newStore.getQueueStore().getLmqNum()); + lmqNameList.forEach(lmqName -> assertNull(newStore.getQueueStore().getConsumeQueueTable().get(lmqName))); // not in consumeQueueTable + newStore.shutdown(); + } } \ No newline at end of file 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 cfcb9896497..4b623325258 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 @@ -25,8 +25,11 @@ import java.util.Map; import java.util.Objects; import java.util.UUID; +import java.util.concurrent.ThreadPoolExecutor; import java.util.concurrent.TimeUnit; import java.util.concurrent.atomic.AtomicInteger; +import io.grpc.protobuf.services.ChannelzService; +import io.grpc.protobuf.services.ProtoReflectionService; import org.apache.rocketmq.broker.BrokerController; import org.apache.rocketmq.client.exception.MQClientException; import org.apache.rocketmq.common.BrokerConfig; @@ -35,14 +38,28 @@ import org.apache.rocketmq.common.attribute.CQType; import org.apache.rocketmq.common.attribute.TopicMessageType; import org.apache.rocketmq.common.namesrv.NamesrvConfig; +import org.apache.rocketmq.common.thread.ThreadPoolMonitor; +import org.apache.rocketmq.common.utils.AbstractStartAndShutdown; +import org.apache.rocketmq.common.utils.StartAndShutdown; import org.apache.rocketmq.logging.org.slf4j.Logger; import org.apache.rocketmq.logging.org.slf4j.LoggerFactory; import org.apache.rocketmq.namesrv.NamesrvController; +import org.apache.rocketmq.proxy.ProxyMode; +import org.apache.rocketmq.proxy.config.ConfigurationManager; +import org.apache.rocketmq.proxy.config.ProxyConfig; +import org.apache.rocketmq.proxy.grpc.GrpcServer; +import org.apache.rocketmq.proxy.grpc.GrpcServerBuilder; +import org.apache.rocketmq.proxy.grpc.v2.GrpcMessagingApplication; +import org.apache.rocketmq.proxy.processor.DefaultMessagingProcessor; +import org.apache.rocketmq.proxy.processor.MessagingProcessor; +import org.apache.rocketmq.proxy.service.cert.TlsCertificateManager; import org.apache.rocketmq.remoting.netty.NettyClientConfig; import org.apache.rocketmq.remoting.netty.NettyServerConfig; import org.apache.rocketmq.store.config.MessageStoreConfig; import org.apache.rocketmq.test.util.MQAdminTestUtils; +import static org.apache.rocketmq.test.base.BaseConf.brokerController1; + public class IntegrationTestBase { public static Logger logger = LoggerFactory.getLogger(IntegrationTestBase.class); @@ -142,6 +159,7 @@ public static BrokerController createAndStartBroker(String nsAddr) { storeConfig.setEnableConsumeQueueExt(true); brokerConfig.setLoadBalancePollNameServerInterval(500); brokerConfig.setPopConsumerKVServiceInit(true); + brokerConfig.setConfigManagerVersion(System.getProperty("configManagerVersion", "v1")); storeConfig.setStorePathRootDir(baseDir); storeConfig.setStorePathCommitLog(baseDir + SEP + "commitlog"); storeConfig.setMappedFileSizeCommitLog(commitLogSize); @@ -150,9 +168,67 @@ public static BrokerController createAndStartBroker(String nsAddr) { storeConfig.setDeleteWhen("01;02;03;04;05;06;07;08;09;10;11;12;13;14;15;16;17;18;19;20;21;22;23;00"); storeConfig.setMaxTransferCountOnMessageInMemory(1024); storeConfig.setMaxTransferCountOnMessageInDisk(1024); + storeConfig.setEnableLmq(Boolean.valueOf(System.getProperty("enableLmq", "false"))); + storeConfig.setEnableMultiDispatch(Boolean.valueOf(System.getProperty("enableMultiDispatch", "false"))); + storeConfig.setStoreType(System.getProperty("storeType", "default")); return createAndStartBroker(storeConfig, brokerConfig); } + public static void createAndStartProxy(String nsAddr) { + try { + ProxyStartAndShutdown startAndShutdown = new ProxyStartAndShutdown(); + ConfigurationManager.initConfig(); + ProxyConfig config = ConfigurationManager.getProxyConfig(); + config.setNamesrvAddr(nsAddr); + config.setEnableTopicMessageTypeCheck(false); + ThreadPoolExecutor executor = ThreadPoolMonitor.createAndMonitor( + config.getGrpcThreadPoolNums(), + config.getGrpcThreadPoolNums(), + 1, TimeUnit.MINUTES, + "GrpcRequestExecutorThread", + config.getGrpcThreadPoolQueueCapacity() + ); + startAndShutdown.appendShutdown(executor::shutdown); + + String proxyModeStr = ConfigurationManager.getProxyConfig().getProxyMode(); + MessagingProcessor messagingProcessor; + if (ProxyMode.isClusterMode(proxyModeStr)) { + messagingProcessor = DefaultMessagingProcessor.createForClusterMode(); + } else { + messagingProcessor = DefaultMessagingProcessor.createForLocalMode(brokerController1); + } + startAndShutdown.appendStartAndShutdown(messagingProcessor); + + TlsCertificateManager tlsCertificateManager = new TlsCertificateManager(); + startAndShutdown.appendStartAndShutdown(tlsCertificateManager); + + GrpcMessagingApplication application = GrpcMessagingApplication.create(messagingProcessor); + startAndShutdown.appendStartAndShutdown(application); + + GrpcServer grpcServer = GrpcServerBuilder.newBuilder(executor, + ConfigurationManager.getProxyConfig().getGrpcServerPort(), tlsCertificateManager) + .addService(application) + .addService(ChannelzService.newInstance(100)) + .addService(ProtoReflectionService.newInstance()) + .configInterceptor() + .shutdownTime(ConfigurationManager.getProxyConfig().getGrpcShutdownTimeSeconds(), TimeUnit.SECONDS) + .build(); + startAndShutdown.appendStartAndShutdown(grpcServer); + + startAndShutdown.start(); + Runtime.getRuntime().addShutdownHook(new Thread(() -> { + try { + startAndShutdown.preShutdown(); + startAndShutdown.shutdown(); + } catch (Exception e) { + } + })); + } catch (Throwable e) { + logger.error("proxy start failed, will exit", e); + System.exit(1); + } + } + public static BrokerController createAndStartBroker(MessageStoreConfig storeConfig, BrokerConfig brokerConfig) { NettyServerConfig nettyServerConfig = new NettyServerConfig(); NettyClientConfig nettyClientConfig = new NettyClientConfig(); @@ -172,10 +248,16 @@ public static BrokerController createAndStartBroker(MessageStoreConfig storeConf } public static boolean initTopic(String topic, String nsAddr, String clusterName, int queueNumbers, CQType cqType) { - return initTopic(topic, nsAddr, clusterName, queueNumbers, cqType, TopicMessageType.NORMAL); + return initTopic(topic, nsAddr, clusterName, queueNumbers, cqType, TopicMessageType.NORMAL, null); + } + + public static boolean initTopic(String topic, String nsAddr, String clusterName, int queueNumbers, + CQType cqType, TopicMessageType topicMessageType) { + return initTopic(topic, nsAddr, clusterName, queueNumbers, cqType, topicMessageType, null); } - public static boolean initTopic(String topic, String nsAddr, String clusterName, int queueNumbers, CQType cqType, TopicMessageType topicMessageType) { + public static boolean initTopic(String topic, String nsAddr, String clusterName, int queueNumbers, + CQType cqType, TopicMessageType topicMessageType, Long liteTtl) { boolean createResult; Map attributes = new HashMap<>(); if (!Objects.equals(CQType.SimpleCQ, cqType)) { @@ -184,16 +266,20 @@ public static boolean initTopic(String topic, String nsAddr, String clusterName, if (!Objects.equals(TopicMessageType.NORMAL, topicMessageType)) { attributes.put("+" + TopicAttributes.TOPIC_MESSAGE_TYPE_ATTRIBUTE.getName(), topicMessageType.toString()); } + if (Objects.equals(TopicMessageType.LITE, topicMessageType)) { + attributes.put("+" + TopicAttributes.LITE_EXPIRATION_ATTRIBUTE.getName(), liteTtl.toString()); + } createResult = MQAdminTestUtils.createTopic(nsAddr, clusterName, topic, queueNumbers, attributes, topicCreateTime); return createResult; } public static boolean initTopic(String topic, String nsAddr, String clusterName, CQType cqType) { - return initTopic(topic, nsAddr, clusterName, BaseConf.QUEUE_NUMBERS, cqType, TopicMessageType.NORMAL); + return initTopic(topic, nsAddr, clusterName, BaseConf.QUEUE_NUMBERS, cqType, TopicMessageType.NORMAL, null); } - public static boolean initTopic(String topic, String nsAddr, String clusterName, TopicMessageType topicMessageType) { - return initTopic(topic, nsAddr, clusterName, BaseConf.QUEUE_NUMBERS, CQType.SimpleCQ, topicMessageType); + public static boolean initTopic(String topic, String nsAddr, String clusterName, + TopicMessageType topicMessageType) { + return initTopic(topic, nsAddr, clusterName, BaseConf.QUEUE_NUMBERS, CQType.SimpleCQ, topicMessageType, null); } public static void deleteFile(File file) { @@ -211,4 +297,11 @@ public static void initMQAdmin(String nsAddr) { System.exit(1); } } + + private static class ProxyStartAndShutdown extends AbstractStartAndShutdown { + @Override + public void appendStartAndShutdown(StartAndShutdown startAndShutdown) { + super.appendStartAndShutdown(startAndShutdown); + } + } } diff --git a/test/src/test/java/org/apache/rocketmq/test/dledger/DLedgerProduceAndConsumeIT.java b/test/src/test/java/org/apache/rocketmq/test/dledger/DLedgerProduceAndConsumeIT.java index 43fefd61669..8bdde845a15 100644 --- a/test/src/test/java/org/apache/rocketmq/test/dledger/DLedgerProduceAndConsumeIT.java +++ b/test/src/test/java/org/apache/rocketmq/test/dledger/DLedgerProduceAndConsumeIT.java @@ -38,8 +38,6 @@ import org.junit.Assert; import org.junit.Test; -import static sun.util.locale.BaseLocale.SEP; - public class DLedgerProduceAndConsumeIT { public BrokerConfig buildBrokerConfig(String cluster, String brokerName) { @@ -55,7 +53,7 @@ public MessageStoreConfig buildStoreConfig(String brokerName, String peers, Stri MessageStoreConfig storeConfig = new MessageStoreConfig(); String baseDir = IntegrationTestBase.createBaseDir(); storeConfig.setStorePathRootDir(baseDir); - storeConfig.setStorePathCommitLog(baseDir + SEP + "commitlog"); + storeConfig.setStorePathCommitLog(baseDir + "_" + "commitlog"); storeConfig.setHaListenPort(0); storeConfig.setMappedFileSizeCommitLog(10 * 1024 * 1024); storeConfig.setEnableDLegerCommitLog(true); diff --git a/test/src/test/java/org/apache/rocketmq/test/grpc/v2/GrpcBaseIT.java b/test/src/test/java/org/apache/rocketmq/test/grpc/v2/GrpcBaseIT.java index 47d0e974e0b..1b1abd0a101 100644 --- a/test/src/test/java/org/apache/rocketmq/test/grpc/v2/GrpcBaseIT.java +++ b/test/src/test/java/org/apache/rocketmq/test/grpc/v2/GrpcBaseIT.java @@ -257,6 +257,7 @@ public void testQueryFifoAssignment() throws Exception { public void testTransactionCheckThenCommit() { String topic = initTopicOnSampleTopicBroker(BROKER1_NAME, TopicMessageType.TRANSACTION); String group = MQRandomUtils.getRandomConsumerGroup(); + initConsumerGroup(group); AtomicReference telemetryCommandRef = new AtomicReference<>(null); StreamObserver requestStreamObserver = stub.telemetry(new DefaultTelemetryCommandStreamObserver() { @@ -351,6 +352,7 @@ public void testSimpleConsumerSendAndRecvDelayMessage() throws Exception { String topic = initTopicOnSampleTopicBroker(BROKER1_NAME, TopicMessageType.DELAY); String group = MQRandomUtils.getRandomConsumerGroup(); long delayTime = TimeUnit.SECONDS.toMillis(5); + initConsumerGroup(group); // init consumer offset this.sendClientSettings(stub, buildSimpleConsumerClientSettings(group)).get(); @@ -398,6 +400,7 @@ public void testSimpleConsumerSendAndRecvDelayMessage() throws Exception { public void testSimpleConsumerSendAndRecallDelayMessage() throws Exception { String topic = initTopicOnSampleTopicBroker(BROKER1_NAME, TopicMessageType.DELAY); String group = MQRandomUtils.getRandomConsumerGroup(); + initConsumerGroup(group); long delayTime = TimeUnit.SECONDS.toMillis(5); // init consumer offset @@ -461,6 +464,7 @@ public void testSimpleConsumerSendAndRecallDelayMessage() throws Exception { public void testSimpleConsumerSendAndRecvBigMessage() throws Exception { String topic = initTopicOnSampleTopicBroker(BROKER1_NAME); String group = MQRandomUtils.getRandomConsumerGroup(); + initConsumerGroup(group); int bodySize = 4 * 1024; @@ -483,6 +487,7 @@ public void testSimpleConsumerSendAndRecvBigMessage() throws Exception { public void testSimpleConsumerSendAndRecv() throws Exception { String topic = initTopicOnSampleTopicBroker(BROKER1_NAME); String group = MQRandomUtils.getRandomConsumerGroup(); + initConsumerGroup(group); // init consumer offset this.sendClientSettings(stub, buildSimpleConsumerClientSettings(group)).get(); @@ -539,6 +544,7 @@ public void testSimpleConsumerSendAndRecv() throws Exception { public void testSimpleConsumerToDLQ() throws Exception { String topic = initTopicOnSampleTopicBroker(BROKER1_NAME); String group = MQRandomUtils.getRandomConsumerGroup(); + initConsumerGroup(group); int maxDeliveryAttempts = 2; SubscriptionGroupConfig groupConfig = brokerController1.getSubscriptionGroupManager().findSubscriptionGroupConfig(group); @@ -632,6 +638,7 @@ public void testConsumeOrderly() throws Exception { public void testSimpleConsumerSendAndRecvPriorityMessage() throws Exception { String topic = initTopicOnSampleTopicBroker(BROKER1_NAME, TopicMessageType.PRIORITY); String group = MQRandomUtils.getRandomConsumerGroup(); + initConsumerGroup(group); // init consumer offset this.sendClientSettings(stub, buildSimpleConsumerClientSettings(group)).get(); diff --git a/test/src/test/java/org/apache/rocketmq/test/lmq/TestBenchLmqStore.java b/test/src/test/java/org/apache/rocketmq/test/lmq/TestBenchLmqStore.java index cb35b392b21..0f3f7417d6a 100644 --- a/test/src/test/java/org/apache/rocketmq/test/lmq/TestBenchLmqStore.java +++ b/test/src/test/java/org/apache/rocketmq/test/lmq/TestBenchLmqStore.java @@ -30,9 +30,7 @@ import org.apache.rocketmq.client.impl.consumer.RebalanceImpl; import org.apache.rocketmq.client.impl.factory.MQClientInstance; import org.apache.rocketmq.client.producer.DefaultMQProducer; -import org.apache.rocketmq.client.producer.SendResult; import org.apache.rocketmq.common.MixAll; -import org.apache.rocketmq.common.message.Message; import org.apache.rocketmq.common.message.MessageQueue; import org.apache.rocketmq.remoting.exception.RemotingException; import org.apache.rocketmq.remoting.protocol.header.QueryConsumerOffsetRequestHeader; @@ -61,8 +59,8 @@ public void test() throws MQBrokerException, RemotingException, InterruptedExcep System.setProperty("pullConsumerNum", "1"); System.setProperty("consumerThreadNum", "1"); BenchLmqStore.defaultMQProducer = mock(DefaultMQProducer.class); - SendResult sendResult = new SendResult(); - when(BenchLmqStore.defaultMQProducer.send(any(Message.class))).thenReturn(sendResult); +// SendResult sendResult = new SendResult(); +// when(BenchLmqStore.defaultMQProducer.send(any(Message.class))).thenReturn(sendResult); BenchLmqStore.doSend(); Thread.sleep(100L); //verify(BenchLmqStore.defaultMQProducer, atLeastOnce()).send(any(Message.class)); 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 d935fbc746b..d29ffad2540 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 @@ -16,6 +16,11 @@ */ package org.apache.rocketmq.tools.admin; +import java.io.UnsupportedEncodingException; +import java.util.List; +import java.util.Map; +import java.util.Properties; +import java.util.Set; import org.apache.rocketmq.client.ClientConfig; import org.apache.rocketmq.client.QueryResult; import org.apache.rocketmq.client.exception.MQBrokerException; @@ -47,6 +52,11 @@ import org.apache.rocketmq.remoting.protocol.body.ConsumerConnection; import org.apache.rocketmq.remoting.protocol.body.ConsumerRunningInfo; import org.apache.rocketmq.remoting.protocol.body.EpochEntryCache; +import org.apache.rocketmq.remoting.protocol.body.GetBrokerLiteInfoResponseBody; +import org.apache.rocketmq.remoting.protocol.body.GetLiteClientInfoResponseBody; +import org.apache.rocketmq.remoting.protocol.body.GetLiteGroupInfoResponseBody; +import org.apache.rocketmq.remoting.protocol.body.GetLiteTopicInfoResponseBody; +import org.apache.rocketmq.remoting.protocol.body.GetParentTopicInfoResponseBody; import org.apache.rocketmq.remoting.protocol.body.GroupList; import org.apache.rocketmq.remoting.protocol.body.HARuntimeInfo; import org.apache.rocketmq.remoting.protocol.body.KVTable; @@ -70,12 +80,6 @@ import org.apache.rocketmq.tools.admin.api.MessageTrack; import org.apache.rocketmq.tools.admin.common.AdminToolResult; -import java.io.UnsupportedEncodingException; -import java.util.List; -import java.util.Map; -import java.util.Properties; -import java.util.Set; - public class DefaultMQAdminExt extends ClientConfig implements MQAdminExt { private final DefaultMQAdminExtImpl defaultMQAdminExtImpl; private String adminExtGroup = "admin_ext_group"; @@ -997,4 +1001,44 @@ public void switchTimerEngine(String brokerAddr, String desTimerEngine) throws R UnsupportedEncodingException, InterruptedException, MQBrokerException { defaultMQAdminExtImpl.switchTimerEngine(brokerAddr, desTimerEngine); } + + @Override + public GetBrokerLiteInfoResponseBody getBrokerLiteInfo(final String brokerAddr) + throws RemotingException, MQBrokerException, InterruptedException { + return defaultMQAdminExtImpl.getBrokerLiteInfo(brokerAddr); + } + + @Override + public GetParentTopicInfoResponseBody getParentTopicInfo(final String brokerAddr, final String topic) + throws RemotingException, MQBrokerException, InterruptedException, MQClientException { + return defaultMQAdminExtImpl.getParentTopicInfo(brokerAddr, topic); + } + + @Override + public GetLiteTopicInfoResponseBody getLiteTopicInfo(final String brokerAddr, final String parentTopic, + final String liteTopic) + throws RemotingException, MQBrokerException, InterruptedException, MQClientException { + return defaultMQAdminExtImpl.getLiteTopicInfo(brokerAddr, parentTopic, liteTopic); + } + + @Override + public GetLiteClientInfoResponseBody getLiteClientInfo(final String brokerAddr, final String parentTopic, + final String group, final String clientId) + throws RemotingException, MQBrokerException, InterruptedException, MQClientException { + return defaultMQAdminExtImpl.getLiteClientInfo(brokerAddr, parentTopic, group, clientId); + } + + @Override + public GetLiteGroupInfoResponseBody getLiteGroupInfo(final String brokerAddr, final String group, + final String liteTopic, final int topK) + throws RemotingException, MQBrokerException, InterruptedException, MQClientException { + return defaultMQAdminExtImpl.getLiteGroupInfo(brokerAddr, group, liteTopic, topK); + } + + @Override + public void triggerLiteDispatch(final String brokerAddr, final String group, final String clientId) + throws RemotingException, MQBrokerException, InterruptedException, MQClientException { + defaultMQAdminExtImpl.triggerLiteDispatch(brokerAddr, group, clientId); + } + } 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 b64cafd4615..d96b4b03bcc 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 @@ -17,6 +17,25 @@ package org.apache.rocketmq.tools.admin; import com.alibaba.fastjson2.JSON; +import java.io.UnsupportedEncodingException; +import java.util.ArrayList; +import java.util.Arrays; +import java.util.Collections; +import java.util.HashMap; +import java.util.HashSet; +import java.util.Iterator; +import java.util.List; +import java.util.Map; +import java.util.Map.Entry; +import java.util.Properties; +import java.util.Random; +import java.util.Set; +import java.util.concurrent.ConcurrentHashMap; +import java.util.concurrent.CopyOnWriteArrayList; +import java.util.concurrent.CountDownLatch; +import java.util.concurrent.LinkedBlockingQueue; +import java.util.concurrent.ThreadPoolExecutor; +import java.util.concurrent.TimeUnit; import org.apache.commons.collections.CollectionUtils; import org.apache.commons.lang3.StringUtils; import org.apache.rocketmq.client.QueryResult; @@ -70,6 +89,11 @@ import org.apache.rocketmq.remoting.protocol.body.ConsumerConnection; import org.apache.rocketmq.remoting.protocol.body.ConsumerRunningInfo; import org.apache.rocketmq.remoting.protocol.body.EpochEntryCache; +import org.apache.rocketmq.remoting.protocol.body.GetBrokerLiteInfoResponseBody; +import org.apache.rocketmq.remoting.protocol.body.GetLiteClientInfoResponseBody; +import org.apache.rocketmq.remoting.protocol.body.GetLiteGroupInfoResponseBody; +import org.apache.rocketmq.remoting.protocol.body.GetLiteTopicInfoResponseBody; +import org.apache.rocketmq.remoting.protocol.body.GetParentTopicInfoResponseBody; import org.apache.rocketmq.remoting.protocol.body.GroupList; import org.apache.rocketmq.remoting.protocol.body.HARuntimeInfo; import org.apache.rocketmq.remoting.protocol.body.KVTable; @@ -103,26 +127,6 @@ import org.apache.rocketmq.tools.admin.common.AdminToolsResultCodeEnum; import org.apache.rocketmq.tools.command.CommandUtil; -import java.io.UnsupportedEncodingException; -import java.util.ArrayList; -import java.util.Arrays; -import java.util.Collections; -import java.util.HashMap; -import java.util.HashSet; -import java.util.Iterator; -import java.util.List; -import java.util.Map; -import java.util.Map.Entry; -import java.util.Properties; -import java.util.Random; -import java.util.Set; -import java.util.concurrent.ConcurrentHashMap; -import java.util.concurrent.CopyOnWriteArrayList; -import java.util.concurrent.CountDownLatch; -import java.util.concurrent.LinkedBlockingQueue; -import java.util.concurrent.ThreadPoolExecutor; -import java.util.concurrent.TimeUnit; - public class DefaultMQAdminExtImpl implements MQAdminExt, MQAdminExtInner { private static final String SOCKS_PROXY_JSON = "socksProxyJson"; @@ -2061,4 +2065,44 @@ public void switchTimerEngine(String brokerAddr, String desTimerEngine) throws R this.mqClientInstance.getMQClientAPIImpl().switchTimerEngine(brokerAddr, desTimerEngine, timeoutMillis); } + + @Override + public GetBrokerLiteInfoResponseBody getBrokerLiteInfo(String brokerAddr) + throws RemotingException, MQBrokerException, InterruptedException { + return this.mqClientInstance.getMQClientAPIImpl().getBrokerLiteInfo(brokerAddr, timeoutMillis); + } + + @Override + public GetParentTopicInfoResponseBody getParentTopicInfo(String brokerAddr, String topic) + throws RemotingException, MQBrokerException, InterruptedException, MQClientException { + return this.mqClientInstance.getMQClientAPIImpl().getParentTopicInfo(brokerAddr, topic, timeoutMillis); + } + + @Override + public GetLiteTopicInfoResponseBody getLiteTopicInfo(String brokerAddr, String parentTopic, String liteTopic) + throws RemotingException, MQBrokerException, InterruptedException, MQClientException { + return this.mqClientInstance.getMQClientAPIImpl().getLiteTopicInfo(brokerAddr, parentTopic, liteTopic, + timeoutMillis); + } + + @Override + public GetLiteClientInfoResponseBody getLiteClientInfo(String brokerAddr, String parentTopic, String group, + String clientId) + throws RemotingException, MQBrokerException, InterruptedException, MQClientException { + return this.mqClientInstance.getMQClientAPIImpl().getLiteClientInfo(brokerAddr, parentTopic, group, clientId, + timeoutMillis); + } + + @Override + public GetLiteGroupInfoResponseBody getLiteGroupInfo(String brokerAddr, String group, String liteTopic, int topK) + throws RemotingException, MQBrokerException, InterruptedException, MQClientException { + return this.mqClientInstance.getMQClientAPIImpl().getLiteGroupInfo(brokerAddr, group, liteTopic, topK, timeoutMillis); + } + + @Override + public void triggerLiteDispatch(String brokerAddr, String group, String clientId) + throws RemotingException, MQBrokerException, InterruptedException, MQClientException { + this.mqClientInstance.getMQClientAPIImpl().triggerLiteDispatch(brokerAddr, group, clientId, timeoutMillis); + } + } diff --git a/tools/src/main/java/org/apache/rocketmq/tools/admin/MQAdminExt.java b/tools/src/main/java/org/apache/rocketmq/tools/admin/MQAdminExt.java index 0c433b6d91f..980ff5acdb4 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 @@ -16,6 +16,11 @@ */ package org.apache.rocketmq.tools.admin; +import java.io.UnsupportedEncodingException; +import java.util.List; +import java.util.Map; +import java.util.Properties; +import java.util.Set; import org.apache.rocketmq.client.MQAdmin; import org.apache.rocketmq.client.exception.MQBrokerException; import org.apache.rocketmq.client.exception.MQClientException; @@ -43,6 +48,11 @@ import org.apache.rocketmq.remoting.protocol.body.ConsumerConnection; import org.apache.rocketmq.remoting.protocol.body.ConsumerRunningInfo; import org.apache.rocketmq.remoting.protocol.body.EpochEntryCache; +import org.apache.rocketmq.remoting.protocol.body.GetBrokerLiteInfoResponseBody; +import org.apache.rocketmq.remoting.protocol.body.GetLiteClientInfoResponseBody; +import org.apache.rocketmq.remoting.protocol.body.GetLiteGroupInfoResponseBody; +import org.apache.rocketmq.remoting.protocol.body.GetLiteTopicInfoResponseBody; +import org.apache.rocketmq.remoting.protocol.body.GetParentTopicInfoResponseBody; import org.apache.rocketmq.remoting.protocol.body.GroupList; import org.apache.rocketmq.remoting.protocol.body.HARuntimeInfo; import org.apache.rocketmq.remoting.protocol.body.KVTable; @@ -66,12 +76,6 @@ import org.apache.rocketmq.tools.admin.api.MessageTrack; import org.apache.rocketmq.tools.admin.common.AdminToolResult; -import java.io.UnsupportedEncodingException; -import java.util.List; -import java.util.Map; -import java.util.Properties; -import java.util.Set; - public interface MQAdminExt extends MQAdmin { void start() throws MQClientException; @@ -518,4 +522,25 @@ void exportPopRecords(String brokerAddr, long timeout) throws RemotingConnectExc RemotingSendRequestException, RemotingTimeoutException, MQBrokerException, InterruptedException; void switchTimerEngine(String brokerAddr, String desTimerEngine) throws RemotingConnectException, RemotingSendRequestException, RemotingTimeoutException, UnsupportedEncodingException, InterruptedException, MQBrokerException; + + GetBrokerLiteInfoResponseBody getBrokerLiteInfo(final String brokerAddr) + throws RemotingException, MQBrokerException, InterruptedException, MQClientException; + + GetParentTopicInfoResponseBody getParentTopicInfo(final String brokerAddr, final String topic) + throws RemotingException, MQBrokerException, InterruptedException, MQClientException; + + GetLiteTopicInfoResponseBody getLiteTopicInfo(final String brokerAddr, final String parentTopic, + final String liteTopic) + throws RemotingException, MQBrokerException, InterruptedException, MQClientException; + + GetLiteClientInfoResponseBody getLiteClientInfo(final String brokerAddr, final String parentTopic, + final String group, final String clientId) + throws RemotingException, MQBrokerException, InterruptedException, MQClientException; + + GetLiteGroupInfoResponseBody getLiteGroupInfo(final String brokerAddr, final String group, + final String liteTopic, final int topK) + throws RemotingException, MQBrokerException, InterruptedException, MQClientException; + + void triggerLiteDispatch(final String brokerAddr, final String group, final String clientId) + throws RemotingException, MQBrokerException, InterruptedException, MQClientException; } 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 3513f48f320..d7054933e10 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 @@ -16,6 +16,8 @@ */ package org.apache.rocketmq.tools.command; +import java.util.ArrayList; +import java.util.List; import org.apache.commons.cli.CommandLine; import org.apache.commons.cli.DefaultParser; import org.apache.commons.cli.Options; @@ -78,6 +80,12 @@ import org.apache.rocketmq.tools.command.export.ExportPopRecordCommand; import org.apache.rocketmq.tools.command.ha.GetSyncStateSetSubCommand; import org.apache.rocketmq.tools.command.ha.HAStatusSubCommand; +import org.apache.rocketmq.tools.command.lite.GetBrokerLiteInfoSubCommand; +import org.apache.rocketmq.tools.command.lite.GetLiteClientInfoSubCommand; +import org.apache.rocketmq.tools.command.lite.GetLiteGroupInfoSubCommand; +import org.apache.rocketmq.tools.command.lite.GetLiteTopicInfoSubCommand; +import org.apache.rocketmq.tools.command.lite.GetParentTopicInfoSubCommand; +import org.apache.rocketmq.tools.command.lite.TriggerLiteDispatchSubCommand; import org.apache.rocketmq.tools.command.message.CheckMsgSendRTCommand; import org.apache.rocketmq.tools.command.message.ConsumeMessageCommand; import org.apache.rocketmq.tools.command.message.DumpCompactionLogCommand; @@ -116,9 +124,6 @@ import org.apache.rocketmq.tools.command.topic.UpdateTopicPermSubCommand; import org.apache.rocketmq.tools.command.topic.UpdateTopicSubCommand; -import java.util.ArrayList; -import java.util.List; - public class MQAdminStartup { protected static final List SUB_COMMANDS = new ArrayList<>(); @@ -300,6 +305,14 @@ public static void initCommand() { initCommand(new RocksDBConfigToJsonCommand()); initCommand(new CheckRocksdbCqWriteProgressCommand()); initCommand(new SwitchTimerEngineSubCommand()); + + // lite topic related + initCommand(new GetBrokerLiteInfoSubCommand()); + initCommand(new GetParentTopicInfoSubCommand()); + initCommand(new GetLiteTopicInfoSubCommand()); + initCommand(new GetLiteClientInfoSubCommand()); + initCommand(new GetLiteGroupInfoSubCommand()); + initCommand(new TriggerLiteDispatchSubCommand()); } private static void printHelp() { diff --git a/tools/src/main/java/org/apache/rocketmq/tools/command/lite/GetBrokerLiteInfoSubCommand.java b/tools/src/main/java/org/apache/rocketmq/tools/command/lite/GetBrokerLiteInfoSubCommand.java new file mode 100644 index 00000000000..e64c7ad5fc4 --- /dev/null +++ b/tools/src/main/java/org/apache/rocketmq/tools/command/lite/GetBrokerLiteInfoSubCommand.java @@ -0,0 +1,136 @@ +/* + * 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.lite; + +import com.alibaba.fastjson2.JSON; +import java.util.Set; +import org.apache.commons.cli.CommandLine; +import org.apache.commons.cli.Option; +import org.apache.commons.cli.OptionGroup; +import org.apache.commons.cli.Options; +import org.apache.rocketmq.remoting.RPCHook; +import org.apache.rocketmq.remoting.protocol.body.GetBrokerLiteInfoResponseBody; +import org.apache.rocketmq.tools.admin.DefaultMQAdminExt; +import org.apache.rocketmq.tools.command.CommandUtil; +import org.apache.rocketmq.tools.command.SubCommand; +import org.apache.rocketmq.tools.command.SubCommandException; + +public class GetBrokerLiteInfoSubCommand implements SubCommand { + + @Override + public String commandName() { + return "getBrokerLiteInfo"; + } + + @Override + public String commandDesc() { + return "Get broker lite info."; + } + + @Override + public Options buildCommandlineOptions(Options options) { + OptionGroup optionGroup = new OptionGroup(); + + Option opt = new Option("b", "brokerAddr", true, "Broker address"); + optionGroup.addOption(opt); + + opt = new Option("c", "cluster", true, "Cluster name"); + optionGroup.addOption(opt); + + optionGroup.setRequired(true); + options.addOptionGroup(optionGroup); + + opt = new Option("d", "showDetail", false, "Show topic and group detail info"); + 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 { + defaultMQAdminExt.start(); + boolean showDetail = commandLine.hasOption('d'); + + printHeader(); + + if (commandLine.hasOption('b')) { + String brokerAddr = commandLine.getOptionValue('b').trim(); + GetBrokerLiteInfoResponseBody responseBody = defaultMQAdminExt.getBrokerLiteInfo(brokerAddr); + printRow(responseBody, brokerAddr, showDetail); + } else if (commandLine.hasOption('c')) { + String clusterName = commandLine.getOptionValue('c').trim(); + Set masterSet = CommandUtil + .fetchMasterAddrByClusterName(defaultMQAdminExt, clusterName); + for (String brokerAddr : masterSet) { + try { + GetBrokerLiteInfoResponseBody responseBody = defaultMQAdminExt.getBrokerLiteInfo(brokerAddr); + printRow(responseBody, brokerAddr, showDetail); + } catch (Exception e) { + System.out.printf("[%s] error.%n", brokerAddr); + } + } + } + } catch (Exception e) { + throw new SubCommandException(this.getClass().getSimpleName() + " command failed", e); + } finally { + defaultMQAdminExt.shutdown(); + } + } + + static void printHeader() { + System.out.printf("%-30s %-17s %-10s %-14s %-20s %-17s %-15s %-18s %-15s%n", + "#Broker", + "#Store Type", + "#Max LMQ", + "#Current LMQ", + "#SubscriptionCount", + "#OrderInfoCount", + "#CQTableSize", + "#OffsetTableSize", + "#eventMapSize" + ); + } + + static void printRow( + GetBrokerLiteInfoResponseBody responseBody, + String brokerAddr, + boolean showDetail + ) { + System.out.printf("%-30s %-17s %-10s %-14s %-20s %-17s %-15s %-18s %-15s%n", + brokerAddr, + responseBody.getStoreType(), + responseBody.getMaxLmqNum(), + responseBody.getCurrentLmqNum(), + responseBody.getLiteSubscriptionCount(), + responseBody.getOrderInfoCount(), + responseBody.getCqTableSize(), + responseBody.getOffsetTableSize(), + responseBody.getEventMapSize() + ); + + // If showDetail enabled, print Topic Meta and Group Meta on new lines + if (showDetail) { + System.out.printf("Topic Meta: %s%n", JSON.toJSONString(responseBody.getTopicMeta())); + System.out.printf("Group Meta: %s%n%n", JSON.toJSONString(responseBody.getGroupMeta())); + } + } +} diff --git a/tools/src/main/java/org/apache/rocketmq/tools/command/lite/GetLiteClientInfoSubCommand.java b/tools/src/main/java/org/apache/rocketmq/tools/command/lite/GetLiteClientInfoSubCommand.java new file mode 100644 index 00000000000..33227dc3df8 --- /dev/null +++ b/tools/src/main/java/org/apache/rocketmq/tools/command/lite/GetLiteClientInfoSubCommand.java @@ -0,0 +1,130 @@ +/* + * 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.lite; + +import org.apache.commons.cli.CommandLine; +import org.apache.commons.cli.Option; +import org.apache.commons.cli.Options; +import org.apache.rocketmq.common.UtilAll; +import org.apache.rocketmq.remoting.RPCHook; +import org.apache.rocketmq.remoting.protocol.body.GetLiteClientInfoResponseBody; +import org.apache.rocketmq.remoting.protocol.route.BrokerData; +import org.apache.rocketmq.remoting.protocol.route.TopicRouteData; +import org.apache.rocketmq.tools.admin.DefaultMQAdminExt; +import org.apache.rocketmq.tools.command.SubCommand; +import org.apache.rocketmq.tools.command.SubCommandException; + +public class GetLiteClientInfoSubCommand implements SubCommand { + + @Override + public String commandName() { + return "getLiteClientInfo"; + } + + @Override + public String commandDesc() { + return "Get lite client info."; + } + + @Override + public Options buildCommandlineOptions(Options options) { + Option opt = new Option("p", "parentTopic", true, "Parent topic name"); + opt.setRequired(true); + options.addOption(opt); + + opt = new Option("g", "group", true, "Consumer group"); + opt.setRequired(true); + options.addOption(opt); + + opt = new Option("c", "clientId", true, "Client id"); + opt.setRequired(true); + options.addOption(opt); + + opt = new Option("s", "showDetail", false, "Show details"); + 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 { + defaultMQAdminExt.start(); + + String parentTopic = commandLine.getOptionValue('p').trim(); + String group = commandLine.getOptionValue('g').trim(); + String clientId = commandLine.getOptionValue('c').trim(); + boolean showLiteTopic = commandLine.hasOption('s'); + + TopicRouteData topicRouteData = defaultMQAdminExt.examineTopicRouteInfo(parentTopic); + System.out.printf("Lite Client Info: [%s] [%s] [%s]%n", parentTopic, group, clientId); + + printHeader(); + + for (BrokerData brokerData : topicRouteData.getBrokerDatas()) { + String brokerAddr = brokerData.selectBrokerAddr(); + String brokerName = brokerData.getBrokerName(); + if (null == brokerAddr) { + continue; + } + try { + GetLiteClientInfoResponseBody body = defaultMQAdminExt + .getLiteClientInfo(brokerAddr, parentTopic, group, clientId); + printRow(body, brokerName, showLiteTopic); + } catch (Exception e) { + System.out.printf("[%s] error.%n", brokerData.getBrokerName()); + } + } + } catch (Exception e) { + throw new SubCommandException(this.getClass().getSimpleName() + " command failed", e); + } finally { + defaultMQAdminExt.shutdown(); + } + } + + static void printHeader() { + System.out.printf("%-30s %-20s %-30s %-30s %n", + "#Broker", + "#LiteTopicCount", + "#LastAccessTime", + "#LastConsumeTime" + ); + } + + static void printRow( + GetLiteClientInfoResponseBody responseBody, + String brokerName, + boolean showDetail + ) { + System.out.printf("%-30s %-20s %-30s %-30s %n", + brokerName, + responseBody.getLiteTopicCount() > 0 ? responseBody.getLiteTopicCount() : "N/A", + responseBody.getLastAccessTime() > 0 + ? UtilAll.timeMillisToHumanString2(responseBody.getLastAccessTime()) : "N/A", + responseBody.getLastConsumeTime() > 0 + ? UtilAll.timeMillisToHumanString2(responseBody.getLastConsumeTime()) : "N/A" + ); + + if (showDetail && responseBody.getLiteTopicSet() != null) { + System.out.printf("Lite Topics: %s%n%n", responseBody.getLiteTopicSet()); + } + } +} diff --git a/tools/src/main/java/org/apache/rocketmq/tools/command/lite/GetLiteGroupInfoSubCommand.java b/tools/src/main/java/org/apache/rocketmq/tools/command/lite/GetLiteGroupInfoSubCommand.java new file mode 100644 index 00000000000..6fc17dc523c --- /dev/null +++ b/tools/src/main/java/org/apache/rocketmq/tools/command/lite/GetLiteGroupInfoSubCommand.java @@ -0,0 +1,183 @@ +/* + * 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.lite; + +import java.util.ArrayList; +import java.util.Collections; +import java.util.Comparator; +import java.util.List; +import org.apache.commons.cli.CommandLine; +import org.apache.commons.cli.Option; +import org.apache.commons.cli.Options; +import org.apache.commons.lang3.StringUtils; +import org.apache.rocketmq.common.UtilAll; +import org.apache.rocketmq.common.lite.LiteLagInfo; +import org.apache.rocketmq.remoting.RPCHook; +import org.apache.rocketmq.remoting.protocol.admin.OffsetWrapper; +import org.apache.rocketmq.remoting.protocol.body.GetLiteGroupInfoResponseBody; +import org.apache.rocketmq.remoting.protocol.route.BrokerData; +import org.apache.rocketmq.remoting.protocol.route.TopicRouteData; +import org.apache.rocketmq.tools.admin.DefaultMQAdminExt; +import org.apache.rocketmq.tools.command.SubCommand; +import org.apache.rocketmq.tools.command.SubCommandException; + +public class GetLiteGroupInfoSubCommand implements SubCommand { + + @Override + public String commandName() { + return "getLiteGroupInfo"; + } + + @Override + public String commandDesc() { + return "Get lite group info."; + } + + @Override + public Options buildCommandlineOptions(Options options) { + Option opt = new Option("p", "parentTopic", true, "Parent topic name"); + opt.setRequired(true); + options.addOption(opt); + + opt = new Option("g", "group", true, "Consumer group"); + opt.setRequired(true); + options.addOption(opt); + + opt = new Option("l", "liteTopic", true, "query lite topic detail"); + opt.setRequired(false); + options.addOption(opt); + + opt = new Option("k", "topK", true, "topK value of each broker"); + 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 { + defaultMQAdminExt.start(); + + String parentTopic = commandLine.getOptionValue('p').trim(); + String group = commandLine.getOptionValue('g').trim(); + int topK = 20; + if (commandLine.hasOption('k')) { + topK = Integer.parseInt(commandLine.getOptionValue('k').trim()); + } + String liteTopic = commandLine.hasOption('l') ? commandLine.getOptionValue('l').trim() : null; + boolean queryByLiteTopic = StringUtils.isNotEmpty(liteTopic); + + TopicRouteData topicRouteData = defaultMQAdminExt.examineTopicRouteInfo(parentTopic); + System.out.printf("Lite Group Info: [%s] [%s]%n", group, parentTopic); + + long totalLagCount = 0; + long earliestUnconsumedTimestamp = System.currentTimeMillis(); + List lagCountTopK = new ArrayList<>(); + List lagTimestampTopK = new ArrayList<>(); + + if (queryByLiteTopic) { + System.out.printf("%-50s %-16s %-16s %-16s %-30s%n", + "#Broker Name", + "#BrokerOffset", + "#ConsumeOffset", + "#LagCount", + "#LastUpdate" + ); + } + + for (BrokerData brokerData : topicRouteData.getBrokerDatas()) { + String brokerAddr = brokerData.selectBrokerAddr(); + if (null == brokerAddr) { + continue; + } + try { + GetLiteGroupInfoResponseBody body = defaultMQAdminExt.getLiteGroupInfo(brokerAddr, group, liteTopic, topK); + totalLagCount += body.getTotalLagCount() > 0 ? body.getTotalLagCount() : 0; + if (body.getEarliestUnconsumedTimestamp() > 0) { + earliestUnconsumedTimestamp = Math.min(earliestUnconsumedTimestamp, body.getEarliestUnconsumedTimestamp()); + } + printOffsetWrapper(queryByLiteTopic, brokerData.getBrokerName(), body.getLiteTopicOffsetWrapper()); + lagCountTopK.addAll(body.getLagCountTopK() != null ? body.getLagCountTopK() : Collections.emptyList()); + lagTimestampTopK.addAll(body.getLagTimestampTopK() != null ? body.getLagTimestampTopK() : Collections.emptyList()); + } catch (Exception e) { + System.out.printf("[%s] error.%n", brokerData.getBrokerName()); + } + } + + System.out.printf("Total Lag Count: %d%n", totalLagCount); + long lagTime = System.currentTimeMillis() - earliestUnconsumedTimestamp; + System.out.printf("Min Unconsumed Timestamp: %d (%d s ago)%n%n", earliestUnconsumedTimestamp, lagTime / 1000); + + if (queryByLiteTopic) { + return; + } + + // Sort and print topK lagCountTopK + lagCountTopK.sort((o1, o2) -> Long.compare(o2.getLagCount(), o1.getLagCount())); + System.out.printf("------TopK by lag count-----%n"); + System.out.printf("%-6s %-40s %-12s %-30s%n", "NO", "Lite Topic", "Lag Count", "UnconsumedTimestamp"); + for (int i = 0; i < lagCountTopK.size(); i++) { + LiteLagInfo info = lagCountTopK.get(i); + System.out.printf("%-6s %-40s %-12s %-30s%n", + i + 1, info.getLiteTopic(), info.getLagCount(), info.getEarliestUnconsumedTimestamp() > 0 ? + UtilAll.timeMillisToHumanString2(info.getEarliestUnconsumedTimestamp()) : "-"); + } + + // Sort and print topK lagTimestampTopK + lagTimestampTopK.sort(Comparator.comparingLong(LiteLagInfo::getEarliestUnconsumedTimestamp)); + System.out.printf("%n------TopK by lag time------%n"); + System.out.printf("%-6s %-40s %-12s %-30s%n", "NO", "Lite Topic", "Lag Count", "UnconsumedTimestamp"); + for (int i = 0; i < lagTimestampTopK.size(); i++) { + LiteLagInfo info = lagTimestampTopK.get(i); + System.out.printf("%-6s %-40s %-12s %-30s%n", + i + 1, info.getLiteTopic(), info.getLagCount(), info.getEarliestUnconsumedTimestamp() > 0 ? + UtilAll.timeMillisToHumanString2(info.getEarliestUnconsumedTimestamp()) : "-"); + } + + } catch (Exception e) { + throw new SubCommandException(this.getClass().getSimpleName() + " command failed", e); + } finally { + defaultMQAdminExt.shutdown(); + } + } + + private static void printOffsetWrapper(boolean queryByLiteTopic, String brokerName, OffsetWrapper offsetWrapper) { + if (!queryByLiteTopic) { + return; + } + if (null == offsetWrapper) { + System.out.printf("%-50s %-16s %-16s %-16s %-30s%n", + brokerName, + "-", + "-", + "-", + "-"); + return; + } + System.out.printf("%-50s %-16s %-16s %-16s %-30s%n", + brokerName, + offsetWrapper.getBrokerOffset(), + offsetWrapper.getConsumerOffset(), + offsetWrapper.getBrokerOffset() - offsetWrapper.getConsumerOffset(), + offsetWrapper.getLastTimestamp() > 0 + ? UtilAll.timeMillisToHumanString2(offsetWrapper.getLastTimestamp()) : "-"); + } +} diff --git a/tools/src/main/java/org/apache/rocketmq/tools/command/lite/GetLiteTopicInfoSubCommand.java b/tools/src/main/java/org/apache/rocketmq/tools/command/lite/GetLiteTopicInfoSubCommand.java new file mode 100644 index 00000000000..fe708ea74dc --- /dev/null +++ b/tools/src/main/java/org/apache/rocketmq/tools/command/lite/GetLiteTopicInfoSubCommand.java @@ -0,0 +1,124 @@ +/* + * 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.lite; + +import org.apache.commons.cli.CommandLine; +import org.apache.commons.cli.Option; +import org.apache.commons.cli.Options; +import org.apache.rocketmq.common.UtilAll; +import org.apache.rocketmq.common.lite.LiteUtil; +import org.apache.rocketmq.remoting.RPCHook; +import org.apache.rocketmq.remoting.protocol.body.GetLiteTopicInfoResponseBody; +import org.apache.rocketmq.remoting.protocol.route.BrokerData; +import org.apache.rocketmq.remoting.protocol.route.TopicRouteData; +import org.apache.rocketmq.tools.admin.DefaultMQAdminExt; +import org.apache.rocketmq.tools.command.SubCommand; +import org.apache.rocketmq.tools.command.SubCommandException; + +import java.util.Collections; +import java.util.List; +import java.util.stream.Collectors; + +public class GetLiteTopicInfoSubCommand implements SubCommand { + + @Override + public String commandName() { + return "getLiteTopicInfo"; + } + + @Override + public String commandDesc() { + return "Get lite topic info."; + } + + @Override + public Options buildCommandlineOptions(Options options) { + Option opt = new Option("p", "parentTopic", true, "Parent topic name"); + opt.setRequired(true); + options.addOption(opt); + + opt = new Option("l", "liteTopic", true, "Lite topic name"); + opt.setRequired(true); + options.addOption(opt); + + opt = new Option("s", "showClientId", false, "Show all clientId"); + 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 { + defaultMQAdminExt.start(); + String parentTopic = commandLine.getOptionValue('p').trim(); + String liteTopic = commandLine.getOptionValue('l').trim(); + boolean showClientId = commandLine.hasOption('s'); + + TopicRouteData topicRouteData = defaultMQAdminExt.examineTopicRouteInfo(parentTopic); + System.out.printf("Lite Topic Info: [%s] [%s] [%s]%n", + parentTopic, liteTopic, LiteUtil.toLmqName(parentTopic, liteTopic)); + System.out.printf("%-50s %-14s %-14s %-30s %-12s %-18s %n", + "#Broker Name", + "#MinOffset", + "#MaxOffset", + "#LastUpdate", + "#Sharding", + "#SubClientCount" + ); + for (BrokerData brokerData : topicRouteData.getBrokerDatas()) { + String brokerAddr = brokerData.selectBrokerAddr(); + if (null == brokerAddr) { + continue; + } + GetLiteTopicInfoResponseBody body; + try { + body = defaultMQAdminExt.getLiteTopicInfo(brokerAddr, parentTopic, liteTopic); + if (null == body.getSubscriber()) { + body.setSubscriber(Collections.emptySet()); + } + } catch (Exception e) { + System.out.printf("[%s] error.%n", brokerData.getBrokerName()); + continue; + } + System.out.printf("%-50s %-14s %-14s %-30s %-12s %-18s %n", + UtilAll.frontStringAtLeast(brokerData.getBrokerName(), 40), + body.getTopicOffset().getMinOffset(), + body.getTopicOffset().getMaxOffset(), + body.getTopicOffset().getLastUpdateTimestamp() > 0 + ? UtilAll.timeMillisToHumanString2(body.getTopicOffset().getLastUpdateTimestamp()) : "-", + body.isShardingToBroker(), + body.getSubscriber().size() + ); + if (showClientId) { + List displayList = body.getSubscriber().stream() + .map(clientGroup -> clientGroup.clientId + "@" + clientGroup.group) + .collect(Collectors.toList()); + System.out.printf("%s%n", displayList); + } + } + } 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/lite/GetParentTopicInfoSubCommand.java b/tools/src/main/java/org/apache/rocketmq/tools/command/lite/GetParentTopicInfoSubCommand.java new file mode 100644 index 00000000000..aa1f6d25ed7 --- /dev/null +++ b/tools/src/main/java/org/apache/rocketmq/tools/command/lite/GetParentTopicInfoSubCommand.java @@ -0,0 +1,96 @@ +/* + * 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.lite; + +import org.apache.commons.cli.CommandLine; +import org.apache.commons.cli.Option; +import org.apache.commons.cli.Options; +import org.apache.rocketmq.common.UtilAll; +import org.apache.rocketmq.remoting.RPCHook; +import org.apache.rocketmq.remoting.protocol.body.GetParentTopicInfoResponseBody; +import org.apache.rocketmq.remoting.protocol.route.BrokerData; +import org.apache.rocketmq.remoting.protocol.route.TopicRouteData; +import org.apache.rocketmq.tools.admin.DefaultMQAdminExt; +import org.apache.rocketmq.tools.command.SubCommand; +import org.apache.rocketmq.tools.command.SubCommandException; + +public class GetParentTopicInfoSubCommand implements SubCommand { + + @Override + public String commandName() { + return "getParentTopicInfo"; + } + + @Override + public String commandDesc() { + return "Get parent topic info."; + } + + @Override + public Options buildCommandlineOptions(Options options) { + Option opt = new Option("p", "parentTopic", true, "Parent topic name"); + 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 parentTopic = commandLine.getOptionValue('p').trim(); + + TopicRouteData topicRouteData = defaultMQAdminExt.examineTopicRouteInfo(parentTopic); + System.out.printf("Parent Topic Info: [%s]%n", parentTopic); + System.out.printf("%-50s %-8s %-14s %-14s %-100s %n", + "#Broker Name", + "#TTL", + "#Lite Count", + "#LMQ NUM", + "#GROUPS" + ); + for (BrokerData brokerData : topicRouteData.getBrokerDatas()) { + String brokerAddr = brokerData.selectBrokerAddr(); + if (null == brokerAddr) { + continue; + } + GetParentTopicInfoResponseBody body; + try { + body = defaultMQAdminExt.getParentTopicInfo(brokerAddr, parentTopic); + } catch (Exception e) { + System.out.printf("[%s] error.%n", brokerData.getBrokerName()); + continue; + } + System.out.printf("%-50s %-8s %-14s %-14s %-100s %n", + UtilAll.frontStringAtLeast(brokerData.getBrokerName(), 40), + body.getTtl(), + body.getLiteTopicCount(), + body.getLmqNum(), + body.getGroups() + ); + } + } 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/lite/TriggerLiteDispatchSubCommand.java b/tools/src/main/java/org/apache/rocketmq/tools/command/lite/TriggerLiteDispatchSubCommand.java new file mode 100644 index 00000000000..b85691dfaa3 --- /dev/null +++ b/tools/src/main/java/org/apache/rocketmq/tools/command/lite/TriggerLiteDispatchSubCommand.java @@ -0,0 +1,101 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package org.apache.rocketmq.tools.command.lite; + +import org.apache.commons.cli.CommandLine; +import org.apache.commons.cli.Option; +import org.apache.commons.cli.Options; +import org.apache.rocketmq.remoting.RPCHook; +import org.apache.rocketmq.remoting.protocol.route.BrokerData; +import org.apache.rocketmq.remoting.protocol.route.TopicRouteData; +import org.apache.rocketmq.tools.admin.DefaultMQAdminExt; +import org.apache.rocketmq.tools.command.SubCommand; +import org.apache.rocketmq.tools.command.SubCommandException; + +public class TriggerLiteDispatchSubCommand implements SubCommand { + + @Override + public String commandName() { + return "triggerLiteDispatch"; + } + + @Override + public String commandDesc() { + return "Trigger Lite Dispatch."; + } + + @Override + public Options buildCommandlineOptions(Options options) { + Option opt = new Option("p", "parentTopic", true, "Parent topic name"); + opt.setRequired(true); + options.addOption(opt); + + opt = new Option("g", "group", true, "Consumer group"); + opt.setRequired(true); + options.addOption(opt); + + opt = new Option("c", "clientId", true, "clientId (optional)"); + opt.setRequired(false); + options.addOption(opt); + + opt = new Option("b", "brokerName", true, "brokerName (optional)"); + 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 { + defaultMQAdminExt.start(); + + String parentTopic = commandLine.getOptionValue('p').trim(); + String group = commandLine.getOptionValue('g').trim(); + String clientId = commandLine.hasOption('c') ? commandLine.getOptionValue('c').trim() : null; + String brokerName = commandLine.hasOption('b') ? commandLine.getOptionValue('b').trim() : null; + + TopicRouteData topicRouteData = defaultMQAdminExt.examineTopicRouteInfo(parentTopic); + System.out.printf("Group And Topic Info: [%s] [%s]%n%n", group, parentTopic); + + for (BrokerData brokerData : topicRouteData.getBrokerDatas()) { + String brokerAddr = brokerData.selectBrokerAddr(); + if (null == brokerAddr) { + continue; + } + if (brokerName != null && !brokerName.equals(brokerData.getBrokerName())) { + continue; + } + boolean success = true; + try { + defaultMQAdminExt.triggerLiteDispatch(brokerAddr, group, clientId); + } catch (Exception e) { + success = false; + } + System.out.printf("%-30s %-12s%n", brokerData.getBrokerName(), success ? "dispatched" : "error"); + } + } catch (Exception e) { + throw new SubCommandException(this.getClass().getSimpleName() + " command failed", e); + } finally { + defaultMQAdminExt.shutdown(); + } + } + +} diff --git a/tools/src/main/java/org/apache/rocketmq/tools/command/message/QueryMsgByOffsetSubCommand.java b/tools/src/main/java/org/apache/rocketmq/tools/command/message/QueryMsgByOffsetSubCommand.java index 14d0625fd2c..1f7ad4d19a8 100644 --- a/tools/src/main/java/org/apache/rocketmq/tools/command/message/QueryMsgByOffsetSubCommand.java +++ b/tools/src/main/java/org/apache/rocketmq/tools/command/message/QueryMsgByOffsetSubCommand.java @@ -21,6 +21,7 @@ import org.apache.commons.cli.CommandLine; import org.apache.commons.cli.Option; import org.apache.commons.cli.Options; +import org.apache.commons.lang3.StringUtils; import org.apache.rocketmq.client.consumer.DefaultMQPullConsumer; import org.apache.rocketmq.client.consumer.PullResult; import org.apache.rocketmq.common.MixAll; @@ -63,6 +64,10 @@ public Options buildCommandlineOptions(Options options) { opt = new Option("f", "bodyFormat", true, "print message body by the specified format"); opt.setRequired(false); options.addOption(opt); + + opt = new Option("r", "routeTopic", true, "the topic which is used to find route info"); + opt.setRequired(false); + options.addOption(opt); return options; } @@ -79,6 +84,7 @@ public void execute(CommandLine commandLine, Options options, RPCHook rpcHook) t String brokerName = commandLine.getOptionValue('b').trim(); String queueId = commandLine.getOptionValue('i').trim(); String offset = commandLine.getOptionValue('o').trim(); + String routeTopic = commandLine.hasOption('r') ? commandLine.getOptionValue('r').trim() : null; Charset msgBodyCharset = null; if (commandLine.hasOption('f')) { msgBodyCharset = Charset.forName(commandLine.getOptionValue('f').trim()); @@ -92,6 +98,10 @@ public void execute(CommandLine commandLine, Options options, RPCHook rpcHook) t defaultMQPullConsumer.start(); defaultMQAdminExt.start(); + if (StringUtils.isNotEmpty(routeTopic) && !routeTopic.equals(topic)) { + // try to find route info by route topic, to support LMQ + defaultMQPullConsumer.pull(new MessageQueue(routeTopic, brokerName, 0), "*", 0, 1); + } PullResult pullResult = defaultMQPullConsumer.pull(mq, "*", Long.parseLong(offset), 1); if (pullResult != null) { switch (pullResult.getPullStatus()) { diff --git a/tools/src/test/java/org/apache/rocketmq/tools/command/lite/GetBrokerLiteInfoSubCommandTest.java b/tools/src/test/java/org/apache/rocketmq/tools/command/lite/GetBrokerLiteInfoSubCommandTest.java new file mode 100644 index 00000000000..4098efcfe8e --- /dev/null +++ b/tools/src/test/java/org/apache/rocketmq/tools/command/lite/GetBrokerLiteInfoSubCommandTest.java @@ -0,0 +1,63 @@ +/* + * 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.lite; + +import java.util.Arrays; +import java.util.Collections; +import java.util.HashMap; +import java.util.HashSet; +import java.util.Map; +import java.util.Set; +import org.apache.rocketmq.remoting.protocol.body.GetBrokerLiteInfoResponseBody; +import org.junit.Test; + +public class GetBrokerLiteInfoSubCommandTest { + + private GetBrokerLiteInfoResponseBody mockResponseBody() { + GetBrokerLiteInfoResponseBody responseBody = new GetBrokerLiteInfoResponseBody(); + responseBody.setStoreType("RocksDB"); + responseBody.setMaxLmqNum(1000); + responseBody.setCurrentLmqNum(500); + responseBody.setLiteSubscriptionCount(200); + + // Mock topic meta data + Map topicMeta = new HashMap<>(); + topicMeta.put("TopicA", 10); + topicMeta.put("TopicB", 20); + responseBody.setTopicMeta(topicMeta); + + // Mock group meta data + Map> groupMeta = new HashMap<>(); + Set topics1 = new HashSet<>(Arrays.asList("TopicA", "TopicB")); + Set topics2 = new HashSet<>(Collections.singletonList("TopicC")); + groupMeta.put("Group1", topics1); + groupMeta.put("Group2", topics2); + responseBody.setGroupMeta(groupMeta); + + return responseBody; + } + + @Test + public void testPrint() { + GetBrokerLiteInfoResponseBody responseBody = mockResponseBody(); + GetBrokerLiteInfoSubCommand.printHeader(); + GetBrokerLiteInfoSubCommand.printRow(responseBody, "127.0.0.1:10911", true); + GetBrokerLiteInfoSubCommand.printRow(responseBody, "127.0.0.1:10911", true); + } + +} diff --git a/tools/src/test/java/org/apache/rocketmq/tools/command/lite/GetLiteClientInfoSubCommandTest.java b/tools/src/test/java/org/apache/rocketmq/tools/command/lite/GetLiteClientInfoSubCommandTest.java new file mode 100644 index 00000000000..cfab25e1721 --- /dev/null +++ b/tools/src/test/java/org/apache/rocketmq/tools/command/lite/GetLiteClientInfoSubCommandTest.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.tools.command.lite; + +import java.util.HashSet; +import java.util.Set; +import org.apache.rocketmq.remoting.protocol.body.GetLiteClientInfoResponseBody; +import org.junit.Test; + +public class GetLiteClientInfoSubCommandTest { + + private GetLiteClientInfoResponseBody mockResponseBody() { + GetLiteClientInfoResponseBody responseBody = new GetLiteClientInfoResponseBody(); + responseBody.setParentTopic("testParentTopic"); + responseBody.setGroup("testGroup"); + responseBody.setClientId("testClientId"); + responseBody.setLastAccessTime(System.currentTimeMillis()); + responseBody.setLastConsumeTime(System.currentTimeMillis()); + responseBody.setLiteTopicCount(5); + Set liteTopicSet = new HashSet<>(); + liteTopicSet.add("liteTopic1"); + liteTopicSet.add("liteTopic2"); + responseBody.setLiteTopicSet(liteTopicSet); + return responseBody; + } + + @Test + public void testPrint() { + GetLiteClientInfoResponseBody responseBody = mockResponseBody(); + GetLiteClientInfoSubCommand.printHeader(); + GetLiteClientInfoSubCommand.printRow(responseBody, "brokerName1", true); + GetLiteClientInfoSubCommand.printRow(responseBody, "brokerName2", true); + GetLiteClientInfoSubCommand.printHeader(); + } +}